You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by dw...@apache.org on 2019/08/08 12:34:26 UTC

[flink] branch release-1.9 updated: [FLINK-13630][table-api-bridge] Use values retention configuration from TableConfig if not specified explicitly

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

dwysakowicz pushed a commit to branch release-1.9
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/release-1.9 by this push:
     new 448c459  [FLINK-13630][table-api-bridge] Use values retention configuration from TableConfig if not specified explicitly
448c459 is described below

commit 448c459609ea0054e691be29ac6332af5ab4e289
Author: Dawid Wysakowicz <dw...@apache.org>
AuthorDate: Wed Aug 7 12:15:07 2019 +0200

    [FLINK-13630][table-api-bridge] Use values retention configuration from TableConfig if not specified explicitly
    
    If methods that do not specify QueryConfig explicitly are used in
    StreamTableEnvironment than the values from TableConfig will be used.
    If methods that accept QueryConfig are used than those values will be
    used to overwrite the TableConfig.
---
 .../java/internal/StreamTableEnvironmentImpl.java  |  42 +++---
 .../internal/StreamTableEnvironmentImplTest.java   | 145 +++++++++++++++++++++
 .../internal/StreamTableEnvironmentImpl.scala      |  31 +++--
 .../internal/StreamTableEnvironmentImplTest.scala  | 116 +++++++++++++++++
 4 files changed, 300 insertions(+), 34 deletions(-)

diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImpl.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImpl.java
index 8987940..cf2ace7 100644
--- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImpl.java
+++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImpl.java
@@ -214,12 +214,17 @@ public final class StreamTableEnvironmentImpl extends TableEnvironmentImpl imple
 
 	@Override
 	public <T> DataStream<T> toAppendStream(Table table, Class<T> clazz) {
-		return toAppendStream(table, clazz, new StreamQueryConfig());
+		TypeInformation<T> typeInfo = extractTypeInformation(table, clazz);
+		return toAppendStream(table, typeInfo);
 	}
 
 	@Override
 	public <T> DataStream<T> toAppendStream(Table table, TypeInformation<T> typeInfo) {
-		return toAppendStream(table, typeInfo, new StreamQueryConfig());
+		OutputConversionModifyOperation modifyOperation = new OutputConversionModifyOperation(
+			table.getQueryOperation(),
+			TypeConversions.fromLegacyInfoToDataType(typeInfo),
+			OutputConversionModifyOperation.UpdateMode.APPEND);
+		return toDataStream(table, modifyOperation);
 	}
 
 	@Override
@@ -227,8 +232,10 @@ public final class StreamTableEnvironmentImpl extends TableEnvironmentImpl imple
 			Table table,
 			Class<T> clazz,
 			StreamQueryConfig queryConfig) {
-		TypeInformation<T> typeInfo = extractTypeInformation(table, clazz);
-		return toAppendStream(table, typeInfo, queryConfig);
+		tableConfig.setIdleStateRetentionTime(
+			Time.milliseconds(queryConfig.getMinIdleStateRetentionTime()),
+			Time.milliseconds(queryConfig.getMaxIdleStateRetentionTime()));
+		return toAppendStream(table, clazz);
 	}
 
 	@Override
@@ -236,24 +243,25 @@ public final class StreamTableEnvironmentImpl extends TableEnvironmentImpl imple
 			Table table,
 			TypeInformation<T> typeInfo,
 			StreamQueryConfig queryConfig) {
-		OutputConversionModifyOperation modifyOperation = new OutputConversionModifyOperation(
-			table.getQueryOperation(),
-			TypeConversions.fromLegacyInfoToDataType(typeInfo),
-			OutputConversionModifyOperation.UpdateMode.APPEND);
 		tableConfig.setIdleStateRetentionTime(
 			Time.milliseconds(queryConfig.getMinIdleStateRetentionTime()),
 			Time.milliseconds(queryConfig.getMaxIdleStateRetentionTime()));
-		return toDataStream(table, modifyOperation);
+		return toAppendStream(table, typeInfo);
 	}
 
 	@Override
 	public <T> DataStream<Tuple2<Boolean, T>> toRetractStream(Table table, Class<T> clazz) {
-		return toRetractStream(table, clazz, new StreamQueryConfig());
+		TypeInformation<T> typeInfo = extractTypeInformation(table, clazz);
+		return toRetractStream(table, typeInfo);
 	}
 
 	@Override
 	public <T> DataStream<Tuple2<Boolean, T>> toRetractStream(Table table, TypeInformation<T> typeInfo) {
-		return toRetractStream(table, typeInfo, new StreamQueryConfig());
+		OutputConversionModifyOperation modifyOperation = new OutputConversionModifyOperation(
+			table.getQueryOperation(),
+			wrapWithChangeFlag(typeInfo),
+			OutputConversionModifyOperation.UpdateMode.RETRACT);
+		return toDataStream(table, modifyOperation);
 	}
 
 	@Override
