You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by ec...@apache.org on 2018/12/27 13:57:17 UTC

[beam] 01/04: Add source mocks

This is an automated email from the ASF dual-hosted git repository.

echauchot pushed a commit to branch spark-runner_structured-streaming
in repository https://gitbox.apache.org/repos/asf/beam.git

commit 83f84873649f42ffaf8c3e5a5c862b0df4262d9e
Author: Etienne Chauchot <ec...@apache.org>
AuthorDate: Thu Dec 20 15:38:11 2018 +0100

    Add source mocks
---
 .../batch/ReadSourceTranslatorMockBatch.java       |  81 +++++++++++++++
 .../translation/io/DatasetSourceMock.java          | 114 +++++++++++++++++++++
 2 files changed, 195 insertions(+)

diff --git a/runners/spark-structured-streaming/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReadSourceTranslatorMockBatch.java b/runners/spark-structured-streaming/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReadSourceTranslatorMockBatch.java
new file mode 100644
index 0000000..5b1bada
--- /dev/null
+++ b/runners/spark-structured-streaming/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReadSourceTranslatorMockBatch.java
@@ -0,0 +1,81 @@
+/*
+ * 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.runners.spark.structuredstreaming.translation.batch;
+
+import java.io.IOException;
+import org.apache.beam.runners.core.construction.ReadTranslation;
+import org.apache.beam.runners.spark.structuredstreaming.translation.TransformTranslator;
+import org.apache.beam.runners.spark.structuredstreaming.translation.TranslationContext;
+import org.apache.beam.runners.spark.structuredstreaming.translation.io.DatasetSource;
+import org.apache.beam.runners.spark.structuredstreaming.translation.io.DatasetSourceMock;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.runners.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.spark.api.java.function.MapFunction;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoder;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.streaming.DataStreamReader;
+import org.apache.spark.sql.types.StructType;
+import scala.reflect.ClassTag;
+
+class ReadSourceTranslatorMockBatch<T>
+    implements TransformTranslator<PTransform<PBegin, PCollection<T>>> {
+
+  private String SOURCE_PROVIDER_CLASS = DatasetSourceMock.class.getCanonicalName();
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public void translateTransform(
+      PTransform<PBegin, PCollection<T>> transform, TranslationContext context) {
+    AppliedPTransform<PBegin, PCollection<T>, PTransform<PBegin, PCollection<T>>> rootTransform =
+        (AppliedPTransform<PBegin, PCollection<T>, PTransform<PBegin, PCollection<T>>>)
+            context.getCurrentTransform();
+
+        String providerClassName = SOURCE_PROVIDER_CLASS.substring(0, SOURCE_PROVIDER_CLASS.indexOf("$"));
+    SparkSession sparkSession = context.getSparkSession();
+    DataStreamReader dataStreamReader = sparkSession.readStream().format(providerClassName);
+
+    Dataset<Row> rowDataset = dataStreamReader.load();
+
+    MapFunction<Row, WindowedValue<Integer>> func = new MapFunction<Row, WindowedValue<Integer>>() {
+      @Override public WindowedValue<Integer> call(Row value) throws Exception {
+        //there is only one value put in each Row by the InputPartitionReader
+        return value.<WindowedValue<Integer>>getAs(0);
+      }
+    };
+    Dataset<WindowedValue<Integer>> dataset = rowDataset.map(func, new Encoder<WindowedValue<Integer>>() {
+
+      @Override public StructType schema() {
+        return null;
+      }
+
+      @Override public ClassTag<WindowedValue<Integer>> clsTag() {
+        return scala.reflect.ClassTag$.MODULE$.<WindowedValue<Integer>>apply(WindowedValue.class);
+      }
+    });
+
+    PCollection<T> output = (PCollection<T>) context.getOutput();
+    context.putDataset(output, dataset);
+  }
+}
diff --git a/runners/spark-structured-streaming/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/io/DatasetSourceMock.java b/runners/spark-structured-streaming/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/io/DatasetSourceMock.java
new file mode 100644
index 0000000..fa42fdf
--- /dev/null
+++ b/runners/spark-structured-streaming/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/io/DatasetSourceMock.java
@@ -0,0 +1,114 @@
+/*
+ * 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.runners.spark.structuredstreaming.translation.io;
+
+import static scala.collection.JavaConversions.asScalaBuffer;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.sources.v2.DataSourceOptions;
+import org.apache.spark.sql.sources.v2.DataSourceV2;
+import org.apache.spark.sql.sources.v2.MicroBatchReadSupport;
+import org.apache.spark.sql.sources.v2.reader.InputPartition;
+import org.apache.spark.sql.sources.v2.reader.InputPartitionReader;
+import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReader;
+import org.apache.spark.sql.sources.v2.reader.streaming.Offset;
+import org.apache.spark.sql.types.StructType;
+import org.joda.time.Instant;
+
+/**
+ * This is a mock source that gives values between 0 and 999.
+ */
+public class DatasetSourceMock implements DataSourceV2, MicroBatchReadSupport {
+
+  @Override public MicroBatchReader createMicroBatchReader(Optional<StructType> schema, String checkpointLocation, DataSourceOptions options) {
+    return new DatasetMicroBatchReader();
+  }
+
+  /** This class can be mapped to Beam {@link BoundedSource}. */
+  private class DatasetMicroBatchReader implements MicroBatchReader {
+
+    @Override public void setOffsetRange(Optional<Offset> start, Optional<Offset> end) {
+    }
+
+    @Override public Offset getStartOffset() {
+      return null;
+    }
+
+    @Override public Offset getEndOffset() {
+      return null;
+    }
+
+    @Override public Offset deserializeOffset(String json) {
+      return null;
+    }
+
+    @Override public void commit(Offset end) {
+    }
+
+    @Override public void stop() {
+    }
+
+    @Override public StructType readSchema() {
+      return null;
+    }
+
+    @Override public List<InputPartition<InternalRow>> planInputPartitions() {
+      List<InputPartition<InternalRow>> result = new ArrayList<>();
+      result.add(new InputPartition<InternalRow>() {
+
+        @Override public InputPartitionReader<InternalRow> createPartitionReader() {
+          return new DatasetMicroBatchPartitionReaderMock();
+        }
+      });
+      return result;
+    }
+  }
+
+  /** This class is a mocked reader*/
+  private class DatasetMicroBatchPartitionReaderMock implements InputPartitionReader<InternalRow> {
+
+    private ArrayList<Integer> values;
+    private int currentIndex = 0;
+
+    private DatasetMicroBatchPartitionReaderMock() {
+      for (int i = 0; i < 1000; i++){
+        values.add(i);
+      }
+    }
+
+    @Override public boolean next() throws IOException {
+      currentIndex++;
+      return (currentIndex <= values.size());
+    }
+
+    @Override public void close() throws IOException {
+    }
+
+    @Override public InternalRow get() {
+      List<Object> list = new ArrayList<>();
+      list.add(WindowedValue.timestampedValueInGlobalWindow(values.get(currentIndex), new Instant()));
+      return InternalRow.apply(asScalaBuffer(list).toList());
+    }
+  }
+}
\ No newline at end of file