Skip to content

Commit

Permalink
Add some auto-starting runners to the typescript SDK. (apache#17580)
Browse files Browse the repository at this point in the history
Adds out-of-the-box support for FlinkRunner, DataflowRunner, and the Python Universal Local runner. Also adds a DefaultRunner which chooses between the DirectRunner and the ULR depending on the properties of the pipeline.
  • Loading branch information
robertwb authored May 13, 2022
1 parent c110365 commit 66e85da
Show file tree
Hide file tree
Showing 15 changed files with 479 additions and 77 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ def maybe_strict_version(s):
or options.beam_version.startswith('https://')):
# It's a path to a tarball.
beam_version = os.path.basename(options.beam_version)
beam_package = options.beam_version
beam_package = options.beam_version + '[gcp,aws,asure,dataframe]'
else:
beam_version = options.beam_version
beam_package = 'apache_beam[gcp,aws,asure,dataframe]==' + beam_version
Expand Down
15 changes: 13 additions & 2 deletions sdks/typescript/package-lock.json

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

1 change: 1 addition & 0 deletions sdks/typescript/package.json
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
"chai": "^4.3.4",
"date-fns": "^2.28.0",
"fast-deep-equal": "^3.1.3",
"find-git-root": "^1.0.4",
"long": "^4.0.0",
"protobufjs": "^6.10.2",
"queue-typescript": "^1.0.1",
Expand Down
20 changes: 17 additions & 3 deletions sdks/typescript/src/apache_beam/examples/wordcount.ts
Original file line number Diff line number Diff line change
Expand Up @@ -16,10 +16,24 @@
* limitations under the License.
*/

// TODO: Should this be in a top-level examples dir, rather than under apache_beam.
// Run directly with
//
// node dist/src/apache_beam/examples/wordcount.js
//
// A different runner can be chosen via a --runner argument, e.g.
//
// node dist/src/apache_beam/examples/wordcount.js --runner=flink
//
// To run on Dataflow, pass the required arguments:
//
// node dist/src/apache_beam/examples/wordcount.js --runner=dataflow --project=PROJECT_ID --tempLocation=gs://BUCKET/DIR' --region=us-central1

// TODO: Should this be in a top-level examples dir, rather than under apache_beam?

import * as yargs from "yargs";

import * as beam from "../../apache_beam";
import { DirectRunner } from "../runners/direct_runner";
import { createRunner } from "../runners/runner";

import { count } from "../transforms/combiners";
import { GroupBy } from "../transforms/group_and_combine";
Expand All @@ -45,7 +59,7 @@ function wordCount(lines: beam.PCollection<string>): beam.PCollection<any> {
}

