JingsongLi commented on a change in pull request #13915: URL: https://github.com/apache/flink/pull/13915#discussion_r517083047
########## File path: flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSource.java ########## @@ -0,0 +1,168 @@ +/* + * 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.connectors.hive; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.connector.file.src.AbstractFileSource; +import org.apache.flink.connector.file.src.reader.BulkFormat; +import org.apache.flink.connectors.hive.read.HiveBulkFormatAdapter; +import org.apache.flink.connectors.hive.read.HiveSourceSplit; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter; +import org.apache.flink.table.types.DataType; +import org.apache.flink.util.Preconditions; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.mapred.FileSplit; +import org.apache.hadoop.mapred.InputFormat; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.util.ReflectionUtils; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.IntStream; + +import static org.apache.flink.connector.file.src.FileSource.DEFAULT_SPLIT_ASSIGNER; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.hadoop.mapreduce.lib.input.FileInputFormat.INPUT_DIR; + +/** + * A unified data source that reads a hive table. + */ +public class HiveSource extends AbstractFileSource<RowData, HiveSourceSplit> implements ResultTypeQueryable<RowData> { + + private static final long serialVersionUID = 1L; + + // DataType of the records to be returned, with projection applied (if any) + private final DataType producedDataType; + + HiveSource( + JobConf jobConf, + CatalogTable catalogTable, + List<HiveTablePartition> partitions, + int[] projectedFields, + long limit, Review comment: @Nullable Long limit ########## File path: flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSource.java ########## @@ -0,0 +1,168 @@ +/* + * 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.connectors.hive; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.connector.file.src.AbstractFileSource; +import org.apache.flink.connector.file.src.reader.BulkFormat; +import org.apache.flink.connectors.hive.read.HiveBulkFormatAdapter; +import org.apache.flink.connectors.hive.read.HiveSourceSplit; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter; +import org.apache.flink.table.types.DataType; +import org.apache.flink.util.Preconditions; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.mapred.FileSplit; +import org.apache.hadoop.mapred.InputFormat; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.util.ReflectionUtils; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.IntStream; + +import static org.apache.flink.connector.file.src.FileSource.DEFAULT_SPLIT_ASSIGNER; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.hadoop.mapreduce.lib.input.FileInputFormat.INPUT_DIR; + +/** + * A unified data source that reads a hive table. + */ +public class HiveSource extends AbstractFileSource<RowData, HiveSourceSplit> implements ResultTypeQueryable<RowData> { + + private static final long serialVersionUID = 1L; + + // DataType of the records to be returned, with projection applied (if any) + private final DataType producedDataType; + + HiveSource( + JobConf jobConf, + CatalogTable catalogTable, + List<HiveTablePartition> partitions, + int[] projectedFields, + long limit, + String hiveVersion, + boolean useMapRedReader, + boolean isStreamingSource, + DataType producedDataType) { Review comment: A `RowType` is enough ########## File path: flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceSplitSerializer.java ########## @@ -0,0 +1,100 @@ +/* + * 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.connectors.hive; + +import org.apache.flink.connector.file.src.FileSourceSplit; +import org.apache.flink.connector.file.src.FileSourceSplitSerializer; +import org.apache.flink.connectors.hive.read.HiveSourceSplit; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * SerDe for {@link HiveSourceSplit}. + */ +public class HiveSourceSplitSerializer implements SimpleVersionedSerializer<HiveSourceSplit> { + + private static final int VERSION = 1; + + public static final HiveSourceSplitSerializer INSTANCE = new HiveSourceSplitSerializer(); + + private HiveSourceSplitSerializer() { + } + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(HiveSourceSplit split) throws IOException { + checkArgument(split.getClass() == HiveSourceSplit.class, "Cannot serialize subclasses of HiveSourceSplit"); + + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + try (ObjectOutputStream outputStream = new ObjectOutputStream(byteArrayOutputStream)) { + serialize(outputStream, split); + } + return byteArrayOutputStream.toByteArray(); + } + + @Override + public HiveSourceSplit deserialize(int version, byte[] serialized) throws IOException { + if (version == 1) { + try (ObjectInputStream inputStream = new ObjectInputStream(new ByteArrayInputStream(serialized))) { + return deserializeV1(inputStream); + } + } else { + throw new IOException("Unknown version: " + version); + } + } + + private void serialize(ObjectOutputStream outputStream, HiveSourceSplit split) throws IOException { + byte[] superBytes = FileSourceSplitSerializer.INSTANCE.serialize(new FileSourceSplit( Review comment: Why do you want to create a new `FileSourceSplit`? Why not just use `HiveSourceSplit` ########## File path: flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveBulkFormatAdapter.java ########## @@ -0,0 +1,291 @@ +/* + * 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.connectors.hive.read; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.file.src.reader.BulkFormat; +import org.apache.flink.connector.file.src.util.ArrayResultIterator; +import org.apache.flink.connectors.hive.HiveTablePartition; +import org.apache.flink.connectors.hive.JobConfWrapper; +import org.apache.flink.formats.parquet.ParquetColumnarRowInputFormat; +import org.apache.flink.table.catalog.hive.client.HiveShim; +import org.apache.flink.table.catalog.hive.client.HiveShimLoader; +import org.apache.flink.table.catalog.hive.util.HiveTypeUtil; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.filesystem.PartitionValueConverter; +import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +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.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.io.IOConstants; +import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; +import org.apache.hadoop.mapred.JobConf; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +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; + +/** + * A BulkFormat implementation for HiveSource. This implementation delegates reading to other BulkFormat instances, + * because different hive partitions may need different BulkFormat to do the reading. + */ +public class HiveBulkFormatAdapter implements BulkFormat<RowData, HiveSourceSplit> { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(HiveBulkFormatAdapter.class); + + // schema evolution configs are not available in older versions of IOConstants, let's define them ourselves + private static final String SCHEMA_EVOLUTION_COLUMNS = "schema.evolution.columns"; + private static final String SCHEMA_EVOLUTION_COLUMNS_TYPES = "schema.evolution.columns.types"; + + private final JobConfWrapper jobConfWrapper; + private final List<String> partitionKeys; + private final String[] fieldNames; + private final DataType[] fieldTypes; + // indices of fields to be returned, with projection applied (if any) + private final int[] selectedFields; + private final String hiveVersion; + private final HiveShim hiveShim; + private final DataType producedDataType; + private final boolean useMapRedReader; + // We should limit the input read count of the splits, -1 represents no limit. + private final long limit; + + public HiveBulkFormatAdapter(JobConfWrapper jobConfWrapper, List<String> partitionKeys, String[] fieldNames, DataType[] fieldTypes, + int[] selectedFields, String hiveVersion, DataType producedDataType, boolean useMapRedReader, long limit) { + this.jobConfWrapper = jobConfWrapper; + this.partitionKeys = partitionKeys; + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + this.selectedFields = selectedFields; + this.hiveVersion = hiveVersion; + this.hiveShim = HiveShimLoader.loadHiveShim(hiveVersion); + this.producedDataType = producedDataType; + this.useMapRedReader = useMapRedReader; + this.limit = limit; + } + + @Override + public Reader<RowData> createReader(Configuration config, HiveSourceSplit split) + throws IOException { + return createBulkFormatForSplit(split).createReader(config, split); + } + + @Override + public Reader<RowData> restoreReader(Configuration config, HiveSourceSplit split) throws IOException { + return createBulkFormatForSplit(split).restoreReader(config, split); + } + + @Override + public boolean isSplittable() { + return true; + } + + @Override + public TypeInformation<RowData> getProducedType() { + return (TypeInformation<RowData>) TypeInfoDataTypeConverter.fromDataTypeToTypeInfo(producedDataType); Review comment: Ditto ########## File path: flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSource.java ########## @@ -0,0 +1,168 @@ +/* + * 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.connectors.hive; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.connector.file.src.AbstractFileSource; +import org.apache.flink.connector.file.src.reader.BulkFormat; +import org.apache.flink.connectors.hive.read.HiveBulkFormatAdapter; +import org.apache.flink.connectors.hive.read.HiveSourceSplit; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter; +import org.apache.flink.table.types.DataType; +import org.apache.flink.util.Preconditions; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.mapred.FileSplit; +import org.apache.hadoop.mapred.InputFormat; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.util.ReflectionUtils; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.IntStream; + +import static org.apache.flink.connector.file.src.FileSource.DEFAULT_SPLIT_ASSIGNER; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.hadoop.mapreduce.lib.input.FileInputFormat.INPUT_DIR; + +/** + * A unified data source that reads a hive table. + */ +public class HiveSource extends AbstractFileSource<RowData, HiveSourceSplit> implements ResultTypeQueryable<RowData> { + + private static final long serialVersionUID = 1L; + + // DataType of the records to be returned, with projection applied (if any) + private final DataType producedDataType; + + HiveSource( + JobConf jobConf, + CatalogTable catalogTable, + List<HiveTablePartition> partitions, + int[] projectedFields, + long limit, + String hiveVersion, + boolean useMapRedReader, + boolean isStreamingSource, + DataType producedDataType) { + super( + new org.apache.flink.core.fs.Path[1], + new HiveSourceFileEnumerator.Provider(partitions, new JobConfWrapper(jobConf)), + DEFAULT_SPLIT_ASSIGNER, + createBulkFormat(new JobConf(jobConf), catalogTable, projectedFields, hiveVersion, producedDataType, useMapRedReader, limit), + null); + Preconditions.checkArgument(!isStreamingSource, "HiveSource currently only supports bounded mode"); + this.producedDataType = producedDataType; + } + + @Override + public SimpleVersionedSerializer<HiveSourceSplit> getSplitSerializer() { + return HiveSourceSplitSerializer.INSTANCE; + } + + @Override + public TypeInformation<RowData> getProducedType() { + return (TypeInformation<RowData>) TypeInfoDataTypeConverter.fromDataTypeToTypeInfo(producedDataType); Review comment: `InternalTypeInfo.of(rowType)` BTW, why override this? ########## File path: flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveSourceSplit.java ########## @@ -0,0 +1,95 @@ +/* + * 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.connectors.hive.read; + +import org.apache.flink.connector.file.src.FileSourceSplit; +import org.apache.flink.connector.file.src.util.CheckpointedPosition; +import org.apache.flink.connectors.hive.FlinkHiveException; +import org.apache.flink.connectors.hive.HiveTablePartition; +import org.apache.flink.core.fs.Path; + +import org.apache.hadoop.mapred.FileSplit; + +import javax.annotation.Nullable; + +import java.io.IOException; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A wrapper class that wraps info needed for a file input split. + * Right now, it contains info about the partition of the split. + */ +public class HiveSourceSplit extends FileSourceSplit { + + private static final long serialVersionUID = 1L; + + protected final HiveTablePartition hiveTablePartition; + + public HiveSourceSplit( + FileSplit fileSplit, + HiveTablePartition hiveTablePartition, + @Nullable CheckpointedPosition readerPosition) throws IOException { + this( + fileSplit.toString(), + new Path(fileSplit.getPath().toString()), + fileSplit.getStart(), + fileSplit.getLength(), + fileSplit.getLocations(), + readerPosition, + hiveTablePartition + ); + } + + public HiveSourceSplit( + String id, + Path filePath, + long offset, + long length, + String[] hostnames, + @Nullable CheckpointedPosition readerPosition, + HiveTablePartition hiveTablePartition) { + super(id, filePath, offset, length, hostnames, readerPosition); + this.hiveTablePartition = checkNotNull(hiveTablePartition, "hiveTablePartition can not be null"); + } + + public HiveTablePartition getHiveTablePartition() { + return hiveTablePartition; + } + + public FileSplit toMapRedSplit() { + return new FileSplit(new org.apache.hadoop.fs.Path(path().toString()), offset(), length(), hostnames()); + } + + @Override + public FileSourceSplit updateWithCheckpointedPosition(@Nullable CheckpointedPosition position) { + try { + return new HiveSourceSplit(toMapRedSplit(), hiveTablePartition, position); Review comment: Why to `toMapRedSplit`, call another constructor? ########## File path: flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceSplitSerializer.java ########## @@ -0,0 +1,100 @@ +/* + * 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.connectors.hive; + +import org.apache.flink.connector.file.src.FileSourceSplit; +import org.apache.flink.connector.file.src.FileSourceSplitSerializer; +import org.apache.flink.connectors.hive.read.HiveSourceSplit; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * SerDe for {@link HiveSourceSplit}. + */ +public class HiveSourceSplitSerializer implements SimpleVersionedSerializer<HiveSourceSplit> { + + private static final int VERSION = 1; + + public static final HiveSourceSplitSerializer INSTANCE = new HiveSourceSplitSerializer(); + + private HiveSourceSplitSerializer() { + } + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(HiveSourceSplit split) throws IOException { + checkArgument(split.getClass() == HiveSourceSplit.class, "Cannot serialize subclasses of HiveSourceSplit"); + + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + try (ObjectOutputStream outputStream = new ObjectOutputStream(byteArrayOutputStream)) { Review comment: Maybe use `InstantiationUtil.serializeObject` directly? ########## File path: flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSource.java ########## @@ -0,0 +1,168 @@ +/* + * 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.connectors.hive; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.connector.file.src.AbstractFileSource; +import org.apache.flink.connector.file.src.reader.BulkFormat; +import org.apache.flink.connectors.hive.read.HiveBulkFormatAdapter; +import org.apache.flink.connectors.hive.read.HiveSourceSplit; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter; +import org.apache.flink.table.types.DataType; +import org.apache.flink.util.Preconditions; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.mapred.FileSplit; +import org.apache.hadoop.mapred.InputFormat; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.util.ReflectionUtils; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.IntStream; + +import static org.apache.flink.connector.file.src.FileSource.DEFAULT_SPLIT_ASSIGNER; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.hadoop.mapreduce.lib.input.FileInputFormat.INPUT_DIR; + +/** + * A unified data source that reads a hive table. + */ +public class HiveSource extends AbstractFileSource<RowData, HiveSourceSplit> implements ResultTypeQueryable<RowData> { + + private static final long serialVersionUID = 1L; + + // DataType of the records to be returned, with projection applied (if any) + private final DataType producedDataType; + + HiveSource( + JobConf jobConf, + CatalogTable catalogTable, + List<HiveTablePartition> partitions, + int[] projectedFields, + long limit, + String hiveVersion, + boolean useMapRedReader, + boolean isStreamingSource, + DataType producedDataType) { + super( + new org.apache.flink.core.fs.Path[1], + new HiveSourceFileEnumerator.Provider(partitions, new JobConfWrapper(jobConf)), + DEFAULT_SPLIT_ASSIGNER, + createBulkFormat(new JobConf(jobConf), catalogTable, projectedFields, hiveVersion, producedDataType, useMapRedReader, limit), + null); + Preconditions.checkArgument(!isStreamingSource, "HiveSource currently only supports bounded mode"); + this.producedDataType = producedDataType; + } + + @Override + public SimpleVersionedSerializer<HiveSourceSplit> getSplitSerializer() { + return HiveSourceSplitSerializer.INSTANCE; + } + + @Override + public TypeInformation<RowData> getProducedType() { + return (TypeInformation<RowData>) TypeInfoDataTypeConverter.fromDataTypeToTypeInfo(producedDataType); + } + + private static BulkFormat<RowData, HiveSourceSplit> createBulkFormat( + JobConf jobConf, + CatalogTable catalogTable, + @Nullable int[] projectedFields, + String hiveVersion, + DataType producedDataType, + boolean useMapRedReader, + long limit) { + checkNotNull(catalogTable, "catalogTable can not be null."); + return new HiveBulkFormatAdapter( + new JobConfWrapper(jobConf), + catalogTable.getPartitionKeys(), + catalogTable.getSchema().getFieldNames(), + catalogTable.getSchema().getFieldDataTypes(), + projectedFields != null ? projectedFields : IntStream.range(0, catalogTable.getSchema().getFieldCount()).toArray(), + hiveVersion, + producedDataType, + useMapRedReader, + limit); + } + + public static List<HiveSourceSplit> createInputSplits( Review comment: Why in here? Should in `HiveSourceFileEnumerator` ########## File path: flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSource.java ########## @@ -0,0 +1,168 @@ +/* + * 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.connectors.hive; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.connector.file.src.AbstractFileSource; +import org.apache.flink.connector.file.src.reader.BulkFormat; +import org.apache.flink.connectors.hive.read.HiveBulkFormatAdapter; +import org.apache.flink.connectors.hive.read.HiveSourceSplit; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter; +import org.apache.flink.table.types.DataType; +import org.apache.flink.util.Preconditions; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.mapred.FileSplit; +import org.apache.hadoop.mapred.InputFormat; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.util.ReflectionUtils; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.IntStream; + +import static org.apache.flink.connector.file.src.FileSource.DEFAULT_SPLIT_ASSIGNER; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.hadoop.mapreduce.lib.input.FileInputFormat.INPUT_DIR; + +/** + * A unified data source that reads a hive table. + */ +public class HiveSource extends AbstractFileSource<RowData, HiveSourceSplit> implements ResultTypeQueryable<RowData> { + + private static final long serialVersionUID = 1L; + + // DataType of the records to be returned, with projection applied (if any) + private final DataType producedDataType; + + HiveSource( + JobConf jobConf, + CatalogTable catalogTable, + List<HiveTablePartition> partitions, + int[] projectedFields, + long limit, + String hiveVersion, + boolean useMapRedReader, + boolean isStreamingSource, + DataType producedDataType) { + super( + new org.apache.flink.core.fs.Path[1], + new HiveSourceFileEnumerator.Provider(partitions, new JobConfWrapper(jobConf)), + DEFAULT_SPLIT_ASSIGNER, + createBulkFormat(new JobConf(jobConf), catalogTable, projectedFields, hiveVersion, producedDataType, useMapRedReader, limit), + null); + Preconditions.checkArgument(!isStreamingSource, "HiveSource currently only supports bounded mode"); + this.producedDataType = producedDataType; + } + + @Override + public SimpleVersionedSerializer<HiveSourceSplit> getSplitSerializer() { + return HiveSourceSplitSerializer.INSTANCE; + } + + @Override + public TypeInformation<RowData> getProducedType() { + return (TypeInformation<RowData>) TypeInfoDataTypeConverter.fromDataTypeToTypeInfo(producedDataType); + } + + private static BulkFormat<RowData, HiveSourceSplit> createBulkFormat( + JobConf jobConf, + CatalogTable catalogTable, + @Nullable int[] projectedFields, + String hiveVersion, + DataType producedDataType, + boolean useMapRedReader, + long limit) { + checkNotNull(catalogTable, "catalogTable can not be null."); + return new HiveBulkFormatAdapter( + new JobConfWrapper(jobConf), + catalogTable.getPartitionKeys(), + catalogTable.getSchema().getFieldNames(), + catalogTable.getSchema().getFieldDataTypes(), + projectedFields != null ? projectedFields : IntStream.range(0, catalogTable.getSchema().getFieldCount()).toArray(), + hiveVersion, + producedDataType, + useMapRedReader, + limit); + } + + public static List<HiveSourceSplit> createInputSplits( + int minNumSplits, + List<HiveTablePartition> partitions, + JobConf jobConf) throws IOException { + List<HiveSourceSplit> hiveSplits = new ArrayList<>(); + FileSystem fs = null; + for (HiveTablePartition partition : partitions) { + StorageDescriptor sd = partition.getStorageDescriptor(); + Path inputPath = new Path(sd.getLocation()); + if (fs == null) { + fs = inputPath.getFileSystem(jobConf); + } + // it's possible a partition exists in metastore but the data has been removed + if (!fs.exists(inputPath)) { + continue; + } + InputFormat format; + try { + format = (InputFormat) + Class.forName(sd.getInputFormat(), true, Thread.currentThread().getContextClassLoader()).newInstance(); + } catch (Exception e) { + throw new FlinkHiveException("Unable to instantiate the hadoop input format", e); + } + ReflectionUtils.setConf(format, jobConf); + jobConf.set(INPUT_DIR, sd.getLocation()); + //TODO: we should consider how to calculate the splits according to minNumSplits in the future. + org.apache.hadoop.mapred.InputSplit[] splitArray = format.getSplits(jobConf, minNumSplits); + for (org.apache.hadoop.mapred.InputSplit inputSplit : splitArray) { + Preconditions.checkState(inputSplit instanceof FileSplit, + "Unsupported InputSplit type: " + inputSplit.getClass().getName()); + hiveSplits.add(new HiveSourceSplit((FileSplit) inputSplit, partition, null)); + } + } + + return hiveSplits; + } + + public static int getNumFiles(List<HiveTablePartition> partitions, JobConf jobConf) throws IOException { Review comment: Better in HiveSourceFileEnumerator ########## File path: flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSource.java ########## @@ -0,0 +1,168 @@ +/* + * 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.connectors.hive; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.connector.file.src.AbstractFileSource; +import org.apache.flink.connector.file.src.reader.BulkFormat; +import org.apache.flink.connectors.hive.read.HiveBulkFormatAdapter; +import org.apache.flink.connectors.hive.read.HiveSourceSplit; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter; +import org.apache.flink.table.types.DataType; +import org.apache.flink.util.Preconditions; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.mapred.FileSplit; +import org.apache.hadoop.mapred.InputFormat; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.util.ReflectionUtils; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.IntStream; + +import static org.apache.flink.connector.file.src.FileSource.DEFAULT_SPLIT_ASSIGNER; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.hadoop.mapreduce.lib.input.FileInputFormat.INPUT_DIR; + +/** + * A unified data source that reads a hive table. + */ +public class HiveSource extends AbstractFileSource<RowData, HiveSourceSplit> implements ResultTypeQueryable<RowData> { + + private static final long serialVersionUID = 1L; + + // DataType of the records to be returned, with projection applied (if any) + private final DataType producedDataType; + + HiveSource( + JobConf jobConf, + CatalogTable catalogTable, + List<HiveTablePartition> partitions, + int[] projectedFields, Review comment: Already has `producedType`, I think `projectedFields` can be removed ---------------------------------------------------------------- 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: [email protected]
