You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/05/07 09:43:57 UTC

[GitHub] [flink] twalthr commented on a change in pull request #11981: [FLINK-17002] Support CREATE TABLE ... LIKE clause

twalthr commented on a change in pull request #11981:
URL: https://github.com/apache/flink/pull/11981#discussion_r421372670



##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/TableSchema.java
##########
@@ -550,6 +550,16 @@ public Builder field(String name, DataType dataType, String expression) {
 			return this;
 		}
 
+		/**
+		 * Adds a {@link TableColumn} to this builder.
+		 *
+		 * <p>The call order of this method determines the order of fields in the schema.
+		 */
+		public Builder field(TableColumn column) {

Review comment:
       A field that takes a column looks weird to me.

##########
File path: flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/operations/MergeTableLikeUtilTest.java
##########
@@ -0,0 +1,773 @@
+/*
+ * 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.planner.operations;
+
+import org.apache.flink.sql.parser.ddl.SqlTableColumn;
+import org.apache.flink.sql.parser.ddl.SqlTableLike.FeatureOption;
+import org.apache.flink.sql.parser.ddl.SqlTableLike.MergingStrategy;
+import org.apache.flink.sql.parser.ddl.SqlTableLike.SqlTableLikeOption;
+import org.apache.flink.sql.parser.ddl.SqlWatermark;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogManager;
+import org.apache.flink.table.catalog.FunctionCatalog;
+import org.apache.flink.table.module.ModuleManager;
+import org.apache.flink.table.planner.calcite.FlinkCalciteSqlValidator;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.calcite.FlinkTypeSystem;
+import org.apache.flink.table.planner.catalog.CatalogManagerCalciteSchema;
+import org.apache.flink.table.planner.delegation.PlannerContext;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.utils.CatalogManagerMocks;
+
+import org.apache.calcite.avatica.util.TimeUnit;
+import org.apache.calcite.sql.SqlAsOperator;
+import org.apache.calcite.sql.SqlBasicCall;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.calcite.jdbc.CalciteSchemaBuilder.asRootSchema;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for {@link MergeTableLikeUtil}.
+ */
+public class MergeTableLikeUtilTest {
+
+	@Rule
+	public ExpectedException thrown = ExpectedException.none();
+
+	private final FlinkTypeFactory typeFactory = new FlinkTypeFactory(new FlinkTypeSystem());

Review comment:
       Can we move all these 6 instances into one `PlannerMocks.createEmptyPlanner()` next to `CatalogMocks`?

##########
File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/SqlToOperationConverter.java
##########
@@ -244,6 +224,117 @@ private Operation convertCreateTable(SqlCreateTable sqlCreateTable) {
 			sqlCreateTable.isTemporary());
 	}
 
+	private CatalogTable createCatalogTable(SqlCreateTable sqlCreateTable) {

Review comment:
       Can we stop letting this file grow infinitely and somehow move those static methods into separate classes?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org