async function main() {
await new DirectRunner().run((root) => {
await createRunner(yargs.argv).run((root) => {
const lines = root.apply(
new beam.Create([
"In the beginning God created the heaven and the earth.",
Expand Down
44 changes: 44 additions & 0 deletions sdks/typescript/src/apache_beam/runners/dataflow.ts
Original file line number Diff line number Diff line change
@@ -0,0 +1,44 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

import { Pipeline } from "../internal/pipeline";
import { PipelineResult, Runner } from "./runner";
import { PortableRunner } from "./portable_runner/runner";
import { PythonService } from "../utils/service";

export function dataflowRunner(runnerOptions: {
project: string;
tempLocation: string;
region: string;
[others: string]: any;
}): Runner {
return new (class extends Runner {
async runPipeline(
pipeline: Pipeline,
options: Object = {}
): Promise<PipelineResult> {
return new PortableRunner(
runnerOptions as any,
new PythonService("apache_beam.runners.dataflow.dataflow_job_service", [
"--port",
"{{PORT}}",
])
).runPipeline(pipeline, options);
}
})();
}
47 changes: 47 additions & 0 deletions sdks/typescript/src/apache_beam/runners/direct_runner.ts
Original file line number Diff line number Diff line change
Expand Up @@ -44,14 +44,61 @@ import {
} from "../values";
import { PaneInfoCoder } from "../coders/standard_coders";
import { Coder, Context as CoderContext } from "../coders/coders";
import * as environments from "../internal/environments";
import { serializeFn, deserializeFn } from "../internal/serialize";

const SUPPORTED_REQUIREMENTS: string[] = [];

export function directRunner(options: Object = {}): Runner {
return new DirectRunner(options);
}

export class DirectRunner extends Runner {
// All the operators for a given pipeline should share the same state.
// This global mapping allows operators to look up a shared state object for
// a given pipeline on deserialization.
static inMemoryStatesRefs: Map<string, InMemoryStateProvider> = new Map();

constructor(private options: Object = {}) {
super();
}

unsupportedFeatures(pipeline, options: Object = {}): string[] {
return [...this.unsupportedFeaturesIter(pipeline, options)];
}

*unsupportedFeaturesIter(pipeline, options: Object = {}) {
const proto: runnerApi.Pipeline = pipeline.proto;
for (const requirement of proto.requirements) {
if (!SUPPORTED_REQUIREMENTS.includes(requirement)) {
yield requirement;
}
}

for (const env of Object.values(proto.components!.environments)) {
if (
env.urn &&
env.urn != environments.TYPESCRIPT_DEFAULT_ENVIRONMENT_URN
) {
yield env.urn;
}
}

for (const windowing of Object.values(
proto.components!.windowingStrategies
)) {
if (
![
runnerApi.MergeStatus_Enum.UNSPECIFIED,
runnerApi.MergeStatus_Enum.NON_MERGING,
runnerApi.MergeStatus_Enum.ALREADY_MERGED,
].includes(windowing.mergeStatus)
) {
yield "MergeStatus=" + windowing.mergeStatus;
}
}
}

async runPipeline(p): Promise<PipelineResult> {
// console.dir(p.proto, { depth: null });

Expand Down
84 changes: 84 additions & 0 deletions sdks/typescript/src/apache_beam/runners/flink.ts
Original file line number Diff line number Diff line change
@@ -0,0 +1,84 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

const fs = require("fs");
const os = require("os");
const path = require("path");

import { Pipeline } from "../internal/pipeline";
import { PipelineResult, Runner } from "./runner";
import { PortableRunner } from "./portable_runner/runner";
import { JavaJarService } from "../utils/service";

const MAGIC_HOST_NAMES = ["[local]", "[auto]"];

// These should stay in sync with gradle.properties.
const PUBLISHED_FLINK_VERSIONS = ["1.12", "1.13", "1.14"];

const defaultOptions = {
flinkMaster: "[local]",
flinkVersion: PUBLISHED_FLINK_VERSIONS[PUBLISHED_FLINK_VERSIONS.length - 1],
};

export function flinkRunner(runnerOptions: Object = {}): Runner {
return new (class extends Runner {
async runPipeline(
pipeline: Pipeline,
options: Object = {}
): Promise<PipelineResult> {
const allOptions = {
...defaultOptions,
...runnerOptions,
...options,
} as any;
if (
!allOptions.environmentType &&
MAGIC_HOST_NAMES.includes(allOptions.flinkMaster)
) {
allOptions.environmentType = "LOOPBACK";
}
if (!allOptions.artifactsDir) {
allOptions.artifactsDir = fs.mkdtempSync(
path.join(os.tmpdir(), "flinkArtifactsDir")
);
}

const jobServerJar =
allOptions.flinkJobServerJar ||
(await JavaJarService.cachedJar(
JavaJarService.gradleToJar(
`runners:flink:${allOptions.flinkVersion}:job-server:shadowJar`
)
));
const jobServer = new JavaJarService(jobServerJar, [
"--flink-master",
allOptions.flinkMaster,
"--artifacts-dir",
allOptions.artifactsDir,
"--job-port",
"{{PORT}}",
"--artifact-port",
"0",
"--expansion-port",
"0",
]);

return new PortableRunner(allOptions, jobServer).runPipeline(pipeline);
}
})();
}
Loading

0 comments on commit 66e85da

Please sign in to comment.