robertwb commented on code in PR #21723: URL: https://github.com/apache/beam/pull/21723#discussion_r904400499
########## sdks/typescript/src/apache_beam/worker/worker_main.ts: ########## @@ -29,6 +29,8 @@ import { BeamFnLoggingClient } from "../proto/beam_fn_api.grpc-client"; // Needed for registration. import * as row_coder from "../coders/row_coder"; import * as combiners from "../transforms/combiners"; +import * as pubsub from "../io/pubsub"; +import * as assert from "../testing/assert"; Review Comment: Yes, these imports register the (de)serialization hooks. Future work to do this automatically. ########## sdks/typescript/src/apache_beam/runners/portable_runner/runner.ts: ########## @@ -79,6 +79,17 @@ class PortableRunnerPipelineResult implements PipelineResult { return state; } + async cancel() { Review Comment: I'm not sure there's much benefit in doing so (over just using await). If so, we can always add it later (but it wouldn't be possible to remove). ########## sdks/typescript/src/apache_beam/io/pubsub.ts: ########## @@ -0,0 +1,129 @@ +/* + * 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 * as PubSub from "@google-cloud/pubsub"; + +import * as beam from "../pvalue"; +import * as external from "../transforms/external"; +import * as internal from "../transforms/internal"; +import { AsyncPTransform, withName } from "../transforms/transform"; +import { RowCoder } from "../coders/row_coder"; +import { BytesCoder } from "../coders/required_coders"; +import { serviceProviderFromJavaGradleTarget } from "../utils/service"; +import { camelToSnakeOptions } from "../utils/utils"; + +const PUBSUB_EXPANSION_GRADLE_TARGET = + "sdks:java:io:google-cloud-platform:expansion-service:shadowJar"; + +const readSchema = RowCoder.inferSchemaOfJSON({ + topic: "string", + subscription: "string", + idAttribute: "string", + timestampAttribute: "string", +}); + +type ReadOptions = + | { + topic: string; + subscription?: never; + idAttribute?: string; + timestampAttribute?: string; + } + | { + topic?: never; + subscription: string; + idAttribute?: string; + timestampAttribute?: string; + }; + +// TODO: Schema-producing variants. +export function readFromPubSub( + options: ReadOptions +): AsyncPTransform<beam.Root, beam.PCollection<Uint8Array>> { + if (options.topic && options.subscription) { + throw new TypeError( + "Exactly one of topic or subscription must be provided." + ); + } + return withName( + "readFromPubSubRaw", + external.rawExternalTransform<beam.Root, beam.PCollection<Uint8Array>>( + "beam:transform:org.apache.beam:pubsub_read:v1", + camelToSnakeOptions(options), + serviceProviderFromJavaGradleTarget(PUBSUB_EXPANSION_GRADLE_TARGET) + ) + ); +} Review Comment: I think this is worth it as the common case is to just look at the data. In addition, in that case re-encoding and decoding the protos over the cross-langauge boundary can get expensive. ########## sdks/typescript/src/apache_beam/runners/portable_runner/runner.ts: ########## @@ -137,6 +148,11 @@ export class PortableRunner extends Runner { return await call.response; } + async cancelJob(jobId: string) { Review Comment: Same. ########## sdks/typescript/src/apache_beam/io/kafka.ts: ########## @@ -0,0 +1,124 @@ +/* + * 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 * as beam from "../../apache_beam"; +import * as external from "../transforms/external"; +import { Schema } from "../proto/schema"; +import { RowCoder } from "../coders/row_coder"; +import { serviceProviderFromJavaGradleTarget } from "../utils/service"; +import * as protobufjs from "protobufjs"; +import { camelToSnakeOptions } from "../utils/utils"; + +const KAFKA_EXPANSION_GRADLE_TARGET = + "sdks:java:io:expansion-service:shadowJar"; + +export type ReadFromKafkaOptions = { + keyDeserializer?: string; + valueDeserializer?: string; + startReadTime?: number; + maxNumRecords?: number; + maxReadTime?: number; + commitOffsetInFinalize?: boolean; + timestampPolicy?: "ProcessingTime" | "CreateTime" | "LogAppendTime"; +}; + +const defaultReadFromKafkaOptions = { + keyDeserializer: + "org.apache.kafka.common.serialization.ByteArrayDeserializer", + valueDeserializer: + "org.apache.kafka.common.serialization.ByteArrayDeserializer", + timestampPolicy: "ProcessingTime", +}; + +export function readFromKafka( + consumerConfig: { [key: string]: string }, // TODO: Or a map? + topics: string[], + options: ReadFromKafkaOptions = {} +): beam.AsyncPTransform<beam.Root, beam.PCollection<any>> { Review Comment: I added a template parameter so the user can specify the concrete type if desired. If they don't they just get any like before. ########## sdks/typescript/src/apache_beam/io/parquetio.ts: ########## @@ -0,0 +1,69 @@ +/* + * 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 * as beam from "../../apache_beam"; +import { StrUtf8Coder } from "../coders/standard_coders"; +import * as external from "../transforms/external"; +import { withCoderInternal } from "../transforms/internal"; +import { pythonTransform } from "../transforms/python"; +import { PythonService } from "../utils/service"; +import { camelToSnakeOptions } from "../utils/utils"; +import { Schema } from "../proto/schema"; +import { RowCoder } from "../coders/row_coder"; + Review Comment: And it's super fast too :). -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: github-unsubscr...@beam.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org