@@ -261,8 +269,10 @@ public final class StreamTableEnvironmentImpl extends TableEnvironmentImpl imple
 			Table table,
 			Class<T> clazz,
 			StreamQueryConfig queryConfig) {
-		TypeInformation<T> typeInfo = extractTypeInformation(table, clazz);
-		return toRetractStream(table, typeInfo, queryConfig);
+		tableConfig.setIdleStateRetentionTime(
+			Time.milliseconds(queryConfig.getMinIdleStateRetentionTime()),
+			Time.milliseconds(queryConfig.getMaxIdleStateRetentionTime()));
+		return toRetractStream(table, clazz);
 	}
 
 	@Override
@@ -270,14 +280,10 @@ public final class StreamTableEnvironmentImpl extends TableEnvironmentImpl imple
 			Table table,
 			TypeInformation<T> typeInfo,
 			StreamQueryConfig queryConfig) {
-		OutputConversionModifyOperation modifyOperation = new OutputConversionModifyOperation(
-			table.getQueryOperation(),
-			wrapWithChangeFlag(typeInfo),
-			OutputConversionModifyOperation.UpdateMode.RETRACT);
 		tableConfig.setIdleStateRetentionTime(
 			Time.milliseconds(queryConfig.getMinIdleStateRetentionTime()),
 			Time.milliseconds(queryConfig.getMaxIdleStateRetentionTime()));
-		return toDataStream(table, modifyOperation);
+		return toRetractStream(table, typeInfo);
 	}
 
 	@Override
