flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From GitBox <...@apache.org>
Subject [GitHub] [flink] lirui-apache commented on a change in pull request #13479: [FLINK-19414][parquet] Introduce ParquetColumnarRowInputFormat
Date Mon, 12 Oct 2020 06:18:28 GMT

lirui-apache commented on a change in pull request #13479:
URL: https://github.com/apache/flink/pull/13479#discussion_r503060323



##########
File path: flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetVectorizedInputFormat.java
##########
@@ -0,0 +1,471 @@
+/*
+ * 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.formats.parquet;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.base.source.reader.SourceReaderOptions;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.src.util.CheckpointedPosition;
+import org.apache.flink.connector.file.src.util.Pool;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.formats.parquet.utils.SerializableConfiguration;
+import org.apache.flink.formats.parquet.vector.ColumnBatchFactory;
+import org.apache.flink.formats.parquet.vector.ParquetDecimalVector;
+import org.apache.flink.formats.parquet.vector.reader.AbstractColumnReader;
+import org.apache.flink.formats.parquet.vector.reader.ColumnReader;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.flink.table.data.vector.writable.WritableColumnVector;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.filter2.compat.FilterCompat;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.Type;
+import org.apache.parquet.schema.Types;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+import static org.apache.flink.formats.parquet.vector.ParquetSplitReaderUtil.createColumnReader;
+import static org.apache.flink.formats.parquet.vector.ParquetSplitReaderUtil.createWritableColumnVector;
+import static org.apache.parquet.filter2.compat.RowGroupFilter.filterRowGroups;
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.range;
+import static org.apache.parquet.hadoop.ParquetFileReader.readFooter;
+import static org.apache.parquet.hadoop.ParquetInputFormat.getFilter;
+
+/**
+ * Parquet {@link BulkFormat} that reads data from the file to {@link VectorizedColumnBatch}
in
+ * vectorized mode.
+ */
+public abstract class ParquetVectorizedInputFormat<T> implements BulkFormat<T>
{
+
+	private static final long serialVersionUID = 1L;
+
+	private final SerializableConfiguration hadoopConfig;
+	private final String[] projectedFields;
+	private final LogicalType[] projectedTypes;
+	private final ColumnBatchFactory batchFactory;
+	private final int batchSize;
+	private final boolean isUtcTimestamp;
+	private final boolean isCaseSensitive;
+
+	public ParquetVectorizedInputFormat(
+			SerializableConfiguration hadoopConfig,
+			String[] projectedFields,
+			LogicalType[] projectedTypes,
+			ColumnBatchFactory batchFactory,
+			int batchSize,
+			boolean isUtcTimestamp,
+			boolean isCaseSensitive) {
+		Preconditions.checkArgument(
+				projectedFields.length == projectedTypes.length,
+				"The length(%s) of projectedFields should equal to the length(%s) projectedTypes",
+				projectedFields.length,
+				projectedTypes.length);
+
+		this.hadoopConfig = hadoopConfig;
+		this.projectedFields = projectedFields;
+		this.projectedTypes = projectedTypes;
+		this.batchFactory = batchFactory;
+		this.batchSize = batchSize;
+		this.isUtcTimestamp = isUtcTimestamp;
+		this.isCaseSensitive = isCaseSensitive;
+	}
+
+	@Override
+	public ParquetReader createReader(
+			Configuration config,
+			Path filePath,
+			long splitOffset,
+			long splitLength) throws IOException {
+		org.apache.hadoop.fs.Path hadoopPath = new org.apache.hadoop.fs.Path(filePath.toUri());
+		ParquetMetadata footer = readFooter(
+				hadoopConfig.conf(), hadoopPath, range(splitOffset, splitOffset + splitLength));
+		MessageType fileSchema = footer.getFileMetaData().getSchema();
+		FilterCompat.Filter filter = getFilter(hadoopConfig.conf());
+		List<BlockMetaData> blocks = filterRowGroups(filter, footer.getBlocks(), fileSchema);
+
+		MessageType requestedSchema = clipParquetSchema(fileSchema);
+		ParquetFileReader reader = new ParquetFileReader(
+				hadoopConfig.conf(),
+				footer.getFileMetaData(),
+				hadoopPath,
+				blocks,
+				requestedSchema.getColumns());
+
+		long totalRowCount = 0;
+		for (BlockMetaData block : blocks) {
+			totalRowCount += block.getRowCount();
+		}
+
+		checkSchema(fileSchema, requestedSchema);
+
+		final int numBatchesToCirculate = config.getInteger(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY)
+ 1;
+		final Pool<ParquetReaderBatch<T>> poolOfBatches =
+				createPoolOfBatches(filePath, requestedSchema, numBatchesToCirculate);
+
+		return new ParquetReader(reader, requestedSchema, totalRowCount, poolOfBatches);
+	}
+
+	@Override
+	public ParquetReader restoreReader(
+			Configuration config,
+			Path filePath,
+			long splitOffset,
+			long splitLength,
+			CheckpointedPosition checkpointedPosition) throws IOException {
+		ParquetReader reader = createReader(config, filePath, splitOffset, splitLength);
+		// Offset is record count too.
+		reader.seek(checkpointedPosition.getOffset() + checkpointedPosition.getRecordsAfterOffset());

Review comment:
       I wasn't using `offset` for my Hive source PoC. Honestly, I don't quite understand
the difference between `offset` and `recordsAfterOffset`. Could you elaborate on these and
how should they be used?




----------------------------------------------------------------
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



Mime
View raw message