KurtYoung commented on a change in pull request #12049: URL: https://github.com/apache/flink/pull/12049#discussion_r422997187
########## File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/sinks/SelectTableSinkSchemaConverter.java ########## @@ -0,0 +1,61 @@ +/* + * 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.sinks; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.TimestampKind; +import org.apache.flink.table.types.logical.TimestampType; + +/** + * An utility class that changes to default conversion class + * and converts time attributes (proc time / event time) to regular timestamps. + */ +class SelectTableSinkSchemaConverter { + + /** + * Change to default conversion class and + * convert time attributes (proc time / event time) to regular timestamps, + * return a new {@link TableSchema}. + */ + static TableSchema convert(TableSchema tableSchema) { Review comment: This `convert` is too ambiguous, you can do whatever you want in this but reader can not have a clue what you really did. I would suggest split the logics into some atomic ones each with clear semantic. BTW, I think we can create an abstract class for both batch and stream select table sink, and put these methods there. ########## File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/sinks/SelectTableSinkSchemaConverter.java ########## @@ -0,0 +1,61 @@ +/* + * 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.sinks; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.TimestampKind; +import org.apache.flink.table.types.logical.TimestampType; + +/** + * An utility class that changes to default conversion class + * and converts time attributes (proc time / event time) to regular timestamps. + */ +class SelectTableSinkSchemaConverter { + + /** + * Change to default conversion class and + * convert time attributes (proc time / event time) to regular timestamps, + * return a new {@link TableSchema}. + */ + static TableSchema convert(TableSchema tableSchema) { + DataType[] oldTypes = tableSchema.getFieldDataTypes(); + String[] oldNames = tableSchema.getFieldNames(); + + TableSchema.Builder builder = TableSchema.builder(); + for (int i = 0; i < tableSchema.getFieldCount(); i++) { + // change to default conversion class + DataType fieldType = LogicalTypeDataTypeConverter.fromLogicalTypeToDataType( Review comment: `LogicalTypeDataTypeConverter` is deprecated ########## File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/sinks/StreamSelectTableSink.java ########## @@ -0,0 +1,94 @@ +/* + * 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.sinks; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.experimental.CollectSink; +import org.apache.flink.streaming.experimental.SocketStreamIterator; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.internal.SelectTableSink; +import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter; +import org.apache.flink.table.sinks.AppendStreamTableSink; +import org.apache.flink.table.types.DataType; +import org.apache.flink.types.Row; + +import java.io.IOException; +import java.net.InetAddress; +import java.util.Iterator; + +/** + * A {@link SelectTableSink} for streaming select job. + * + * <p><strong>NOTES:</strong> This is a temporary solution, + * once FLINK-14807 is finished, the implementation should be changed. + * Currently, only insert changes (AppendStreamTableSink) is supported. + * Once FLINK-16998 is finished, all kinds of changes will be supported. + */ +public class StreamSelectTableSink implements AppendStreamTableSink<Row>, SelectTableSink { + private final TableSchema tableSchema; + private final TypeSerializer<Row> typeSerializer; + private final SocketStreamIterator<Row> iterator; + + @SuppressWarnings("unchecked") + public StreamSelectTableSink(TableSchema tableSchema) { + this.tableSchema = SelectTableSinkSchemaConverter.convert(tableSchema); + this.typeSerializer = (TypeSerializer<Row>) TypeInfoDataTypeConverter Review comment: `TypeInfoDataTypeConverter ` is deprecated ########## File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/sinks/BatchSelectTableSink.java ########## @@ -0,0 +1,112 @@ +/* + * 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.sinks; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.accumulators.SerializedListAccumulator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.Utils; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.internal.SelectTableSink; +import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter; +import org.apache.flink.table.sinks.StreamTableSink; +import org.apache.flink.table.types.DataType; +import org.apache.flink.types.Row; +import org.apache.flink.util.AbstractID; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.ExecutionException; + +/** + * A {@link SelectTableSink} for batch select job. + * + * <p><strong>NOTES:</strong> This is a temporary solution, + * once FLINK-14807 is finished, the implementation should be changed. + */ +public class BatchSelectTableSink implements StreamTableSink<Row>, SelectTableSink { + private final TableSchema tableSchema; + private final String accumulatorName; + private final TypeSerializer<Row> typeSerializer; + private JobClient jobClient; + + @SuppressWarnings("unchecked") + public BatchSelectTableSink(TableSchema tableSchema) { + this.tableSchema = SelectTableSinkSchemaConverter.convert(tableSchema); + this.accumulatorName = new AbstractID().toString(); + this.typeSerializer = (TypeSerializer<Row>) TypeInfoDataTypeConverter Review comment: why use this deprecated `TypeInfoDataTypeConverter`? ---------------------------------------------------------------- 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: us...@infra.apache.org