diff --git a/flink-table/flink-table-api-java-bridge/src/test/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImplTest.java b/flink-table/flink-table-api-java-bridge/src/test/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImplTest.java
new file mode 100644
index 0000000..1348ec2
--- /dev/null
+++ b/flink-table/flink-table-api-java-bridge/src/test/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImplTest.java
@@ -0,0 +1,145 @@
+/*
+ * 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.flink.table.api.java.internal;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.catalog.CatalogManager;
+import org.apache.flink.table.catalog.FunctionCatalog;
+import org.apache.flink.table.catalog.GenericInMemoryCatalog;
+import org.apache.flink.table.delegation.Executor;
+import org.apache.flink.table.delegation.Planner;
+import org.apache.flink.table.operations.ModifyOperation;
+import org.apache.flink.table.operations.Operation;
+import org.apache.flink.types.Row;
+
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.List;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for {@link StreamTableEnvironmentImpl}.
+ */
+public class StreamTableEnvironmentImplTest {
+	@Test
+	public void testAppendStreamDoesNotOverwriteTableConfig() {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		DataStreamSource<Integer> elements = env.fromElements(1, 2, 3);
+
+		StreamTableEnvironmentImpl tEnv = getStreamTableEnvironment(env, elements);
+
+		Time minRetention = Time.minutes(1);
+		Time maxRetention = Time.minutes(10);
+		tEnv.getConfig().setIdleStateRetentionTime(minRetention, maxRetention);
+		Table table = tEnv.fromDataStream(elements);
+		tEnv.toAppendStream(table, Row.class);
+
+		assertThat(
+			tEnv.getConfig().getMinIdleStateRetentionTime(),
+			equalTo(minRetention.toMilliseconds()));
+		assertThat(
+			tEnv.getConfig().getMaxIdleStateRetentionTime(),
+			equalTo(maxRetention.toMilliseconds()));
+	}
+
+	@Test
+	public void testRetractStreamDoesNotOverwriteTableConfig() {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		DataStreamSource<Integer> elements = env.fromElements(1, 2, 3);
+
+		StreamTableEnvironmentImpl tEnv = getStreamTableEnvironment(env, elements);
+
+		Time minRetention = Time.minutes(1);
+		Time maxRetention = Time.minutes(10);
+		tEnv.getConfig().setIdleStateRetentionTime(minRetention, maxRetention);
+		Table table = tEnv.fromDataStream(elements);
+		tEnv.toRetractStream(table, Row.class);
+
+		assertThat(
+			tEnv.getConfig().getMinIdleStateRetentionTime(),
+			equalTo(minRetention.toMilliseconds()));
+		assertThat(
+			tEnv.getConfig().getMaxIdleStateRetentionTime(),
+			equalTo(maxRetention.toMilliseconds()));
+	}
+
+	private StreamTableEnvironmentImpl getStreamTableEnvironment(
+			StreamExecutionEnvironment env,
+			DataStreamSource<Integer> elements) {
+		CatalogManager catalogManager = new CatalogManager("cat", new GenericInMemoryCatalog("cat", "db"));
+		return new StreamTableEnvironmentImpl(
+			catalogManager,
+			new FunctionCatalog(catalogManager),
+			new TableConfig(),
+			env,
+			new TestPlanner(elements.getTransformation()),
+			executor,
+			true
+		);
+	}
+
+	private static class TestPlanner implements Planner {
+		private final Transformation<?> transformation;
+
+		private TestPlanner(Transformation<?> transformation) {
+			this.transformation = transformation;
+		}
+
+		@Override
+		public List<Operation> parse(String statement) {
+			throw new AssertionError("Should not be called");
+		}
+
+		@Override
+		public List<Transformation<?>> translate(List<ModifyOperation> modifyOperations) {
+			return Collections.singletonList(transformation);
+		}
+
+		@Override
+		public String explain(List<Operation> operations, boolean extended) {
+			throw new AssertionError("Should not be called");
+		}
+
+		@Override
+		public String[] getCompletionHints(String statement, int position) {
+			throw new AssertionError("Should not be called");
+		}
+	}
+
+	private final Executor executor = new Executor() {
+		@Override
+		public void apply(List<Transformation<?>> transformations) {
+
+		}
+
+		@Override
+		public JobExecutionResult execute(String jobName) throws Exception {
+			throw new AssertionError("Should not be called");
+		}
+	};
+}
diff --git a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImpl.scala b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImpl.scala
index 67021cb..c45d324 100644
--- a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImpl.scala
+++ b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImpl.scala
@@ -95,44 +95,43 @@ class StreamTableEnvironmentImpl (
   }
 
   override def toAppendStream[T: TypeInformation](table: Table): DataStream[T] = {
-    toAppendStream(table, new StreamQueryConfig)
-  }
-
-  override def toAppendStream[T: TypeInformation](
-      table: Table,
-      queryConfig: StreamQueryConfig)
-    : DataStream[T] = {
     val returnType = createTypeInformation[T]
 
     val modifyOperation = new OutputConversionModifyOperation(
       table.getQueryOperation,
       TypeConversions.fromLegacyInfoToDataType(returnType),
       OutputConversionModifyOperation.UpdateMode.APPEND)
+    toDataStream[T](table, modifyOperation)
+  }
+
+  override def toAppendStream[T: TypeInformation](
+      table: Table,
+      queryConfig: StreamQueryConfig)
+    : DataStream[T] = {
     tableConfig.setIdleStateRetentionTime(
       Time.milliseconds(queryConfig.getMinIdleStateRetentionTime),
       Time.milliseconds(queryConfig.getMaxIdleStateRetentionTime))
-    toDataStream(table, modifyOperation)
+    toAppendStream(table)
   }
 
   override def toRetractStream[T: TypeInformation](table: Table): DataStream[(Boolean, T)] = {
-    toRetractStream(table, new StreamQueryConfig)
-  }
-
-  override def toRetractStream[T: TypeInformation](
-      table: Table,
-      queryConfig: StreamQueryConfig)
-    : DataStream[(Boolean, T)] = {
     val returnType = createTypeInformation[(Boolean, T)]
 
     val modifyOperation = new OutputConversionModifyOperation(
       table.getQueryOperation,
       TypeConversions.fromLegacyInfoToDataType(returnType),
       OutputConversionModifyOperation.UpdateMode.RETRACT)
+    toDataStream(table, modifyOperation)
+  }
 
