You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/06/06 16:19:48 UTC

[GitHub] [beam] robertwb opened a new pull request, #21723: Add several IOs to the typescript SDK.

robertwb opened a new pull request, #21723:
URL: https://github.com/apache/beam/pull/21723

   
   ------------------------
   
   Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
   
    - [ ] [**Choose reviewer(s)**](https://beam.apache.org/contribute/#make-your-change) and mention them in a comment (`R: @username`).
    - [ ] Add a link to the appropriate issue in your description, if applicable. This will automatically link the pull request to the issue.
    - [ ] Update `CHANGES.md` with noteworthy changes.
    - [ ] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
   
   See the [Contributor Guide](https://beam.apache.org/contribute) for more tips on [how to make review process smoother](https://beam.apache.org/contribute/#make-reviewers-job-easier).
   
   To check the build health, please visit [https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md](https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md)
   
   GitHub Actions Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   [![Build python source distribution and wheels](https://github.com/apache/beam/workflows/Build%20python%20source%20distribution%20and%20wheels/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule)
   [![Python tests](https://github.com/apache/beam/workflows/Python%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule)
   [![Java tests](https://github.com/apache/beam/workflows/Java%20Tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule)
   
   See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more information about GitHub Actions CI.
   


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] robertwb commented on a diff in pull request #21723: Add several IOs to the typescript SDK.

Posted by GitBox <gi...@apache.org>.
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] TheNeuralBit commented on pull request #21723: Add several IOs to the typescript SDK.

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on PR #21723:
URL: https://github.com/apache/beam/pull/21723#issuecomment-1151583113

   Please let me know if there's anything else I need to review.


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] robertwb commented on pull request #21723: Add several IOs to the typescript SDK.

Posted by GitBox <gi...@apache.org>.
robertwb commented on PR #21723:
URL: https://github.com/apache/beam/pull/21723#issuecomment-1151474570

   Rebased on master now that Python changes have been merged.


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] robertwb commented on pull request #21723: Add several IOs to the typescript SDK.

Posted by GitBox <gi...@apache.org>.
robertwb commented on PR #21723:
URL: https://github.com/apache/beam/pull/21723#issuecomment-1151607415

   Your part is done, Brian. Thanks.
   
   On Thu, Jun 9, 2022 at 1:24 PM Brian Hulette ***@***.***>
   wrote:
   
   > Please let me know if there's anything else I need to review.
   >
   > —
   > Reply to this email directly, view it on GitHub
   > <https://github.com/apache/beam/pull/21723#issuecomment-1151583113>, or
   > unsubscribe
   > <https://github.com/notifications/unsubscribe-auth/AADWVAOO4DLCY2LLJCDM6S3VOJHJBANCNFSM5X74ABAQ>
   > .
   > You are receiving this because you authored the thread.Message ID:
   > ***@***.***>
   >
   


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] robertwb commented on pull request #21723: Add several IOs to the typescript SDK.

Posted by GitBox <gi...@apache.org>.
robertwb commented on PR #21723:
URL: https://github.com/apache/beam/pull/21723#issuecomment-1147650301

   Note that this change is only half as big as it looks due to the lockfile. 
   
   @TheNeuralBit Could you review the changes to apache_beam/dataframe and iobase? @pabloem Could you take the typescript ones?


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] pabloem commented on a diff in pull request #21723: Add several IOs to the typescript SDK.

Posted by GitBox <gi...@apache.org>.
pabloem commented on code in PR #21723:
URL: https://github.com/apache/beam/pull/21723#discussion_r894923232


##########
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:
   the return type - is it `any`? Can we do something like parameterize it as `T`? I don't know what's best practice in typescript
   
   @KonradJanica - we're trying to have a `Row` type - basically, an element with a schema. I guess in JSON these are just objects - is there a better way of delimiting them rather than `any`? In this case, it's just a simple JSON-like object without functions, only data attached to it.



##########
sdks/typescript/src/apache_beam/runners/portable_runner/runner.ts:
##########
@@ -79,6 +79,17 @@ class PortableRunnerPipelineResult implements PipelineResult {
     return state;
   }
 
+  async cancel() {

Review Comment:
   does it make sense to pass a completion callback to the cancel function itself?



##########
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:
   ditto: "does it make sense to pass a completion callback to the cancel function itself?"



##########
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:
   is it worth having separate `read` and `readWithAttributes`? I would personally prefer always including attributes, and only dropping them for some kind of readBytes / readStrings transform. Thoughts?



##########
sdks/typescript/src/apache_beam/runners/portable_runner/runner.ts:
##########
@@ -79,6 +79,17 @@ class PortableRunnerPipelineResult implements PipelineResult {
     return state;
   }
 
+  async cancel() {

Review Comment:
   feel free to add as TODO : )



##########
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:
   so cool that this can be done with pandas



##########
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:
   are these needed?



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] codecov[bot] commented on pull request #21723: Add several IOs to the typescript SDK.

Posted by GitBox <gi...@apache.org>.
codecov[bot] commented on PR #21723:
URL: https://github.com/apache/beam/pull/21723#issuecomment-1147724356

   # [Codecov](https://codecov.io/gh/apache/beam/pull/21723?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#21723](https://codecov.io/gh/apache/beam/pull/21723?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (9aa2ad2) into [master](https://codecov.io/gh/apache/beam/commit/044313637c9eea2e3c2b0baa60bc853a948c12ee?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0443136) will **decrease** coverage by `0.04%`.
   > The diff coverage is `60.71%`.
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #21723      +/-   ##
   ==========================================
   - Coverage   74.10%   74.06%   -0.05%     
   ==========================================
     Files         697      697              
     Lines       92040    92095      +55     
   ==========================================
   + Hits        68209    68210       +1     
   - Misses      22580    22634      +54     
     Partials     1251     1251              
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | python | `83.67% <60.71%> (-0.07%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/beam/pull/21723?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/io/iobase.py](https://codecov.io/gh/apache/beam/pull/21723/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vaW8vaW9iYXNlLnB5) | `86.25% <ø> (ø)` | |
   | [sdks/python/apache\_beam/dataframe/io.py](https://codecov.io/gh/apache/beam/pull/21723/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZGF0YWZyYW1lL2lvLnB5) | `89.90% <41.17%> (-2.13%)` | :arrow_down: |
   | [sdks/python/apache\_beam/dataframe/schemas.py](https://codecov.io/gh/apache/beam/pull/21723/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZGF0YWZyYW1lL3NjaGVtYXMucHk=) | `96.92% <90.00%> (-0.72%)` | :arrow_down: |
   | [sdks/python/apache\_beam/dataframe/convert.py](https://codecov.io/gh/apache/beam/pull/21723/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZGF0YWZyYW1lL2NvbnZlcnQucHk=) | `90.36% <100.00%> (ø)` | |
   | [...ache\_beam/runners/dataflow/dataflow\_job\_service.py](https://codecov.io/gh/apache/beam/pull/21723/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kYXRhZmxvdy9kYXRhZmxvd19qb2Jfc2VydmljZS5weQ==) | `50.00% <0.00%> (-12.17%)` | :arrow_down: |
   | [...eam/runners/portability/fn\_api\_runner/fn\_runner.py](https://codecov.io/gh/apache/beam/pull/21723/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9wb3J0YWJpbGl0eS9mbl9hcGlfcnVubmVyL2ZuX3J1bm5lci5weQ==) | `87.51% <0.00%> (-2.50%)` | :arrow_down: |
   | [...nners/portability/fn\_api\_runner/worker\_handlers.py](https://codecov.io/gh/apache/beam/pull/21723/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9wb3J0YWJpbGl0eS9mbl9hcGlfcnVubmVyL3dvcmtlcl9oYW5kbGVycy5weQ==) | `77.89% <0.00%> (-1.45%)` | :arrow_down: |
   | [...hon/apache\_beam/runners/direct/test\_stream\_impl.py](https://codecov.io/gh/apache/beam/pull/21723/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kaXJlY3QvdGVzdF9zdHJlYW1faW1wbC5weQ==) | `93.28% <0.00%> (-0.75%)` | :arrow_down: |
   | [...eam/runners/portability/fn\_api\_runner/execution.py](https://codecov.io/gh/apache/beam/pull/21723/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9wb3J0YWJpbGl0eS9mbl9hcGlfcnVubmVyL2V4ZWN1dGlvbi5weQ==) | `92.44% <0.00%> (-0.65%)` | :arrow_down: |
   | [sdks/python/apache\_beam/transforms/combiners.py](https://codecov.io/gh/apache/beam/pull/21723/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdHJhbnNmb3Jtcy9jb21iaW5lcnMucHk=) | `93.05% <0.00%> (-0.39%)` | :arrow_down: |
   | ... and [9 more](https://codecov.io/gh/apache/beam/pull/21723/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/21723?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/21723?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [0443136...9aa2ad2](https://codecov.io/gh/apache/beam/pull/21723?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] robertwb merged pull request #21723: Add several IOs to the typescript SDK.

Posted by GitBox <gi...@apache.org>.
robertwb merged PR #21723:
URL: https://github.com/apache/beam/pull/21723


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] pabloem commented on pull request #21723: Add several IOs to the typescript SDK.

Posted by GitBox <gi...@apache.org>.
pabloem commented on PR #21723:
URL: https://github.com/apache/beam/pull/21723#issuecomment-1151577424

   thanks Robert - taking a look!


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] pabloem commented on pull request #21723: Add several IOs to the typescript SDK.

Posted by GitBox <gi...@apache.org>.
pabloem commented on PR #21723:
URL: https://github.com/apache/beam/pull/21723#issuecomment-1163787806

   LGTM! Thanks @robertwb 


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org