JingsongLi commented on a change in pull request #13957: URL: https://github.com/apache/flink/pull/13957#discussion_r519119368
########## File path: flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/DeserializationSchemaAdapter.java ########## @@ -0,0 +1,316 @@ +/* + * 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.filesystem; + +import org.apache.flink.api.common.io.DelimitedInputFormat; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.file.src.FileSourceSplit; +import org.apache.flink.connector.file.src.reader.BulkFormat; +import org.apache.flink.connector.file.src.util.ArrayResultIterator; +import org.apache.flink.core.fs.FileInputSplit; +import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.utils.PartitionPathUtils; +import org.apache.flink.util.Collector; +import org.apache.flink.util.InstantiationUtil; +import org.apache.flink.util.UserCodeClassLoader; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.Arrays; +import java.util.List; +import java.util.Queue; +import java.util.stream.Collectors; + +import static org.apache.flink.connector.file.src.util.CheckpointedPosition.NO_OFFSET; +import static org.apache.flink.table.data.vector.VectorizedColumnBatch.DEFAULT_SIZE; + +/** + * Adapter to turn a {@link DeserializationSchema} into a {@link BulkFormat}. + */ +public class DeserializationSchemaAdapter implements BulkFormat<RowData, FileSourceSplit> { + + private static final int BATCH_SIZE = 100; + + // NOTE, deserializationSchema produce full format fields with original order + private final DeserializationSchema<RowData> deserializationSchema; + + private final String[] fieldNames; + private final DataType[] fieldTypes; + private final int[] projectFields; + private final RowType projectedRowType; + + private final List<String> partitionKeys; + private final String defaultPartValue; + + private final int[] toProjectedField; + private final RowData.FieldGetter[] formatFieldGetters; + + public DeserializationSchemaAdapter( + DeserializationSchema<RowData> deserializationSchema, + TableSchema schema, + int[] projectFields, + List<String> partitionKeys, + String defaultPartValue) { + this.deserializationSchema = deserializationSchema; + this.fieldNames = schema.getFieldNames(); + this.fieldTypes = schema.getFieldDataTypes(); + this.projectFields = projectFields; + this.partitionKeys = partitionKeys; + this.defaultPartValue = defaultPartValue; + + List<String> projectedNames = Arrays.stream(projectFields) + .mapToObj(idx -> schema.getFieldNames()[idx]) + .collect(Collectors.toList()); + + this.projectedRowType = RowType.of( + Arrays.stream(projectFields).mapToObj(idx -> + schema.getFieldDataTypes()[idx].getLogicalType()).toArray(LogicalType[]::new), + projectedNames.toArray(new String[0])); + + List<String> formatFields = Arrays.stream(schema.getFieldNames()) + .filter(field -> !partitionKeys.contains(field)) + .collect(Collectors.toList()); + + List<String> formatProjectedFields = projectedNames.stream() + .filter(field -> !partitionKeys.contains(field)) + .collect(Collectors.toList()); + + this.toProjectedField = formatProjectedFields.stream() + .mapToInt(projectedNames::indexOf) + .toArray(); + + this.formatFieldGetters = new RowData.FieldGetter[formatProjectedFields.size()]; + for (int i = 0; i < formatProjectedFields.size(); i++) { + String name = formatProjectedFields.get(i); + this.formatFieldGetters[i] = RowData.createFieldGetter( + schema.getFieldDataType(name).get().getLogicalType(), + formatFields.indexOf(name)); + } + } + + private DeserializationSchema<RowData> createDeserialization() throws IOException { + try { + DeserializationSchema<RowData> deserialization = InstantiationUtil.clone(deserializationSchema); + deserialization.open(new DeserializationSchema.InitializationContext() { + @Override + public MetricGroup getMetricGroup() { + throw new UnsupportedOperationException("MetricGroup is unsupported in BulkFormat."); + } + + @Override + public UserCodeClassLoader getUserCodeClassLoader() { + return (UserCodeClassLoader) Thread.currentThread().getContextClassLoader(); + } + }); + return deserialization; + } catch (Exception e) { + throw new IOException(e); + } + } + + @Override + public Reader createReader(Configuration config, FileSourceSplit split) throws IOException { + return new Reader(config, split); + } + + @Override + public Reader restoreReader(Configuration config, FileSourceSplit split) throws IOException { + Reader reader = new Reader(config, split); + reader.seek(split.getReaderPosition().get().getRecordsAfterOffset()); + return null; Review comment: Streaming restore case: no test cover ########## File path: flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemTableSink.java ########## @@ -251,6 +258,14 @@ private RowDataPartitionComputer partitionComputer() { //noinspection unchecked return Optional.of(FileInputFormatCompactReader.factory((FileInputFormat<RowData>) format)); } + } else if (deserializationFormat != null) { + // NOTE, we need pass full format types to deserializationFormat + DeserializationSchema<RowData> decoder = deserializationFormat.createRuntimeDecoder( + createSourceContext(context), getFormatDataType()); + int[] projectedFields = IntStream.of(0, schema.getFieldCount()).toArray(); Review comment: Streaming sink compaction case: no test cover ---------------------------------------------------------------- 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