You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2020/06/13 21:03:03 UTC
[hive] branch master updated: HIVE-23554 : Adding FilterContext as
part of LLAP ColumnVectorBatch (propagated in ReadPipeline). Also moving
the code that prepares VectorBatches in EncodedDataConsumer in a separate
method -- this will be handy when dealing with row-filters later on. (Panos
G via Ashutosh Chauhan)
This is an automated email from the ASF dual-hosted git repository.
hashutosh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git
The following commit(s) were added to refs/heads/master by this push:
new 3065904 HIVE-23554 : Adding FilterContext as part of LLAP ColumnVectorBatch (propagated in ReadPipeline). Also moving the code that prepares VectorBatches in EncodedDataConsumer in a separate method -- this will be handy when dealing with row-filters later on. (Panos G via Ashutosh Chauhan)
3065904 is described below
commit 30659041e470702e4a52f024eabe28d2d848d478
Author: Panos Garefalakis <pg...@cloudera.com>
AuthorDate: Wed May 27 11:51:48 2020 +0100
HIVE-23554 : Adding FilterContext as part of LLAP ColumnVectorBatch (propagated in ReadPipeline). Also moving the code that prepares VectorBatches in EncodedDataConsumer in a separate method -- this will be handy when dealing with row-filters later on. (Panos G via Ashutosh Chauhan)
Change-Id: I0177756e842e60f6850c966cfa44fe0d53df4a28
Signed-off-by: Ashutosh Chauhan <ha...@apache.org>
---
.../hive/llap/io/api/impl/ColumnVectorBatch.java | 15 ++++++++++++
.../llap/io/decode/OrcEncodedDataConsumer.java | 27 +++++++++++++---------
2 files changed, 31 insertions(+), 11 deletions(-)
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/ColumnVectorBatch.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/ColumnVectorBatch.java
index 19b0b55..52dc072 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/ColumnVectorBatch.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/ColumnVectorBatch.java
@@ -20,12 +20,17 @@ package org.apache.hadoop.hive.llap.io.api.impl;
import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.io.filter.MutableFilterContext;
+
+import java.util.Arrays;
/**
* Unlike VRB, doesn't have some fields, and doesn't have all columns
* (non-selected, partition cols, cols for downstream ops, etc.)
+ * It does, however, hold the FilterContext of the VRB.
*/
public class ColumnVectorBatch {
+ public MutableFilterContext filterContext;
public ColumnVector[] cols;
public int size;
@@ -34,6 +39,7 @@ public class ColumnVectorBatch {
}
public ColumnVectorBatch(int columnCount, int batchSize) {
+ this.filterContext = new VectorizedRowBatch(0);
this.cols = new ColumnVector[columnCount];
this.size = batchSize;
}
@@ -51,6 +57,15 @@ public class ColumnVectorBatch {
return "";
}
StringBuilder b = new StringBuilder();
+ b.append("FilterContext used: ");
+ b.append(filterContext.isSelectedInUse());
+ b.append(", size: ");
+ b.append(filterContext.getSelectedSize());
+ b.append('\n');
+ b.append("Selected: ");
+ b.append(filterContext.isSelectedInUse() ? Arrays.toString(filterContext.getSelected()) : "[]");
+ b.append('\n');
+
b.append("Column vector types: ");
for (int k = 0; k < cols.length; k++) {
ColumnVector cv = cols[k];
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java
index 79dba42..9459a4f 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java
@@ -152,17 +152,10 @@ public class OrcEncodedDataConsumer
}
ColumnVectorBatch cvb = cvbPool.take();
+ cvb.filterContext.reset();
// assert cvb.cols.length == batch.getColumnIxs().length; // Must be constant per split.
cvb.size = batchSize;
for (int idx = 0; idx < columnReaders.length; ++idx) {
- TreeReader reader = columnReaders[idx];
- if (cvb.cols[idx] == null) {
- // Orc store rows inside a root struct (hive writes it this way).
- // When we populate column vectors we skip over the root struct.
- cvb.cols[idx] = createColumn(batchSchemas[idx], VectorizedRowBatch.DEFAULT_SIZE, useDecimal64ColumnVectors);
- }
- trace.logTreeReaderNextVector(idx);
-
/*
* Currently, ORC's TreeReaderFactory class does this:
*
@@ -198,9 +191,8 @@ public class OrcEncodedDataConsumer
* it doesn't get confused.
*
*/
- ColumnVector cv = cvb.cols[idx];
- cv.reset();
- cv.ensureSize(batchSize, false);
+ TreeReader reader = columnReaders[idx];
+ ColumnVector cv = prepareColumnVector(cvb, idx, batchSize);
reader.nextVector(cv, null, batchSize);
}
@@ -218,6 +210,19 @@ public class OrcEncodedDataConsumer
}
}
+ private ColumnVector prepareColumnVector(ColumnVectorBatch cvb, int idx, int batchSize) {
+ if (cvb.cols[idx] == null) {
+ // Orc store rows inside a root struct (hive writes it this way).
+ // When we populate column vectors we skip over the root struct.
+ cvb.cols[idx] = createColumn(batchSchemas[idx], VectorizedRowBatch.DEFAULT_SIZE, useDecimal64ColumnVectors);
+ }
+ trace.logTreeReaderNextVector(idx);
+ ColumnVector cv = cvb.cols[idx];
+ cv.reset();
+ cv.ensureSize(batchSize, false);
+ return cv;
+ }
+
private void createColumnReaders(OrcEncodedColumnBatch batch,
ConsumerStripeMetadata stripeMetadata, TypeDescription fileSchema) throws IOException {
TreeReaderFactory.Context context = new TreeReaderFactory.ReaderContext()