flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From GitBox <...@apache.org>
Subject [GitHub] [flink] KurtYoung commented on a change in pull request #10022: [FLINK-14135][hive][orc] Introduce orc ColumnarRow reader for hive connector
Date Fri, 15 Nov 2019 12:06:58 GMT
KurtYoung commented on a change in pull request #10022: [FLINK-14135][hive][orc] Introduce
orc ColumnarRow reader for hive connector
URL: https://github.com/apache/flink/pull/10022#discussion_r346713929
 
 

 ##########
 File path: flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcSplitReader.java
 ##########
 @@ -0,0 +1,577 @@
+/*
+ * 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.orc;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.fs.Path;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.orc.OrcConf;
+import org.apache.orc.OrcFile;
+import org.apache.orc.Reader;
+import org.apache.orc.RecordReader;
+import org.apache.orc.StripeInformation;
+import org.apache.orc.TypeDescription;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Orc split reader to read record from orc file. The reader is only responsible for reading
the data
+ * of a single split.
+ */
+public abstract class OrcSplitReader<T> implements Closeable {
+
+	// the ORC reader
+	private RecordReader orcRowsReader;
+
+	// the vectorized row data to be read in a batch
+	protected final VectorizedRowBatch rowBatch;
+
+	private final Reader.Options options;
+
+	// the number of rows in the current batch
+	private int rowsInBatch;
+	// the index of the next row to return
+	protected int nextRow;
+
+	public OrcSplitReader(
+			Configuration conf,
+			TypeDescription schema,
+			int[] selectedFields,
+			List<Predicate> conjunctPredicates,
+			int batchSize,
+			Path path,
+			long splitStart,
+			long splitLength) throws IOException {
+		// open ORC file and create reader
+		org.apache.hadoop.fs.Path hPath = new org.apache.hadoop.fs.Path(path.getPath());
+		Reader orcReader = OrcFile.createReader(hPath, OrcFile.readerOptions(conf));
+
+		// get offset and length for the stripes that start in the split
+		Tuple2<Long, Long> offsetAndLength = getOffsetAndLengthForSplit(
+				splitStart, splitLength, getStripes(orcReader));
+
+		// create ORC row reader configuration
+		this.options = orcReader.options()
+				.schema(schema)
+				.range(offsetAndLength.f0, offsetAndLength.f1)
+				.useZeroCopy(OrcConf.USE_ZEROCOPY.getBoolean(conf))
+				.skipCorruptRecords(OrcConf.SKIP_CORRUPT_DATA.getBoolean(conf))
+				.tolerateMissingSchema(OrcConf.TOLERATE_MISSING_SCHEMA.getBoolean(conf));
+
+		// configure filters
+		if (!conjunctPredicates.isEmpty()) {
+			SearchArgument.Builder b = SearchArgumentFactory.newBuilder();
+			b = b.startAnd();
+			for (Predicate predicate : conjunctPredicates) {
+				predicate.add(b);
+			}
+			b = b.end();
+			options.searchArgument(b.build(), new String[]{});
+		}
+
+		// configure selected fields
+		options.include(computeProjectionMask(schema, selectedFields));
+
+		// create ORC row reader
+		this.orcRowsReader = orcReader.rows(options);
+
+		// assign ids
+		schema.getId();
+		// create row batch
+		this.rowBatch = schema.createRowBatch(batchSize);
+		rowsInBatch = 0;
+		nextRow = 0;
+	}
+
+	/**
+	 * Method used to check if the end of the input is reached.
+	 *
+	 * @return True if the end is reached, otherwise false.
+	 * @throws IOException Thrown, if an I/O error occurred.
+	 */
+	public boolean reachedEnd() throws IOException {
+		return !ensureBatch();
+	}
+
+	/**
+	 * Fills an ORC batch into an array of Row.
+	 *
+	 * @return The number of rows that were filled.
+	 */
+	protected abstract int fillRows();
+
+	/**
+	 * Reads the next record from the input.
+	 *
+	 * @param reuse Object that may be reused.
+	 * @return Read record.
+	 *
+	 * @throws IOException Thrown, if an I/O error occurred.
+	 */
+	public abstract T nextRecord(T reuse) throws IOException;
+
+	/**
+	 * Checks if there is at least one row left in the batch to return.
+	 * If no more row are available, it reads another batch of rows.
+	 *
+	 * @return Returns true if there is one more row to return, false otherwise.
+	 * @throws IOException throw if an exception happens while reading a batch.
+	 */
+	private boolean ensureBatch() throws IOException {
+
+		if (nextRow >= rowsInBatch) {
+			// No more rows available in the Rows array.
+			nextRow = 0;
+			// Try to read the next batch if rows from the ORC file.
+			boolean moreRows = orcRowsReader.nextBatch(rowBatch);
+
+			if (moreRows) {
+				// Load the data into the Rows array.
+				rowsInBatch = fillRows();
+			}
+			return moreRows;
+		}
+		// there is at least one Row left in the Rows array.
+		return true;
+	}
+
+	private Tuple2<Long, Long> getOffsetAndLengthForSplit(
+			long splitStart, long splitLength, List<StripeInformation> stripes) {
+		long splitEnd = splitStart + splitLength;
+		long readStart = Long.MAX_VALUE;
+		long readEnd = Long.MIN_VALUE;
+
+		for (StripeInformation s : stripes) {
+			if (splitStart <= s.getOffset() && s.getOffset() < splitEnd) {
+				// stripe starts in split, so it is included
+				readStart = Math.min(readStart, s.getOffset());
+				readEnd = Math.max(readEnd, s.getOffset() + s.getLength());
+			}
+		}
+
+		if (readStart < Long.MAX_VALUE) {
+			// at least one split is included
+			return Tuple2.of(readStart, readEnd - readStart);
+		} else {
+			return Tuple2.of(0L, 0L);
+		}
+	}
+
+	@VisibleForTesting
+	Reader.Options getOptions() {
+		return options;
+	}
+
+	@VisibleForTesting
+	List<StripeInformation> getStripes(Reader orcReader) {
+		return orcReader.getStripes();
+	}
+
+	/**
+	 * Computes the ORC projection mask of the fields to include from the selected fields.rowOrcInputFormat.nextRecord(null).
+	 *
+	 * @return The ORC projection mask.
+	 */
+	private boolean[] computeProjectionMask(TypeDescription schema, int[] selectedFields) {
+		// mask with all fields of the schema
+		boolean[] projectionMask = new boolean[schema.getMaximumId() + 1];
+		// for each selected field
+		for (int inIdx : selectedFields) {
+			// set all nested fields of a selected field to true
+			TypeDescription fieldSchema = schema.getChildren().get(inIdx);
+			for (int i = fieldSchema.getId(); i <= fieldSchema.getMaximumId(); i++) {
+				projectionMask[i] = true;
+			}
+		}
+		return projectionMask;
+	}
+
+	@Override
+	public void close() throws IOException {
+		if (orcRowsReader != null) {
+			this.orcRowsReader.close();
+		}
+		this.orcRowsReader = null;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Classes to define predicates
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * A filter predicate that can be evaluated by the OrcInputFormat.
+	 */
+	public abstract static class Predicate implements Serializable {
 
 Review comment:
   Could you move these predicates classes out of this? It doesn't look like be related to
`SplitReader`

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


With regards,
Apache Git Services

Mime
View raw message