[
https://issues.apache.org/jira/browse/BEAM-1754?focusedWorklogId=762471&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-762471
]
ASF GitHub Bot logged work on BEAM-1754:
----------------------------------------
Author: ASF GitHub Bot
Created on: 26/Apr/22 18:34
Start Date: 26/Apr/22 18:34
Worklog Time Spent: 10m
Work Description: damccorm commented on code in PR #17341:
URL: https://github.com/apache/beam/pull/17341#discussion_r859008628
##########
sdks/typescript/src/apache_beam/internal/pipeline.ts:
##########
@@ -0,0 +1,341 @@
+/*
+ * 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 equal from "fast-deep-equal";
+
+import * as runnerApi from "../proto/beam_runner_api";
+import * as fnApi from "../proto/beam_fn_api";
+import {
+ PTransform,
+ AsyncPTransform,
+ extractName,
+} from "../transforms/transform";
+import { GlobalWindows } from "../transforms/windowings";
+import * as pvalue from "../pvalue";
+import { WindowInto } from "../transforms/window";
+import * as environments from "./environments";
+import { Coder, globalRegistry as globalCoderRegistry } from
"../coders/coders";
+
+type Components = runnerApi.Components | fnApi.ProcessBundleDescriptor;
+
+// TODO: Cleanup. Where to put this.
+export class PipelineContext {
+ counter: number = 0;
+
+ private coders: { [key: string]: Coder<any> } = {};
+
+ constructor(public components: Components) {}
+
+ getCoder<T>(coderId: string): Coder<T> {
+ const this_ = this;
+ if (this.coders[coderId] == undefined) {
+ const coderProto = this.components.coders[coderId];
+ const coderConstructor = globalCoderRegistry().get(coderProto.spec!.urn);
+ const components = (coderProto.componentCoderIds || []).map(
+ this_.getCoder.bind(this_)
+ );
+ if (coderProto.spec!.payload && coderProto.spec!.payload.length) {
Review Comment:
```suggestion
if (coderProto.spec!.payload?.length) {
```
##########
sdks/typescript/webpack.config.js:
##########
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
Review Comment:
@kerrydc bumping Robert's question from spreadsheet here - do we need this
file? It looks like the tsconfig is handling the ts-closure-transform usage,
and I don't think this file is actually being used at the moment
Issue Time Tracking
-------------------
Worklog Id: (was: 762471)
Time Spent: 5h 10m (was: 5h)
> Will Dataflow ever support Node.js with an SDK similar to Java or Python?
> -------------------------------------------------------------------------
>
> Key: BEAM-1754
> URL: https://issues.apache.org/jira/browse/BEAM-1754
> Project: Beam
> Issue Type: New Feature
> Components: sdk-ideas
> Reporter: Diego Zuluaga
> Assignee: Kerry Donny-Clark
> Priority: P3
> Labels: node.js
> Time Spent: 5h 10m
> Remaining Estimate: 0h
>
> I like the philosophy behind DataFlow and found the Java and Python samples
> highly comprehensible. However, I have to admit that for most Node.js
> developers who have little background on typed languages and are used to get
> up to speed with frameworks incredibly fast, learning Dataflow might take
> some learning curve that they/we're not used to. So, I wonder if at any point
> in time Dataflow will provide a Node.js SDK. Maybe this is out of the
> question, but I wanted to run it by the team as it would be awesome to have
> something along these lines!
> Thanks,
> Diego
> Question originaly posted in SO:
> http://stackoverflow.com/questions/42893436/will-dataflow-ever-support-node-js-with-and-sdk-similar-to-java-or-python
--
This message was sent by Atlassian Jira
(v8.20.7#820007)