+  override def toRetractStream[T: TypeInformation](
+      table: Table,
+      queryConfig: StreamQueryConfig)
+    : DataStream[(Boolean, T)] = {
     tableConfig.setIdleStateRetentionTime(
         Time.milliseconds(queryConfig.getMinIdleStateRetentionTime),
         Time.milliseconds(queryConfig.getMaxIdleStateRetentionTime))
-    toDataStream(table, modifyOperation)
+    toRetractStream(table)
   }
 
   override def registerFunction[T: TypeInformation](name: String, tf: TableFunction[T]): Unit = {
diff --git a/flink-table/flink-table-api-scala-bridge/src/test/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImplTest.scala b/flink-table/flink-table-api-scala-bridge/src/test/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImplTest.scala
new file mode 100644
index 0000000..91b37d0
--- /dev/null
+++ b/flink-table/flink-table-api-scala-bridge/src/test/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImplTest.scala
@@ -0,0 +1,116 @@
+/*
+ * 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.flink.table.api.scala.internal
+
+import org.apache.flink.api.common.time.Time
+import org.apache.flink.api.dag.Transformation
+import org.apache.flink.streaming.api.scala.{StreamExecutionEnvironment, _}
+import org.apache.flink.table.api.TableConfig
+import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, GenericInMemoryCatalog}
+import org.apache.flink.table.delegation.{Executor, Planner}
+import org.apache.flink.table.operations.{ModifyOperation, Operation}
+import org.apache.flink.types.Row
+
+import org.hamcrest.CoreMatchers.equalTo
+import org.junit.Assert.assertThat
+import org.junit.Test
+
+import java.util.{Collections, List => JList}
+
+/**
+ * Tests for [[StreamTableEnvironmentImpl]].
+ */
+class StreamTableEnvironmentImplTest {
+  @Test
+  def testAppendStreamDoesNotOverwriteTableConfig(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val elements = env.fromElements(1, 2, 3)
+    val tEnv: StreamTableEnvironmentImpl = getStreamTableEnvironment(env, elements)
+
+    val minRetention = Time.minutes(1)
+    val maxRetention = Time.minutes(10)
+    tEnv.getConfig.setIdleStateRetentionTime(minRetention, maxRetention)
+    val table = tEnv.fromDataStream(elements)
+    tEnv.toAppendStream[Row](table)
+
+    assertThat(
+      tEnv.getConfig.getMinIdleStateRetentionTime,
+      equalTo(minRetention.toMilliseconds))
+    assertThat(
+      tEnv.getConfig.getMaxIdleStateRetentionTime,
+      equalTo(maxRetention.toMilliseconds))
+  }
+
+  @Test
+  def testRetractStreamDoesNotOverwriteTableConfig(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val elements = env.fromElements(1, 2, 3)
+    val tEnv: StreamTableEnvironmentImpl = getStreamTableEnvironment(env, elements)
+
+    val minRetention = Time.minutes(1)
+    val maxRetention = Time.minutes(10)
+    tEnv.getConfig.setIdleStateRetentionTime(minRetention, maxRetention)
+    val table = tEnv.fromDataStream(elements)
+    tEnv.toRetractStream[Row](table)
+
+    assertThat(
+      tEnv.getConfig.getMinIdleStateRetentionTime,
+      equalTo(minRetention.toMilliseconds))
+    assertThat(
+      tEnv.getConfig.getMaxIdleStateRetentionTime,
+      equalTo(maxRetention.toMilliseconds))
+  }
+
+  private def getStreamTableEnvironment(
+      env: StreamExecutionEnvironment,
+      elements: DataStream[Int]) = {
+    val catalogManager = new CatalogManager(
+      "cat",
+      new GenericInMemoryCatalog("cat", "db"))
+    new StreamTableEnvironmentImpl(
+      catalogManager,
+      new FunctionCatalog(catalogManager),
+      new TableConfig,
+      env,
+      new TestPlanner(elements.javaStream.getTransformation),
+      executor,
+      true)
+  }
+
+  private class TestPlanner(transformation: Transformation[_]) extends Planner {
+    override def parse(statement: String) = throw new AssertionError("Should not be called")
+
+    override def translate(modifyOperations: JList[ModifyOperation])
+      : JList[Transformation[_]] = {
+      Collections.singletonList(transformation)
+    }
+
+    override def explain(operations: JList[Operation], extended: Boolean) =
+      throw new AssertionError("Should not be called")
+
+    override def getCompletionHints(statement: String, position: Int) =
+      throw new AssertionError("Should not be called")
+  }
+
+  private val executor = new Executor() {
+    override def apply(transformations: JList[Transformation[_]]): Unit = {}
+
+    override def execute(jobName: String) = throw new AssertionError("Should not be called")
+  }
+}