You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@wayang.apache.org by be...@apache.org on 2022/04/07 20:57:00 UTC

[incubator-wayang] branch main updated: Issue 134 (#135)

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

bertty pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-wayang.git


The following commit(s) were added to refs/heads/main by this push:
     new a8bfdd47 Issue 134 (#135)
a8bfdd47 is described below

commit a8bfdd47f80e4c352c06116b418fba68fe8ab333
Author: Matthew JS <57...@users.noreply.github.com>
AuthorDate: Thu Apr 7 22:56:56 2022 +0200

    Issue 134 (#135)
    
    * Update README.md
    
    Add notes regarding issue with JAVA_HOME variable and setting up Wayang as well as specifying (current) supported Java version.
    
    * added partial test class for flink
    
    * fixed flinktestbase
---
 .../operators/FlinkCartesianOperatorTest.java      | 59 +++++++++++++
 .../flink/operators/FlinkOperatorTestBase.java     | 96 ++++++++++++++++++++++
 .../apache/wayang/flink/test/ChannelFactory.java   | 72 ++++++++++++++++
 3 files changed, 227 insertions(+)

diff --git a/wayang-platforms/wayang-flink/code/test/java/org/apache/wayang/flink/operators/FlinkCartesianOperatorTest.java b/wayang-platforms/wayang-flink/code/test/java/org/apache/wayang/flink/operators/FlinkCartesianOperatorTest.java
new file mode 100644
index 00000000..73fd91fa
--- /dev/null
+++ b/wayang-platforms/wayang-flink/code/test/java/org/apache/wayang/flink/operators/FlinkCartesianOperatorTest.java
@@ -0,0 +1,59 @@
+/*
+ * 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.wayang.flink.operators;
+
+import org.apache.wayang.basic.data.Tuple2;
+import org.apache.wayang.core.platform.ChannelInstance;
+import org.apache.wayang.core.types.DataSetType;
+import org.apache.wayang.flink.channels.DataSetChannel;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class FlinkCartesianOperatorTest extends FlinkOperatorTestBase {
+
+    @Test
+    public void testExecution() throws Exception {
+        // Prepare test data.
+        DataSetChannel.Instance input0 = this.createDataSetChannelInstance(Arrays.asList(1, 2));
+        DataSetChannel.Instance input1 = this.createDataSetChannelInstance(Arrays.asList("a", "b", "c"));
+        DataSetChannel.Instance output = this.createDataSetChannelInstance();
+
+        // Build the Cartesian operator.
+        FlinkCartesianOperator<Integer, String> cartesianOperator =
+                new FlinkCartesianOperator<>(
+                        DataSetType.createDefaultUnchecked(Integer.class),
+                        DataSetType.createDefaultUnchecked(String.class));
+
+        // Set up the ChannelInstances.
+        final ChannelInstance[] inputs = new ChannelInstance[]{input0, input1};
+        final ChannelInstance[] outputs = new ChannelInstance[]{output};
+
+        // Execute.
+        this.evaluate(cartesianOperator, inputs, outputs);
+
+        // Verify the outcome.
+        final List<Tuple2<Integer, String>> result = output.<Tuple2<Integer, String>>provideDataSet().collect();
+        Assert.assertEquals(6, result.size());
+        Assert.assertEquals(result.get(0), new Tuple2(1, "a"));
+
+    }
+}
diff --git a/wayang-platforms/wayang-flink/code/test/java/org/apache/wayang/flink/operators/FlinkOperatorTestBase.java b/wayang-platforms/wayang-flink/code/test/java/org/apache/wayang/flink/operators/FlinkOperatorTestBase.java
new file mode 100644
index 00000000..e926b0ab
--- /dev/null
+++ b/wayang-platforms/wayang-flink/code/test/java/org/apache/wayang/flink/operators/FlinkOperatorTestBase.java
@@ -0,0 +1,96 @@
+/*
+ * 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.wayang.flink.operators;
+
+import org.junit.Before;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.wayang.core.api.Configuration;
+import org.apache.wayang.core.api.Job;
+import org.apache.wayang.core.optimizer.DefaultOptimizationContext;
+import org.apache.wayang.core.optimizer.OptimizationContext;
+import org.apache.wayang.core.plan.wayangplan.Operator;
+import org.apache.wayang.core.platform.ChannelInstance;
+import org.apache.wayang.core.platform.CrossPlatformExecutor;
+import org.apache.wayang.core.profiling.FullInstrumentationStrategy;
+import org.apache.wayang.flink.channels.DataSetChannel;
+import org.apache.wayang.flink.execution.FlinkExecutor;
+import org.apache.wayang.flink.platform.FlinkPlatform;
+import org.apache.wayang.flink.test.ChannelFactory;
+import org.apache.wayang.java.channels.CollectionChannel;
+import org.apache.wayang.flink.operators.FlinkExecutionOperator;
+
+import java.util.Collection;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class FlinkOperatorTestBase {
+
+    protected Configuration configuration;
+
+    protected FlinkExecutor flinkExecutor;
+
+    @Before
+    public void setUp(){
+        this.configuration = new Configuration();
+        if(flinkExecutor == null)
+            this.flinkExecutor = (FlinkExecutor) FlinkPlatform.getInstance().getExecutorFactory().create(this.mockJob());
+    }
+
+    Job mockJob() {
+        final Job job = mock(Job.class);
+        when(job.getConfiguration()).thenReturn(this.configuration);
+        when(job.getCrossPlatformExecutor()).thenReturn(new CrossPlatformExecutor(job, new FullInstrumentationStrategy()));
+        return job;
+    }
+
+    protected OptimizationContext.OperatorContext createOperatorContext(Operator operator) {
+        OptimizationContext optimizationContext = new DefaultOptimizationContext(mockJob());
+        return optimizationContext.addOneTimeOperator(operator);
+    }
+
+
+    protected void evaluate(FlinkExecutionOperator operator,
+                            ChannelInstance[] inputs,
+                            ChannelInstance[] outputs) throws Exception {
+        operator.evaluate(inputs, outputs, this.flinkExecutor, this.createOperatorContext(operator));
+    }
+
+
+    DataSetChannel.Instance createDataSetChannelInstance() {
+        return ChannelFactory.createDataSetChannelInstance(this.configuration);
+    }
+
+    DataSetChannel.Instance createDataSetChannelInstance(Collection<?> collection) {
+        return ChannelFactory.createDataSetChannelInstance(collection, this.flinkExecutor, this.configuration);
+    }
+
+    protected CollectionChannel.Instance createCollectionChannelInstance() {
+        return ChannelFactory.createCollectionChannelInstance(this.configuration);
+    }
+
+    protected CollectionChannel.Instance createCollectionChannelInstance(Collection<?> collection) {
+        return ChannelFactory.createCollectionChannelInstance(collection, this.configuration);
+    }
+
+    public ExecutionEnvironment getEnv() {
+        return this.flinkExecutor.fee;
+    }
+
+}
diff --git a/wayang-platforms/wayang-flink/code/test/java/org/apache/wayang/flink/test/ChannelFactory.java b/wayang-platforms/wayang-flink/code/test/java/org/apache/wayang/flink/test/ChannelFactory.java
new file mode 100644
index 00000000..5c6da87a
--- /dev/null
+++ b/wayang-platforms/wayang-flink/code/test/java/org/apache/wayang/flink/test/ChannelFactory.java
@@ -0,0 +1,72 @@
+/*
+ * 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.wayang.flink.test;
+
+import org.apache.wayang.core.api.Configuration;
+import org.apache.wayang.core.platform.ChannelDescriptor;
+import org.apache.wayang.core.util.WayangCollections;
+import org.apache.wayang.flink.channels.DataSetChannel;
+import org.apache.wayang.java.channels.CollectionChannel;
+import org.junit.Before;
+import org.apache.wayang.flink.execution.FlinkExecutor;
+
+import java.util.Collection;
+
+import static org.mockito.Mockito.mock;
+
+public class ChannelFactory {
+
+    private static FlinkExecutor flinkExecutor;
+
+    @Before
+    public void setUp() {
+        flinkExecutor = mock(FlinkExecutor.class);
+    }
+
+    public static DataSetChannel.Instance createDataSetChannelInstance(ChannelDescriptor dataSetChannelDescriptor, Configuration configuration){
+        return (DataSetChannel.Instance) dataSetChannelDescriptor
+                .createChannel(null,configuration)
+                .createInstance(flinkExecutor,null,-1);
+    }
+
+    public static DataSetChannel.Instance createDataSetChannelInstance(Configuration configuration){
+        return createDataSetChannelInstance(DataSetChannel.DESCRIPTOR, configuration);
+    }
+
+    public static DataSetChannel.Instance createDataSetChannelInstance(Collection<?> data,
+                                                                       FlinkExecutor flinkExecutor,
+                                                                       Configuration configuration){
+        DataSetChannel.Instance instance = createDataSetChannelInstance(configuration);
+        instance.accept(flinkExecutor.fee.fromCollection(WayangCollections.asList(data)),flinkExecutor);
+        return instance;
+    }
+
+    public static CollectionChannel.Instance createCollectionChannelInstance(Configuration configuration) {
+        return (CollectionChannel.Instance) CollectionChannel.DESCRIPTOR
+                .createChannel(null, configuration)
+                .createInstance(flinkExecutor, null, -1);
+    }
+
+    public static CollectionChannel.Instance createCollectionChannelInstance(Collection<?> collection, Configuration configuration) {
+        CollectionChannel.Instance instance = createCollectionChannelInstance(configuration);
+        instance.accept(collection);
+        return instance;
+    }
+
+}