ayushtkn commented on code in PR #4199: URL: https://github.com/apache/hive/pull/4199#discussion_r1395295308
########## hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableInputFormatV2.java: ########## @@ -0,0 +1,375 @@ +/* + * 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.hadoop.hive.hbase; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.mapred.TableMapReduceUtil; +import org.apache.hadoop.hbase.mapreduce.TableSplit; +import org.apache.hadoop.hive.hbase.ColumnMappings.ColumnMapping; +import org.apache.hadoop.hive.ql.exec.SerializationUtilities; +import org.apache.hadoop.hive.ql.index.IndexPredicateAnalyzer; +import org.apache.hadoop.hive.ql.index.IndexSearchCondition; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.plan.TableScanDesc; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.mapred.InputFormat; +import org.apache.hadoop.mapred.InputSplit; +import org.apache.hadoop.mapred.RecordReader; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; +import org.apache.hadoop.security.UserGroupInformation; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.security.PrivilegedExceptionAction; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * It`s the pair with {@link HiveHBaseTableInputFormat}, instead of directly extending TableInputFormatBase, using delegate + * to make the class ONLY inherit from mapred.*, which makes the hierarchy more clear and avoid downstream application + * like spark issue, ex https://github.com/apache/spark/pull/31302 + */ +public class HiveHBaseTableInputFormatV2 implements InputFormat<ImmutableBytesWritable, ResultWritable> { + + static final Logger LOG = LoggerFactory.getLogger(HiveHBaseTableInputFormatV2.class); + private static final Object HBASE_TABLE_MONITOR = new Object(); + + private HiveHBaseTableInputFormatDelegate delegate = new HiveHBaseTableInputFormatDelegate(); Review Comment: should be final ########## common/src/java/org/apache/hadoop/hive/conf/HiveConf.java: ########## @@ -3504,6 +3504,12 @@ public static enum ConfVars { HIVE_HBASE_GENERATE_HFILES("hive.hbase.generatehfiles", false, "True when HBaseStorageHandler should generate hfiles instead of operate against the online table."), HIVE_HBASE_SNAPSHOT_NAME("hive.hbase.snapshot.name", null, "The HBase table snapshot name to use."), + + HIVE_HBASE_INPUTFORMAT_V2("hive.hbase.inputformat.v2", false, "If enabled, the new " + + "version (V2) of the input format, which inherits only the mapred version of InputFormat, will be " + + "utilized as the default input format for reading the Hbase table. " + + "Currently, the old version (HiveHBaseTableInputFormat) is used by default, " + + "which inherits both the mapred and mapreduce versions of InputFormat."), Review Comment: I think this config isn't right, we should just have ``hive.hbase.inputformat`` and specify ``1`` as default and use ``2`` for v2, else if tmrw v3 or v4 pitches in we would be having 1 config for each ########## hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableInputFormatV2.java: ########## @@ -0,0 +1,375 @@ +/* + * 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.hadoop.hive.hbase; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.mapred.TableMapReduceUtil; +import org.apache.hadoop.hbase.mapreduce.TableSplit; +import org.apache.hadoop.hive.hbase.ColumnMappings.ColumnMapping; +import org.apache.hadoop.hive.ql.exec.SerializationUtilities; +import org.apache.hadoop.hive.ql.index.IndexPredicateAnalyzer; +import org.apache.hadoop.hive.ql.index.IndexSearchCondition; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.plan.TableScanDesc; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.mapred.InputFormat; +import org.apache.hadoop.mapred.InputSplit; +import org.apache.hadoop.mapred.RecordReader; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; +import org.apache.hadoop.security.UserGroupInformation; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.security.PrivilegedExceptionAction; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * It`s the pair with {@link HiveHBaseTableInputFormat}, instead of directly extending TableInputFormatBase, using delegate + * to make the class ONLY inherit from mapred.*, which makes the hierarchy more clear and avoid downstream application + * like spark issue, ex https://github.com/apache/spark/pull/31302 + */ +public class HiveHBaseTableInputFormatV2 implements InputFormat<ImmutableBytesWritable, ResultWritable> { + + static final Logger LOG = LoggerFactory.getLogger(HiveHBaseTableInputFormatV2.class); + private static final Object HBASE_TABLE_MONITOR = new Object(); + + private HiveHBaseTableInputFormatDelegate delegate = new HiveHBaseTableInputFormatDelegate(); + + @Override public RecordReader<ImmutableBytesWritable, ResultWritable> getRecordReader(InputSplit split, + JobConf jobConf, final Reporter reporter) throws IOException { + + HBaseSplit hbaseSplit = (HBaseSplit) split; + TableSplit tableSplit = hbaseSplit.getTableSplit(); + + final org.apache.hadoop.mapreduce.RecordReader<ImmutableBytesWritable, Result> recordReader; + + Job job = new Job(jobConf); + TaskAttemptContext tac = ShimLoader.getHadoopShims().newTaskAttemptContext(job.getConfiguration(), reporter); + + final Connection conn; + + synchronized (HBASE_TABLE_MONITOR) { + conn = ConnectionFactory.createConnection(HBaseConfiguration.create(jobConf)); + delegate.initializeTableDelegate(conn, tableSplit.getTable()); + delegate.setScan(HiveHBaseInputFormatUtil.getScan(jobConf)); + recordReader = delegate.createRecordReader(tableSplit, tac); + try { + recordReader.initialize(tableSplit, tac); + } catch (InterruptedException e) { + delegate.closeTableDelegate(); // Free up the HTable connections + conn.close(); + throw new IOException("Failed to initialize RecordReader: ", e); + } + } + + return new RecordReader<ImmutableBytesWritable, ResultWritable>() { + + @Override public void close() throws IOException { + synchronized (HBASE_TABLE_MONITOR) { + recordReader.close(); + delegate.closeTableDelegate(); + conn.close(); + } + } + + @Override public ImmutableBytesWritable createKey() { + return new ImmutableBytesWritable(); + } + + @Override public ResultWritable createValue() { + return new ResultWritable(new Result()); + } + + @Override public long getPos() throws IOException { + return 0; + } + + @Override public float getProgress() throws IOException { + float progress = 0.0F; + + try { + progress = recordReader.getProgress(); + } catch (InterruptedException e) { + throw new IOException(e); + } + + return progress; + } + + @Override public boolean next(ImmutableBytesWritable rowKey, ResultWritable value) throws IOException { + + boolean next = false; + + try { + next = recordReader.nextKeyValue(); + + if (next) { + rowKey.set(recordReader.getCurrentValue().getRow()); + value.setResult(recordReader.getCurrentValue()); + } + } catch (InterruptedException e) { + throw new IOException(e); + } + + return next; + } + }; + } + + /** + * Converts a filter (which has been pushed down from Hive's optimizer) + * into corresponding restrictions on the HBase scan. The + * filter should already be in a form which can be fully converted. + * + * @param jobConf configuration for the scan + * + * @param iKey 0-based offset of key column within Hive table + * + * @return converted table split if any + */ + private Scan createFilterScan(JobConf jobConf, int iKey, int iTimestamp, boolean isKeyBinary) throws IOException { + + // TODO: assert iKey is HBaseSerDe#HBASE_KEY_COL + + Scan scan = new Scan(); + String filterObjectSerialized = jobConf.get(TableScanDesc.FILTER_OBJECT_CONF_STR); + if (filterObjectSerialized != null) { + HiveHBaseInputFormatUtil.setupScanRange(scan, filterObjectSerialized, jobConf, false); + return scan; + } + + String filterExprSerialized = jobConf.get(TableScanDesc.FILTER_EXPR_CONF_STR); + if (filterExprSerialized == null) { + return scan; + } + + ExprNodeGenericFuncDesc filterExpr = SerializationUtilities.deserializeExpression(filterExprSerialized); + + String keyColName = jobConf.get(serdeConstants.LIST_COLUMNS).split(",")[iKey]; + ArrayList<TypeInfo> cols = TypeInfoUtils.getTypeInfosFromTypeString(jobConf.get(serdeConstants.LIST_COLUMN_TYPES)); + String colType = cols.get(iKey).getTypeName(); + boolean isKeyComparable = isKeyBinary || "string".equalsIgnoreCase(colType); + + String tsColName = null; + if (iTimestamp >= 0) { + tsColName = jobConf.get(serdeConstants.LIST_COLUMNS).split(",")[iTimestamp]; + } + + IndexPredicateAnalyzer analyzer = newIndexPredicateAnalyzer(keyColName, isKeyComparable, tsColName); + + List<IndexSearchCondition> conditions = new ArrayList<IndexSearchCondition>(); + ExprNodeDesc residualPredicate = analyzer.analyzePredicate(filterExpr, conditions); + + // There should be no residual since we already negotiated that earlier in + // HBaseStorageHandler.decomposePredicate. However, with hive.optimize.index.filter + // OpProcFactory#pushFilterToStorageHandler pushes the original filter back down again. + // Since pushed-down filters are not omitted at the higher levels (and thus the + // contract of negotiation is ignored anyway), just ignore the residuals. + // Re-assess this when negotiation is honored and the duplicate evaluation is removed. + // THIS IGNORES RESIDUAL PARSING FROM HBaseStorageHandler#decomposePredicate + if (residualPredicate != null) { + LOG.debug("Ignoring residual predicate " + residualPredicate.getExprString()); + } + + Map<String, List<IndexSearchCondition>> split = HiveHBaseInputFormatUtil.decompose(conditions); + List<IndexSearchCondition> keyConditions = split.get(keyColName); + if (keyConditions != null && !keyConditions.isEmpty()) { + HiveHBaseInputFormatUtil.setupKeyRange(scan, keyConditions, isKeyBinary); + } + List<IndexSearchCondition> tsConditions = split.get(tsColName); + if (tsConditions != null && !tsConditions.isEmpty()) { + HiveHBaseInputFormatUtil.setupTimeRange(scan, tsConditions); + } + return scan; + } + + /** + * Instantiates a new predicate analyzer suitable for + * determining how to push a filter down into the HBase scan, + * based on the rules for what kinds of pushdown we currently support. + * + * @param keyColumnName name of the Hive column mapped to the HBase row key + * + * @return preconfigured predicate analyzer + */ + static IndexPredicateAnalyzer newIndexPredicateAnalyzer(String keyColumnName, boolean isKeyComparable, + String timestampColumn) { + + IndexPredicateAnalyzer analyzer = new IndexPredicateAnalyzer(); + + // We can always do equality predicate. Just need to make sure we get appropriate + // BA representation of constant of filter condition. + // We can do other comparisons only if storage format in hbase is either binary + // or we are dealing with string types since there lexicographic ordering will suffice. + String genericUDFOPEqualClass = "org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual"; + String genericUDFOPEqualOrGreaterThanClass = "org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrGreaterThan"; + String genericUDFOPEqualOrLessThanClass = "org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrLessThan"; + String genericUDFOPLessThanClass = "org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan"; + String genericUDFOPGreaterThanClass = "org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPGreaterThan"; + if (isKeyComparable) { + analyzer.addComparisonOp(keyColumnName, genericUDFOPEqualClass, + genericUDFOPEqualOrGreaterThanClass, + genericUDFOPEqualOrLessThanClass, + genericUDFOPLessThanClass, + genericUDFOPGreaterThanClass); + } else { + analyzer.addComparisonOp(keyColumnName, genericUDFOPEqualClass); + } + + if (timestampColumn != null) { + analyzer.addComparisonOp(timestampColumn, genericUDFOPEqualClass, + genericUDFOPEqualOrGreaterThanClass, + genericUDFOPEqualOrLessThanClass, + genericUDFOPLessThanClass, + genericUDFOPGreaterThanClass); + } + + return analyzer; + } + + @Override public InputSplit[] getSplits(final JobConf jobConf, final int numSplits) throws IOException { + synchronized (HBASE_TABLE_MONITOR) { Review Comment: this is also pretty much a copy from v1 ########## hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableInputFormatV2.java: ########## @@ -0,0 +1,375 @@ +/* + * 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.hadoop.hive.hbase; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.mapred.TableMapReduceUtil; +import org.apache.hadoop.hbase.mapreduce.TableSplit; +import org.apache.hadoop.hive.hbase.ColumnMappings.ColumnMapping; +import org.apache.hadoop.hive.ql.exec.SerializationUtilities; +import org.apache.hadoop.hive.ql.index.IndexPredicateAnalyzer; +import org.apache.hadoop.hive.ql.index.IndexSearchCondition; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.plan.TableScanDesc; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.mapred.InputFormat; +import org.apache.hadoop.mapred.InputSplit; +import org.apache.hadoop.mapred.RecordReader; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; +import org.apache.hadoop.security.UserGroupInformation; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.security.PrivilegedExceptionAction; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * It`s the pair with {@link HiveHBaseTableInputFormat}, instead of directly extending TableInputFormatBase, using delegate + * to make the class ONLY inherit from mapred.*, which makes the hierarchy more clear and avoid downstream application + * like spark issue, ex https://github.com/apache/spark/pull/31302 + */ +public class HiveHBaseTableInputFormatV2 implements InputFormat<ImmutableBytesWritable, ResultWritable> { + + static final Logger LOG = LoggerFactory.getLogger(HiveHBaseTableInputFormatV2.class); Review Comment: shouldn't be ``private``? ########## hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableInputFormatV2.java: ########## @@ -0,0 +1,375 @@ +/* + * 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.hadoop.hive.hbase; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.mapred.TableMapReduceUtil; +import org.apache.hadoop.hbase.mapreduce.TableSplit; +import org.apache.hadoop.hive.hbase.ColumnMappings.ColumnMapping; +import org.apache.hadoop.hive.ql.exec.SerializationUtilities; +import org.apache.hadoop.hive.ql.index.IndexPredicateAnalyzer; +import org.apache.hadoop.hive.ql.index.IndexSearchCondition; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.plan.TableScanDesc; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.mapred.InputFormat; +import org.apache.hadoop.mapred.InputSplit; +import org.apache.hadoop.mapred.RecordReader; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; +import org.apache.hadoop.security.UserGroupInformation; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.security.PrivilegedExceptionAction; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * It`s the pair with {@link HiveHBaseTableInputFormat}, instead of directly extending TableInputFormatBase, using delegate + * to make the class ONLY inherit from mapred.*, which makes the hierarchy more clear and avoid downstream application + * like spark issue, ex https://github.com/apache/spark/pull/31302 + */ +public class HiveHBaseTableInputFormatV2 implements InputFormat<ImmutableBytesWritable, ResultWritable> { + + static final Logger LOG = LoggerFactory.getLogger(HiveHBaseTableInputFormatV2.class); + private static final Object HBASE_TABLE_MONITOR = new Object(); + + private HiveHBaseTableInputFormatDelegate delegate = new HiveHBaseTableInputFormatDelegate(); + + @Override public RecordReader<ImmutableBytesWritable, ResultWritable> getRecordReader(InputSplit split, + JobConf jobConf, final Reporter reporter) throws IOException { + + HBaseSplit hbaseSplit = (HBaseSplit) split; + TableSplit tableSplit = hbaseSplit.getTableSplit(); + + final org.apache.hadoop.mapreduce.RecordReader<ImmutableBytesWritable, Result> recordReader; + + Job job = new Job(jobConf); + TaskAttemptContext tac = ShimLoader.getHadoopShims().newTaskAttemptContext(job.getConfiguration(), reporter); + + final Connection conn; + + synchronized (HBASE_TABLE_MONITOR) { + conn = ConnectionFactory.createConnection(HBaseConfiguration.create(jobConf)); + delegate.initializeTableDelegate(conn, tableSplit.getTable()); + delegate.setScan(HiveHBaseInputFormatUtil.getScan(jobConf)); + recordReader = delegate.createRecordReader(tableSplit, tac); + try { + recordReader.initialize(tableSplit, tac); + } catch (InterruptedException e) { + delegate.closeTableDelegate(); // Free up the HTable connections + conn.close(); + throw new IOException("Failed to initialize RecordReader: ", e); + } + } + + return new RecordReader<ImmutableBytesWritable, ResultWritable>() { + + @Override public void close() throws IOException { + synchronized (HBASE_TABLE_MONITOR) { + recordReader.close(); + delegate.closeTableDelegate(); + conn.close(); + } + } + + @Override public ImmutableBytesWritable createKey() { + return new ImmutableBytesWritable(); + } + + @Override public ResultWritable createValue() { + return new ResultWritable(new Result()); + } + + @Override public long getPos() throws IOException { + return 0; + } + + @Override public float getProgress() throws IOException { + float progress = 0.0F; + + try { + progress = recordReader.getProgress(); + } catch (InterruptedException e) { + throw new IOException(e); + } + + return progress; + } + + @Override public boolean next(ImmutableBytesWritable rowKey, ResultWritable value) throws IOException { + + boolean next = false; + + try { + next = recordReader.nextKeyValue(); + + if (next) { + rowKey.set(recordReader.getCurrentValue().getRow()); + value.setResult(recordReader.getCurrentValue()); + } + } catch (InterruptedException e) { + throw new IOException(e); + } + + return next; + } + }; + } Review Comment: this is an exact dupe from ``HiveHBaseTableInputFormat``, avoid duplication and refactor ########## hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableInputFormatV2.java: ########## @@ -0,0 +1,375 @@ +/* + * 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.hadoop.hive.hbase; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.mapred.TableMapReduceUtil; +import org.apache.hadoop.hbase.mapreduce.TableSplit; +import org.apache.hadoop.hive.hbase.ColumnMappings.ColumnMapping; +import org.apache.hadoop.hive.ql.exec.SerializationUtilities; +import org.apache.hadoop.hive.ql.index.IndexPredicateAnalyzer; +import org.apache.hadoop.hive.ql.index.IndexSearchCondition; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.plan.TableScanDesc; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.mapred.InputFormat; +import org.apache.hadoop.mapred.InputSplit; +import org.apache.hadoop.mapred.RecordReader; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; +import org.apache.hadoop.security.UserGroupInformation; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.security.PrivilegedExceptionAction; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * It`s the pair with {@link HiveHBaseTableInputFormat}, instead of directly extending TableInputFormatBase, using delegate + * to make the class ONLY inherit from mapred.*, which makes the hierarchy more clear and avoid downstream application + * like spark issue, ex https://github.com/apache/spark/pull/31302 + */ +public class HiveHBaseTableInputFormatV2 implements InputFormat<ImmutableBytesWritable, ResultWritable> { + + static final Logger LOG = LoggerFactory.getLogger(HiveHBaseTableInputFormatV2.class); + private static final Object HBASE_TABLE_MONITOR = new Object(); + + private HiveHBaseTableInputFormatDelegate delegate = new HiveHBaseTableInputFormatDelegate(); + + @Override public RecordReader<ImmutableBytesWritable, ResultWritable> getRecordReader(InputSplit split, + JobConf jobConf, final Reporter reporter) throws IOException { + + HBaseSplit hbaseSplit = (HBaseSplit) split; + TableSplit tableSplit = hbaseSplit.getTableSplit(); + + final org.apache.hadoop.mapreduce.RecordReader<ImmutableBytesWritable, Result> recordReader; + + Job job = new Job(jobConf); + TaskAttemptContext tac = ShimLoader.getHadoopShims().newTaskAttemptContext(job.getConfiguration(), reporter); + + final Connection conn; + + synchronized (HBASE_TABLE_MONITOR) { + conn = ConnectionFactory.createConnection(HBaseConfiguration.create(jobConf)); + delegate.initializeTableDelegate(conn, tableSplit.getTable()); + delegate.setScan(HiveHBaseInputFormatUtil.getScan(jobConf)); + recordReader = delegate.createRecordReader(tableSplit, tac); + try { + recordReader.initialize(tableSplit, tac); + } catch (InterruptedException e) { + delegate.closeTableDelegate(); // Free up the HTable connections + conn.close(); + throw new IOException("Failed to initialize RecordReader: ", e); + } + } + + return new RecordReader<ImmutableBytesWritable, ResultWritable>() { + + @Override public void close() throws IOException { + synchronized (HBASE_TABLE_MONITOR) { + recordReader.close(); + delegate.closeTableDelegate(); + conn.close(); + } + } + + @Override public ImmutableBytesWritable createKey() { + return new ImmutableBytesWritable(); + } + + @Override public ResultWritable createValue() { + return new ResultWritable(new Result()); + } + + @Override public long getPos() throws IOException { + return 0; + } + + @Override public float getProgress() throws IOException { + float progress = 0.0F; + + try { + progress = recordReader.getProgress(); + } catch (InterruptedException e) { + throw new IOException(e); + } + + return progress; + } + + @Override public boolean next(ImmutableBytesWritable rowKey, ResultWritable value) throws IOException { + + boolean next = false; + + try { + next = recordReader.nextKeyValue(); + + if (next) { + rowKey.set(recordReader.getCurrentValue().getRow()); + value.setResult(recordReader.getCurrentValue()); + } + } catch (InterruptedException e) { + throw new IOException(e); + } + + return next; + } + }; + } + + /** + * Converts a filter (which has been pushed down from Hive's optimizer) + * into corresponding restrictions on the HBase scan. The + * filter should already be in a form which can be fully converted. + * + * @param jobConf configuration for the scan + * + * @param iKey 0-based offset of key column within Hive table + * + * @return converted table split if any + */ + private Scan createFilterScan(JobConf jobConf, int iKey, int iTimestamp, boolean isKeyBinary) throws IOException { Review Comment: This method is also a complete copy from v1 -- 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. To unsubscribe, e-mail: gitbox-unsubscr...@hive.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: gitbox-unsubscr...@hive.apache.org For additional commands, e-mail: gitbox-h...@hive.apache.org