beam-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From dhalp...@apache.org
Subject [1/2] incubator-beam git commit: Add a compatibility stub of GABWViaWindowSetDoFn
Date Thu, 19 May 2016 01:41:51 GMT
Repository: incubator-beam
Updated Branches:
  refs/heads/master 3edae9b8b -> f184bcf37


Add a compatibility stub of GABWViaWindowSetDoFn

Direct calls to
`org.apache.beam.sdk.util.GroupAlsoByWindowViaWindowSetDoFn#create`
failed following the rename in 892ead2c.

This re-introduces that method as a redirect to the new name to
unbreak things while waiting for a complete fix.


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/6c942435
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/6c942435
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/6c942435

Branch: refs/heads/master
Commit: 6c942435b52ab7f79ead3342e79154aab5e133bc
Parents: 3edae9b
Author: bchambers <bchambers@google.com>
Authored: Wed May 18 17:20:18 2016 -0700
Committer: bchambers <bchambers@google.com>
Committed: Wed May 18 17:33:54 2016 -0700

----------------------------------------------------------------------
 runners/google-cloud-dataflow-java/pom.xml      |  5 +++
 .../util/GroupAlsoByWindowViaWindowSetDoFn.java | 40 ++++++++++++++++++++
 2 files changed, 45 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6c942435/runners/google-cloud-dataflow-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml
index f7f1d80..2b7b49d 100644
--- a/runners/google-cloud-dataflow-java/pom.xml
+++ b/runners/google-cloud-dataflow-java/pom.xml
@@ -288,6 +288,11 @@
     </dependency>
 
     <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>runners-core</artifactId>
+    </dependency>
+
+    <dependency>
       <groupId>org.apache.avro</groupId>
       <artifactId>avro</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6c942435/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowViaWindowSetDoFn.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowViaWindowSetDoFn.java
b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowViaWindowSetDoFn.java
new file mode 100644
index 0000000..42ebf94
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowViaWindowSetDoFn.java
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.KV;
+
+/**
+ * Stub class that exists for compatibility with code expecting
+ * {@link org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn} in the old location.
+ */
+public class GroupAlsoByWindowViaWindowSetDoFn {
+
+  public static <K, InputT, OutputT, W extends BoundedWindow>
+  DoFn<KeyedWorkItem<K, InputT>, KV<K, OutputT>> create(
+      WindowingStrategy<?, W> strategy, SystemReduceFn<K, InputT, ?, OutputT, W>
reduceFn) {
+    return org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn.create(
+        strategy, reduceFn);
+  }
+
+  /** Prohibit instantiation. */
+  private GroupAlsoByWindowViaWindowSetDoFn() {}
+}
+


Mime
View raw message