This is an automated email from the ASF dual-hosted git repository. vihangk1 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 0a77914 HIVE-23821: Send tableId in request for all the new HMS get_partition APIs 0a77914 is described below commit 0a7791443dbb562e6c8a0493319fc52af793fe6b Author: kishendas <kishen....@gmail.com> AuthorDate: Fri Aug 7 14:00:55 2020 -0700 HIVE-23821: Send tableId in request for all the new HMS get_partition APIs --- .../add/AlterTableAddPartitionOperation.java | 2 +- .../org/apache/hadoop/hive/ql/metadata/Hive.java | 45 ++++++- .../hadoop/hive/ql/lockmgr/TestTxnManager.java | 71 +++++++++++ ...TestHiveMetaStoreClientApiArgumentsChecker.java | 141 +++++++++++++++++++++ .../hive/metastore/TestHiveMetaStoreClient.java | 119 +++++++++++++++++ 5 files changed, 372 insertions(+), 6 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/add/AlterTableAddPartitionOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/add/AlterTableAddPartitionOperation.java index ddc47a4..e5dfcb6 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/add/AlterTableAddPartitionOperation.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/add/AlterTableAddPartitionOperation.java @@ -175,7 +175,7 @@ public class AlterTableAddPartitionOperation extends DDLOperation<AlterTableAddP for (Partition partition : partitions){ partitionNames.add(getPartitionName(table, partition)); try { - Partition p = context.getDb().getPartition(desc.getDbName(), desc.getTableName(), partition.getValues()); + Partition p = context.getDb().getPartition(table, desc.getDbName(), desc.getTableName(), partition.getValues()); if (desc.getReplicationSpec().allowReplacementInto(p.getParameters())){ ReplicationSpec.copyLastReplId(p.getParameters(), partition.getParameters()); partitionsToAlter.add(partition); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java index 4c12927..ea8c584 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java @@ -137,6 +137,8 @@ import org.apache.hadoop.hive.metastore.api.Function; import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse; import org.apache.hadoop.hive.metastore.api.GetPartitionNamesPsRequest; import org.apache.hadoop.hive.metastore.api.GetPartitionNamesPsResponse; +import org.apache.hadoop.hive.metastore.api.GetPartitionRequest; +import org.apache.hadoop.hive.metastore.api.GetPartitionResponse; import org.apache.hadoop.hive.metastore.api.GetPartitionsPsWithAuthRequest; import org.apache.hadoop.hive.metastore.api.GetPartitionsPsWithAuthResponse; import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalRequest; @@ -474,6 +476,13 @@ public class Hive { return get(true); } + @VisibleForTesting + public static Hive get(IMetaStoreClient msc) throws HiveException, MetaException { + Hive hive = get(true); + hive.setMSC(msc); + return hive; + } + public static Hive get(boolean doRegisterAllFns) throws HiveException { return getInternal(null, false, false, doRegisterAllFns); } @@ -3256,10 +3265,20 @@ private void constructOneLBLocationMap(FileStatus fSta, } } - public org.apache.hadoop.hive.metastore.api.Partition getPartition(String dbName, String tableName, + public org.apache.hadoop.hive.metastore.api.Partition getPartition(Table t, String dbName, String tableName, List<String> params) throws HiveException { try { - return getMSC().getPartition(dbName, tableName, params); + GetPartitionRequest req = new GetPartitionRequest(); + req.setDbName(dbName); + req.setTblName(tableName); + req.setPartVals(params); + if (AcidUtils.isTransactionalTable(t)) { + ValidWriteIdList validWriteIdList = getValidWriteIdList(dbName, tableName); + req.setValidWriteIdList(validWriteIdList != null ? validWriteIdList.toString() : null); + req.setId(t.getTTable().getId()); + } + GetPartitionResponse res = getMSC().getPartitionRequest(req); + return res.getPartition(); } catch (Exception e) { LOG.error(StringUtils.stringifyException(e)); throw new HiveException(e); @@ -3623,6 +3642,7 @@ private void constructOneLBLocationMap(FileStatus fSta, if (AcidUtils.isTransactionalTable(t)) { ValidWriteIdList validWriteIdList = getValidWriteIdList(dbName, tblName); req.setValidWriteIdList(validWriteIdList != null ? validWriteIdList.toString() : null); + req.setId(t.getTTable().getId()); } GetPartitionNamesPsResponse res = getMSC().listPartitionNamesRequest(req); names = res.getNames(); @@ -3661,6 +3681,7 @@ private void constructOneLBLocationMap(FileStatus fSta, if (AcidUtils.isTransactionalTable(tbl)) { ValidWriteIdList validWriteIdList = getValidWriteIdList(tbl.getDbName(), tbl.getTableName()); req.setValidWriteIdList(validWriteIdList != null ? validWriteIdList.toString() : null); + req.setId(tbl.getTTable().getId()); } names = getMSC().listPartitionNames(req); @@ -3697,6 +3718,7 @@ private void constructOneLBLocationMap(FileStatus fSta, if (AcidUtils.isTransactionalTable(tbl)) { ValidWriteIdList validWriteIdList = getValidWriteIdList(tbl.getDbName(), tbl.getTableName()); req.setValidWriteIdList(validWriteIdList != null ? validWriteIdList.toString() : null); + req.setId(tbl.getTTable().getId()); } GetPartitionsPsWithAuthResponse res = getMSC().listPartitionsWithAuthInfoRequest(req); tParts = res.getPartitions(); @@ -4006,13 +4028,16 @@ private void constructOneLBLocationMap(FileStatus fSta, List<org.apache.hadoop.hive.metastore.api.PartitionSpec> msParts = new ArrayList<>(); ValidWriteIdList validWriteIdList = null; + + PartitionsByExprRequest req = buildPartitionByExprRequest(tbl, exprBytes, defaultPartitionName, conf, + null); + if (AcidUtils.isTransactionalTable(tbl)) { validWriteIdList = getValidWriteIdList(tbl.getDbName(), tbl.getTableName()); + req.setValidWriteIdList(validWriteIdList != null ? validWriteIdList.toString() : null); + req.setId(tbl.getTTable().getId()); } - PartitionsByExprRequest req = buildPartitionByExprRequest(tbl, exprBytes, defaultPartitionName, conf, - validWriteIdList != null ? validWriteIdList.toString() : null); - boolean hasUnknownParts = getMSC().listPartitionsSpecByExpr(req, msParts); partitions.addAll(convertFromPartSpec(msParts.iterator(), tbl)); @@ -5260,6 +5285,16 @@ private void constructOneLBLocationMap(FileStatus fSta, } /** + * Sets the metastore client for the current thread + * @throws MetaException + */ + @VisibleForTesting + public synchronized void setMSC(IMetaStoreClient client) + throws MetaException { + metaStoreClient = client; + } + + /** * @return the metastore client for the current thread * @throws MetaException */ diff --git a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestTxnManager.java b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestTxnManager.java new file mode 100644 index 0000000..e1e09ec --- /dev/null +++ b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestTxnManager.java @@ -0,0 +1,71 @@ +/* + * 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.ql.lockmgr; + +import org.apache.hadoop.hive.common.FileUtils; +import org.apache.hadoop.hive.common.ValidTxnWriteIdList; +import org.apache.hadoop.hive.metastore.api.CommitTxnRequest; +import org.apache.hadoop.hive.metastore.api.GetOpenTxnsResponse; +import org.apache.hadoop.hive.metastore.api.TxnToWriteId; +import org.apache.hadoop.hive.metastore.api.TxnType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.hive.common.ValidTxnList; +import org.apache.hadoop.hive.common.ValidReadTxnList; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.ErrorMsg; +import org.apache.hadoop.hive.ql.DriverState; +import org.apache.hadoop.hive.ql.QueryPlan; +import org.apache.hadoop.hive.ql.hooks.ReadEntity; +import org.apache.hadoop.hive.ql.hooks.WriteEntity; +import org.apache.hadoop.hive.ql.metadata.DummyPartition; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.Partition; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.util.ReflectionUtils; + +import java.util.*; + +/** + * An implementation of {@link HiveTxnManager} that does not support + * transactions. + * This class is only used in test. + */ +class TestTxnManager extends DummyTxnManager { + final static Character COLON = ':'; + final static Character DOLLAR = '$'; + + + @Override + public long getCurrentTxnId() { + return 1L; + } + + @Override + public ValidTxnWriteIdList getValidWriteIds(List<String> tableList, + String validTxnList) throws LockException { + // Format : <txnId>$<table_name>:<hwm>:<minOpenWriteId>:<open_writeids>:<abort_writeids> + return new ValidTxnWriteIdList(getCurrentTxnId() + DOLLAR.toString() + "db.table" + COLON + + getCurrentTxnId() + COLON + + getCurrentTxnId() + COLON + + getCurrentTxnId() + COLON); + } +} + diff --git a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreClientApiArgumentsChecker.java b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreClientApiArgumentsChecker.java new file mode 100644 index 0000000..def25ab --- /dev/null +++ b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreClientApiArgumentsChecker.java @@ -0,0 +1,141 @@ +/* + * 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.ql.metadata; + +import com.google.common.collect.Lists; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hive.common.ValidTxnList; +import org.apache.hadoop.hive.common.ValidWriteIdList; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.TableType; +import org.apache.hadoop.hive.metastore.TestHiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; +import org.apache.hadoop.hive.metastore.conf.MetastoreConf; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrGreaterThan; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.apache.thrift.TException; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * TestHiveMetaStoreClientApiArgumentsChecker + * + * This class works with {@link TestHiveMetaStoreClient} in order to verify the arguments that + * are sent from HS2 to HMS APIs. + * + */ +public class TestHiveMetaStoreClientApiArgumentsChecker { + + private Hive hive; + private IMetaStoreClient msc; + private FileSystem fs; + final static String DB_NAME = "db"; + final static String TABLE_NAME = "table"; + private IMetaStoreClient client; + private Table t; + + protected static final String USER_NAME = "user0"; + + @Before + public void setUp() throws Exception { + + client = new TestHiveMetaStoreClient(new HiveConf(Hive.class)); + hive = Hive.get(client); + hive.getConf().set(MetastoreConf.ConfVars.FS_HANDLER_THREADS_COUNT.getVarname(), "15"); + hive.getConf().set(MetastoreConf.ConfVars.MSCK_PATH_VALIDATION.getVarname(), "throw"); + msc = new HiveMetaStoreClient(hive.getConf()); + + hive.getConf().setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, + "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); + HiveConf.setBoolVar(hive.getConf(), HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false); + hive.getConf().set(ValidTxnList.VALID_TXNS_KEY, "1:"); + hive.getConf().set(ValidWriteIdList.VALID_WRITEIDS_KEY, TABLE_NAME + ":1:"); + hive.getConf().setVar(HiveConf.ConfVars.HIVE_TXN_MANAGER, "org.apache.hadoop.hive.ql.lockmgr.TestTxnManager"); + SessionState.start(hive.getConf()); + SessionState.get().initTxnMgr(hive.getConf()); + Context ctx = new Context(hive.getConf()); + SessionState.get().getTxnMgr().openTxn(ctx, USER_NAME); + + t = new Table(); + org.apache.hadoop.hive.metastore.api.Table tTable = new org.apache.hadoop.hive.metastore.api.Table(); + tTable.setId(Long.MAX_VALUE); + t.setTTable(tTable); + Map<String, String> parameters = new HashMap<>(); + parameters.put(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, "true"); + tTable.setParameters(parameters); + tTable.setTableType(TableType.MANAGED_TABLE.toString()); + t.setTableName(TABLE_NAME); + t.setDbName(DB_NAME); + + List<FieldSchema> partCols = new ArrayList<>(); + partCols.add(new FieldSchema()); + t.setPartCols(partCols); + + } + + @Test + public void testGetPartition() throws HiveException { + hive.getPartition(t, DB_NAME, TABLE_NAME, null); + } + + @Test + public void testGetPartitions() throws HiveException { + hive.getPartitions(t); + } + + @Test + public void testGetPartitionNames() throws HiveException { + hive.getPartitionNames(DB_NAME, TABLE_NAME, null, (short) -1); + } + + @Test + public void testGetPartitionNames2() throws HiveException { + hive.getPartitionNames(t, null, null, (short) -1); + } + + @Test + public void testGetPartitionsByExpr() throws HiveException, TException { + List<Partition> partitions = new ArrayList<>(); + ExprNodeDesc column = new ExprNodeColumnDesc(TypeInfoFactory.stringTypeInfo, "rid", + null, false); + ExprNodeDesc constant = new ExprNodeConstantDesc(TypeInfoFactory.stringTypeInfo, "f"); + List<ExprNodeDesc> children = Lists.newArrayList(); + children.add(column); + children.add(constant); + ExprNodeGenericFuncDesc node = + new ExprNodeGenericFuncDesc(TypeInfoFactory.stringTypeInfo, + new GenericUDFOPEqualOrGreaterThan(), children); + hive.getPartitionsByExpr(t, node, hive.getConf(), partitions); + } +} + + diff --git a/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreClient.java b/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreClient.java new file mode 100644 index 0000000..5d7ca15 --- /dev/null +++ b/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreClient.java @@ -0,0 +1,119 @@ +/* + * 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.metastore; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClientWithLocalCache; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.GetPartitionNamesPsRequest; +import org.apache.hadoop.hive.metastore.api.GetPartitionNamesPsResponse; +import org.apache.hadoop.hive.metastore.api.GetPartitionRequest; +import org.apache.hadoop.hive.metastore.api.GetPartitionResponse; +import org.apache.hadoop.hive.metastore.api.GetPartitionsPsWithAuthRequest; +import org.apache.hadoop.hive.metastore.api.GetPartitionsPsWithAuthResponse; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.PartitionSpec; +import org.apache.hadoop.hive.metastore.api.PartitionsByExprRequest; +import org.apache.hadoop.hive.metastore.api.SerDeInfo; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; +import org.apache.thrift.TException; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertNotNull; + +/** + * TestHiveMetaStoreClient + * + * This class provides implementation of certain methods from IMetaStoreClient interface. + * It's mainly used to verify the arguments that are sent from HS2 to HMS APIs. + * This class' methods are used when HS2 methods are invoked from + * {@link TestHiveMetaStoreClientApiArgumentsChecker}. + * + * Tests in this class ensure that both tableId and validWriteIdList are sent from HS2 in the input of + * HMS get_* APIs. + * tableId and validWriteIdList are used to determine whether the response of a given get_* API should + * be served from the cache or the backing DB. + * So, if we want consistent read from cache, it is important to send both validWriteIdList and tableId + * in the get_* API request, so that we can compare them and decide where to send the data from. + * Right now only few get_* APIs take validWriteIdList and tableId in the input. + * As we support more APIs, we should add them here with appropriate test cases. + * + */ +public class TestHiveMetaStoreClient extends HiveMetaStoreClientWithLocalCache implements IMetaStoreClient { + + public TestHiveMetaStoreClient(Configuration conf) throws MetaException { + super(conf); + } + + public GetPartitionResponse getPartitionRequest(GetPartitionRequest req) + throws NoSuchObjectException, MetaException, TException { + assertNotNull(req.getId()); + assertNotNull(req.getValidWriteIdList()); + GetPartitionResponse res = new GetPartitionResponse(); + return res; + } + + public GetPartitionNamesPsResponse listPartitionNamesRequest(GetPartitionNamesPsRequest req) + throws NoSuchObjectException, MetaException, TException { + assertNotNull(req.getId()); + assertNotNull(req.getValidWriteIdList()); + GetPartitionNamesPsResponse res = new GetPartitionNamesPsResponse(); + return res; + } + + public GetPartitionsPsWithAuthResponse listPartitionsWithAuthInfoRequest(GetPartitionsPsWithAuthRequest req) + throws MetaException, TException, NoSuchObjectException { + assertNotNull(req.getId()); + assertNotNull(req.getValidWriteIdList()); + GetPartitionsPsWithAuthResponse res = new GetPartitionsPsWithAuthResponse(); + return res; + + } + + public boolean listPartitionsSpecByExpr(PartitionsByExprRequest req, List<PartitionSpec> result) throws TException { + assertNotNull(req.getId()); + assertNotNull(req.getValidWriteIdList()); + return false; + } + + public Table getTable(String dbName, String tableName, boolean getColumnStats, String engine) + throws MetaException, TException, NoSuchObjectException { + Table tTable = new Table(); + tTable.setId(Long.MAX_VALUE); + Map<String, String> parameters = new HashMap<>(); + parameters.put(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, "true"); + tTable.setParameters(parameters); + Map<String, String> sdParameters = new HashMap<>(); + StorageDescriptor sd = new StorageDescriptor(); + sd.setParameters(sdParameters); + SerDeInfo si = new SerDeInfo(); + sd.setSerdeInfo(si); + tTable.setSd(sd); + + return tTable; + + } + +}