[hive] branch master updated: HIVE-26071: JWT authentication mechanism for Thrift over HTTP in HiveMetastore (#3233) (Sourabh Goyal, reviewed by Yu-wen, Deng and Sai)
This is an automated email from the ASF dual-hosted git repository. ychena 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 d30db8cbaf HIVE-26071: JWT authentication mechanism for Thrift over HTTP in HiveMetastore (#3233) (Sourabh Goyal, reviewed by Yu-wen, Deng and Sai) d30db8cbaf is described below commit d30db8cbafba110f6519354df7504b36643a8e60 Author: Sourabh Goyal AuthorDate: Thu May 12 11:40:56 2022 -0700 HIVE-26071: JWT authentication mechanism for Thrift over HTTP in HiveMetastore (#3233) (Sourabh Goyal, reviewed by Yu-wen, Deng and Sai) What changes were proposed in this pull request? This PR is a follow up of #3105. It adds a support for JWT authentication in HiveMetastore server when run in HTTP transport mode. Why are the changes needed? It supports a new authentication mechanism ie JWT in HiveMetastore server. Does this PR introduce any user-facing change? No How was this patch tested? Added new unit tests that cover cases like successfully authenticating valid JWT failing to authenticate expired, invalid JWTs * Add JWTValidator and URLBasedJWKSProvider code from HS2 Change-Id: I969f57daf640adb16f228e95b1b522f8ffc24ffe * Add JWT authentication in HiveMetastore Change-Id: I6d84517a1ee97df492ad3816ec866c0b785ed5ed * Better error handling for authentication failures. Added integration tests for validating JWT Change-Id: I6b9da531db4e4a805d8daa1ba6d941c5643bf514 * Added test JWTs for jwt authentication tests Change-Id: Ice36a703d8af7d4dbf28a48c9bb96127100fd8c7 * moved jwt test keys under jwt directory Change-Id: I8bf0b4bbc101a0acb3f69bb1963b9c4bcda5b719 * Fixes failures in metastore jwt unit tests Change-Id: I2877730a34dff7d3184b100ec04031032611838a * Addresses review comments Change-Id: I8498e85212476c663cf735211848a28baaa3bad5 * Addresses nits from review comments Change-Id: Id67588c106104732a0f6e49e5c983cb5f7287c3e * Added more comments in the code Change-Id: Ia51f490362985d109778a6a0aa92a281436d5d21 * removes unsed import statement Change-Id: I94633bdce0db87a9085968dde79d8ff6cd9bf4a3 --- .../hadoop/hive/metastore/HiveMetaStoreClient.java | 57 +++-- .../hadoop/hive/metastore/conf/MetastoreConf.java | 15 +- standalone-metastore/metastore-server/pom.xml | 16 ++ .../hadoop/hive/metastore/HiveMetaStore.java | 2 +- .../hive/metastore/HmsThriftHttpServlet.java | 156 .../auth/HttpAuthenticationException.java | 47 .../hive/metastore/auth/jwt/JWTValidator.java | 109 .../metastore/auth/jwt/URLBasedJWKSProvider.java | 88 +++ .../TestRemoteHiveMetastoreWithHttpJwt.java| 283 + .../resources/auth/jwt/jwt-authorized-key.json | 12 + .../resources/auth/jwt/jwt-unauthorized-key.json | 12 + .../resources/auth/jwt/jwt-verification-jwks.json | 20 ++ standalone-metastore/pom.xml | 3 + 13 files changed, 746 insertions(+), 74 deletions(-) diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java index 651546caa5..db8477d89b 100644 --- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java +++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java @@ -80,6 +80,7 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.StringUtils; import org.apache.http.HttpException; +import org.apache.http.HttpHeaders; import org.apache.http.HttpRequest; import org.apache.http.HttpRequestInterceptor; import org.apache.http.impl.client.HttpClientBuilder; @@ -601,32 +602,52 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable { client.rename_partition_req(req); } + /* + Creates a THttpClient if HTTP mode is enabled. If Client auth mode is set to JWT, + then the method fetches JWT from environment variable: HMS_JWT and sets in auth + header in http request + */ private THttpClient createHttpClient(URI store, boolean useSSL) throws MetaException, TTransportException { String path = MetaStoreUtils.getHttpPath(MetastoreConf.getVar(conf, ConfVars.THRIFT_HTTP_PATH)); String httpUrl = (useSSL ? "https://; : "http://;) + store.getHost() + ":" + store.getPort() + path; -String user = MetastoreConf.getVar(conf, ConfVars.METASTORE_CLIENT_PLAIN_USERNAME); -
[hive] branch master updated: HIVE-21456: Thrift over Http for Hive Metastore (#3105) (Sourabh Goyal, reviewed by Sai Gantasala, Peter Vary, Naveen and Yongzhi)
This is an automated email from the ASF dual-hosted git repository. ychena 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 b7da71856b HIVE-21456: Thrift over Http for Hive Metastore (#3105) (Sourabh Goyal, reviewed by Sai Gantasala, Peter Vary, Naveen and Yongzhi) b7da71856b is described below commit b7da71856b1bb51af68a5ba6890b65f9843f3606 Author: Sourabh Goyal AuthorDate: Wed Apr 20 06:33:37 2022 -0700 HIVE-21456: Thrift over Http for Hive Metastore (#3105) (Sourabh Goyal, reviewed by Sai Gantasala, Peter Vary, Naveen and Yongzhi) * [WIP]HIVE-21456: Thrift over Http for Hive Metastore Change-Id: Ie610b7351fe6279353c1f781b0602da0f1860443 * Addresses review comments. Also fixes build failure Change-Id: Idc8dc3448156e7e2715dc9ea979edf007d4d53d4 * fixes test failures Change-Id: Ibf6210985248f88cf7011b048703e95fd99dee49 * Refactors creation of Binary and HTTP clients in seprate methods in HiveMetastoreClient Change-Id: Ib080e24eede76104e10458343f85ac746022f16d * Addresses review comments Change-Id: I5ec4fb201bd65bc358c38160348b200fc16d730c * Fixes validation of maxIdleTimeout in metastore http server Change-Id: I52990b3904cd8d42da9cff9f282e1f099323e3d7 * Disabled HTTP TRACE in embedded jetty server in HMS Change-Id: Idcdec4ee0ff7d3ded67816cca4505627a1e5b33b * Addresses review comments Change-Id: Ie046c512f2095b0d71743a9485620e369dc75b17 * Addresses nits. Adds some more comments Change-Id: I39b50cd549af62e5d460fa99167c5aab221edaf8 --- .../java/org/hadoop/hive/jdbc/SSLTestUtils.java| 6 + .../test/java/org/apache/hive/jdbc/TestSSL.java| 24 +- .../hadoop/hive/metastore/HiveMetaStoreClient.java | 352 + .../hadoop/hive/metastore/conf/MetastoreConf.java | 27 ++ .../hive/metastore/utils/MetaStoreUtils.java | 25 ++ .../hadoop/hive/metastore/utils/SecurityUtils.java | 46 +++ .../src/test/resources/log4j2.properties | 11 +- .../hadoop/hive/metastore/HiveMetaStore.java | 285 +++-- .../hive/metastore/HmsThriftHttpServlet.java | 113 +++ ...Store.java => TestRemoteHiveHttpMetaStore.java} | 49 +-- .../hive/metastore/TestRemoteHiveMetaStore.java| 3 + standalone-metastore/pom.xml | 6 + 12 files changed, 752 insertions(+), 195 deletions(-) diff --git a/itests/hive-unit/src/main/java/org/hadoop/hive/jdbc/SSLTestUtils.java b/itests/hive-unit/src/main/java/org/hadoop/hive/jdbc/SSLTestUtils.java index b8e7e3de65..3917a3b457 100644 --- a/itests/hive-unit/src/main/java/org/hadoop/hive/jdbc/SSLTestUtils.java +++ b/itests/hive-unit/src/main/java/org/hadoop/hive/jdbc/SSLTestUtils.java @@ -67,6 +67,12 @@ public class SSLTestUtils { KEY_STORE_TRUST_STORE_PASSWORD); } + public static void setMetastoreHttpsConf(HiveConf conf) { +setMetastoreSslConf(conf); +MetastoreConf.setVar(conf, MetastoreConf.ConfVars.THRIFT_TRANSPORT_MODE, "http"); +MetastoreConf.setVar(conf, MetastoreConf.ConfVars.METASTORE_CLIENT_THRIFT_TRANSPORT_MODE, "http"); + } + public static void clearSslConfOverlay(Map confOverlay) { confOverlay.put(HiveConf.ConfVars.HIVE_SERVER2_USE_SSL.varname, "false"); } diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestSSL.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestSSL.java index 1d170ec309..ec6c65f75a 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestSSL.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestSSL.java @@ -53,7 +53,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@org.junit.Ignore("HIVE-22620") public class TestSSL { private static final Logger LOG = LoggerFactory.getLogger(TestSSL.class); @@ -65,6 +64,7 @@ public class TestSSL { private static final String JAVA_TRUST_STORE_PROP = "javax.net.ssl.trustStore"; private static final String JAVA_TRUST_STORE_PASS_PROP = "javax.net.ssl.trustStorePassword"; private static final String JAVA_TRUST_STORE_TYPE_PROP = "javax.net.ssl.trustStoreType"; + private static final String KEY_MANAGER_FACTORY_ALGORITHM = "SunX509"; private MiniHS2 miniHS2 = null; private static HiveConf conf = new HiveConf(); @@ -290,6 +290,7 @@ public class TestSSL { * Test SSL client connection to SSL server * @throws Exception */ + @Ignore @Test public void testSSLConnectionWithProperty() throws Exception { SSLTestUtils.setSslConfOverlay(confOverlay); @@ -390,6 +391,7 @@ public class TestSSL { * Opening a new connection with this wrong certificate should fail * @throws Exception */ + @Ignore @Test public voi
[hive] branch master updated (1e853b9 -> a62af3a)
This is an automated email from the ASF dual-hosted git repository. ychena pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/hive.git. from 1e853b9 HIVE-21152: Rewrite if expression to case and recognize simple case as an if (#2791) (Zoltan Haindrich reviewed by Krisztian Kasa) add a62af3a HIVE-25957: Fix password based authentication with SAML enabled (#3028) (Yu-Wen Lai, reviewed by Naveen Gangam) No new revisions were added by this update. Summary of changes: .../java/org/apache/hadoop/hive/conf/HiveConf.java | 2 +- .../org/apache/hive/service/auth/AuthType.java | 112 + .../hive/service/cli/thrift/ThriftHttpServlet.java | 59 +-- .../org/apache/hive/service/auth/TestAuthType.java | 112 + .../service/cli/thrift/ThriftHttpServletTest.java | 2 +- 5 files changed, 253 insertions(+), 34 deletions(-) create mode 100644 service/src/java/org/apache/hive/service/auth/AuthType.java create mode 100644 service/src/test/org/apache/hive/service/auth/TestAuthType.java
[hive] branch master updated: HIVE-25887 - Add external_table_concatenate.q to testconfiguration.properties. (#2959) (Harish Jaiprakash, reviewed by Naveen Gangam)
This is an automated email from the ASF dual-hosted git repository. ychena 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 28dc8c1 HIVE-25887 - Add external_table_concatenate.q to testconfiguration.properties. (#2959) (Harish Jaiprakash, reviewed by Naveen Gangam) 28dc8c1 is described below commit 28dc8c17a49f861ec03689369c981097b0daa5d6 Author: Harish Jaiprakash AuthorDate: Mon Jan 24 21:28:19 2022 +0530 HIVE-25887 - Add external_table_concatenate.q to testconfiguration.properties. (#2959) (Harish Jaiprakash, reviewed by Naveen Gangam) --- itests/src/test/resources/testconfiguration.properties | 1 + 1 file changed, 1 insertion(+) diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties index 99901ac..aa99e7e 100644 --- a/itests/src/test/resources/testconfiguration.properties +++ b/itests/src/test/resources/testconfiguration.properties @@ -65,6 +65,7 @@ minillap.query.files=\ except_distinct.q,\ explainanalyze_acid_with_direct_insert.q,\ explainuser_2.q,\ + external_table_concatenate.q,\ external_table_purge.q,\ external_table_with_space_in_location_path.q,\ file_with_header_footer.q,\
[hive] branch master updated: HIVE-25757: Use cached database type to choose metastore backend queries (#2833) (Yongzhi Chen, reviewed by Peter Vary)
This is an automated email from the ASF dual-hosted git repository. ychena 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 08f1ea2 HIVE-25757: Use cached database type to choose metastore backend queries (#2833) (Yongzhi Chen, reviewed by Peter Vary) 08f1ea2 is described below commit 08f1ea28caa4973cc42fd24b4fd08b458391ce2a Author: Yongzhi Chen AuthorDate: Thu Dec 2 08:18:28 2021 -0500 HIVE-25757: Use cached database type to choose metastore backend queries (#2833) (Yongzhi Chen, reviewed by Peter Vary) Co-authored-by: Yongzhi Chen --- .../src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java index a221eef..fa8ee52 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java @@ -5286,8 +5286,8 @@ public class ObjectStore implements RawStore, Configurable { */ try { // HIVE-21075: Fix Postgres performance regression caused by HIVE-9447 - DatabaseProduct dbProduct = DatabaseProduct.determineDatabaseProduct(MetaStoreDirectSql.getProductName(pm), conf); - if (dbProduct.isPOSTGRES() || dbProduct.isMYSQL()) { + LOG.debug("The dbType is {} ", dbType.getHiveSchemaPostfix()); + if (dbType.isPOSTGRES() || dbType.isMYSQL()) { query = pm.newQuery(MStorageDescriptor.class, "this.cd == inCD"); query.declareParameters("MColumnDescriptor inCD"); List referencedSDs = null;
[hive] branch master updated (52a0399 -> 1a6414a)
This is an automated email from the ASF dual-hosted git repository. ychena pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/hive.git. from 52a0399 HIVE-25609: Preserve XAttrs in normal file copy case. (Haymant Mangla, reviewed by Ayush Saxena) add 1a6414a HIVE-21075 : Metastore: Drop partition performance downgrade with Postgres (#2826) (Oleksiy Sayankin, reviewed by Peter Vary, Yongzhi Chen)) No new revisions were added by this update. Summary of changes: .../apache/hadoop/hive/metastore/ObjectStore.java | 67 ++ 1 file changed, 55 insertions(+), 12 deletions(-)
[hive] branch master updated: HIVE-25521 - Fix concatenate file handling when files of different compressions are in same table/partition. (#2639)( Harish Jaiprakash, reviewed by Panagiotis Garefalaki
This is an automated email from the ASF dual-hosted git repository. ychena 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 f0169cd HIVE-25521 - Fix concatenate file handling when files of different compressions are in same table/partition. (#2639)( Harish Jaiprakash, reviewed by Panagiotis Garefalakis) f0169cd is described below commit f0169cdd472b08a77b7a72666899d904f6ce1e9a Author: Harish Jaiprakash AuthorDate: Sat Oct 9 00:04:31 2021 +0530 HIVE-25521 - Fix concatenate file handling when files of different compressions are in same table/partition. (#2639)( Harish Jaiprakash, reviewed by Panagiotis Garefalakis) * HIVE-25521 - Fix concatenate file handling when files of different compressions are in same table/partition. * Split test cases into 2 part, and not creating the reader at all for ignored splits. --- .../ql/io/orc/OrcFileStripeMergeRecordReader.java | 67 +++--- .../io/orc/TestOrcFileStripeMergeRecordReader.java | 34 +-- 2 files changed, 64 insertions(+), 37 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFileStripeMergeRecordReader.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFileStripeMergeRecordReader.java index 2ebfd29..e677842 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFileStripeMergeRecordReader.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFileStripeMergeRecordReader.java @@ -40,19 +40,23 @@ public class OrcFileStripeMergeRecordReader implements protected Iterator iter; protected List stripeStatistics; private int stripeIdx; - private long start; - private long end; private boolean skipFile; public OrcFileStripeMergeRecordReader(Configuration conf, FileSplit split) throws IOException { path = split.getPath(); -start = split.getStart(); -end = start + split.getLength(); -FileSystem fs = path.getFileSystem(conf); -this.reader = OrcFile.createReader(path, OrcFile.readerOptions(conf).filesystem(fs)); -this.iter = reader.getStripes().iterator(); -this.stripeIdx = 0; -this.stripeStatistics = ((ReaderImpl) reader).getOrcProtoStripeStatistics(); +long start = split.getStart(); +// if the combined split has only part of the file split, the entire file will be handled by the mapper that +// owns the start of file split. +skipFile = start > 0; // skip the file if start is not 0 +if (!skipFile) { + FileSystem fs = path.getFileSystem(conf); + this.reader = OrcFile.createReader(path, OrcFile.readerOptions(conf).filesystem(fs)); + this.iter = reader.getStripes().iterator(); + this.stripeIdx = 0; + this.stripeStatistics = ((ReaderImpl) reader).getOrcProtoStripeStatistics(); +} else { + reader = null; +} } public Class getKeyClass() { @@ -90,33 +94,27 @@ public class OrcFileStripeMergeRecordReader implements return true; } -while (iter.hasNext()) { +// file split starts with 0 and hence this mapper owns concatenate of all stripes in the file. +if (iter.hasNext()) { StripeInformation si = iter.next(); - - // if stripe offset is outside the split boundary then ignore the current - // stripe as it will be handled by some other mapper. - if (si.getOffset() >= start && si.getOffset() < end) { -valueWrapper.setStripeStatistics(stripeStatistics.get(stripeIdx++)); -valueWrapper.setStripeInformation(si); -if (!iter.hasNext()) { - valueWrapper.setLastStripeInFile(true); - Map userMeta = new HashMap<>(); - for(String key: reader.getMetadataKeys()) { -userMeta.put(key, reader.getMetadataValue(key)); - } - valueWrapper.setUserMetadata(userMeta); + valueWrapper.setStripeStatistics(stripeStatistics.get(stripeIdx)); + valueWrapper.setStripeInformation(si); + if (!iter.hasNext()) { +valueWrapper.setLastStripeInFile(true); +Map userMeta = new HashMap<>(); +for(String key: reader.getMetadataKeys()) { + userMeta.put(key, reader.getMetadataValue(key)); } -keyWrapper.setInputPath(path); -keyWrapper.setCompression(reader.getCompressionKind()); -keyWrapper.setCompressBufferSize(reader.getCompressionSize()); -keyWrapper.setFileVersion(reader.getFileVersion()); -keyWrapper.setWriterVersion(reader.getWriterVersion()); -keyWrapper.setRowIndexStride(reader.getRowIndexStride()); -keyWrapper.setFileSchema(reader.getSchema()); - } else { -stripeIdx++; -continue; +valueWrapper.setUserMetadata(userMeta); } + keyWrapper.setInputPath(path); + keyWrapper.setCompression(reader.getCompressionKind()); + keyWrapper.setCompressBufferS
[hive] branch master updated (71583e3 -> 6f478eb)
This is an automated email from the ASF dual-hosted git repository. ychena pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/hive.git. from 71583e3 HIVE-25503: Add cleanup for the duplicate COMPLETED_TXN_COMPONENTS entries (Denys Kuzmenko, reviewed by Peter Vary and Krisztian Kasa) add 6f478eb Hive-25460 : Advance the Write ID by default (#2624) (Kishen Das, reviewed by Yu-Wen Lai) No new revisions were added by this update. Summary of changes: ql/src/java/org/apache/hadoop/hive/ql/ddl/DDLDesc.java | 7 ++- 1 file changed, 6 insertions(+), 1 deletion(-)
[hive] branch master updated: HIVE-24936 - Fix file name parsing and copy file move (#2628) (Harish Jaiprakash, reviewed by Kishen Das)
This is an automated email from the ASF dual-hosted git repository. ychena 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 aefbf40 HIVE-24936 - Fix file name parsing and copy file move (#2628) (Harish Jaiprakash, reviewed by Kishen Das) aefbf40 is described below commit aefbf40f0b1b2e07ede073418200f8940f91dba7 Author: Harish Jaiprakash AuthorDate: Mon Sep 13 23:41:45 2021 +0530 HIVE-24936 - Fix file name parsing and copy file move (#2628) (Harish Jaiprakash, reviewed by Kishen Das) * HIVE-24936 - Fix file name parsing and copy file move. * HIVE-25130: handle spark inserted files during alter table concat. --- .../hive/ql/exec/AbstractFileMergeOperator.java| 36 ++--- .../hadoop/hive/ql/exec/ParsedOutputFileName.java | 156 ++ .../org/apache/hadoop/hive/ql/exec/Utilities.java | 168 ++- .../hive/ql/exec/ParsedOutputFileNameTest.java | 177 + .../apache/hadoop/hive/ql/exec/TestUtilities.java | 45 ++ 5 files changed, 473 insertions(+), 109 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractFileMergeOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractFileMergeOperator.java index 86cb983..264573b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractFileMergeOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractFileMergeOperator.java @@ -275,39 +275,33 @@ public abstract class AbstractFileMergeOperator throw new HiveException("Incompatible files should not happen in MM tables."); } Path destDir = finalPath.getParent(); -Path destPath = destDir; // move any incompatible files to final path if (incompatFileSet != null && !incompatFileSet.isEmpty()) { for (Path incompatFile : incompatFileSet) { -// check if path conforms to Hive's file name convention. Hive expects filenames to be in specific format +// Hive expects filenames to be in specific format // like 00_0, but "LOAD DATA" commands can let you add any files to any partitions/tables without -// renaming. This can cause MoveTask to remove files in some cases where MoveTask assumes the files are -// are generated by speculatively executed tasks. +// renaming. +// This can cause a few issues: +// MoveTask will remove files in some cases where MoveTask assumes the files are are generated by +// speculatively executed tasks. // Example: MoveTask thinks the following files are same // part-m-0_1417075294718 // part-m-1_1417075294718 // Assumes 1417075294718 as taskId and retains only large file supposedly generated by speculative execution. -// This can result in data loss in case of CONCATENATE/merging. Filter out files that does not match Hive's -// filename convention. -if (!Utilities.isHiveManagedFile(incompatFile)) { - // rename un-managed files to conform to Hive's naming standard - // Example: - // /warehouse/table/part-m-0_1417075294718 will get renamed to /warehouse/table/.hive-staging/00_0 - // If staging directory already contains the file, taskId_copy_N naming will be used. - final String taskId = Utilities.getTaskId(jc); - Path destFilePath = new Path(destDir, new Path(taskId)); - for (int counter = 1; fs.exists(destFilePath); counter++) { -destFilePath = new Path(destDir, taskId + (Utilities.COPY_KEYWORD + counter)); - } - LOG.warn("Path doesn't conform to Hive's expectation. Renaming {} to {}", incompatFile, destFilePath); - destPath = destFilePath; -} +// This can result in data loss in case of CONCATENATE/merging. +// If filename is consistent with XX_N and another task with same task-id runs after this move, then +// the same file name is used in the other task which will result in task failure and retry of task and +// subsequent removal of this file as duplicate. +// Example: if the file name is 01_0 and another task runs with taskid 01_0, it will fail to create +// the file and next attempt will create 01_1, both the files will be considered as output of same task +// and only 01_1 will be picked resulting it loss of existing file 01_0. +final String destFileName = Utilities.getTaskId(jc) + Utilities.COPY_KEYWORD + 1; try { - Utilities.renameOrMoveFiles(fs, incompatFile, destPath); + Path destPath = Utilit
[hive] branch master updated (ad1f6c5 -> fa42214)
This is an automated email from the ASF dual-hosted git repository. ychena pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/hive.git. from ad1f6c5 HIVE-25477: Clean Up JDBC Code (David Mollitor reviewed by Miklos Gergely) add fa42214 HIVE-25445 - Enable JdbcStorageHandler to get password from AWS Secret Service. (#2584) (Harish Jai Prakash Perumal, reviewed by Naveen Gangam) No new revisions were added by this update. Summary of changes: .../org/apache/hadoop/hive/conf/Constants.java | 1 + .../jdbc/conf/JdbcStorageConfigManager.java| 54 -- .../jdbc/dao/GenericJdbcDatabaseAccessor.java | 25 ++--- pom.xml| 24 + ql/pom.xml | 9 ++ .../org/apache/hadoop/hive/ql/exec/Utilities.java | 51 ++ .../ql/secrets/AWSSecretsManagerSecretSource.java | 109 + .../hadoop/hive/ql/secrets/SecretSource.java | 41 .../hadoop/hive/ql/secrets/URISecretSource.java| 74 ++ .../org.apache.hadoop.hive.ql.secrets.SecretSource | 1 + .../secrets/AWSSecretsManagerSecretSourceTest.java | 74 ++ .../hive/ql/secrets/URISecretSourceTest.java | 87 12 files changed, 488 insertions(+), 62 deletions(-) create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/secrets/AWSSecretsManagerSecretSource.java copy shims/common/src/main/java/org/apache/hadoop/hive/io/HiveIOExceptionHandler.java => ql/src/java/org/apache/hadoop/hive/ql/secrets/SecretSource.java (52%) create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/secrets/URISecretSource.java create mode 100644 ql/src/main/resources/META-INF/services/org.apache.hadoop.hive.ql.secrets.SecretSource create mode 100644 ql/src/test/org/apache/hadoop/hive/ql/secrets/AWSSecretsManagerSecretSourceTest.java create mode 100644 ql/src/test/org/apache/hadoop/hive/ql/secrets/URISecretSourceTest.java
[hive] branch master updated: HIVE-25408: AlterTableSetOwnerAnalyzer should send Hive Privilege Objects for Authorization (#2560) (Sai Hemanth Gantasala, reviewed by Yongzhi Chen)
This is an automated email from the ASF dual-hosted git repository. ychena 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 620b2b1 HIVE-25408: AlterTableSetOwnerAnalyzer should send Hive Privilege Objects for Authorization (#2560) (Sai Hemanth Gantasala, reviewed by Yongzhi Chen) 620b2b1 is described below commit 620b2b197269041d7f508bd0e4564ed8e5edfcfd Author: Sai Hemanth Gantasala <68923650+saihemanth-cloud...@users.noreply.github.com> AuthorDate: Tue Aug 24 12:35:28 2021 -0700 HIVE-25408: AlterTableSetOwnerAnalyzer should send Hive Privilege Objects for Authorization (#2560) (Sai Hemanth Gantasala, reviewed by Yongzhi Chen) --- .../hive/ql/ddl/table/misc/owner/AlterTableSetOwnerAnalyzer.java| 3 +++ ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java| 1 + ql/src/test/results/clientpositive/llap/table_set_owner.q.out | 6 ++ 3 files changed, 10 insertions(+) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/misc/owner/AlterTableSetOwnerAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/misc/owner/AlterTableSetOwnerAnalyzer.java index d2b18fd..420cfab 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/misc/owner/AlterTableSetOwnerAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/misc/owner/AlterTableSetOwnerAnalyzer.java @@ -27,6 +27,8 @@ import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory.DDLType; import org.apache.hadoop.hive.ql.ddl.privilege.PrincipalDesc; import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableAnalyzer; import org.apache.hadoop.hive.ql.exec.TaskFactory; +import org.apache.hadoop.hive.ql.io.AcidUtils; +import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.hadoop.hive.ql.parse.ASTNode; import org.apache.hadoop.hive.ql.parse.HiveParser; import org.apache.hadoop.hive.ql.parse.SemanticException; @@ -55,6 +57,7 @@ public class AlterTableSetOwnerAnalyzer extends AbstractAlterTableAnalyzer { } AlterTableSetOwnerDesc desc = new AlterTableSetOwnerDesc(tableName, ownerPrincipal); +addInputsOutputsAlterTable(tableName, null, desc, desc.getType(), false); rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc), conf)); setAcidDdlDesc(getTable(tableName), desc); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java index f5a8d63..e383c34 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java @@ -226,6 +226,7 @@ public class WriteEntity extends Entity implements Serializable { case TRUNCATE: case MERGEFILES: case DROP_CONSTRAINT: +case OWNER: return WriteType.DDL_EXCLUSIVE; case ADDPARTITION: diff --git a/ql/src/test/results/clientpositive/llap/table_set_owner.q.out b/ql/src/test/results/clientpositive/llap/table_set_owner.q.out index 023ccee..7d7bbd8 100644 --- a/ql/src/test/results/clientpositive/llap/table_set_owner.q.out +++ b/ql/src/test/results/clientpositive/llap/table_set_owner.q.out @@ -8,8 +8,12 @@ POSTHOOK: Output: database:default POSTHOOK: Output: default@t PREHOOK: query: EXPLAIN ALTER TABLE t SET OWNER USER user1 PREHOOK: type: ALTERTABLE_OWNER +PREHOOK: Input: default@t +PREHOOK: Output: default@t POSTHOOK: query: EXPLAIN ALTER TABLE t SET OWNER USER user1 POSTHOOK: type: ALTERTABLE_OWNER +POSTHOOK: Input: default@t +POSTHOOK: Output: default@t STAGE DEPENDENCIES: Stage-0 is a root stage @@ -23,6 +27,8 @@ STAGE PLANS: PREHOOK: query: ALTER TABLE t SET OWNER USER user1 PREHOOK: type: ALTERTABLE_OWNER +PREHOOK: Input: default@t +PREHOOK: Output: default@t POSTHOOK: query: ALTER TABLE t SET OWNER USER user1 POSTHOOK: type: ALTERTABLE_OWNER POSTHOOK: Input: default@t
[hive] branch master updated (9b5feec -> a903002)
This is an automated email from the ASF dual-hosted git repository. ychena pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/hive.git. from 9b5feec HIVE-25410: CommonMergeJoin fails for ARRAY join keys with varying sizes (okumin reviewed by Stamatis Zampetakis) add a903002 HIVE-25423: Add new test driver to automatically launch and load external database (Same as PR#2559)(#2580) (Dantong Dong, reviewed by Sai Hemanth Gantasala and Yongzhi Chen) No new revisions were added by this update. Summary of changes: data/scripts/derby/q_test_extDB_cleanup.derby.sql | 3 + data/scripts/derby/q_test_extDB_init.derby.sql | 27 ++ data/scripts/mysql/q_test_extDB_cleanup.mysql.sql | 3 + data/scripts/mysql/q_test_extDB_init.mysql.sql | 23 ++ .../postgres/q_test_extDB_cleanup.postgres.sql | 3 + .../postgres/q_test_extDB_init.postgres.sql| 23 ++ itests/qtest/pom.xml | 18 ++ .../hive/cli/TestMiniLlapExtDBCliDriver.java | 65 + .../test/resources/testconfiguration.properties| 5 + .../hadoop/hive/cli/control/AbstractCliConfig.java | 5 + .../apache/hadoop/hive/cli/control/CliConfigs.java | 24 ++ .../hadoop/hive/cli/control/CoreCliDriver.java | 5 + .../org/apache/hadoop/hive/ql/QTestArguments.java | 17 ++ .../org/apache/hadoop/hive/ql/QTestExternalDB.java | 57 .../java/org/apache/hadoop/hive/ql/QTestUtil.java | 62 - .../hive/ql/externalDB/AbstractExternalDB.java | 288 + .../hadoop/hive/ql/externalDB/MySQLExternalDB.java | 59 + .../hive/ql/externalDB/PostgresExternalDB.java | 67 + .../queries/clientpositive/dataconnector_mysql.q | 19 ++ .../clientpositive/llap/dataconnector_mysql.q.out | 68 + .../dataconnector/jdbc/MySQLConnectorProvider.java | 2 +- .../ptest2/conf/deployed/master-mr2.properties | 9 +- 22 files changed, 849 insertions(+), 3 deletions(-) create mode 100644 data/scripts/derby/q_test_extDB_cleanup.derby.sql create mode 100644 data/scripts/derby/q_test_extDB_init.derby.sql create mode 100644 data/scripts/mysql/q_test_extDB_cleanup.mysql.sql create mode 100644 data/scripts/mysql/q_test_extDB_init.mysql.sql create mode 100644 data/scripts/postgres/q_test_extDB_cleanup.postgres.sql create mode 100644 data/scripts/postgres/q_test_extDB_init.postgres.sql create mode 100644 itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMiniLlapExtDBCliDriver.java create mode 100644 itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestExternalDB.java create mode 100644 itests/util/src/main/java/org/apache/hadoop/hive/ql/externalDB/AbstractExternalDB.java create mode 100644 itests/util/src/main/java/org/apache/hadoop/hive/ql/externalDB/MySQLExternalDB.java create mode 100644 itests/util/src/main/java/org/apache/hadoop/hive/ql/externalDB/PostgresExternalDB.java create mode 100644 ql/src/test/queries/clientpositive/dataconnector_mysql.q create mode 100644 ql/src/test/results/clientpositive/llap/dataconnector_mysql.q.out
[hive] branch master updated (7f99b5f -> 773b929)
This is an automated email from the ASF dual-hosted git repository. ychena pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/hive.git. from 7f99b5f HIVE-25391: Turn off priv synchronizer by default (#2535) add 773b929 HIVE-25380: Remove the Hive Privilege Object for Database in the ReadTableEvent and CreateTableEvent (#2525) (Sai Hemanth Gantasala, reviewed by Yongzhi Chen) No new revisions were added by this update. Summary of changes: .../authorization/plugin/metastore/events/CreateTableEvent.java| 3 --- .../security/authorization/plugin/metastore/events/ReadTableEvent.java | 2 -- 2 files changed, 5 deletions(-)
[hive] branch master updated (588d44d -> 7eb5ab0)
This is an automated email from the ASF dual-hosted git repository. ychena pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/hive.git. from 588d44d HIVE-25224: Multi insert statements involving tables with different bucketing_versions results in error (#2381) (Zoltan Haindrich reviewed by Krisztian Kasa) add 7eb5ab0 HIVE-25238: Make SSL cipher suites configurable for Hive Web UI and HS2 (#2385) (Yongzhi Chen, reviewed by Naveen Gangam) No new revisions were added by this update. Summary of changes: .../hadoop/hive/common/auth/HiveAuthUtils.java | 21 ++--- .../java/org/apache/hadoop/hive/conf/HiveConf.java | 12 ++-- .../src/java/org/apache/hive/http/HttpServer.java | 14 ++ .../service/cli/thrift/ThriftBinaryCLIService.java | 5 - .../service/cli/thrift/ThriftHttpCLIService.java| 12 .../org/apache/hive/service/server/HiveServer2.java | 2 ++ .../server/TestHS2HttpServerPamConfiguration.java | 3 +++ 7 files changed, 63 insertions(+), 6 deletions(-)
[hive] branch master updated (fadaaa9 -> 799b2c2)
This is an automated email from the ASF dual-hosted git repository. ychena pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/hive.git. from fadaaa9 HIVE-25201: Remove Caffein shading from Iceberg (Peter Vary reviewed by Laszlo Pinter) add 799b2c2 HIVE-25211: Create database throws NPE (#2362) (Yongzhi Chen, reviewed by Naveen Gangam) No new revisions were added by this update. Summary of changes: .../src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-)
[hive] branch master updated (93c5800 -> eff0edd)
This is an automated email from the ASF dual-hosted git repository. ychena pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/hive.git. from 93c5800 HIVE-25113: Connection starvation in TxnHandler.getValidWriteIds (#2272) (Yu-Wen Lai, reviewed by Denys Kuzmenko and Kishen Das ) add eff0edd HIVE-25132: HiveOperationType should be ReadDatabases for default policy in ranger (#2294) (Sai Hemanth Gantasala, reviewed by Yongzhi Chen) No new revisions were added by this update. Summary of changes: .../authorization/plugin/metastore/events/ReadDatabaseEvent.java| 2 +- 1 file changed, 1 insertion(+), 1 deletion(-)
[hive] branch master updated (da1a4e6 -> 93c5800)
This is an automated email from the ASF dual-hosted git repository. ychena pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/hive.git. from da1a4e6 HIVE-25080: Create metric about oldest entry in "ready for cleaning" state (Antal Sinkovits, reviewed by Karen Coppage) add 93c5800 HIVE-25113: Connection starvation in TxnHandler.getValidWriteIds (#2272) (Yu-Wen Lai, reviewed by Denys Kuzmenko and Kishen Das ) No new revisions were added by this update. Summary of changes: .../txn/TestTxnHandlerWithOneConnection.java | 77 ++ .../hadoop/hive/metastore/txn/TxnHandler.java | 30 +++-- 2 files changed, 100 insertions(+), 7 deletions(-) create mode 100644 ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandlerWithOneConnection.java
[hive] branch master updated (c852460 -> e78d728)
This is an automated email from the ASF dual-hosted git repository. ychena pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/hive.git. from c852460 HIVE-25107: Classpath logging should be on DEBUG level (#2271) (Laszlo Bodor reviewed by Panagiotis Garefalakis) add e78d728 HIVE-25121: Fix qfile results due to disabling discovery.partitions (#2279) (Yu-Wen Lai, reviewed by Yongzhi Chen) No new revisions were added by this update. Summary of changes: .../test/results/positive/alter_multi_part_table_to_iceberg.q.out | 6 -- .../src/test/results/positive/alter_part_table_to_iceberg.q.out | 6 -- .../test/results/clientpositive/llap/create_table_explain_ddl.q.out | 6 -- 3 files changed, 18 deletions(-)
[hive] branch master updated (eb5e904 -> 1e8cc12)
This is an automated email from the ASF dual-hosted git repository. ychena pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/hive.git. from eb5e904 HIVE-25039: Disabling discovery.partitions config for external tables (#2201) (Sai Hemanth Gantasala, reviewed by Yongzhi Chen) add 1e8cc12 HIVE-24816: Jackson version changed to 2.11.0 in the pom files (#2075) (Sai Hemanth Gantasala, reviewed by Yongzhi Chen) No new revisions were added by this update. Summary of changes: .../test/java/org/apache/hadoop/hive/cli/TestMiniDruidCliDriver.java | 3 ++- .../java/org/apache/hadoop/hive/cli/TestMiniDruidKafkaCliDriver.java | 3 ++- pom.xml| 2 +- ql/src/test/queries/clientpositive/druid_timestamptz2.q| 1 + ql/src/test/queries/clientpositive/druidkafkamini_avro.q | 1 + ql/src/test/queries/clientpositive/druidkafkamini_csv.q| 1 + ql/src/test/queries/clientpositive/druidmini_extractTime.q | 1 + ql/src/test/queries/clientpositive/druidmini_floorTime.q | 1 + ql/src/test/queries/clientpositive/druidmini_mv.q | 1 + .../queries/clientpositive/druidmini_semijoin_reduction_all_types.q| 1 + ql/src/test/queries/clientpositive/druidmini_test1.q | 1 + ql/src/test/queries/clientpositive/druidmini_test_alter.q | 1 + ql/src/test/queries/clientpositive/druidmini_test_insert.q | 1 + ql/src/test/queries/clientpositive/druidmini_test_ts.q | 1 + standalone-metastore/pom.xml | 2 +- testutils/ptest2/conf/deployed/master-mr2.properties | 1 + testutils/ptest2/pom.xml | 2 +- 17 files changed, 19 insertions(+), 5 deletions(-)
[hive] branch master updated (8f4ee59 -> eb5e904)
This is an automated email from the ASF dual-hosted git repository. ychena pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/hive.git. from 8f4ee59 HIVE-25105: Support Parquet as MV storage format (Jesus Camacho Rodriguez, reviewed by Krisztian Kasa) add eb5e904 HIVE-25039: Disabling discovery.partitions config for external tables (#2201) (Sai Hemanth Gantasala, reviewed by Yongzhi Chen) No new revisions were added by this update. Summary of changes: .../org/apache/hadoop/hive/ql/ddl/table/create/CreateTableDesc.java | 5 - .../hive/ql/ddl/table/create/like/CreateTableLikeOperation.java | 5 - ql/src/test/results/clientpositive/llap/create_like2.q.out | 2 -- ql/src/test/results/clientpositive/llap/external_table_purge.q.out | 2 -- ql/src/test/results/clientpositive/llap/partition_discovery.q.out | 6 -- .../results/clientpositive/llap/partition_params_postgres.q.out | 1 - .../clientpositive/llap/rename_external_partition_location.q.out| 2 -- ql/src/test/results/clientpositive/llap/repl_2_exim_basic.q.out | 2 -- .../results/clientpositive/llap/show_create_table_partitioned.q.out | 1 - 9 files changed, 26 deletions(-)
[hive] branch master updated (ae3be9a -> 7b0a4b7)
This is an automated email from the ASF dual-hosted git repository. ychena pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/hive.git. from ae3be9a HIVE-24958: Create new iceberg-catalog module (Marton Bod reviewed by Laszlo Pinter and Peter Vary) add 7b0a4b7 HIVE-24876: Disable /logconf.jsp page in HS2 web UI if the user doesn… (#2063) No new revisions were added by this update. Summary of changes: service/src/resources/hive-webapps/hiveserver2/logconf.jsp | 9 +++-- 1 file changed, 7 insertions(+), 2 deletions(-)
[hive] branch master updated (2454a3e -> 12b99f7)
This is an automated email from the ASF dual-hosted git repository. ychena pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/hive.git. from 2454a3e HIVE-24868: Support specifying Respect/Ignore Nulls in function parameter list (Krisztian Kasa, reviewed by Zoltan Haindrich) add 12b99f7 HIVE-24890: Upgrade the cron-utils library from 8.1.1 to 9.1.3 because of CVE-2020-26238 (#2081) (Abhay Chennagiri, reviewed by Zoltan Haindrich) No new revisions were added by this update. Summary of changes: standalone-metastore/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-)
[hive] branch master updated (75fec20 -> fae1c83)
This is an automated email from the ASF dual-hosted git repository. ychena pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/hive.git. from 75fec20 HIVE-24809: Build failure while resolving javax.el dependency (#2003) (Stamatis Zampetakis reviewed by Zoltan Haindrich) add fae1c83 HIVE-24768: Using jackson bom everywhere, so that it'll be easy to ma… (#1969) (Sai Hemanth Gantasala, reviewed by Kevin Risden) No new revisions were added by this update. Summary of changes: itests/qtest-druid/pom.xml | 2 -- standalone-metastore/pom.xml | 12 ++-- testutils/ptest2/pom.xml | 10 +++--- 3 files changed, 17 insertions(+), 7 deletions(-)
[hive] branch master updated (b63a7df -> b5b16d7)
This is an automated email from the ASF dual-hosted git repository. ychena pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/hive.git. from b63a7df HIVE-23882: Compiler extensions for MJ probe optimization add b5b16d7 HIVE-24743: Send tableId in get_partitions_by_names HMS API (#1956) (Kishen Das, reviewed by Yongzhi Chen) No new revisions were added by this update. Summary of changes: .../add/AlterTableAddPartitionOperation.java | 2 +- .../org/apache/hadoop/hive/ql/metadata/Hive.java | 42 +++-- ...TestHiveMetaStoreClientApiArgumentsChecker.java | 26 .../hadoop/hive/metastore/HiveMetaStoreClient.java | 58 - .../hadoop/hive/metastore/IMetaStoreClient.java| 72 ++ .../hive/metastore/TestHiveMetaStoreClient.java| 10 +++ .../metastore/HiveMetaStoreClientPreCatalog.java | 27 7 files changed, 228 insertions(+), 9 deletions(-)
[hive] branch master updated (12eb9c1 -> 2e59376)
This is an automated email from the ASF dual-hosted git repository. ychena pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/hive.git. from 12eb9c1 HIVE-24744: Deletion of previous dump dir fails with NPE for ptests (Arko Sharma, reviewed by Aasha Medhi) add 2e59376 HIVE-24456: Added HiveConf to check if FIPS mode is enabled, if so column masking should be done with SHA512 (#1721) (Sai Hemanth Gantasala, reviewed by Yongzhi Chen) No new revisions were added by this update. Summary of changes: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java| 3 +++ common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java| 4 .../apache/hadoop/hive/ql/udf/generic/GenericUDFMaskHash.java| 9 +++-- ql/src/test/queries/clientpositive/udf_mask_hash.q | 3 +++ ql/src/test/results/clientpositive/llap/udf_mask_hash.q.out | 9 + 5 files changed, 26 insertions(+), 2 deletions(-)
[hive] branch master updated (63ea278 -> 8d87112)
This is an automated email from the ASF dual-hosted git repository. ychena pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/hive.git. from 63ea278 HIVE-24674. Set repl.source.for property in the db if db is under repication (Ayush Saxena, reviewed by Aasha Medhi) add 8d87112 HIVE-24392: Add table id and getFileMetadata to GetPartitionsByNamesRequest (#1909) (Yongzhi Chen, reviewed by reviewed by Kishen Das) No new revisions were added by this update. Summary of changes: .../gen/thrift/gen-cpp/hive_metastore_types.cpp| 44 + .../src/gen/thrift/gen-cpp/hive_metastore_types.h | 20 +- .../metastore/api/GetPartitionsByNamesRequest.java | 206 - .../metastore/GetPartitionsByNamesRequest.php | 48 + .../src/gen/thrift/gen-py/hive_metastore/ttypes.py | 26 ++- .../src/gen/thrift/gen-rb/hive_metastore_types.rb | 6 +- .../src/main/thrift/hive_metastore.thrift | 4 +- 7 files changed, 345 insertions(+), 9 deletions(-)
[hive] branch master updated (c357ebf -> cd37d36)
This is an automated email from the ASF dual-hosted git repository. ychena pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/hive.git. from c357ebf HIVE-24656: CBO fails for queries with is null on map and array types (Adam Szita, reviewed by Zoltan Haindrich) add cd37d36 HIVE-24500: Bump up the log4j version from 2.12.1 to 2.13.2 to avoid CVE-2020-9488 (#1754)(Sai Hemanth Gantasala, reviewed by Yongzhi Chen) No new revisions were added by this update. Summary of changes: pom.xml | 2 +- standalone-metastore/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-)
[hive] branch master updated (fe23eaf -> 347a662)
This is an automated email from the ASF dual-hosted git repository. ychena pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/hive.git. from fe23eaf HIVE-24550: Cleanup only transaction information for the current DriverContext (Peter Varga via Peter Vary) add 347a662 HIVE-24570: Hive on spark tmp file should be delete when driver process finished (#1816) (Long Zhao, reviewed by Yongzhi Chen) No new revisions were added by this update. Summary of changes: .../java/org/apache/hive/spark/client/AbstractSparkClient.java | 7 ++- 1 file changed, 6 insertions(+), 1 deletion(-)
[hive] branch master updated (4846df0 -> ecdcc7c)
This is an automated email from the ASF dual-hosted git repository. ychena pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/hive.git. from 4846df0 HIVE-24535: Cleanup AcidUtils.Directory and remove unnecessary filesystem listing (Peter Varga, reviewed by Karen Coppage) add ecdcc7c HIVE-24552: Possible HMS connections leak or accumulation in loadDynamicPartitions (#1802) (Yongzhi Chen, reviewed by László Pintér) No new revisions were added by this update. Summary of changes: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java | 3 +++ 1 file changed, 3 insertions(+)
[hive] branch master updated: HIVE-24342: isPathEncrypted should make sure resolved path also from HDFS (Uma Maheswara Rao Gangumalla, reviewed by Yongzhi Chen) (#1639)
This is an automated email from the ASF dual-hosted git repository. ychena 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 fdc2c5c HIVE-24342: isPathEncrypted should make sure resolved path also from HDFS (Uma Maheswara Rao Gangumalla, reviewed by Yongzhi Chen) (#1639) fdc2c5c is described below commit fdc2c5c9c8e3a22f68a160144003a98ac95d76bd Author: Uma Maheswara Rao G AuthorDate: Tue Nov 3 19:50:37 2020 -0800 HIVE-24342: isPathEncrypted should make sure resolved path also from HDFS (Uma Maheswara Rao Gangumalla, reviewed by Yongzhi Chen) (#1639) --- .../apache/hadoop/hive/shims/Hadoop23Shims.java| 29 +- 1 file changed, 28 insertions(+), 1 deletion(-) diff --git a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java index 858a799..3825434 100644 --- a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java +++ b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java @@ -1236,13 +1236,40 @@ public class Hadoop23Shims extends HadoopShimsSecure { } else { fullPath = path.getFileSystem(conf).makeQualified(path); } - if(!"hdfs".equalsIgnoreCase(path.toUri().getScheme())) { + if (!isFileInHdfs(path.getFileSystem(conf), path)) { return false; } return (getEncryptionZoneForPath(fullPath) != null); } +/** + * Returns true if the given fs supports mount functionality. In general we + * can have child file systems only in the case of mount fs like + * ViewFsOverloadScheme or ViewDistributedFileSystem. Returns false if the + * getChildFileSystems API returns null. + */ +private boolean isMountedFs(FileSystem fs) { + return fs.getChildFileSystems() != null; +} + +private boolean isFileInHdfs(FileSystem fs, Path path) throws IOException { + String hdfsScheme = "hdfs"; + boolean isHdfs = hdfsScheme.equalsIgnoreCase(path.toUri().getScheme()); + // The ViewHDFS supports that, any non-hdfs paths can be mounted as hdfs + // paths. Here HDFSEncryptionShim actually works only for hdfs paths. But + // in the case of ViewHDFS, paths can be with hdfs scheme, but they might + // actually resolve to other fs. + // ex: hdfs://ns1/test ---> o3fs://b.v.ozone1/test + // So, we need to lookup where the actual file is to know the filesystem + // in use. The resolvePath is a sure shot way of knowing which file system + // the file is. + if (isHdfs && isMountedFs(fs)) { +isHdfs = hdfsScheme.equals(fs.resolvePath(path).toUri().getScheme()); + } + return isHdfs; +} + public EncryptionZone getEncryptionZoneForPath(Path path) throws IOException { if (path.getFileSystem(conf).exists(path)) { return hdfsAdmin.getEncryptionZoneForPath(path);
[hive] branch master updated: HIVE-24292: hive webUI should support keystoretype by config (Yongzhi Chen, reviewed by Kevin Risden) (#1594)
This is an automated email from the ASF dual-hosted git repository. ychena 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 377c874 HIVE-24292: hive webUI should support keystoretype by config (Yongzhi Chen, reviewed by Kevin Risden) (#1594) 377c874 is described below commit 377c8741b7714d37fcff82ecb38ead1b7dcce890 Author: Yongzhi Chen AuthorDate: Thu Oct 22 12:44:19 2020 -0400 HIVE-24292: hive webUI should support keystoretype by config (Yongzhi Chen, reviewed by Kevin Risden) (#1594) Add: hive.server2.webui.keystore.type hive.server2.webui.keymanagerfactory.algorithm * Fix test after review. --- .../java/org/apache/hadoop/hive/conf/HiveConf.java| 4 common/src/java/org/apache/hive/http/HttpServer.java | 19 ++- .../org/apache/hive/service/server/HiveServer2.java | 3 +++ .../server/TestHS2HttpServerPamConfiguration.java | 2 ++ 4 files changed, 27 insertions(+), 1 deletion(-) diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index b8eaefe..aab4913 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -3615,6 +3615,10 @@ public class HiveConf extends Configuration { "SSL certificate keystore location for HiveServer2 WebUI."), HIVE_SERVER2_WEBUI_SSL_KEYSTORE_PASSWORD("hive.server2.webui.keystore.password", "", "SSL certificate keystore password for HiveServer2 WebUI."), +HIVE_SERVER2_WEBUI_SSL_KEYSTORE_TYPE("hive.server2.webui.keystore.type", "", +"SSL certificate keystore type for HiveServer2 WebUI."), + HIVE_SERVER2_WEBUI_SSL_KEYMANAGERFACTORY_ALGORITHM("hive.server2.webui.keymanagerfactory.algorithm", +"","SSL certificate key manager factory algorithm for HiveServer2 WebUI."), HIVE_SERVER2_WEBUI_USE_SPNEGO("hive.server2.webui.use.spnego", false, "If true, the HiveServer2 WebUI will be secured with SPNEGO. Clients must authenticate with Kerberos."), HIVE_SERVER2_WEBUI_SPNEGO_KEYTAB("hive.server2.webui.spnego.keytab", "", diff --git a/common/src/java/org/apache/hive/http/HttpServer.java b/common/src/java/org/apache/hive/http/HttpServer.java index 31646ab..1d1db2f 100644 --- a/common/src/java/org/apache/hive/http/HttpServer.java +++ b/common/src/java/org/apache/hive/http/HttpServer.java @@ -35,6 +35,7 @@ import java.util.Set; import java.util.regex.Matcher; import java.util.regex.Pattern; +import javax.net.ssl.KeyManagerFactory; import javax.servlet.Filter; import javax.servlet.FilterChain; import javax.servlet.FilterConfig; @@ -155,6 +156,8 @@ public class HttpServer { private final Map contextAttrs = new HashMap(); private String keyStorePassword; private String keyStorePath; +private String keyStoreType; +private String keyManagerFactoryAlgorithm; private String spnegoPrincipal; private String spnegoKeytab; private boolean useSPNEGO; @@ -221,6 +224,16 @@ public class HttpServer { return this; } +public Builder setKeyStoreType(String keyStoreType) { + this.keyStoreType = keyStoreType; + return this; +} + +public Builder setKeyManagerFactoryAlgorithm(String keyManagerFactoryAlgorithm) { + this.keyManagerFactoryAlgorithm = keyManagerFactoryAlgorithm; + return this; +} + public Builder setUseSSL(boolean useSSL) { this.useSSL = useSSL; return this; @@ -519,7 +532,11 @@ public class HttpServer { } else { SslContextFactory sslContextFactory = new SslContextFactory(); sslContextFactory.setKeyStorePath(b.keyStorePath); - sslContextFactory.setKeyStoreType(KeyStore.getDefaultType()); + sslContextFactory.setKeyStoreType(b.keyStoreType == null || b.keyStoreType.isEmpty() ? + KeyStore.getDefaultType(): b.keyStoreType); + sslContextFactory.setKeyManagerFactoryAlgorithm( + b.keyManagerFactoryAlgorithm == null || b.keyManagerFactoryAlgorithm.isEmpty()? + KeyManagerFactory.getDefaultAlgorithm() : b.keyManagerFactoryAlgorithm); Set excludedSSLProtocols = Sets.newHashSet( Splitter.on(",").trimResults().omitEmptyStrings().split( Strings.nullToEmpty(b.conf.getVar(ConfVars.HIVE_SSL_PROTOCOL_BLACKLIST; diff --git a/service/src/java/org/apache/hive/service/server/HiveServer2.java b/service/src/java/org/apache/hive/service/server/HiveServer2.java index 4783298..2bf2505 100644 --- a/service/src/java/org/apache/hive/service/server/HiveServer2.java +++ b/service/src/java/org/apache/hive/service/server/HiveServer2.java @@ -360,6 +360,9 @@ public c
[hive] branch master updated: Hive-24287: Using SHA-512 for Cookie signature (#1589) (Sai Hemanth Gantasala, reviewed by Yongzhi Chen)
This is an automated email from the ASF dual-hosted git repository. ychena 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 6816a7e Hive-24287: Using SHA-512 for Cookie signature (#1589) (Sai Hemanth Gantasala, reviewed by Yongzhi Chen) 6816a7e is described below commit 6816a7ed17c6ccc6b57ebc1cf7583cd297847315 Author: saihemanth-cloudera <68923650+saihemanth-cloud...@users.noreply.github.com> AuthorDate: Wed Oct 21 09:41:25 2020 -0700 Hive-24287: Using SHA-512 for Cookie signature (#1589) (Sai Hemanth Gantasala, reviewed by Yongzhi Chen) --- service/src/java/org/apache/hive/service/CookieSigner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/service/src/java/org/apache/hive/service/CookieSigner.java b/service/src/java/org/apache/hive/service/CookieSigner.java index d1a41d3..9b2646b 100644 --- a/service/src/java/org/apache/hive/service/CookieSigner.java +++ b/service/src/java/org/apache/hive/service/CookieSigner.java @@ -32,7 +32,7 @@ import org.slf4j.LoggerFactory; */ public class CookieSigner { private static final String SIGNATURE = "="; - private static final String SHA_STRING = "SHA-256"; + private static final String SHA_STRING = "SHA-512"; private byte[] secretBytes; private static final Logger LOG = LoggerFactory.getLogger(CookieSigner.class);
[hive] branch master updated: HIVE-24253: HMS and HS2 needs to support keystore/truststores types by config (#1580) (Yongzhi Chen, reviewed by Kevin Risden and Naveen Gangam)
This is an automated email from the ASF dual-hosted git repository. ychena 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 036ee74 HIVE-24253: HMS and HS2 needs to support keystore/truststores types by config (#1580) (Yongzhi Chen, reviewed by Kevin Risden and Naveen Gangam) 036ee74 is described below commit 036ee740eb423714af99a06f32d76c9708f4201b Author: Yongzhi Chen AuthorDate: Thu Oct 15 17:00:00 2020 -0400 HIVE-24253: HMS and HS2 needs to support keystore/truststores types by config (#1580) (Yongzhi Chen, reviewed by Kevin Risden and Naveen Gangam) HMS and HS2 needs to support keystore/truststores types besides JKS by config Add following optional properties for SSL related configs: HiveServer2: Add new properties: hive.server2.keystore.type hive.server2.keymanagerfactory.algorithm JDBC: Remove hard-coded SSL_TRUST_STORE_TYPE(JKS) from jdbc connection for HS2 Add trustStoreType param for jdbc connection Add trustManagerFactoryAlgorithm Hive MetaStore: New properties for service and client: metastore.keystore.type metastore.keymanagerfactory.algorithm metastore.truststore.type metastore.trustmanagerfactory.algorithm Add bcfks into metastore.dbaccess.ssl.truststore.type stringvalidator Tests: Unit tests for HS2 and HMS --- .../hadoop/hive/common/auth/HiveAuthUtils.java | 17 +-- .../java/org/apache/hadoop/hive/conf/HiveConf.java | 4 ++ .../java/org/hadoop/hive/jdbc/SSLTestUtils.java| 11 ++-- .../test/java/org/apache/hive/jdbc/TestSSL.java| 58 +- .../java/org/apache/hive/jdbc/HiveConnection.java | 24 +++-- jdbc/src/java/org/apache/hive/jdbc/Utils.java | 6 +-- .../service/cli/thrift/ThriftBinaryCLIService.java | 4 +- .../service/cli/thrift/ThriftHttpCLIService.java | 12 - .../hadoop/hive/metastore/HiveMetaStoreClient.java | 6 ++- .../hadoop/hive/metastore/conf/MetastoreConf.java | 10 +++- .../hadoop/hive/metastore/utils/SecurityUtils.java | 22 ++-- .../hadoop/hive/metastore/HiveMetaStore.java | 7 ++- .../metastore/HiveMetaStoreClientPreCatalog.java | 7 ++- .../hadoop/hive/metastore/tools/HMSClient.java | 6 ++- 14 files changed, 162 insertions(+), 32 deletions(-) diff --git a/common/src/java/org/apache/hadoop/hive/common/auth/HiveAuthUtils.java b/common/src/java/org/apache/hadoop/hive/common/auth/HiveAuthUtils.java index 44cde6f..938fd52 100644 --- a/common/src/java/org/apache/hadoop/hive/common/auth/HiveAuthUtils.java +++ b/common/src/java/org/apache/hadoop/hive/common/auth/HiveAuthUtils.java @@ -57,11 +57,14 @@ public class HiveAuthUtils { } public static TTransport getSSLSocket(String host, int port, int loginTimeout, -String trustStorePath, String trustStorePassWord) throws TTransportException { + String trustStorePath, String trustStorePassWord, String trustStoreType, + String trustStoreAlgorithm) throws TTransportException { TSSLTransportFactory.TSSLTransportParameters params = new TSSLTransportFactory.TSSLTransportParameters(); -params.setTrustStore(trustStorePath, trustStorePassWord, -TrustManagerFactory.getDefaultAlgorithm(), KeyStore.getDefaultType()); +String tStoreType = trustStoreType.isEmpty()? KeyStore.getDefaultType() : trustStoreType; +String tStoreAlgorithm = trustStoreAlgorithm.isEmpty()? +TrustManagerFactory.getDefaultAlgorithm() : trustStoreAlgorithm; +params.setTrustStore(trustStorePath, trustStorePassWord, tStoreAlgorithm, tStoreType); params.requireClientAuth(true); // The underlying SSLSocket object is bound to host:port with the given SO_TIMEOUT and // SSLContext created with the given params @@ -92,12 +95,16 @@ public class HiveAuthUtils { } public static TServerSocket getServerSSLSocket(String hiveHost, int portNum, String keyStorePath, - String keyStorePassWord, List sslVersionBlacklist) throws TTransportException, + String keyStorePassWord, String keyStoreType, String keyStoreAlgorithm, + List sslVersionBlacklist) throws TTransportException, UnknownHostException { TSSLTransportFactory.TSSLTransportParameters params = new TSSLTransportFactory.TSSLTransportParameters(); +String kStoreType = keyStoreType.isEmpty()? KeyStore.getDefaultType() : keyStoreType; +String kStoreAlgorithm = keyStoreAlgorithm.isEmpty()? +KeyManagerFactory.getDefaultAlgorithm() : keyStoreAlgorithm; params.setKeyStore(keyStorePath, keyStorePassWord, -KeyManagerFactory.getDefaultAlgorithm(), KeyStore.getDefaultType()); +kStoreAlgorithm, kStoreType); InetSocketAddress serverAddress; if (hiveHost == null || hiveHost.isEmpty()) { // Wildcard bind diff --git a/common/src/java/org
[hive] branch master updated: HIVE-24236: Fixed possible Connection leaks in TxnHandler (Yongzhi Chen, reviewed by Denys Kuzmenko) (#1559)
This is an automated email from the ASF dual-hosted git repository. ychena 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 7269eda HIVE-24236: Fixed possible Connection leaks in TxnHandler (Yongzhi Chen, reviewed by Denys Kuzmenko) (#1559) 7269eda is described below commit 7269eda4be6193928dcdffb1ed6f06505b2c43b6 Author: yongzhi <47003498+yong...@users.noreply.github.com> AuthorDate: Thu Oct 8 12:26:26 2020 -0400 HIVE-24236: Fixed possible Connection leaks in TxnHandler (Yongzhi Chen, reviewed by Denys Kuzmenko) (#1559) --- .../hadoop/hive/metastore/txn/TestTxnHandler.java | 28 -- .../hadoop/hive/metastore/txn/TxnHandler.java | 10 2 files changed, 22 insertions(+), 16 deletions(-) diff --git a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java index 85cc26e..0d82fef 100644 --- a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java +++ b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java @@ -1434,21 +1434,25 @@ public class TestTxnHandler { @Ignore("Wedges Derby") public void deadlockDetected() throws Exception { LOG.debug("Starting deadlock test"); + if (txnHandler instanceof TxnHandler) { final TxnHandler tHndlr = (TxnHandler)txnHandler; Connection conn = tHndlr.getDbConn(Connection.TRANSACTION_SERIALIZABLE); - Statement stmt = conn.createStatement(); - long now = tHndlr.getDbTime(conn); - stmt.executeUpdate("INSERT INTO \"TXNS\" (\"TXN_ID\", \"TXN_STATE\", \"TXN_STARTED\", \"TXN_LAST_HEARTBEAT\", " + - "txn_user, txn_host) values (1, 'o', " + now + ", " + now + ", 'shagy', " + - "'scooby.com')"); - stmt.executeUpdate("INSERT INTO \"HIVE_LOCKS\" (\"HL_LOCK_EXT_ID\", \"HL_LOCK_INT_ID\", \"HL_TXNID\", " + - "\"HL_DB\", \"HL_TABLE\", \"HL_PARTITION\", \"HL_LOCK_STATE\", \"HL_LOCK_TYPE\", \"HL_LAST_HEARTBEAT\", " + - "\"HL_USER\", \"HL_HOST\") VALUES (1, 1, 1, 'MYDB', 'MYTABLE', 'MYPARTITION', '" + - tHndlr.LOCK_WAITING + "', '" + getEncoding(LockType.EXCLUSIVE) + "', " + now + ", 'fred', " + - "'scooby.com')"); - conn.commit(); - tHndlr.closeDbConn(conn); + try { +Statement stmt = conn.createStatement(); +long now = tHndlr.getDbTime(conn); +stmt.executeUpdate("INSERT INTO \"TXNS\" (\"TXN_ID\", \"TXN_STATE\", \"TXN_STARTED\", \"TXN_LAST_HEARTBEAT\", " + +"txn_user, txn_host) values (1, 'o', " + now + ", " + now + ", 'shagy', " + +"'scooby.com')"); +stmt.executeUpdate("INSERT INTO \"HIVE_LOCKS\" (\"HL_LOCK_EXT_ID\", \"HL_LOCK_INT_ID\", \"HL_TXNID\", " + +"\"HL_DB\", \"HL_TABLE\", \"HL_PARTITION\", \"HL_LOCK_STATE\", \"HL_LOCK_TYPE\", \"HL_LAST_HEARTBEAT\", " + +"\"HL_USER\", \"HL_HOST\") VALUES (1, 1, 1, 'MYDB', 'MYTABLE', 'MYPARTITION', '" + +tHndlr.LOCK_WAITING + "', '" + getEncoding(LockType.EXCLUSIVE) + "', " + now + ", 'fred', " + +"'scooby.com')"); +conn.commit(); + } finally { +tHndlr.closeDbConn(conn); + } final AtomicBoolean sawDeadlock = new AtomicBoolean(); diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java index 40c5e5c..f3dc6be 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java @@ -5213,6 +5213,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI { Connection dbConn = null; Statement stmt = null; ResultSet rs = null; +boolean needToCloseConn = true; try { try { String sqlStmt = sqlGenerator.addForUpdateClause("SELECT \"MT_COMMENT\" FROM \"AUX_TABLE\" WHERE \"MT_KEY1\"=" + quoteString(key) + " and \"MT_KEY2\"=0"); @@ -5250,20 +5251,21 @@ ab
[hive] branch master updated: Show tables in database with owner policy (#1380)
This is an automated email from the ASF dual-hosted git repository. ychena 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 595d2f3 Show tables in database with owner policy (#1380) 595d2f3 is described below commit 595d2f3fdfc5b61ea9916690714e3111cab45cd8 Author: saihemanth-cloudera <68923650+saihemanth-cloud...@users.noreply.github.com> AuthorDate: Mon Oct 5 16:45:24 2020 -0700 Show tables in database with owner policy (#1380) * Show tables in database with owner policy * Trigger ptests * HIVE-23969: Show tables in database with owner policy * Trigger ptests again * Trigger ptests again --- .../plugin/TestHiveAuthorizerCheckInvocation.java | 17 + .../metastore/filtercontext/TableFilterContext.java | 3 +-- .../org/apache/hadoop/hive/metastore/HiveMetaStore.java | 11 +-- 3 files changed, 27 insertions(+), 4 deletions(-) diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java index d046822..506bb7c 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java @@ -619,6 +619,23 @@ public class TestHiveAuthorizerCheckInvocation { assertEquals("table name", inDbTableName.toLowerCase(), dbObj.getObjectName()); } + @Test + public void showTablesInDB() throws Exception{ +final String tableName1 = "table1"; +driver.run("create table " + dbName+"."+tableName1 + "(eid int, yoj int)"); +final String tableName2 = "table2"; +driver.run("create table " + dbName+"."+tableName2 + "(eid int, ecode int)"); +reset(mockedAuthorizer); + +int status = driver.compile("show tables in "+dbName, true); +assertEquals(0, status); +Pair, List> io = getHivePrivilegeObjectInputs(); +List inputs = io.getLeft(); +HivePrivilegeObject dbObj = inputs.get(0); +assertEquals("input type", HivePrivilegeObjectType.DATABASE, dbObj.getType()); +assertTrue(dbObj.getOwnerName() != null); + } + private void checkSingleTableInput(List inputs) { assertEquals("number of inputs", 1, inputs.size()); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/filtercontext/TableFilterContext.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/filtercontext/TableFilterContext.java index b140200..2b6814b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/filtercontext/TableFilterContext.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/filtercontext/TableFilterContext.java @@ -69,8 +69,7 @@ public class TableFilterContext extends HiveMetaStoreAuthorizableEvent { HivePrivilegeObjectType type = HivePrivilegeObjectType.TABLE_OR_VIEW; HivePrivObjectActionType objectActionType = HivePrivilegeObject.HivePrivObjectActionType.OTHER; HivePrivilegeObject hivePrivilegeObject = -new HivePrivilegeObject(type, table.getDbName(), table.getTableName(), null, null, objectActionType, null, -null); +new HivePrivilegeObject(type, table.getDbName(), table.getTableName(), null, null, objectActionType, null, null, table.getOwner(), table.getOwnerType()); ret.add(hivePrivilegeObject); } } else { diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java index 8645994..d00dcd9 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java @@ -5841,8 +5841,15 @@ public class HiveMetaStore extends ThriftHiveMetastore { String[] parsedDbName = parseDbName(dbname, conf); try { ret = getMS().getTables(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], pattern); -ret = FilterUtils.filterTableNamesIfEnabled(isServerFilterEnabled, filterHook, -parsedDbName[CAT_NAME], parsedDbName[DB_NAME], ret); +if(ret != null && !ret.isEmpty()) { + List tableInfo = new ArrayList<>(); + tableInfo = getMS().getTableObjectsByName(pa
[hive] branch master updated: HIVE-22461: NPE Metastore Transformer (Yongzhi Chen, reviewed by Naveen Gangam)
This is an automated email from the ASF dual-hosted git repository. ychena 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 eb72a0c HIVE-22461: NPE Metastore Transformer (Yongzhi Chen, reviewed by Naveen Gangam) eb72a0c is described below commit eb72a0c01d3dc5dc50c220a548c7986a440eef3f Author: Yongzhi Chen AuthorDate: Wed Dec 11 09:41:49 2019 -0500 HIVE-22461: NPE Metastore Transformer (Yongzhi Chen, reviewed by Naveen Gangam) --- .../metastore/MetastoreDefaultTransformer.java | 70 -- 1 file changed, 40 insertions(+), 30 deletions(-) diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetastoreDefaultTransformer.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetastoreDefaultTransformer.java index 8d018d7..5283e07 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetastoreDefaultTransformer.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetastoreDefaultTransformer.java @@ -109,7 +109,7 @@ public class MetastoreDefaultTransformer implements IMetaStoreMetadataTransforme Map params = table.getParameters(); String tableType = table.getTableType(); String tCapabilities = params.get(OBJCAPABILITIES); - int numBuckets = table.getSd().getNumBuckets(); + int numBuckets = table.isSetSd()? table.getSd().getNumBuckets() : 0; boolean isBucketed = (numBuckets > 0) ? true : false; LOG.info("Table " + table.getTableName() + ",#bucket=" + numBuckets + ",isBucketed:" + isBucketed + ",tableType=" + tableType + ",tableCapabilities=" + tCapabilities); @@ -466,6 +466,8 @@ public class MetastoreDefaultTransformer implements IMetaStoreMetadataTransforme if (partition.getSd() != null) { partBuckets = partition.getSd().getNumBuckets(); LOG.info("Number of original part buckets=" + partBuckets); + } else { +partBuckets = 0; } if (tCapabilities == null) { @@ -473,34 +475,34 @@ public class MetastoreDefaultTransformer implements IMetaStoreMetadataTransforme switch (tableType) { case "EXTERNAL_TABLE": + if (partBuckets > 0 && !processorCapabilities.contains(HIVEBUCKET2)) { +Partition newPartition = new Partition(partition); +StorageDescriptor newSd = new StorageDescriptor(partition.getSd()); +newSd.setNumBuckets(-1); // remove bucketing info +newPartition.setSd(newSd); +ret.add(newPartition); + } else { +ret.add(partition); + } + break; + case "MANAGED_TABLE": + String txnal = params.get(TABLE_IS_TRANSACTIONAL); + if (txnal == null || "FALSE".equalsIgnoreCase(txnal)) { // non-ACID MANAGED table if (partBuckets > 0 && !processorCapabilities.contains(HIVEBUCKET2)) { Partition newPartition = new Partition(partition); StorageDescriptor newSd = new StorageDescriptor(partition.getSd()); newSd.setNumBuckets(-1); // remove bucketing info newPartition.setSd(newSd); ret.add(newPartition); -} else { - ret.add(partition); -} -break; - case "MANAGED_TABLE": -String txnal = params.get(TABLE_IS_TRANSACTIONAL); -if (txnal == null || txnal.equalsIgnoreCase("FALSE")) { // non-ACID MANAGED table - if (partBuckets > 0 && !processorCapabilities.contains(HIVEBUCKET2)) { -Partition newPartition = new Partition(partition); -StorageDescriptor newSd = new StorageDescriptor(partition.getSd()); -newSd.setNumBuckets(-1); // remove bucketing info -newPartition.setSd(newSd); -ret.add(newPartition); -break; - } + break; } -// INSERT or FULL ACID table, bucketing info to be retained -ret.add(partition); -break; + } + // INSERT or FULL ACID table, bucketing info to be retained + ret.add(partition); + break; default: -ret.add(partition); -break; + ret.add(partition); + break; } } else { // table has capabilities tCapabilities = tCapabilities.replaceAll("\\s","").toUpperCase(); // remove spaces between tCapabilities + toUppercase @@ -578,7 +580,7 @@ public class MetastoreDefaultTransformer implements IMetaStoreMetadataTrans
[hive] branch master updated: HIVE-21796: ArrayWritableObjectInspector.equals can take O(2^nesting_depth) time (Zoltan Matyus and Csaba Ringhofer, reviewed by Yongzhi Chen)
This is an automated email from the ASF dual-hosted git repository. ychena 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 936fe48 HIVE-21796: ArrayWritableObjectInspector.equals can take O(2^nesting_depth) time (Zoltan Matyus and Csaba Ringhofer, reviewed by Yongzhi Chen) 936fe48 is described below commit 936fe482576595f671669a52d9827b82806e4284 Author: Yongzhi Chen AuthorDate: Tue Jun 11 22:22:40 2019 -0400 HIVE-21796: ArrayWritableObjectInspector.equals can take O(2^nesting_depth) time (Zoltan Matyus and Csaba Ringhofer, reviewed by Yongzhi Chen) --- .../serde/ArrayWritableObjectInspector.java| 3 +- .../serde/TestArrayWritableObjectInspector.java| 75 ++ 2 files changed, 76 insertions(+), 2 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ArrayWritableObjectInspector.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ArrayWritableObjectInspector.java index ba69795..742721d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ArrayWritableObjectInspector.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ArrayWritableObjectInspector.java @@ -246,7 +246,7 @@ public class ArrayWritableObjectInspector extends SettableStructObjectInspector return false; } -return fieldsByName != null ? fieldsByName.equals(that.fieldsByName) : that.fieldsByName == null; +return true; } @Override @@ -255,7 +255,6 @@ public class ArrayWritableObjectInspector extends SettableStructObjectInspector result = 31 * result + (fieldInfos != null ? fieldInfos.hashCode() : 0); result = 31 * result + (fieldNames != null ? fieldNames.hashCode() : 0); result = 31 * result + (fields != null ? fields.hashCode() : 0); -result = 31 * result + (fieldsByName != null ? fieldsByName.hashCode() : 0); result = 31 * result + (isRoot ? 1 : 0); return result; } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestArrayWritableObjectInspector.java b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestArrayWritableObjectInspector.java new file mode 100644 index 000..a3234a5 --- /dev/null +++ b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestArrayWritableObjectInspector.java @@ -0,0 +1,75 @@ +/* + * 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.io.parquet.serde; + +import java.util.Arrays; + +import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.junit.Assert; + +import junit.framework.TestCase; + +/** + * Tests for ArrayWritableObjectInspector. At the moment only behavior related to HIVE-21796 covered. + */ +public class TestArrayWritableObjectInspector extends TestCase { + + private StructTypeInfo nestOnce(TypeInfo nestedType) { +return (StructTypeInfo) TypeInfoFactory.getStructTypeInfo(Arrays.asList("value"), Arrays.asList(nestedType)); + } + + private StructTypeInfo createNestedStruct(TypeInfo nestedType, int depth) { +StructTypeInfo result = nestOnce(nestedType); +for(int i = 0; i < depth; i++){ + result = nestOnce(result); +} +return result; + } + + /** Regression tests for HIVE-21796: equals and hash takes forever if HIVE-21796 is reverted / reintroduced. */ + public void testIdenticalInspectorsEquals() { +StructTypeInfo nestedStruct = createNestedStruct(TypeInfoFactory.getPrimitiveTypeInfo("int"), 1000); +ArrayWritableObjectInspector inspectorX = new ArrayWritableObjectInspector(true, nestedStruct, null); +ArrayWritableObjectInspector inspectorY = new ArrayWritableObjectInspector(true, nestedStruct, null); +Assert.assertEquals(inspectorX, inspectorY); +Assert.assertEquals(inspectorX.hashCode(), inspectorY.hashCode()); + } + + /** Regression tests for HIVE-21796: equals and hash takes forever if HIVE-21796 is reverted / reintroduced. */ + public void testEq
[hive] branch branch-3 updated: HIVE-21045: Add HMS total api count stats and connection pool stats to metrics (Karthik Manamcheri, reviewed by Naveen Gangam and Yongzhi Chen)
This is an automated email from the ASF dual-hosted git repository. ychena pushed a commit to branch branch-3 in repository https://gitbox.apache.org/repos/asf/hive.git The following commit(s) were added to refs/heads/branch-3 by this push: new c1c33cc HIVE-21045: Add HMS total api count stats and connection pool stats to metrics (Karthik Manamcheri, reviewed by Naveen Gangam and Yongzhi Chen) c1c33cc is described below commit c1c33cce5d185067df191aa57f4e1d31f1a5d582 Author: Yongzhi Chen AuthorDate: Thu Jan 31 10:02:55 2019 -0500 HIVE-21045: Add HMS total api count stats and connection pool stats to metrics (Karthik Manamcheri, reviewed by Naveen Gangam and Yongzhi Chen) --- .../hive/metastore/PersistenceManagerProvider.java | 48 + .../datasource/BoneCPDataSourceProvider.java | 109 + .../metastore/datasource/DataSourceProvider.java | 18 ++-- .../datasource/DataSourceProviderFactory.java | 32 +++--- .../datasource/HikariCPDataSourceProvider.java | 35 --- .../hive/metastore/metrics/MetricsConstants.java | 2 + .../hadoop/hive/metastore/metrics/PerfLogger.java | 17 +++- .../datasource/TestDataSourceProviderFactory.java | 22 ++--- 8 files changed, 188 insertions(+), 95 deletions(-) diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/PersistenceManagerProvider.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/PersistenceManagerProvider.java index 20f0738..876437b 100644 --- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/PersistenceManagerProvider.java +++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/PersistenceManagerProvider.java @@ -19,6 +19,28 @@ package org.apache.hadoop.hive.metastore; +import java.io.IOException; +import java.lang.reflect.Field; +import java.sql.SQLException; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Supplier; +import javax.jdo.JDOCanRetryException; +import javax.jdo.JDOHelper; +import javax.jdo.PersistenceManager; +import javax.jdo.PersistenceManagerFactory; +import javax.jdo.datastore.DataStoreCache; +import javax.sql.DataSource; + import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; @@ -33,7 +55,6 @@ import org.apache.hadoop.hive.metastore.model.MSerDeInfo; import org.apache.hadoop.hive.metastore.model.MStorageDescriptor; import org.apache.hadoop.hive.metastore.model.MTable; import org.apache.hadoop.hive.metastore.model.MType; -import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; import org.datanucleus.AbstractNucleusContext; import org.datanucleus.ClassLoaderResolver; import org.datanucleus.ClassLoaderResolverImpl; @@ -46,29 +67,6 @@ import org.datanucleus.util.WeakValueMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.jdo.JDOCanRetryException; -import javax.jdo.JDOHelper; -import javax.jdo.PersistenceManager; -import javax.jdo.PersistenceManagerFactory; -import javax.jdo.datastore.DataStoreCache; -import javax.sql.DataSource; -import java.io.IOException; -import java.lang.reflect.Field; -import java.lang.reflect.Method; -import java.sql.SQLException; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Properties; -import java.util.Set; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantReadWriteLock; -import java.util.function.Supplier; - /** * This class is a wrapper class around PersistenceManagerFactory and its properties * These objects are static and need to be carefully modified together such that there are no @@ -221,7 +219,7 @@ public class PersistenceManagerProvider { } private static void initPMF(Configuration conf) { -DataSourceProvider dsp = DataSourceProviderFactory.getDataSourceProvider(conf); +DataSourceProvider dsp = DataSourceProviderFactory.tryGetDataSourceProviderOrNull(conf); if (dsp == null) { pmf = JDOHelper.getPersistenceManagerFactory(prop); diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/BoneCPDataSourceProvider.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/BoneCPDataSourceProvider.java index 4ff2bb7..60594fe 100644 --- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/datasource/BoneCPDataSourceProvider.java +++ b/standalone
hive git commit: HIVE-21045: Add HMS total api count stats and connection pool stats to metrics (Karthik Manamcheri, reviewed by Naveen Gangam and Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/master a3bac4d7e -> fd92d8865 HIVE-21045: Add HMS total api count stats and connection pool stats to metrics (Karthik Manamcheri, reviewed by Naveen Gangam and Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/fd92d886 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/fd92d886 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/fd92d886 Branch: refs/heads/master Commit: fd92d88655a93673f4cdebfa4bd46d059ea57e9c Parents: a3bac4d Author: Yongzhi Chen Authored: Fri Jan 25 17:14:47 2019 -0500 Committer: Yongzhi Chen Committed: Sat Jan 26 00:19:08 2019 -0500 -- .../metastore/PersistenceManagerProvider.java | 5 +- .../datasource/BoneCPDataSourceProvider.java| 107 --- .../datasource/DataSourceProvider.java | 18 ++-- .../datasource/DataSourceProviderFactory.java | 48 +++-- .../datasource/DbCPDataSourceProvider.java | 7 +- .../datasource/HikariCPDataSourceProvider.java | 23 ++-- .../metastore/metrics/MetricsConstants.java | 2 + .../hive/metastore/metrics/PerfLogger.java | 17 ++- .../hadoop/hive/metastore/txn/TxnHandler.java | 2 +- .../TestDataSourceProviderFactory.java | 87 ++- 10 files changed, 168 insertions(+), 148 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/fd92d886/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PersistenceManagerProvider.java -- diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PersistenceManagerProvider.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PersistenceManagerProvider.java index dfd7abf..084ae83 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PersistenceManagerProvider.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PersistenceManagerProvider.java @@ -54,7 +54,6 @@ import javax.jdo.datastore.DataStoreCache; import javax.sql.DataSource; import java.io.IOException; import java.lang.reflect.Field; -import java.lang.reflect.Method; import java.sql.SQLException; import java.util.Arrays; import java.util.Collections; @@ -225,9 +224,7 @@ public class PersistenceManagerProvider { } private static void initPMF(Configuration conf) { -DataSourceProvider dsp = DataSourceProviderFactory -.hasProviderSpecificConfigurations(conf) ? DataSourceProviderFactory -.getDataSourceProvider(conf) : null; +DataSourceProvider dsp = DataSourceProviderFactory.tryGetDataSourceProviderOrNull(conf); if (dsp == null) { pmf = JDOHelper.getPersistenceManagerFactory(prop); http://git-wip-us.apache.org/repos/asf/hive/blob/fd92d886/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/datasource/BoneCPDataSourceProvider.java -- diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/datasource/BoneCPDataSourceProvider.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/datasource/BoneCPDataSourceProvider.java index 7e33c51..f3243c5 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/datasource/BoneCPDataSourceProvider.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/datasource/BoneCPDataSourceProvider.java @@ -17,17 +17,27 @@ */ package org.apache.hadoop.hive.metastore.datasource; +import java.sql.SQLException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import javax.sql.DataSource; + +import com.codahale.metrics.Gauge; +import com.codahale.metrics.Metric; +import com.codahale.metrics.MetricRegistry; +import com.codahale.metrics.MetricSet; import com.jolbox.bonecp.BoneCPConfig; import com.jolbox.bonecp.BoneCPDataSource; +import com.jolbox.bonecp.StatisticsMBean; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; +import org.apache.hadoop.hive.metastore.metrics.Metrics; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.sql.DataSource; -import java.sql.SQLException; -import java.util.Properties; - /** * DataSourceProvider for the BoneCP connection pool. */ @@ -61,15 +71,16 @@ public class BoneCPDataSourceProvider implements DataSourceProvider { throw new SQLException("Cannot create BoneCP configuration: ", e);
[5/9] hive git commit: HIVE-20915: Make dynamic sort partition optimization available to HoS and MR (Yongzhi Chen, reviewed by Naveen Gangam)
http://git-wip-us.apache.org/repos/asf/hive/blob/9f2e8e61/ql/src/test/results/clientpositive/load_dyn_part8.q.out -- diff --git a/ql/src/test/results/clientpositive/load_dyn_part8.q.out b/ql/src/test/results/clientpositive/load_dyn_part8.q.out index 97b8886..1037a33 100644 --- a/ql/src/test/results/clientpositive/load_dyn_part8.q.out +++ b/ql/src/test/results/clientpositive/load_dyn_part8.q.out @@ -59,9 +59,9 @@ STAGE DEPENDENCIES: Stage-2 is a root stage Stage-0 depends on stages: Stage-2 Stage-3 depends on stages: Stage-0 - Stage-1 depends on stages: Stage-2 - Stage-4 depends on stages: Stage-1, Stage-5 - Stage-5 depends on stages: Stage-2 + Stage-4 depends on stages: Stage-2 + Stage-1 depends on stages: Stage-4 + Stage-5 depends on stages: Stage-1 STAGE PLANS: Stage: Stage-2 @@ -79,54 +79,14 @@ STAGE PLANS: expressions: key (type: string), value (type: string), ds (type: string), hr (type: string) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 666 Data size: 7075 Basic stats: COMPLETE Column stats: NONE -File Output Operator - compressed: false - GlobalTableId: 1 - A masked pattern was here - NumFilesPerFileSink: 1 - Statistics: Num rows: 666 Data size: 7075 Basic stats: COMPLETE Column stats: NONE - A masked pattern was here - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: -bucket_count -1 -column.name.delimiter , -columns key,value -columns.comments 'default','default' -columns.types string:string - A masked pattern was here -name default.nzhang_part8_n0 -partition_columns ds/hr -partition_columns.types string:string -serialization.ddl struct nzhang_part8_n0 { string key, string value} -serialization.format 1 -serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - A masked pattern was here - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.nzhang_part8_n0 - TotalFiles: 1 - GatherStats: true - MultiFileSpray: false -Select Operator - expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string) - outputColumnNames: key, value, ds, hr - Statistics: Num rows: 666 Data size: 7075 Basic stats: COMPLETE Column stats: NONE - Group By Operator -aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll') -keys: ds (type: string), hr (type: string) -mode: hash -outputColumnNames: _col0, _col1, _col2, _col3 -Statistics: Num rows: 666 Data size: 7075 Basic stats: COMPLETE Column stats: NONE -Reduce Output Operator - key expressions: _col0 (type: string), _col1 (type: string) - null sort order: aa - sort order: ++ - Map-reduce partition columns: _col0 (type: string), _col1 (type: string) - Statistics: Num rows: 666 Data size: 7075 Basic stats: COMPLETE Column stats: NONE - tag: -1 - value expressions: _col2 (type: struct), _col3 (type: struct) - auto parallelism: false +Reduce Output Operator + key expressions: _col2 (type: string), _col3 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: _col2 (type: string), _col3 (type: string) + tag: -1 + value expressions: _col0 (type: string), _col1 (type: string) + auto parallelism: false Filter Operator isSamplingPred: false predicate: (ds > '2008-04-08') (type: boolean) @@ -137,62 +97,23 @@ STAGE PLANS: Statistics: Num rows: 666 Data size: 7075 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - GlobalTableId: 2 + GlobalTableId: 0 A masked pattern was here NumFilesPerFileSink: 1 - Static Partition Specification:
[3/9] hive git commit: HIVE-20915: Make dynamic sort partition optimization available to HoS and MR (Yongzhi Chen, reviewed by Naveen Gangam)
http://git-wip-us.apache.org/repos/asf/hive/blob/9f2e8e61/ql/src/test/results/clientpositive/spark/dynpart_sort_optimization.q.out -- diff --git a/ql/src/test/results/clientpositive/spark/dynpart_sort_optimization.q.out b/ql/src/test/results/clientpositive/spark/dynpart_sort_optimization.q.out new file mode 100644 index 000..ddc5106 --- /dev/null +++ b/ql/src/test/results/clientpositive/spark/dynpart_sort_optimization.q.out @@ -0,0 +1,3504 @@ +PREHOOK: query: create table over1k_n3( + t tinyint, + si smallint, + i int, + b bigint, + f float, + d double, + bo boolean, + s string, + ts timestamp, + `dec` decimal(4,2), + bin binary) + row format delimited + fields terminated by '|' +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@over1k_n3 +POSTHOOK: query: create table over1k_n3( + t tinyint, + si smallint, + i int, + b bigint, + f float, + d double, + bo boolean, + s string, + ts timestamp, + `dec` decimal(4,2), + bin binary) + row format delimited + fields terminated by '|' +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@over1k_n3 +PREHOOK: query: load data local inpath '../../data/files/over1k' into table over1k_n3 +PREHOOK: type: LOAD + A masked pattern was here +PREHOOK: Output: default@over1k_n3 +POSTHOOK: query: load data local inpath '../../data/files/over1k' into table over1k_n3 +POSTHOOK: type: LOAD + A masked pattern was here +POSTHOOK: Output: default@over1k_n3 +PREHOOK: query: create table over1k_part( + si smallint, + i int, + b bigint, + f float) + partitioned by (ds string, t tinyint) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@over1k_part +POSTHOOK: query: create table over1k_part( + si smallint, + i int, + b bigint, + f float) + partitioned by (ds string, t tinyint) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@over1k_part +PREHOOK: query: create table over1k_part_limit like over1k_part +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@over1k_part_limit +POSTHOOK: query: create table over1k_part_limit like over1k_part +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@over1k_part_limit +PREHOOK: query: create table over1k_part_buck( + si smallint, + i int, + b bigint, + f float) + partitioned by (t tinyint) + clustered by (si) into 4 buckets +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@over1k_part_buck +POSTHOOK: query: create table over1k_part_buck( + si smallint, + i int, + b bigint, + f float) + partitioned by (t tinyint) + clustered by (si) into 4 buckets +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@over1k_part_buck +PREHOOK: query: create table over1k_part_buck_sort( + si smallint, + i int, + b bigint, + f float) + partitioned by (t tinyint) + clustered by (si) + sorted by (f) into 4 buckets +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@over1k_part_buck_sort +POSTHOOK: query: create table over1k_part_buck_sort( + si smallint, + i int, + b bigint, + f float) + partitioned by (t tinyint) + clustered by (si) + sorted by (f) into 4 buckets +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@over1k_part_buck_sort +PREHOOK: query: explain insert overwrite table over1k_part partition(ds="foo", t) select si,i,b,f,t from over1k_n3 where t is null or t=27 +PREHOOK: type: QUERY +PREHOOK: Input: default@over1k_n3 +PREHOOK: Output: default@over1k_part@ds=foo +POSTHOOK: query: explain insert overwrite table over1k_part partition(ds="foo", t) select si,i,b,f,t from over1k_n3 where t is null or t=27 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@over1k_n3 +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + Stage-2 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 +Spark + Edges: +Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2) + A masked pattern was here + Vertices: +Map 1 +Map Operator Tree: +TableScan + alias: over1k_n3 + filterExpr: (t is null or (t = 27Y)) (type: boolean) +
[7/9] hive git commit: HIVE-20915: Make dynamic sort partition optimization available to HoS and MR (Yongzhi Chen, reviewed by Naveen Gangam)
http://git-wip-us.apache.org/repos/asf/hive/blob/9f2e8e61/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out -- diff --git a/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out b/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out new file mode 100644 index 000..427f819 --- /dev/null +++ b/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out @@ -0,0 +1,3797 @@ +PREHOOK: query: create table over1k_n3( + t tinyint, + si smallint, + i int, + b bigint, + f float, + d double, + bo boolean, + s string, + ts timestamp, + `dec` decimal(4,2), + bin binary) + row format delimited + fields terminated by '|' +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@over1k_n3 +POSTHOOK: query: create table over1k_n3( + t tinyint, + si smallint, + i int, + b bigint, + f float, + d double, + bo boolean, + s string, + ts timestamp, + `dec` decimal(4,2), + bin binary) + row format delimited + fields terminated by '|' +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@over1k_n3 +PREHOOK: query: load data local inpath '../../data/files/over1k' into table over1k_n3 +PREHOOK: type: LOAD + A masked pattern was here +PREHOOK: Output: default@over1k_n3 +POSTHOOK: query: load data local inpath '../../data/files/over1k' into table over1k_n3 +POSTHOOK: type: LOAD + A masked pattern was here +POSTHOOK: Output: default@over1k_n3 +PREHOOK: query: create table over1k_part( + si smallint, + i int, + b bigint, + f float) + partitioned by (ds string, t tinyint) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@over1k_part +POSTHOOK: query: create table over1k_part( + si smallint, + i int, + b bigint, + f float) + partitioned by (ds string, t tinyint) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@over1k_part +PREHOOK: query: create table over1k_part_limit like over1k_part +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@over1k_part_limit +POSTHOOK: query: create table over1k_part_limit like over1k_part +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@over1k_part_limit +PREHOOK: query: create table over1k_part_buck( + si smallint, + i int, + b bigint, + f float) + partitioned by (t tinyint) + clustered by (si) into 4 buckets +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@over1k_part_buck +POSTHOOK: query: create table over1k_part_buck( + si smallint, + i int, + b bigint, + f float) + partitioned by (t tinyint) + clustered by (si) into 4 buckets +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@over1k_part_buck +PREHOOK: query: create table over1k_part_buck_sort( + si smallint, + i int, + b bigint, + f float) + partitioned by (t tinyint) + clustered by (si) + sorted by (f) into 4 buckets +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@over1k_part_buck_sort +POSTHOOK: query: create table over1k_part_buck_sort( + si smallint, + i int, + b bigint, + f float) + partitioned by (t tinyint) + clustered by (si) + sorted by (f) into 4 buckets +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@over1k_part_buck_sort +PREHOOK: query: explain insert overwrite table over1k_part partition(ds="foo", t) select si,i,b,f,t from over1k_n3 where t is null or t=27 +PREHOOK: type: QUERY +PREHOOK: Input: default@over1k_n3 +PREHOOK: Output: default@over1k_part@ds=foo +POSTHOOK: query: explain insert overwrite table over1k_part partition(ds="foo", t) select si,i,b,f,t from over1k_n3 where t is null or t=27 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@over1k_n3 +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + Stage-2 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 +Map Reduce + Map Operator Tree: + TableScan +alias: over1k_n3 +filterExpr: (t is null or (t = 27Y)) (type: boolean) +Statistics: Num rows: 1 Data size: 1066360 Basic stats: COMPLETE Column stats: NONE +Filter Operator + predicate: ((t = 27Y) or t is null) (type: boolean) +
[4/9] hive git commit: HIVE-20915: Make dynamic sort partition optimization available to HoS and MR (Yongzhi Chen, reviewed by Naveen Gangam)
http://git-wip-us.apache.org/repos/asf/hive/blob/9f2e8e61/ql/src/test/results/clientpositive/orc_merge10.q.out -- diff --git a/ql/src/test/results/clientpositive/orc_merge10.q.out b/ql/src/test/results/clientpositive/orc_merge10.q.out index f9dda11..0184fa9 100644 --- a/ql/src/test/results/clientpositive/orc_merge10.q.out +++ b/ql/src/test/results/clientpositive/orc_merge10.q.out @@ -69,48 +69,24 @@ STAGE PLANS: expressions: UDFToInteger(key) (type: int), value (type: string), (hash(key) pmod 2) (type: int) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator -compressed: false -Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE -table: -input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat -output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat -serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde -name: default.orcfile_merge1 - Select Operator -expressions: _col0 (type: int), _col1 (type: string), '1' (type: string), CAST( _col2 AS STRING) (type: string) -outputColumnNames: key, value, ds, part -Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE -Group By Operator - aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll') - keys: ds (type: string), part (type: string) - mode: hash - outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator -key expressions: _col0 (type: string), _col1 (type: string) -sort order: ++ -Map-reduce partition columns: _col0 (type: string), _col1 (type: string) -Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE -value expressions: _col2 (type: struct), _col3 (type: struct) + Reduce Output Operator +key expressions: _col2 (type: int) +sort order: + +Map-reduce partition columns: _col2 (type: int) +value expressions: _col0 (type: int), _col1 (type: string) Reduce Operator Tree: -Group By Operator - aggregations: compute_stats(VALUE._col0), compute_stats(VALUE._col1) - keys: KEY._col0 (type: string), KEY._col1 (type: string) - mode: mergepartial - outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - Select Operator -expressions: _col2 (type: struct), _col3 (type: struct), _col0 (type: string), _col1 (type: string) -outputColumnNames: _col0, _col1, _col2, _col3 -Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE -File Output Operator - compressed: false - Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - table: - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), KEY._col2 (type: int) + outputColumnNames: _col0, _col1, _col2 + File Output Operator +compressed: false +Dp Sort State: PARTITION_SORTED +Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE +table: +input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat +output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat +serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde +name: default.orcfile_merge1 Stage: Stage-0 Move Operator @@ -150,7 +126,7 @@ POSTHOOK: Lineage: orcfile_merge1 PARTITION(ds=1,part=0).key EXPRESSION [(src)sr POSTHOOK: Lineage: orcfile_merge1 PARTITION(ds=1,part=0).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] POSTHOOK: Lineage: orcfile_merge1 PARTITION(ds=1,part=1).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: orcfile_merge1 PARTITION(ds=1,part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string,
[8/9] hive git commit: HIVE-20915: Make dynamic sort partition optimization available to HoS and MR (Yongzhi Chen, reviewed by Naveen Gangam)
http://git-wip-us.apache.org/repos/asf/hive/blob/9f2e8e61/ql/src/test/results/clientpositive/autoColumnStats_8.q.out -- diff --git a/ql/src/test/results/clientpositive/autoColumnStats_8.q.out b/ql/src/test/results/clientpositive/autoColumnStats_8.q.out index 492acd2..d0c6602 100644 --- a/ql/src/test/results/clientpositive/autoColumnStats_8.q.out +++ b/ql/src/test/results/clientpositive/autoColumnStats_8.q.out @@ -59,9 +59,9 @@ STAGE DEPENDENCIES: Stage-2 is a root stage Stage-0 depends on stages: Stage-2 Stage-3 depends on stages: Stage-0 - Stage-1 depends on stages: Stage-2 - Stage-4 depends on stages: Stage-1, Stage-5 - Stage-5 depends on stages: Stage-2 + Stage-4 depends on stages: Stage-2 + Stage-1 depends on stages: Stage-4 + Stage-5 depends on stages: Stage-1 STAGE PLANS: Stage: Stage-2 @@ -79,54 +79,14 @@ STAGE PLANS: expressions: key (type: string), value (type: string), ds (type: string), hr (type: string) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 666 Data size: 7075 Basic stats: COMPLETE Column stats: NONE -File Output Operator - compressed: false - GlobalTableId: 1 - A masked pattern was here - NumFilesPerFileSink: 1 - Statistics: Num rows: 666 Data size: 7075 Basic stats: COMPLETE Column stats: NONE - A masked pattern was here - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: -bucket_count -1 -column.name.delimiter , -columns key,value -columns.comments 'default','default' -columns.types string:string - A masked pattern was here -name default.nzhang_part8 -partition_columns ds/hr -partition_columns.types string:string -serialization.ddl struct nzhang_part8 { string key, string value} -serialization.format 1 -serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - A masked pattern was here - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.nzhang_part8 - TotalFiles: 1 - GatherStats: true - MultiFileSpray: false -Select Operator - expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string) - outputColumnNames: key, value, ds, hr - Statistics: Num rows: 666 Data size: 7075 Basic stats: COMPLETE Column stats: NONE - Group By Operator -aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll') -keys: ds (type: string), hr (type: string) -mode: hash -outputColumnNames: _col0, _col1, _col2, _col3 -Statistics: Num rows: 666 Data size: 7075 Basic stats: COMPLETE Column stats: NONE -Reduce Output Operator - key expressions: _col0 (type: string), _col1 (type: string) - null sort order: aa - sort order: ++ - Map-reduce partition columns: _col0 (type: string), _col1 (type: string) - Statistics: Num rows: 666 Data size: 7075 Basic stats: COMPLETE Column stats: NONE - tag: -1 - value expressions: _col2 (type: struct), _col3 (type: struct) - auto parallelism: false +Reduce Output Operator + key expressions: _col2 (type: string), _col3 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: _col2 (type: string), _col3 (type: string) + tag: -1 + value expressions: _col0 (type: string), _col1 (type: string) + auto parallelism: false Filter Operator isSamplingPred: false predicate: (ds > '2008-04-08') (type: boolean) @@ -137,62 +97,23 @@ STAGE PLANS: Statistics: Num rows: 666 Data size: 7075 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - GlobalTableId: 2 + GlobalTableId: 0 A masked pattern was here NumFilesPerFileSink: 1 - Static Partition Specification:
[6/9] hive git commit: HIVE-20915: Make dynamic sort partition optimization available to HoS and MR (Yongzhi Chen, reviewed by Naveen Gangam)
http://git-wip-us.apache.org/repos/asf/hive/blob/9f2e8e61/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid2.q.out -- diff --git a/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid2.q.out b/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid2.q.out index 5bc9442..6c71200 100644 --- a/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid2.q.out +++ b/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid2.q.out @@ -31,10 +31,8 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-2 depends on stages: Stage-1 - Stage-0 depends on stages: Stage-2 - Stage-3 depends on stages: Stage-0, Stage-4 - Stage-4 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + Stage-2 depends on stages: Stage-0 STAGE PLANS: Stage: Stage-1 @@ -48,62 +46,24 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator -key expressions: _col1 (type: string) -sort order: + -Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE -value expressions: _col0 (type: string), _col2 (type: string), _col3 (type: string) +key expressions: _col2 (type: string), _col3 (type: string), _bucket_number (type: string), _col1 (type: string) +sort order: +Map-reduce partition columns: _col2 (type: string), _col3 (type: string) +value expressions: _col0 (type: string) Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: VALUE._col0 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE - File Output Operator -compressed: false -table: -input format: org.apache.hadoop.mapred.SequenceFileInputFormat -output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat -serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - - Stage: Stage-2 -Map Reduce - Map Operator Tree: - TableScan -Reduce Output Operator - sort order: - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string) - Execution mode: vectorized - Reduce Operator Tree: -Select Operator - expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + expressions: VALUE._col0 (type: string), KEY._col1 (type: string), KEY._col2 (type: string), KEY._col3 (type: string), KEY._bucket_number (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _bucket_number File Output Operator compressed: false +Dp Sort State: PARTITION_BUCKET_SORTED Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde name: default.non_acid - Select Operator -expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), UDFToInteger(_col3) (type: int) -outputColumnNames: key, value, ds, hr -Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE -Group By Operator - aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll') - keys: ds (type: string), hr (type: int) - mode: hash - outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE - File Output Operator -compressed: false -table: -input format: org.apache.hadoop.mapred.SequenceFileInputFormat -output format:
[2/9] hive git commit: HIVE-20915: Make dynamic sort partition optimization available to HoS and MR (Yongzhi Chen, reviewed by Naveen Gangam)
http://git-wip-us.apache.org/repos/asf/hive/blob/9f2e8e61/ql/src/test/results/clientpositive/spark/load_dyn_part1.q.out -- diff --git a/ql/src/test/results/clientpositive/spark/load_dyn_part1.q.out b/ql/src/test/results/clientpositive/spark/load_dyn_part1.q.out index 090e9cb..3ca922d 100644 --- a/ql/src/test/results/clientpositive/spark/load_dyn_part1.q.out +++ b/ql/src/test/results/clientpositive/spark/load_dyn_part1.q.out @@ -73,9 +73,12 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-2 Spark + Edges: +Reducer 2 <- Map 4 (PARTITION-LEVEL SORT, 2) +Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 2) A masked pattern was here Vertices: -Map 1 +Map 4 Map Operator Tree: TableScan alias: srcpart @@ -87,14 +90,17 @@ STAGE PLANS: expressions: key (type: string), value (type: string), ds (type: string), hr (type: string) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 666 Data size: 7075 Basic stats: COMPLETE Column stats: NONE - File Output Operator -compressed: false -Statistics: Num rows: 666 Data size: 7075 Basic stats: COMPLETE Column stats: NONE -table: -input format: org.apache.hadoop.mapred.TextInputFormat -output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -name: default.nzhang_part1_n0 + Reduce Output Operator +key expressions: _col2 (type: string), _col3 (type: string) +sort order: ++ +Map-reduce partition columns: _col2 (type: string), _col3 (type: string) +value expressions: _col0 (type: string), _col1 (type: string) +Execution mode: vectorized +Map 5 +Map Operator Tree: +TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (ds > '2008-04-08') (type: boolean) Statistics: Num rows: 666 Data size: 7075 Basic stats: COMPLETE Column stats: NONE @@ -102,15 +108,42 @@ STAGE PLANS: expressions: key (type: string), value (type: string), hr (type: string) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 666 Data size: 7075 Basic stats: COMPLETE Column stats: NONE - File Output Operator -compressed: false -Statistics: Num rows: 666 Data size: 7075 Basic stats: COMPLETE Column stats: NONE -table: -input format: org.apache.hadoop.mapred.TextInputFormat -output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -name: default.nzhang_part2_n0 + Reduce Output Operator +key expressions: _col2 (type: string) +sort order: + +Map-reduce partition columns: _col2 (type: string) +value expressions: _col0 (type: string), _col1 (type: string) +Execution mode: vectorized +Reducer 2 +Execution mode: vectorized +Reduce Operator Tree: + Select Operator +expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), KEY._col2 (type: string), KEY._col3 (type: string) +outputColumnNames: _col0, _col1, _col2, _col3 +File Output Operator + compressed: false + Dp Sort State: PARTITION_SORTED + Statistics: Num rows: 666 Data size: 7075 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.nzhang_part1_n0 +Reducer 3 Execution mode: vectorized +Reduce Operator Tree: + Select Operator +expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), KEY._col2 (type: string) +outputColumnNames: _col0, _col1, _col2 +
hive git commit: HIVE-20695: HoS Query fails with hive.exec.parallel=true (Yongzhi Chen, reviewed by Peter Vary)
Repository: hive Updated Branches: refs/heads/master beccce398 -> f6726f84e HIVE-20695: HoS Query fails with hive.exec.parallel=true (Yongzhi Chen, reviewed by Peter Vary) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/f6726f84 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/f6726f84 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/f6726f84 Branch: refs/heads/master Commit: f6726f84ea17cd825aa832c9c9a134c598ec8755 Parents: beccce3 Author: Yongzhi Chen Authored: Fri Oct 12 23:17:13 2018 -0400 Committer: Yongzhi Chen Committed: Fri Oct 12 23:39:49 2018 -0400 -- .../hive/ql/exec/spark/RemoteHiveSparkClient.java| 4 +++- .../hadoop/hive/ql/exec/spark/SparkUtilities.java| 1 + .../hive/ql/exec/spark/session/SparkSessionImpl.java | 15 --- 3 files changed, 12 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/f6726f84/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java index d31a202..49b7614 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java @@ -229,7 +229,7 @@ public class RemoteHiveSparkClient implements HiveSparkClient { return new RemoteSparkJobRef(hiveConf, jobHandle, sparkJobStatus); } - private void refreshLocalResources(SparkWork sparkWork, HiveConf conf) throws IOException { + private synchronized void refreshLocalResources(SparkWork sparkWork, HiveConf conf) throws IOException { // add hive-exec jar addJars((new JobConf(this.getClass())).getJar()); @@ -264,6 +264,7 @@ public class RemoteHiveSparkClient implements HiveSparkClient { addResources(addedArchives); } + //This method is not thread safe private void addResources(String addedFiles) throws IOException { for (String addedFile : CSV_SPLITTER.split(Strings.nullToEmpty(addedFiles))) { try { @@ -281,6 +282,7 @@ public class RemoteHiveSparkClient implements HiveSparkClient { } } + //This method is not thread safe private void addJars(String addedJars) throws IOException { for (String addedJar : CSV_SPLITTER.split(Strings.nullToEmpty(addedJars))) { try { http://git-wip-us.apache.org/repos/asf/hive/blob/f6726f84/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java index fdc5361..d384ed6 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java @@ -78,6 +78,7 @@ public class SparkUtilities { /** * Uploads a local file to HDFS + * This method is not thread safe * * @param source * @param conf http://git-wip-us.apache.org/repos/asf/hive/blob/f6726f84/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java index 6a8b42e..bb50129 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java @@ -98,7 +98,6 @@ public class SparkSessionImpl implements SparkSession { private final String sessionId; private volatile HiveSparkClient hiveSparkClient; private volatile Path scratchDir; - private final Object dirLock = new Object(); /** * The timestamp of the last completed Spark job. @@ -317,6 +316,7 @@ public class SparkSessionImpl implements SparkSession { return result; } + //This method is not thread safe private void cleanScratchDir() throws IOException { if (scratchDir != null) { FileSystem fs = scratchDir.getFileSystem(conf); @@ -324,15 +324,16 @@ public class SparkSessionImpl implements SparkSession { scratchDir = null; } } - + /** + * Create scratch directory for spark session if it does not exist. + * This method is not thread safe. + * @return Path to Spark session scratch directory. + * @throws IOException + */ @Override public Path getHDFSSessionDir() throws IOException { if (scratchDir ==
hive git commit: HIVE-12812: Enable mapred.input.dir.recursive by default to support union with aggregate function (Chaoyu Tang and Alice Fan, reviewed by Naveen Gangam and Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/master b38239165 -> fb7291ac4 HIVE-12812: Enable mapred.input.dir.recursive by default to support union with aggregate function (Chaoyu Tang and Alice Fan, reviewed by Naveen Gangam and Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/fb7291ac Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/fb7291ac Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/fb7291ac Branch: refs/heads/master Commit: fb7291ac4e8686a0bf91eba5a945b8a8d1ea8b6d Parents: b382391 Author: Yongzhi Chen Authored: Thu Sep 27 12:58:10 2018 -0400 Committer: Yongzhi Chen Committed: Thu Sep 27 12:58:10 2018 -0400 -- .../apache/hadoop/hive/ql/parse/MapReduceCompiler.java | 13 + .../queries/clientpositive/skewjoin_union_remove_1.q | 1 - .../queries/clientpositive/skewjoin_union_remove_2.q | 1 - ql/src/test/queries/clientpositive/union_remove_1.q| 1 - ql/src/test/queries/clientpositive/union_remove_10.q | 1 - ql/src/test/queries/clientpositive/union_remove_11.q | 1 - ql/src/test/queries/clientpositive/union_remove_12.q | 1 - ql/src/test/queries/clientpositive/union_remove_13.q | 1 - ql/src/test/queries/clientpositive/union_remove_14.q | 1 - ql/src/test/queries/clientpositive/union_remove_15.q | 1 - ql/src/test/queries/clientpositive/union_remove_16.q | 1 - ql/src/test/queries/clientpositive/union_remove_17.q | 1 - ql/src/test/queries/clientpositive/union_remove_18.q | 1 - ql/src/test/queries/clientpositive/union_remove_19.q | 1 - ql/src/test/queries/clientpositive/union_remove_2.q| 1 - ql/src/test/queries/clientpositive/union_remove_20.q | 1 - ql/src/test/queries/clientpositive/union_remove_21.q | 1 - ql/src/test/queries/clientpositive/union_remove_22.q | 1 - ql/src/test/queries/clientpositive/union_remove_23.q | 1 - ql/src/test/queries/clientpositive/union_remove_24.q | 1 - ql/src/test/queries/clientpositive/union_remove_25.q | 1 - ql/src/test/queries/clientpositive/union_remove_26.q | 2 -- ql/src/test/queries/clientpositive/union_remove_3.q| 1 - ql/src/test/queries/clientpositive/union_remove_4.q| 1 - ql/src/test/queries/clientpositive/union_remove_5.q| 1 - ql/src/test/queries/clientpositive/union_remove_6.q| 1 - .../test/queries/clientpositive/union_remove_6_subq.q | 1 - ql/src/test/queries/clientpositive/union_remove_7.q| 1 - ql/src/test/queries/clientpositive/union_remove_8.q| 1 - ql/src/test/queries/clientpositive/union_remove_9.q| 1 - ql/src/test/queries/clientpositive/union_remove_plan.q | 1 - 31 files changed, 13 insertions(+), 31 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/fb7291ac/ql/src/java/org/apache/hadoop/hive/ql/parse/MapReduceCompiler.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/MapReduceCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/MapReduceCompiler.java index 9f4a201..302b350 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/MapReduceCompiler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/MapReduceCompiler.java @@ -34,6 +34,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.exec.ConditionalTask; import org.apache.hadoop.hive.ql.exec.FileSinkOperator; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; @@ -54,6 +55,7 @@ import org.apache.hadoop.hive.ql.lib.Node; import org.apache.hadoop.hive.ql.lib.NodeProcessor; import org.apache.hadoop.hive.ql.lib.Rule; import org.apache.hadoop.hive.ql.lib.RuleRegExp; +import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.optimizer.GenMRFileSink1; import org.apache.hadoop.hive.ql.optimizer.GenMROperator; import org.apache.hadoop.hive.ql.optimizer.GenMRProcContext; @@ -70,6 +72,7 @@ import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.MoveWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; +import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; import org.apache.hadoop.hive.shims.ShimLoader; public class MapReduceCompiler extends TaskCompiler { @@ -79,6 +82,16 @@ public class MapReduceCompiler extends TaskCompiler { public MapReduceCompiler() { } + @Override + public void init(QueryState queryState, LogHelper console, Hive db) { +super.init(queryState, console, db); + +//It is required the use of recursive input dirs when hive.optimize.union.remove = true +
hive git commit: HIVE-19897: Add more tests for parallel compilation (Yongzhi Chen, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/master edc53cc0d -> 48ce7ce29 HIVE-19897: Add more tests for parallel compilation (Yongzhi Chen, reviewed by Aihua Xu) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/48ce7ce2 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/48ce7ce2 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/48ce7ce2 Branch: refs/heads/master Commit: 48ce7ce2963ce918c6b4e87b8b446dc0df5c1fe4 Parents: edc53cc Author: Yongzhi Chen Authored: Thu Jun 21 10:08:01 2018 -0400 Committer: Yongzhi Chen Committed: Tue Jul 31 09:12:39 2018 -0400 -- .../apache/hive/jdbc/TestJdbcWithMiniHS2.java | 67 ++-- 1 file changed, 62 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/48ce7ce2/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java -- diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java index 7ef2ced..2139709 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java @@ -281,21 +281,75 @@ public class TestJdbcWithMiniHS2 { } @Test + public void testParallelCompilation3() throws Exception { +Statement stmt = conTestDb.createStatement(); +stmt.execute("set hive.driver.parallel.compilation=true"); +stmt.execute("set hive.server2.async.exec.async.compile=true"); +stmt.close(); +Connection conn = getConnection(testDbName); +stmt = conn.createStatement(); +stmt.execute("set hive.driver.parallel.compilation=true"); +stmt.execute("set hive.server2.async.exec.async.compile=true"); +stmt.close(); +int poolSize = 100; +SynchronousQueue executorQueue1 = new SynchronousQueue(); +ExecutorService workers1 = +new ThreadPoolExecutor(1, poolSize, 20, TimeUnit.SECONDS, executorQueue1); +SynchronousQueue executorQueue2 = new SynchronousQueue(); +ExecutorService workers2 = +new ThreadPoolExecutor(1, poolSize, 20, TimeUnit.SECONDS, executorQueue2); +List> list1 = startTasks(workers1, conTestDb, tableName, 10); +List> list2 = startTasks(workers2, conn, tableName, 10); +finishTasks(list1, workers1); +finishTasks(list2, workers2); +conn.close(); + } + + @Test + public void testParallelCompilation4() throws Exception { +Statement stmt = conTestDb.createStatement(); +stmt.execute("set hive.driver.parallel.compilation=true"); +stmt.execute("set hive.server2.async.exec.async.compile=false"); +stmt.close(); +Connection conn = getConnection(testDbName); +stmt = conn.createStatement(); +stmt.execute("set hive.driver.parallel.compilation=true"); +stmt.execute("set hive.server2.async.exec.async.compile=false"); +stmt.close(); +int poolSize = 100; +SynchronousQueue executorQueue1 = new SynchronousQueue(); +ExecutorService workers1 = +new ThreadPoolExecutor(1, poolSize, 20, TimeUnit.SECONDS, executorQueue1); +SynchronousQueue executorQueue2 = new SynchronousQueue(); +ExecutorService workers2 = +new ThreadPoolExecutor(1, poolSize, 20, TimeUnit.SECONDS, executorQueue2); +List> list1 = startTasks(workers1, conTestDb, tableName, 10); +List> list2 = startTasks(workers2, conn, tableName, 10); +finishTasks(list1, workers1); +finishTasks(list2, workers2); +conn.close(); + } + + @Test public void testConcurrentStatements() throws Exception { startConcurrencyTest(conTestDb, tableName, 50); } private static void startConcurrencyTest(Connection conn, String tableName, int numTasks) { // Start concurrent testing -int POOL_SIZE = 100; -int TASK_COUNT = numTasks; - +int poolSize = 100; SynchronousQueue executorQueue = new SynchronousQueue(); ExecutorService workers = -new ThreadPoolExecutor(1, POOL_SIZE, 20, TimeUnit.SECONDS, executorQueue); +new ThreadPoolExecutor(1, poolSize, 20, TimeUnit.SECONDS, executorQueue); +List> list = startTasks(workers, conn, tableName, numTasks); +finishTasks(list, workers); + } + + private static List> startTasks(ExecutorService workers, Connection conn, + String tableName, int numTasks) { List> list = new ArrayList>(); int i = 0; -while (i < TASK_COUNT) { +while (i < numTasks) { try { Future future = workers.submit(new JDBCTask(conn, i, tableName)); list.add(future); @@ -308,7 +362,10 @@ public class TestJdbcWithMiniHS2 { } } } +return list; + } + private static void
hive git commit: HIVE-20226: HMS getNextNotification will throw exception when request maxEvents exceed table's max_rows (Alice Fan, reviewed by Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/master 1e437e2b1 -> 369490860 HIVE-20226: HMS getNextNotification will throw exception when request maxEvents exceed table's max_rows (Alice Fan, reviewed by Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/36949086 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/36949086 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/36949086 Branch: refs/heads/master Commit: 3694908602c3182eea1171f2696fae1facc245fb Parents: 1e437e2 Author: Yongzhi Chen Authored: Fri Jul 27 10:05:37 2018 -0400 Committer: Yongzhi Chen Committed: Fri Jul 27 10:10:22 2018 -0400 -- .../apache/hadoop/hive/metastore/ObjectStore.java | 3 ++- .../hadoop/hive/metastore/conf/MetastoreConf.java | 2 ++ .../hadoop/hive/metastore/TestObjectStore.java| 18 ++ 3 files changed, 22 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/36949086/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java -- diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java index 606ae50..e6f9acb 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java @@ -9795,10 +9795,11 @@ public class ObjectStore implements RawStore, Configurable { try { openTransaction(); long lastEvent = rqst.getLastEvent(); - int maxEvents = rqst.getMaxEvents() > 0 ? rqst.getMaxEvents() : Integer.MAX_VALUE; query = pm.newQuery(MNotificationLog.class, "eventId > lastEvent"); query.declareParameters("java.lang.Long lastEvent"); query.setOrdering("eventId ascending"); + int maxEventResponse = MetastoreConf.getIntVar(conf, ConfVars.METASTORE_MAX_EVENT_RESPONSE); + int maxEvents = (rqst.getMaxEvents() < maxEventResponse && rqst.getMaxEvents() > 0) ? rqst.getMaxEvents() : maxEventResponse; query.setRange(0, maxEvents); Collection events = (Collection) query.execute(lastEvent); commited = commitTransaction(); http://git-wip-us.apache.org/repos/asf/hive/blob/36949086/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java -- diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java index 5791375..cf5fbbe 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java @@ -594,6 +594,8 @@ public class MetastoreConf { "When used in conjunction with the org.apache.hadoop.hive.ql.parse.MetaDataExportListener pre event listener, \n" + "it is the location to which the metadata will be exported. The default is an empty string, which results in the \n" + "metadata being exported to the current user's home directory on HDFS."), +METASTORE_MAX_EVENT_RESPONSE("metastore.max.event.response", "hive.metastore.max.event.response", 100, +"The parameter will decide the maximum number of events that HMS will respond."), MOVE_EXPORTED_METADATA_TO_TRASH("metastore.metadata.move.exported.metadata.to.trash", "hive.metadata.move.exported.metadata.to.trash", true, "When used in conjunction with the org.apache.hadoop.hive.ql.parse.MetaDataExportListener pre event listener, \n" + http://git-wip-us.apache.org/repos/asf/hive/blob/36949086/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java -- diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java index e53ad77..b74c304 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java @@ -811,6 +811,24 @@ public class
hive git commit: HIVE-19424: NPE In MetaDataFormatters (Alice Fan, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/branch-3 bf5791cde -> f233b9375 HIVE-19424: NPE In MetaDataFormatters (Alice Fan, reviewed by Aihua Xu) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/f233b937 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/f233b937 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/f233b937 Branch: refs/heads/branch-3 Commit: f233b93759a2e4b81ccf493895bfa9c2cb317c5b Parents: bf5791c Author: Aihua Xu Authored: Wed May 16 11:27:53 2018 -0700 Committer: Yongzhi Chen Committed: Tue Jul 24 09:56:50 2018 -0400 -- .../hive/ql/metadata/formatting/JsonMetaDataFormatter.java | 8 .../hive/ql/metadata/formatting/TextMetaDataFormatter.java | 8 2 files changed, 8 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/f233b937/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java index e7b5af6..df0a237 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java @@ -204,15 +204,15 @@ public class JsonMetaDataFormatter implements MetaDataFormatter { if (par.getLocation() != null) { tblLoc = par.getDataLocation().toString(); } -inputFormattCls = par.getInputFormatClass().getName(); -outputFormattCls = par.getOutputFormatClass().getName(); +inputFormattCls = par.getInputFormatClass() == null ? null : par.getInputFormatClass().getName(); +outputFormattCls = par.getOutputFormatClass() == null ? null : par.getOutputFormatClass().getName(); } } else { if (tbl.getPath() != null) { tblLoc = tbl.getDataLocation().toString(); } - inputFormattCls = tbl.getInputFormatClass().getName(); - outputFormattCls = tbl.getOutputFormatClass().getName(); + inputFormattCls = tbl.getInputFormatClass() == null ? null : tbl.getInputFormatClass().getName(); + outputFormattCls = tbl.getOutputFormatClass() == null ? null : tbl.getOutputFormatClass().getName(); } MapBuilder builder = MapBuilder.create(); http://git-wip-us.apache.org/repos/asf/hive/blob/f233b937/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java index 2529923..326cbed 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java @@ -291,15 +291,15 @@ class TextMetaDataFormatter implements MetaDataFormatter { if (par.getLocation() != null) { tblLoc = par.getDataLocation().toString(); } -inputFormattCls = par.getInputFormatClass().getName(); -outputFormattCls = par.getOutputFormatClass().getName(); +inputFormattCls = par.getInputFormatClass() == null ? null : par.getInputFormatClass().getName(); +outputFormattCls = par.getOutputFormatClass() == null ? null : par.getOutputFormatClass().getName(); } } else { if (tbl.getPath() != null) { tblLoc = tbl.getDataLocation().toString(); } - inputFormattCls = tbl.getInputFormatClass().getName(); - outputFormattCls = tbl.getOutputFormatClass().getName(); + inputFormattCls = tbl.getInputFormatClass() == null ? null : tbl.getInputFormatClass().getName(); + outputFormattCls = tbl.getOutputFormatClass() == null ? null : tbl.getOutputFormatClass().getName(); } String owner = tbl.getOwner();
hive git commit: HIVE-20016: Investigate TestJdbcWithMiniHS2.testParallelCompilation3 random failure (Yongzhi Chen, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/master 6311e0b03 -> 3b6d4e279 HIVE-20016: Investigate TestJdbcWithMiniHS2.testParallelCompilation3 random failure (Yongzhi Chen, reviewed by Aihua Xu) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/3b6d4e27 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/3b6d4e27 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/3b6d4e27 Branch: refs/heads/master Commit: 3b6d4e279f717ae82a159b33fc858996bf3a3b53 Parents: 6311e0b Author: Yongzhi Chen Authored: Wed Jul 4 07:58:07 2018 -0400 Committer: Yongzhi Chen Committed: Wed Jul 4 07:58:07 2018 -0400 -- ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/3b6d4e27/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java index 6762ee6..27f0216 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java @@ -897,7 +897,7 @@ public class SessionState { /** * Setup authentication and authorization plugins for this session. */ - private void setupAuth() { + private synchronized void setupAuth() { if (authenticator != null) { // auth has been initialized
hive git commit: HIVE-19404: Revise DDL Task Result Logging (Andrew Sherman, reviewed by Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/master ae008b79b -> 777d62ca8 HIVE-19404: Revise DDL Task Result Logging (Andrew Sherman, reviewed by Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/777d62ca Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/777d62ca Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/777d62ca Branch: refs/heads/master Commit: 777d62ca8ae883c3b0f9dcedcc2bdd3e53647e35 Parents: ae008b7 Author: Yongzhi Chen Authored: Thu Jun 28 15:16:49 2018 -0400 Committer: Yongzhi Chen Committed: Thu Jun 28 15:16:49 2018 -0400 -- ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java | 8 1 file changed, 4 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/777d62ca/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java index 8438dad..b7babd6 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java @@ -2787,7 +2787,7 @@ public class DDLTask extends Task implements Serializable { } else { databases = db.getAllDatabases(); } -LOG.info("results : {}", databases.size()); +LOG.info("Found {} database(s) matching the SHOW DATABASES statement.", databases.size()); // write the results in the file DataOutputStream outStream = getOutputStream(showDatabasesDesc.getResFile()); @@ -2827,7 +2827,7 @@ public class DDLTask extends Task implements Serializable { LOG.debug("pattern: {}", pattern); tablesOrViews = db.getTablesByType(dbName, pattern, type); -LOG.debug("results : {}", tablesOrViews.size()); +LOG.debug("Found {} tables/view(s) matching the SHOW TABLES/VIEWS statement.", tablesOrViews.size()); // write the results in the file DataOutputStream outStream = null; @@ -2943,7 +2943,7 @@ public class DDLTask extends Task implements Serializable { console.printInfo("SHOW FUNCTIONS is deprecated, please use SHOW FUNCTIONS LIKE instead."); funcs = FunctionRegistry.getFunctionNames(showFuncs.getPattern()); } - LOG.info("results : {}", funcs.size()); + LOG.info("Found {} function(s) matching the SHOW FUNCTIONS statement.", funcs.size()); } else { funcs = FunctionRegistry.getFunctionNames(); } @@ -3528,7 +3528,7 @@ public class DDLTask extends Task implements Serializable { Table tbl = db.getTable(showTblStatus.getDbName(), tblName); tbls.add(tbl); } - LOG.info("results : {}", tblStr.size()); + LOG.info("Found {} table(s) matching the SHOW TABLE EXTENDED statement.", tblStr.size()); } // write the results in the file
hive git commit: Revert "HIVE-19897: Add more tests for parallel compilation (Yongzhi Chen, reviewed by Aihua Xu)"
Repository: hive Updated Branches: refs/heads/master 15d7d6dc9 -> 09a73bfba Revert "HIVE-19897: Add more tests for parallel compilation (Yongzhi Chen, reviewed by Aihua Xu)" This reverts commit 596edd181624afb5175fb98eea2c672775c2f378. Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/09a73bfb Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/09a73bfb Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/09a73bfb Branch: refs/heads/master Commit: 09a73bfba3213d24f227714426eecb09a7677ba6 Parents: 15d7d6d Author: Yongzhi Chen Authored: Thu Jun 28 06:17:33 2018 -0400 Committer: Yongzhi Chen Committed: Thu Jun 28 06:17:33 2018 -0400 -- .../apache/hive/jdbc/TestJdbcWithMiniHS2.java | 67 ++-- 1 file changed, 5 insertions(+), 62 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/09a73bfb/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java -- diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java index dba0721..d7d7097 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java @@ -281,75 +281,21 @@ public class TestJdbcWithMiniHS2 { } @Test - public void testParallelCompilation3() throws Exception { -Statement stmt = conTestDb.createStatement(); -stmt.execute("set hive.driver.parallel.compilation=true"); -stmt.execute("set hive.server2.async.exec.async.compile=true"); -stmt.close(); -Connection conn = getConnection(testDbName); -stmt = conn.createStatement(); -stmt.execute("set hive.driver.parallel.compilation=true"); -stmt.execute("set hive.server2.async.exec.async.compile=true"); -stmt.close(); -int poolSize = 100; -SynchronousQueue executorQueue1 = new SynchronousQueue(); -ExecutorService workers1 = -new ThreadPoolExecutor(1, poolSize, 20, TimeUnit.SECONDS, executorQueue1); -SynchronousQueue executorQueue2 = new SynchronousQueue(); -ExecutorService workers2 = -new ThreadPoolExecutor(1, poolSize, 20, TimeUnit.SECONDS, executorQueue2); -List> list1 = startTasks(workers1, conTestDb, tableName, 10); -List> list2 = startTasks(workers2, conn, tableName, 10); -finishTasks(list1, workers1); -finishTasks(list2, workers2); -conn.close(); - } - - @Test - public void testParallelCompilation4() throws Exception { -Statement stmt = conTestDb.createStatement(); -stmt.execute("set hive.driver.parallel.compilation=true"); -stmt.execute("set hive.server2.async.exec.async.compile=false"); -stmt.close(); -Connection conn = getConnection(testDbName); -stmt = conn.createStatement(); -stmt.execute("set hive.driver.parallel.compilation=true"); -stmt.execute("set hive.server2.async.exec.async.compile=false"); -stmt.close(); -int poolSize = 100; -SynchronousQueue executorQueue1 = new SynchronousQueue(); -ExecutorService workers1 = -new ThreadPoolExecutor(1, poolSize, 20, TimeUnit.SECONDS, executorQueue1); -SynchronousQueue executorQueue2 = new SynchronousQueue(); -ExecutorService workers2 = -new ThreadPoolExecutor(1, poolSize, 20, TimeUnit.SECONDS, executorQueue2); -List> list1 = startTasks(workers1, conTestDb, tableName, 10); -List> list2 = startTasks(workers2, conn, tableName, 10); -finishTasks(list1, workers1); -finishTasks(list2, workers2); -conn.close(); - } - - @Test public void testConcurrentStatements() throws Exception { startConcurrencyTest(conTestDb, tableName, 50); } private static void startConcurrencyTest(Connection conn, String tableName, int numTasks) { // Start concurrent testing -int poolSize = 100; +int POOL_SIZE = 100; +int TASK_COUNT = numTasks; + SynchronousQueue executorQueue = new SynchronousQueue(); ExecutorService workers = -new ThreadPoolExecutor(1, poolSize, 20, TimeUnit.SECONDS, executorQueue); -List> list = startTasks(workers, conn, tableName, numTasks); -finishTasks(list, workers); - } - - private static List> startTasks(ExecutorService workers, Connection conn, - String tableName, int numTasks) { +new ThreadPoolExecutor(1, POOL_SIZE, 20, TimeUnit.SECONDS, executorQueue); List> list = new ArrayList>(); int i = 0; -while (i < numTasks) { +while (i < TASK_COUNT) { try { Future future = workers.submit(new JDBCTask(conn, i, tableName)); list.add(future); @@ -362,10 +308,7 @@ public class TestJdbcWithMiniHS2 {
hive git commit: HIVE-19897: Add more tests for parallel compilation (Yongzhi Chen, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/master 8f0973b28 -> 596edd181 HIVE-19897: Add more tests for parallel compilation (Yongzhi Chen, reviewed by Aihua Xu) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/596edd18 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/596edd18 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/596edd18 Branch: refs/heads/master Commit: 596edd181624afb5175fb98eea2c672775c2f378 Parents: 8f0973b Author: Yongzhi Chen Authored: Thu Jun 21 10:08:01 2018 -0400 Committer: Yongzhi Chen Committed: Thu Jun 21 10:27:11 2018 -0400 -- .../apache/hive/jdbc/TestJdbcWithMiniHS2.java | 67 ++-- 1 file changed, 62 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/596edd18/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java -- diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java index d7d7097..dba0721 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java @@ -281,21 +281,75 @@ public class TestJdbcWithMiniHS2 { } @Test + public void testParallelCompilation3() throws Exception { +Statement stmt = conTestDb.createStatement(); +stmt.execute("set hive.driver.parallel.compilation=true"); +stmt.execute("set hive.server2.async.exec.async.compile=true"); +stmt.close(); +Connection conn = getConnection(testDbName); +stmt = conn.createStatement(); +stmt.execute("set hive.driver.parallel.compilation=true"); +stmt.execute("set hive.server2.async.exec.async.compile=true"); +stmt.close(); +int poolSize = 100; +SynchronousQueue executorQueue1 = new SynchronousQueue(); +ExecutorService workers1 = +new ThreadPoolExecutor(1, poolSize, 20, TimeUnit.SECONDS, executorQueue1); +SynchronousQueue executorQueue2 = new SynchronousQueue(); +ExecutorService workers2 = +new ThreadPoolExecutor(1, poolSize, 20, TimeUnit.SECONDS, executorQueue2); +List> list1 = startTasks(workers1, conTestDb, tableName, 10); +List> list2 = startTasks(workers2, conn, tableName, 10); +finishTasks(list1, workers1); +finishTasks(list2, workers2); +conn.close(); + } + + @Test + public void testParallelCompilation4() throws Exception { +Statement stmt = conTestDb.createStatement(); +stmt.execute("set hive.driver.parallel.compilation=true"); +stmt.execute("set hive.server2.async.exec.async.compile=false"); +stmt.close(); +Connection conn = getConnection(testDbName); +stmt = conn.createStatement(); +stmt.execute("set hive.driver.parallel.compilation=true"); +stmt.execute("set hive.server2.async.exec.async.compile=false"); +stmt.close(); +int poolSize = 100; +SynchronousQueue executorQueue1 = new SynchronousQueue(); +ExecutorService workers1 = +new ThreadPoolExecutor(1, poolSize, 20, TimeUnit.SECONDS, executorQueue1); +SynchronousQueue executorQueue2 = new SynchronousQueue(); +ExecutorService workers2 = +new ThreadPoolExecutor(1, poolSize, 20, TimeUnit.SECONDS, executorQueue2); +List> list1 = startTasks(workers1, conTestDb, tableName, 10); +List> list2 = startTasks(workers2, conn, tableName, 10); +finishTasks(list1, workers1); +finishTasks(list2, workers2); +conn.close(); + } + + @Test public void testConcurrentStatements() throws Exception { startConcurrencyTest(conTestDb, tableName, 50); } private static void startConcurrencyTest(Connection conn, String tableName, int numTasks) { // Start concurrent testing -int POOL_SIZE = 100; -int TASK_COUNT = numTasks; - +int poolSize = 100; SynchronousQueue executorQueue = new SynchronousQueue(); ExecutorService workers = -new ThreadPoolExecutor(1, POOL_SIZE, 20, TimeUnit.SECONDS, executorQueue); +new ThreadPoolExecutor(1, poolSize, 20, TimeUnit.SECONDS, executorQueue); +List> list = startTasks(workers, conn, tableName, numTasks); +finishTasks(list, workers); + } + + private static List> startTasks(ExecutorService workers, Connection conn, + String tableName, int numTasks) { List> list = new ArrayList>(); int i = 0; -while (i < TASK_COUNT) { +while (i < numTasks) { try { Future future = workers.submit(new JDBCTask(conn, i, tableName)); list.add(future); @@ -308,7 +362,10 @@ public class TestJdbcWithMiniHS2 { } } } +return list; + } + private static void
hive git commit: Revert "HIVE-13745: UDF current_date、current_timestamp、unix_timestamp NPE (Biao Wu, reviewed by Yongzhi Chen)"
Repository: hive Updated Branches: refs/heads/master 0d787cbc0 -> 50f52b728 Revert "HIVE-13745: UDF current_dateãcurrent_timestampãunix_timestamp NPE (Biao Wu, reviewed by Yongzhi Chen)" This reverts commit fb79870592d775cd836d5611e21ab1c7030aadba. Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/50f52b72 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/50f52b72 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/50f52b72 Branch: refs/heads/master Commit: 50f52b728f911634e03b8ff6251c15edf3b987cb Parents: 0d787cb Author: Yongzhi ChenAuthored: Mon May 14 14:56:15 2018 -0400 Committer: Yongzhi Chen Committed: Mon May 14 14:56:15 2018 -0400 -- .../org/apache/hadoop/hive/conf/HiveConf.java | 1 - .../hadoop/hive/ql/session/SessionState.java| 1 - .../ql/udf/generic/GenericUDFCurrentDate.java | 26 +-- .../udf/generic/GenericUDFCurrentTimestamp.java | 26 +-- .../ql/udf/generic/GenericUDFUnixTimeStamp.java | 27 +--- 5 files changed, 3 insertions(+), 78 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/50f52b72/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java -- diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 514257f..9df9cca 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -1843,7 +1843,6 @@ public class HiveConf extends Configuration { TESTMODE_BUCKET_CODEC_VERSION("hive.test.bucketcodec.version", 1, "For testing only. Will make ACID subsystem write RecordIdentifier.bucketId in specified\n" + "format", false), -HIVE_QUERY_TIMESTAMP("hive.query.timestamp", System.currentTimeMillis(), "query execute time."), HIVEMERGEMAPFILES("hive.merge.mapfiles", true, "Merge small files at the end of a map-only job"), http://git-wip-us.apache.org/repos/asf/hive/blob/50f52b72/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java index 9f65a77..6bb756c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java @@ -1924,7 +1924,6 @@ public class SessionState { */ public void setupQueryCurrentTimestamp() { queryCurrentTimestamp = new Timestamp(System.currentTimeMillis()); -sessionConf.setLongVar(ConfVars.HIVE_QUERY_TIMESTAMP, queryCurrentTimestamp.getTime()); // Provide a facility to set current timestamp during tests if (sessionConf.getBoolVar(ConfVars.HIVE_IN_TEST)) { http://git-wip-us.apache.org/repos/asf/hive/blob/50f52b72/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentDate.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentDate.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentDate.java index 91fd08f..7d3c3f4 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentDate.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentDate.java @@ -18,12 +18,8 @@ package org.apache.hadoop.hive.ql.udf.generic; import java.sql.Date; -import java.sql.Timestamp; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.exec.Description; -import org.apache.hadoop.hive.ql.exec.MapredContext; import org.apache.hadoop.hive.ql.exec.UDFArgumentException; import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException; import org.apache.hadoop.hive.ql.metadata.HiveException; @@ -43,13 +39,6 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn public class GenericUDFCurrentDate extends GenericUDF { protected DateWritable currentDate; - private Configuration conf; - - @Override - public void configure(MapredContext context) { -super.configure(context); -conf = context.getJobConf(); - } @Override public ObjectInspector initialize(ObjectInspector[] arguments) @@ -61,21 +50,8 @@ public class GenericUDFCurrentDate extends GenericUDF { } if (currentDate == null) { - SessionState ss = SessionState.get(); - Timestamp queryTimestamp; - if (ss == null) { -if (conf == null) { - queryTimestamp = new
hive git commit: HIVE-13745: UDF current_date、current_timestamp、unix_timestamp NPE (Biao Wu, reviewed by Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/master 68b66a64f -> fb7987059 HIVE-13745: UDF current_dateãcurrent_timestampãunix_timestamp NPE (Biao Wu, reviewed by Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/fb798705 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/fb798705 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/fb798705 Branch: refs/heads/master Commit: fb79870592d775cd836d5611e21ab1c7030aadba Parents: 68b66a6 Author: Yongzhi ChenAuthored: Fri May 11 06:30:30 2018 -0400 Committer: Yongzhi Chen Committed: Fri May 11 06:30:30 2018 -0400 -- .../org/apache/hadoop/hive/conf/HiveConf.java | 1 + .../hadoop/hive/ql/session/SessionState.java| 1 + .../ql/udf/generic/GenericUDFCurrentDate.java | 26 ++- .../udf/generic/GenericUDFCurrentTimestamp.java | 26 ++- .../ql/udf/generic/GenericUDFUnixTimeStamp.java | 27 +++- 5 files changed, 78 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/fb798705/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java -- diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 33c863d..44b9eb2 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -1843,6 +1843,7 @@ public class HiveConf extends Configuration { TESTMODE_BUCKET_CODEC_VERSION("hive.test.bucketcodec.version", 1, "For testing only. Will make ACID subsystem write RecordIdentifier.bucketId in specified\n" + "format", false), +HIVE_QUERY_TIMESTAMP("hive.query.timestamp", System.currentTimeMillis(), "query execute time."), HIVEMERGEMAPFILES("hive.merge.mapfiles", true, "Merge small files at the end of a map-only job"), http://git-wip-us.apache.org/repos/asf/hive/blob/fb798705/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java index 6bb756c..9f65a77 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java @@ -1924,6 +1924,7 @@ public class SessionState { */ public void setupQueryCurrentTimestamp() { queryCurrentTimestamp = new Timestamp(System.currentTimeMillis()); +sessionConf.setLongVar(ConfVars.HIVE_QUERY_TIMESTAMP, queryCurrentTimestamp.getTime()); // Provide a facility to set current timestamp during tests if (sessionConf.getBoolVar(ConfVars.HIVE_IN_TEST)) { http://git-wip-us.apache.org/repos/asf/hive/blob/fb798705/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentDate.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentDate.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentDate.java index 7d3c3f4..91fd08f 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentDate.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCurrentDate.java @@ -18,8 +18,12 @@ package org.apache.hadoop.hive.ql.udf.generic; import java.sql.Date; +import java.sql.Timestamp; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.MapredContext; import org.apache.hadoop.hive.ql.exec.UDFArgumentException; import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException; import org.apache.hadoop.hive.ql.metadata.HiveException; @@ -39,6 +43,13 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn public class GenericUDFCurrentDate extends GenericUDF { protected DateWritable currentDate; + private Configuration conf; + + @Override + public void configure(MapredContext context) { +super.configure(context); +conf = context.getJobConf(); + } @Override public ObjectInspector initialize(ObjectInspector[] arguments) @@ -50,8 +61,21 @@ public class GenericUDFCurrentDate extends GenericUDF { } if (currentDate == null) { + SessionState ss = SessionState.get(); + Timestamp queryTimestamp; + if (ss == null) { +if (conf == null) { + queryTimestamp = new Timestamp(System.currentTimeMillis()); +} else { + queryTimestamp = new
hive git commit: HIVE-18975: NPE when inserting NULL value in structure and array with HBase table (Oleksiy Sayankin, reviewed by Ashutosh Chauhan and Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/master dec3625b9 -> 8f804af05 HIVE-18975: NPE when inserting NULL value in structure and array with HBase table (Oleksiy Sayankin, reviewed by Ashutosh Chauhan and Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/8f804af0 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/8f804af0 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/8f804af0 Branch: refs/heads/master Commit: 8f804af05389edd7d5fa91e21ab9bdd71cc2c7a5 Parents: dec3625 Author: Yongzhi ChenAuthored: Tue Mar 20 13:12:05 2018 -0400 Committer: Yongzhi Chen Committed: Tue Mar 20 13:12:05 2018 -0400 -- .../hadoop/hive/hbase/HBaseRowSerializer.java | 136 +-- .../src/test/queries/positive/hbase_queries.q | 16 +++ .../test/results/positive/hbase_queries.q.out | 82 +++ 3 files changed, 166 insertions(+), 68 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/8f804af0/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseRowSerializer.java -- diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseRowSerializer.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseRowSerializer.java index be6aa1e..bc4e146 100644 --- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseRowSerializer.java +++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseRowSerializer.java @@ -226,83 +226,83 @@ public class HBaseRowSerializer { return output.toByteArray(); } - private boolean serialize( - Object obj, - ObjectInspector objInspector, - int level, ByteStream.Output ss) throws IOException { + private boolean serialize(Object obj, ObjectInspector objInspector, int level, ByteStream.Output ss) + throws IOException { switch (objInspector.getCategory()) { - case PRIMITIVE: -LazyUtils.writePrimitiveUTF8(ss, obj, -(PrimitiveObjectInspector) objInspector, escaped, escapeChar, needsEscape); -return true; - case LIST: -char separator = (char) separators[level]; -ListObjectInspector loi = (ListObjectInspector)objInspector; -List list = loi.getList(obj); -ObjectInspector eoi = loi.getListElementObjectInspector(); -if (list == null) { - return false; -} else { - for (int i = 0; i < list.size(); i++) { -if (i > 0) { - ss.write(separator); -} -serialize(list.get(i), eoi, level + 1, ss); +case PRIMITIVE: + LazyUtils.writePrimitiveUTF8(ss, obj, (PrimitiveObjectInspector) objInspector, escaped, escapeChar, needsEscape); + return true; +case LIST: + char separator = (char) separators[level]; + ListObjectInspector loi = (ListObjectInspector) objInspector; + List list = loi.getList(obj); + ObjectInspector eoi = loi.getListElementObjectInspector(); + if (list == null) { +return false; + } else { +for (int i = 0; i < list.size(); i++) { + if (i > 0) { +ss.write(separator); } -} -return true; - case MAP: -char sep = (char) separators[level]; -char keyValueSeparator = (char) separators[level+1]; -MapObjectInspector moi = (MapObjectInspector) objInspector; -ObjectInspector koi = moi.getMapKeyObjectInspector(); -ObjectInspector voi = moi.getMapValueObjectInspector(); - -Map map = moi.getMap(obj); -if (map == null) { - return false; -} else { - boolean first = true; - for (Map.Entry entry: map.entrySet()) { -if (first) { - first = false; -} else { - ss.write(sep); -} -serialize(entry.getKey(), koi, level+2, ss); - -if ( entry.getValue() != null) { - ss.write(keyValueSeparator); - serialize(entry.getValue(), voi, level+2, ss); -} + Object currentItem = list.get(i); + if (currentItem != null) { +serialize(currentItem, eoi, level + 1, ss); } } -return true; - case STRUCT: -sep = (char)separators[level]; -StructObjectInspector soi = (StructObjectInspector)objInspector; -List fields = soi.getAllStructFieldRefs(); -list = soi.getStructFieldsDataAsList(obj); -if (list == null) { - return false; -} else { - for (int i = 0; i < list.size(); i++) { -if (i > 0) { - ss.write(sep); -} + } + return true; +case MAP: + char sep = (char)
hive git commit: HIVE-18671: Lock not released after Hive on Spark query was cancelled (Yongzhi Chen, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/branch-2 6dbec04dd -> 042296fbc HIVE-18671: Lock not released after Hive on Spark query was cancelled (Yongzhi Chen, reviewed by Aihua Xu) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/042296fb Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/042296fb Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/042296fb Branch: refs/heads/branch-2 Commit: 042296fbcd4f3a6dee6f06aa3c997a594bc73391 Parents: 6dbec04 Author: Yongzhi ChenAuthored: Tue Feb 13 10:03:53 2018 -0500 Committer: Yongzhi Chen Committed: Tue Feb 13 10:35:59 2018 -0500 -- .../ql/exec/spark/status/RemoteSparkJobMonitor.java | 6 ++ .../hadoop/hive/ql/exec/spark/TestSparkTask.java| 16 2 files changed, 22 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/042296fb/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java index dd73f3e..dc6e951 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java @@ -148,6 +148,12 @@ public class RemoteSparkJobMonitor extends SparkJobMonitor { done = true; rc = 3; break; +case CANCELLED: + console.printInfo("Status: Cancelled"); + running = false; + done = true; + rc = 3; + break; } if (!done) { http://git-wip-us.apache.org/repos/asf/hive/blob/042296fb/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java -- diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java index 4c7ec76..3229ea8 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java @@ -20,11 +20,18 @@ package org.apache.hadoop.hive.ql.exec.spark; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import java.io.IOException; import org.apache.hadoop.hive.common.metrics.common.Metrics; import org.apache.hadoop.hive.common.metrics.common.MetricsConstant; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.spark.status.RemoteSparkJobMonitor; +import org.apache.hadoop.hive.ql.exec.spark.status.impl.RemoteSparkJobStatus; +import org.apache.hadoop.hive.ql.plan.SparkWork; +import org.apache.hive.spark.client.JobHandle.State; +import org.junit.Assert; import org.junit.Test; import org.mockito.Mockito; @@ -43,4 +50,13 @@ public class TestSparkTask { verify(mockMetrics, never()).incrementCounter(MetricsConstant.HIVE_MR_TASKS); } + @Test + public void testRemoteSparkCancel() { +RemoteSparkJobStatus jobSts = Mockito.mock(RemoteSparkJobStatus.class); +when(jobSts.getRemoteJobState()).thenReturn(State.CANCELLED); +when(jobSts.isRemoteActive()).thenReturn(true); +HiveConf hiveConf = new HiveConf(); +RemoteSparkJobMonitor remoteSparkJobMonitor = new RemoteSparkJobMonitor(hiveConf, jobSts); +Assert.assertEquals(remoteSparkJobMonitor.startMonitor(), 3); + } }
hive git commit: HIVE-18671: Lock not released after Hive on Spark query was cancelled (Yongzhi Chen, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/master 1d15990ad -> 9a02aa86b HIVE-18671: Lock not released after Hive on Spark query was cancelled (Yongzhi Chen, reviewed by Aihua Xu) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/9a02aa86 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/9a02aa86 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/9a02aa86 Branch: refs/heads/master Commit: 9a02aa86b9fe4b68681ba1c7129d5028f24791c9 Parents: 1d15990 Author: Yongzhi ChenAuthored: Tue Feb 13 10:03:53 2018 -0500 Committer: Yongzhi Chen Committed: Tue Feb 13 10:24:34 2018 -0500 -- .../ql/exec/spark/status/RemoteSparkJobMonitor.java | 6 ++ .../hadoop/hive/ql/exec/spark/TestSparkTask.java| 16 2 files changed, 22 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/9a02aa86/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java index 22f7024..fc4e4de 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java @@ -174,6 +174,12 @@ public class RemoteSparkJobMonitor extends SparkJobMonitor { done = true; rc = 3; break; +case CANCELLED: + console.printInfo("Status: Cancelled"); + running = false; + done = true; + rc = 3; + break; } if (!done) { http://git-wip-us.apache.org/repos/asf/hive/blob/9a02aa86/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java -- diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java index 928ecc0..435c6b6 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.exec.spark; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import java.io.IOException; import java.util.ArrayList; @@ -27,10 +28,14 @@ import java.util.List; import org.apache.hadoop.hive.common.metrics.common.Metrics; import org.apache.hadoop.hive.common.metrics.common.MetricsConstant; +import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.exec.Task; +import org.apache.hadoop.hive.ql.exec.spark.status.RemoteSparkJobMonitor; +import org.apache.hadoop.hive.ql.exec.spark.status.impl.RemoteSparkJobStatus; import org.apache.hadoop.hive.ql.plan.BaseWork; import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.SparkWork; +import org.apache.hive.spark.client.JobHandle.State; import org.junit.Assert; import org.junit.Test; import org.mockito.Mockito; @@ -81,6 +86,17 @@ public class TestSparkTask { Assert.assertEquals(child1.getParentTasks().size(), 0); } + @Test + public void testRemoteSparkCancel() { +RemoteSparkJobStatus jobSts = Mockito.mock(RemoteSparkJobStatus.class); +when(jobSts.getRemoteJobState()).thenReturn(State.CANCELLED); +when(jobSts.isRemoteActive()).thenReturn(true); +HiveConf hiveConf = new HiveConf(); +RemoteSparkJobMonitor remoteSparkJobMonitor = new RemoteSparkJobMonitor(hiveConf, jobSts); +Assert.assertEquals(remoteSparkJobMonitor.startMonitor(), 3); + } + + private boolean isEmptySparkWork(SparkWork sparkWork) { List allWorks = sparkWork.getAllWork(); boolean allWorksIsEmtpy = true;
hive git commit: HIVE-11609: Capability to add a filter to hbase scan via composite key doesn't work (Barna Zsombor Klara, Swarnim Kulkarni, reviewed by Aihua Xue and Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/branch-2 070649d8b -> 47ff26073 HIVE-11609: Capability to add a filter to hbase scan via composite key doesn't work (Barna Zsombor Klara, Swarnim Kulkarni, reviewed by Aihua Xue and Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/47ff2607 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/47ff2607 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/47ff2607 Branch: refs/heads/branch-2 Commit: 47ff2607332dd6206ef0fdbb55a7321161750451 Parents: 070649d Author: Yongzhi ChenAuthored: Tue Jan 9 11:08:25 2018 -0500 Committer: Yongzhi Chen Committed: Tue Jan 9 11:47:20 2018 -0500 -- .../hadoop/hive/hbase/HBaseScanRange.java | 28 ++- .../hive/hbase/HiveHBaseInputFormatUtil.java| 185 +++ .../hive/hbase/HiveHBaseTableInputFormat.java | 155 +--- .../hive/hbase/SampleHBaseKeyFactory3.java | 20 +- .../results/positive/hbase_custom_key3.q.out| 1 - .../hadoop/hive/ql/plan/TableScanDesc.java | 2 +- 6 files changed, 217 insertions(+), 174 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/47ff2607/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseScanRange.java -- diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseScanRange.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseScanRange.java index 8b64321..7b64a03 100644 --- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseScanRange.java +++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseScanRange.java @@ -18,17 +18,17 @@ package org.apache.hadoop.hive.hbase; +import java.io.Serializable; +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.List; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.FilterList; import org.apache.hadoop.io.BytesWritable; -import java.io.Serializable; -import java.lang.reflect.Method; -import java.util.ArrayList; -import java.util.List; - public class HBaseScanRange implements Serializable { private byte[] startRow; @@ -59,12 +59,20 @@ public class HBaseScanRange implements Serializable { } public void setup(Scan scan, Configuration conf) throws Exception { -if (startRow != null) { - scan.setStartRow(startRow); -} -if (stopRow != null) { - scan.setStopRow(stopRow); +setup(scan, conf, false); + } + + public void setup(Scan scan, Configuration conf, boolean filterOnly) throws Exception { +if (!filterOnly) { + // Set the start and stop rows only if asked to + if (startRow != null) { +scan.setStartRow(startRow); + } + if (stopRow != null) { +scan.setStopRow(stopRow); + } } + if (filterDescs.isEmpty()) { return; } http://git-wip-us.apache.org/repos/asf/hive/blob/47ff2607/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseInputFormatUtil.java -- diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseInputFormatUtil.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseInputFormatUtil.java index 6054d53..f97ff65 100644 --- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseInputFormatUtil.java +++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseInputFormatUtil.java @@ -26,15 +26,38 @@ import java.util.Map; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.client.HTable; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.filter.FilterList; import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; import org.apache.hadoop.hbase.filter.KeyOnlyFilter; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hive.hbase.ColumnMappings.ColumnMapping; +import org.apache.hadoop.hive.ql.exec.ExprNodeConstantEvaluator; +import org.apache.hadoop.hive.ql.exec.SerializationUtilities; +import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.index.IndexSearchCondition; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.TableScanDesc; +import org.apache.hadoop.hive.serde2.ByteStream; import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.io.ByteWritable; +import
hive git commit: HIVE-11609: Capability to add a filter to hbase scan via composite key doesn't work (Barna Zsombor Klara, Swarnim Kulkarni, reviewed by Aihua Xue and Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/master 8412748a7 -> c253955e1 HIVE-11609: Capability to add a filter to hbase scan via composite key doesn't work (Barna Zsombor Klara, Swarnim Kulkarni, reviewed by Aihua Xue and Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/c253955e Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/c253955e Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/c253955e Branch: refs/heads/master Commit: c253955e1a786ab93e569de6520f021f6359d1fe Parents: 8412748 Author: Yongzhi ChenAuthored: Tue Jan 9 11:08:25 2018 -0500 Committer: Yongzhi Chen Committed: Tue Jan 9 11:08:25 2018 -0500 -- .../hadoop/hive/hbase/HBaseScanRange.java | 28 ++- .../hive/hbase/HiveHBaseInputFormatUtil.java| 186 +++ .../hive/hbase/HiveHBaseTableInputFormat.java | 155 +--- .../hive/hbase/SampleHBaseKeyFactory3.java | 20 +- .../results/positive/hbase_custom_key3.q.out| 1 - .../hadoop/hive/ql/plan/TableScanDesc.java | 2 +- 6 files changed, 218 insertions(+), 174 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/c253955e/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseScanRange.java -- diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseScanRange.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseScanRange.java index 8b64321..7b64a03 100644 --- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseScanRange.java +++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseScanRange.java @@ -18,17 +18,17 @@ package org.apache.hadoop.hive.hbase; +import java.io.Serializable; +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.List; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.FilterList; import org.apache.hadoop.io.BytesWritable; -import java.io.Serializable; -import java.lang.reflect.Method; -import java.util.ArrayList; -import java.util.List; - public class HBaseScanRange implements Serializable { private byte[] startRow; @@ -59,12 +59,20 @@ public class HBaseScanRange implements Serializable { } public void setup(Scan scan, Configuration conf) throws Exception { -if (startRow != null) { - scan.setStartRow(startRow); -} -if (stopRow != null) { - scan.setStopRow(stopRow); +setup(scan, conf, false); + } + + public void setup(Scan scan, Configuration conf, boolean filterOnly) throws Exception { +if (!filterOnly) { + // Set the start and stop rows only if asked to + if (startRow != null) { +scan.setStartRow(startRow); + } + if (stopRow != null) { +scan.setStopRow(stopRow); + } } + if (filterDescs.isEmpty()) { return; } http://git-wip-us.apache.org/repos/asf/hive/blob/c253955e/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseInputFormatUtil.java -- diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseInputFormatUtil.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseInputFormatUtil.java index d42b7ec..e60fd7d 100644 --- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseInputFormatUtil.java +++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseInputFormatUtil.java @@ -24,14 +24,38 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.filter.FilterList; import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; import org.apache.hadoop.hbase.filter.KeyOnlyFilter; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hive.hbase.ColumnMappings.ColumnMapping; +import org.apache.hadoop.hive.ql.exec.ExprNodeConstantEvaluator; +import org.apache.hadoop.hive.ql.exec.SerializationUtilities; +import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.index.IndexSearchCondition; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.TableScanDesc; +import org.apache.hadoop.hive.serde2.ByteStream; import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.io.ByteWritable; +import org.apache.hadoop.hive.serde2.io.DoubleWritable;
hive git commit: HIVE-17640: Comparison of date return null if time part is provided in string. (Yongzhi Chen, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/branch-2 a93e99ad4 -> 2106bf777 HIVE-17640: Comparison of date return null if time part is provided in string. (Yongzhi Chen, reviewed by Aihua Xu) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/2106bf77 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/2106bf77 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/2106bf77 Branch: refs/heads/branch-2 Commit: 2106bf777ced8b1a076396db9a8398270f1d922f Parents: a93e99a Author: Yongzhi ChenAuthored: Fri Oct 20 10:29:49 2017 -0400 Committer: Yongzhi Chen Committed: Fri Oct 20 10:31:29 2017 -0400 -- .../queries/clientpositive/date_withtimestamp.q | 3 +++ .../clientpositive/annotate_stats_select.q.out | 4 +-- .../clientpositive/constantfolding.q.out| 4 +-- .../clientpositive/date_withtimestamp.q.out | 27 .../PrimitiveObjectInspectorUtils.java | 16 +--- 5 files changed, 47 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/2106bf77/ql/src/test/queries/clientpositive/date_withtimestamp.q -- diff --git a/ql/src/test/queries/clientpositive/date_withtimestamp.q b/ql/src/test/queries/clientpositive/date_withtimestamp.q new file mode 100644 index 000..b6d04f5 --- /dev/null +++ b/ql/src/test/queries/clientpositive/date_withtimestamp.q @@ -0,0 +1,3 @@ +select "2016-12-29 23:59:59" < cast("2016-12-30" as date); +select "2016-12-30 00:00:00" = cast("2016-12-30" as date); +select "2016-12-31 00:00:01" > cast("2016-12-30" as date); http://git-wip-us.apache.org/repos/asf/hive/blob/2106bf77/ql/src/test/results/clientpositive/annotate_stats_select.q.out -- diff --git a/ql/src/test/results/clientpositive/annotate_stats_select.q.out b/ql/src/test/results/clientpositive/annotate_stats_select.q.out index 873f1ab..489f4d2 100644 --- a/ql/src/test/results/clientpositive/annotate_stats_select.q.out +++ b/ql/src/test/results/clientpositive/annotate_stats_select.q.out @@ -470,9 +470,9 @@ STAGE PLANS: alias: alltypes_orc Statistics: Num rows: 2 Data size: 1686 Basic stats: COMPLETE Column stats: COMPLETE Select Operator -expressions: null (type: date) +expressions: 1970-12-31 (type: date) outputColumnNames: _col0 -Statistics: Num rows: 2 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE +Statistics: Num rows: 2 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE ListSink PREHOOK: query: explain select cast("58.174" as DECIMAL) from alltypes_orc http://git-wip-us.apache.org/repos/asf/hive/blob/2106bf77/ql/src/test/results/clientpositive/constantfolding.q.out -- diff --git a/ql/src/test/results/clientpositive/constantfolding.q.out b/ql/src/test/results/clientpositive/constantfolding.q.out index 10e185f..f9a9d24 100644 --- a/ql/src/test/results/clientpositive/constantfolding.q.out +++ b/ql/src/test/results/clientpositive/constantfolding.q.out @@ -205,9 +205,9 @@ STAGE PLANS: alias: src Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE Select Operator -expressions: null (type: date) +expressions: 1970-12-31 (type: date) outputColumnNames: _col0 -Statistics: Num rows: 500 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE +Statistics: Num rows: 500 Data size: 28000 Basic stats: COMPLETE Column stats: COMPLETE ListSink PREHOOK: query: CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE http://git-wip-us.apache.org/repos/asf/hive/blob/2106bf77/ql/src/test/results/clientpositive/date_withtimestamp.q.out -- diff --git a/ql/src/test/results/clientpositive/date_withtimestamp.q.out b/ql/src/test/results/clientpositive/date_withtimestamp.q.out new file mode 100644 index 000..3661888 --- /dev/null +++ b/ql/src/test/results/clientpositive/date_withtimestamp.q.out @@ -0,0 +1,27 @@ +PREHOOK: query: select "2016-12-29 23:59:59" < cast("2016-12-30" as date) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table + A masked pattern was here +POSTHOOK: query: select "2016-12-29 23:59:59" < cast("2016-12-30" as date) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table + A masked pattern was here +true +PREHOOK: query: select "2016-12-30 00:00:00" = cast("2016-12-30" as date) +PREHOOK: type: QUERY
hive git commit: HIVE-16914: Change HiveMetaStoreClient to AutoCloseable (Xiaomeng Zhang, reviewed by Sergio Pena, Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/master 1b0d8df58 -> 86c0bcb42 HIVE-16914: Change HiveMetaStoreClient to AutoCloseable (Xiaomeng Zhang, reviewed by Sergio Pena, Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/86c0bcb4 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/86c0bcb4 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/86c0bcb4 Branch: refs/heads/master Commit: 86c0bcb42cff2c615ea376074f60db68e84bd00e Parents: 1b0d8df Author: Yongzhi ChenAuthored: Fri Jun 30 14:23:13 2017 -0400 Committer: Yongzhi Chen Committed: Fri Jun 30 14:28:34 2017 -0400 -- .../java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/86c0bcb4/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java -- diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java index b99f40a..4c96532 100644 --- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java +++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java @@ -90,7 +90,7 @@ import com.google.common.collect.Lists; */ @Public @Unstable -public class HiveMetaStoreClient implements IMetaStoreClient { +public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable { /** * Capabilities of the current client. If this client talks to a MetaStore server in a manner * implying the usage of some expanded features that require client-side support that this client
hive git commit: HIVE-16869: Hive returns wrong result when predicates on non-existing columns are pushed down to Parquet reader (Yibing Shi, reviewed by Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/master 055f6a0de -> cb0350ce5 HIVE-16869: Hive returns wrong result when predicates on non-existing columns are pushed down to Parquet reader (Yibing Shi, reviewed by Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/cb0350ce Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/cb0350ce Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/cb0350ce Branch: refs/heads/master Commit: cb0350ce56c2957f4985b1ae60537c20faf52475 Parents: 055f6a0 Author: Yongzhi ChenAuthored: Tue Jun 27 09:19:47 2017 -0400 Committer: Yongzhi Chen Committed: Tue Jun 27 09:19:47 2017 -0400 -- .../ql/io/parquet/ParquetRecordReaderBase.java | 2 +- .../read/ParquetFilterPredicateConverter.java | 13 ++--- .../read/TestParquetFilterPredicate.java| 56 +--- .../parquet_predicate_pushdown_2.q | 7 +++ .../parquet_predicate_pushdown_2.q.out | 38 + 5 files changed, 103 insertions(+), 13 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/cb0350ce/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ParquetRecordReaderBase.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ParquetRecordReaderBase.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ParquetRecordReaderBase.java index be9fb10..d788784 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ParquetRecordReaderBase.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ParquetRecordReaderBase.java @@ -197,7 +197,7 @@ public class ParquetRecordReaderBase { FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema); if (p != null) { // Filter may have sensitive information. Do not send to debug. - LOG.debug("PARQUET predicate push down generated."); + LOG.debug("PARQUET predicate push down generated. Predicates = [" + p + "]"); ParquetInputFormat.setFilterPredicate(conf, p); return FilterCompat.get(p); } else { http://git-wip-us.apache.org/repos/asf/hive/blob/cb0350ce/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetFilterPredicateConverter.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetFilterPredicateConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetFilterPredicateConverter.java index 4f8..b7a6bc5 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetFilterPredicateConverter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetFilterPredicateConverter.java @@ -66,14 +66,15 @@ public class ParquetFilterPredicateConverter { switch (root.getOperator()) { case OR: for(ExpressionTree child: root.getChildren()) { + FilterPredicate childPredicate = translate(child, leaves, columns, schema); + if (childPredicate == null) { +return null; + } + if (p == null) { -p = translate(child, leaves, columns, schema); +p = childPredicate; } else { -FilterPredicate right = translate(child, leaves, columns, schema); -// constant means no filter, ignore it when it is null -if(right != null){ - p = FilterApi.or(p, right); -} +p = FilterApi.or(p, childPredicate); } } return p; http://git-wip-us.apache.org/repos/asf/hive/blob/cb0350ce/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/read/TestParquetFilterPredicate.java -- diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/read/TestParquetFilterPredicate.java b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/read/TestParquetFilterPredicate.java index bd1f5e0..8e7acd2 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/read/TestParquetFilterPredicate.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/read/TestParquetFilterPredicate.java @@ -33,19 +33,63 @@ public class TestParquetFilterPredicate { public void testFilterColumnsThatDoNoExistOnSchema() { MessageType schema = MessageTypeParser.parseMessageType("message test { required int32 a; required binary stinger; }"); SearchArgument sarg = SearchArgumentFactory.newBuilder() -.startNot() +.startNot() +.startOr() +.isNull("a", PredicateLeaf.Type.LONG) +.between("y", PredicateLeaf.Type.LONG, 10L, 20L) // Column will be removed from filter +.in("z",
[2/2] hive git commit: HIVE-16875: Query against view with partitioned child on HoS fails with privilege exception. (Yongzhi Chen, reviewed by Aihua Xu)
HIVE-16875: Query against view with partitioned child on HoS fails with privilege exception. (Yongzhi Chen, reviewed by Aihua Xu) Conflicts: ql/src/test/results/clientpositive/spark/louter_join_ppr.q.out ql/src/test/results/clientpositive/spark/outer_join_ppr.q.out ql/src/test/results/clientpositive/spark/router_join_ppr.q.out Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/3298e7f4 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/3298e7f4 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/3298e7f4 Branch: refs/heads/branch-2 Commit: 3298e7f4d0e1cf372163cb19c60a68b1c1c221e2 Parents: 26ac418 Author: Yongzhi ChenAuthored: Mon Jun 12 16:31:12 2017 -0400 Committer: Yongzhi Chen Committed: Thu Jun 22 09:36:50 2017 -0400 -- .../test/resources/testconfiguration.properties | 1 + .../hive/ql/parse/spark/GenSparkUtils.java | 20 +-- .../clientpositive/explaindenpendencydiffengs.q | 10 .../explaindenpendencydiffengs.q.out| 52 ++ .../spark/auto_join_reordering_values.q.out | 10 ++-- .../spark/auto_sortmerge_join_1.q.out | 6 +- .../spark/auto_sortmerge_join_12.q.out | 10 ++-- .../spark/auto_sortmerge_join_3.q.out | 6 +- .../spark/auto_sortmerge_join_4.q.out | 6 +- .../spark/auto_sortmerge_join_5.q.out | 4 +- .../spark/auto_sortmerge_join_7.q.out | 8 +-- .../spark/auto_sortmerge_join_8.q.out | 8 +-- .../spark/bucket_map_join_spark1.q.out | 8 +-- .../spark/bucket_map_join_spark2.q.out | 8 +-- .../spark/bucket_map_join_spark3.q.out | 8 +-- .../spark/bucket_map_join_spark4.q.out | 12 ++-- .../clientpositive/spark/bucketmapjoin1.q.out | 8 +-- .../clientpositive/spark/bucketmapjoin2.q.out | 14 ++--- .../clientpositive/spark/bucketmapjoin3.q.out | 8 +-- .../clientpositive/spark/bucketmapjoin4.q.out | 8 +-- .../spark/explaindenpendencydiffengs.q.out | 52 ++ .../spark/filter_join_breaktask.q.out | 6 +- .../clientpositive/spark/groupby_map_ppr.q.out | 4 +- .../spark/groupby_map_ppr_multi_distinct.q.out | 4 +- .../clientpositive/spark/groupby_ppr.q.out | 4 +- .../spark/groupby_ppr_multi_distinct.q.out | 4 +- .../spark/groupby_sort_1_23.q.out | 42 +++--- .../spark/groupby_sort_skew_1_23.q.out | 42 +++--- .../results/clientpositive/spark/join17.q.out | 4 +- .../results/clientpositive/spark/join26.q.out | 6 +- .../results/clientpositive/spark/join32.q.out | 6 +- .../clientpositive/spark/join32_lessSize.q.out | 26 - .../results/clientpositive/spark/join33.q.out | 6 +- .../results/clientpositive/spark/join34.q.out | 6 +- .../results/clientpositive/spark/join35.q.out | 6 +- .../results/clientpositive/spark/join9.q.out| 4 +- .../spark/join_filters_overlap.q.out| 34 ++-- .../clientpositive/spark/louter_join_ppr.q.out | 28 +- .../clientpositive/spark/mapjoin_mapjoin.q.out | 12 ++-- .../spark/optimize_nullscan.q.out | 58 ++-- .../clientpositive/spark/outer_join_ppr.q.out | 16 +++--- .../test/results/clientpositive/spark/pcr.q.out | 8 +-- .../clientpositive/spark/ppd_join_filter.q.out | 16 +++--- .../clientpositive/spark/router_join_ppr.q.out | 28 +- .../clientpositive/spark/smb_mapjoin_15.q.out | 16 +++--- .../clientpositive/spark/transform_ppr1.q.out | 8 +-- .../clientpositive/spark/transform_ppr2.q.out | 4 +- .../results/clientpositive/spark/union22.q.out | 6 +- .../results/clientpositive/spark/union24.q.out | 24 .../clientpositive/spark/union_ppr.q.out| 8 +-- 50 files changed, 414 insertions(+), 289 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/3298e7f4/itests/src/test/resources/testconfiguration.properties -- diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties index 825c900..3ef27e4 100644 --- a/itests/src/test/resources/testconfiguration.properties +++ b/itests/src/test/resources/testconfiguration.properties @@ -882,6 +882,7 @@ spark.query.files=add_part_multiple.q, \ escape_distributeby1.q, \ escape_orderby1.q, \ escape_sortby1.q, \ + explaindenpendencydiffengs.q, \ filter_join_breaktask.q, \ filter_join_breaktask2.q, \ groupby1.q, \ http://git-wip-us.apache.org/repos/asf/hive/blob/3298e7f4/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java -- diff --git
[1/2] hive git commit: HIVE-16875: Query against view with partitioned child on HoS fails with privilege exception. (Yongzhi Chen, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/branch-2 26ac4188c -> 3298e7f4d http://git-wip-us.apache.org/repos/asf/hive/blob/3298e7f4/ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out -- diff --git a/ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out b/ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out index 35cbe54..76d1f38 100644 --- a/ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out +++ b/ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out @@ -164,7 +164,7 @@ STAGE PLANS: tag: 0 auto parallelism: false Path -> Alias: - nullscan://null/default.src/part_ [src] + nullscan://null/default.src/part_ [a:src] Path -> Partition: nullscan://null/default.src/part_ Partition @@ -212,7 +212,7 @@ STAGE PLANS: name: default.src name: default.src Truncated Path -> Alias: - nullscan://null/default.src/part_ [src] + nullscan://null/default.src/part_ [a:src] Map 3 Map Operator Tree: TableScan @@ -234,10 +234,10 @@ STAGE PLANS: value expressions: _col0 (type: string) auto parallelism: false Path -> Alias: - nullscan://null/default.srcpart/part_ds=2008-04-08_hr=11_ [srcpart] - nullscan://null/default.srcpart/part_ds=2008-04-08_hr=12_ [srcpart] - nullscan://null/default.srcpart/part_ds=2008-04-09_hr=11_ [srcpart] - nullscan://null/default.srcpart/part_ds=2008-04-09_hr=12_ [srcpart] + nullscan://null/default.srcpart/part_ds=2008-04-08_hr=11_ [b:srcpart] + nullscan://null/default.srcpart/part_ds=2008-04-08_hr=12_ [b:srcpart] + nullscan://null/default.srcpart/part_ds=2008-04-09_hr=11_ [b:srcpart] + nullscan://null/default.srcpart/part_ds=2008-04-09_hr=12_ [b:srcpart] Path -> Partition: nullscan://null/default.srcpart/part_ds=2008-04-08_hr=11_ Partition @@ -428,10 +428,10 @@ STAGE PLANS: name: default.srcpart name: default.srcpart Truncated Path -> Alias: - nullscan://null/default.srcpart/part_ds=2008-04-08_hr=11_ [srcpart] - nullscan://null/default.srcpart/part_ds=2008-04-08_hr=12_ [srcpart] - nullscan://null/default.srcpart/part_ds=2008-04-09_hr=11_ [srcpart] - nullscan://null/default.srcpart/part_ds=2008-04-09_hr=12_ [srcpart] + nullscan://null/default.srcpart/part_ds=2008-04-08_hr=11_ [b:srcpart] + nullscan://null/default.srcpart/part_ds=2008-04-08_hr=12_ [b:srcpart] + nullscan://null/default.srcpart/part_ds=2008-04-09_hr=11_ [b:srcpart] + nullscan://null/default.srcpart/part_ds=2008-04-09_hr=12_ [b:srcpart] Reducer 2 Needs Tagging: true Reduce Operator Tree: @@ -549,7 +549,7 @@ STAGE PLANS: value expressions: _col0 (type: bigint) auto parallelism: false Path -> Alias: - nullscan://null/default.src/part_ [src] + nullscan://null/default.src/part_ [null-subquery1:_u1-subquery1:src] Path -> Partition: nullscan://null/default.src/part_ Partition @@ -597,7 +597,7 @@ STAGE PLANS: name: default.src name: default.src Truncated Path -> Alias: - nullscan://null/default.src/part_ [src] + nullscan://null/default.src/part_ [null-subquery1:_u1-subquery1:src] Map 3 Map Operator Tree: TableScan @@ -816,10 +816,10 @@ STAGE PLANS: name: default.srcpart name: default.srcpart Truncated Path -> Alias: - /srcpart/ds=2008-04-08/hr=11 [srcpart] - /srcpart/ds=2008-04-08/hr=12 [srcpart] - /srcpart/ds=2008-04-09/hr=11 [srcpart] - /srcpart/ds=2008-04-09/hr=12 [srcpart] + /srcpart/ds=2008-04-08/hr=11 [null-subquery2:_u1-subquery2:srcpart] + /srcpart/ds=2008-04-08/hr=12 [null-subquery2:_u1-subquery2:srcpart] + /srcpart/ds=2008-04-09/hr=11 [null-subquery2:_u1-subquery2:srcpart] + /srcpart/ds=2008-04-09/hr=12 [null-subquery2:_u1-subquery2:srcpart] Reducer 2 Needs Tagging: false Reduce Operator Tree: @@ -948,7 +948,7 @@ STAGE PLANS: value expressions: _col0 (type: string) auto parallelism: false Path -> Alias: -
[2/2] hive git commit: HIVE-16875: Query against view with partitioned child on HoS fails with privilege exception. (Yongzhi Chen, reviewed by Aihua Xu)
HIVE-16875: Query against view with partitioned child on HoS fails with privilege exception. (Yongzhi Chen, reviewed by Aihua Xu) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/71f52d8a Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/71f52d8a Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/71f52d8a Branch: refs/heads/master Commit: 71f52d8ad512904b3f2c4f04fe39a33f2834f1f2 Parents: 94e9285 Author: Yongzhi ChenAuthored: Mon Jun 12 16:31:12 2017 -0400 Committer: Yongzhi Chen Committed: Thu Jun 22 08:58:27 2017 -0400 -- .../test/resources/testconfiguration.properties | 1 + .../hive/ql/parse/spark/GenSparkUtils.java | 20 +-- .../clientpositive/explaindenpendencydiffengs.q | 10 .../explaindenpendencydiffengs.q.out| 52 ++ .../spark/auto_join_reordering_values.q.out | 10 ++-- .../spark/auto_sortmerge_join_1.q.out | 6 +- .../spark/auto_sortmerge_join_12.q.out | 10 ++-- .../spark/auto_sortmerge_join_3.q.out | 6 +- .../spark/auto_sortmerge_join_4.q.out | 6 +- .../spark/auto_sortmerge_join_5.q.out | 4 +- .../spark/auto_sortmerge_join_7.q.out | 8 +-- .../spark/auto_sortmerge_join_8.q.out | 8 +-- .../spark/bucket_map_join_spark1.q.out | 8 +-- .../spark/bucket_map_join_spark2.q.out | 8 +-- .../spark/bucket_map_join_spark3.q.out | 8 +-- .../spark/bucket_map_join_spark4.q.out | 12 ++-- .../clientpositive/spark/bucketmapjoin1.q.out | 8 +-- .../clientpositive/spark/bucketmapjoin2.q.out | 14 ++--- .../clientpositive/spark/bucketmapjoin3.q.out | 8 +-- .../clientpositive/spark/bucketmapjoin4.q.out | 8 +-- .../spark/explaindenpendencydiffengs.q.out | 52 ++ .../spark/filter_join_breaktask.q.out | 6 +- .../clientpositive/spark/groupby_map_ppr.q.out | 4 +- .../spark/groupby_map_ppr_multi_distinct.q.out | 4 +- .../clientpositive/spark/groupby_ppr.q.out | 4 +- .../spark/groupby_ppr_multi_distinct.q.out | 4 +- .../spark/groupby_sort_1_23.q.out | 42 +++--- .../spark/groupby_sort_skew_1_23.q.out | 42 +++--- .../results/clientpositive/spark/join17.q.out | 4 +- .../results/clientpositive/spark/join26.q.out | 6 +- .../results/clientpositive/spark/join32.q.out | 6 +- .../clientpositive/spark/join32_lessSize.q.out | 26 - .../results/clientpositive/spark/join33.q.out | 6 +- .../results/clientpositive/spark/join34.q.out | 6 +- .../results/clientpositive/spark/join35.q.out | 6 +- .../results/clientpositive/spark/join9.q.out| 4 +- .../spark/join_filters_overlap.q.out| 34 ++-- .../clientpositive/spark/louter_join_ppr.q.out | 24 .../clientpositive/spark/mapjoin_mapjoin.q.out | 12 ++-- .../spark/optimize_nullscan.q.out | 58 ++-- .../clientpositive/spark/outer_join_ppr.q.out | 12 ++-- .../test/results/clientpositive/spark/pcr.q.out | 20 +++ .../clientpositive/spark/ppd_join_filter.q.out | 16 +++--- .../clientpositive/spark/router_join_ppr.q.out | 24 .../clientpositive/spark/smb_mapjoin_15.q.out | 16 +++--- .../clientpositive/spark/transform_ppr1.q.out | 8 +-- .../clientpositive/spark/transform_ppr2.q.out | 4 +- .../results/clientpositive/spark/union22.q.out | 6 +- .../results/clientpositive/spark/union24.q.out | 24 .../clientpositive/spark/union_ppr.q.out| 8 +-- 50 files changed, 414 insertions(+), 289 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/71f52d8a/itests/src/test/resources/testconfiguration.properties -- diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties index 07fd5bf..ac62dcc 100644 --- a/itests/src/test/resources/testconfiguration.properties +++ b/itests/src/test/resources/testconfiguration.properties @@ -912,6 +912,7 @@ spark.query.files=add_part_multiple.q, \ escape_distributeby1.q, \ escape_orderby1.q, \ escape_sortby1.q, \ + explaindenpendencydiffengs.q, \ filter_join_breaktask.q, \ filter_join_breaktask2.q, \ groupby1.q, \ http://git-wip-us.apache.org/repos/asf/hive/blob/71f52d8a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java index d0a82af..b9901da 100644 ---
[1/2] hive git commit: HIVE-16875: Query against view with partitioned child on HoS fails with privilege exception. (Yongzhi Chen, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/master 94e92858a -> 71f52d8ad http://git-wip-us.apache.org/repos/asf/hive/blob/71f52d8a/ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out -- diff --git a/ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out b/ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out index b3478bf..d294f49 100644 --- a/ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out +++ b/ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out @@ -164,7 +164,7 @@ STAGE PLANS: tag: 0 auto parallelism: false Path -> Alias: - nullscan://null/default.src/part_ [src] + nullscan://null/default.src/part_ [a:src] Path -> Partition: nullscan://null/default.src/part_ Partition @@ -212,7 +212,7 @@ STAGE PLANS: name: default.src name: default.src Truncated Path -> Alias: - nullscan://null/default.src/part_ [src] + nullscan://null/default.src/part_ [a:src] Map 3 Map Operator Tree: TableScan @@ -234,10 +234,10 @@ STAGE PLANS: value expressions: _col0 (type: string) auto parallelism: false Path -> Alias: - nullscan://null/default.srcpart/part_ds=2008-04-08_hr=11_ [srcpart] - nullscan://null/default.srcpart/part_ds=2008-04-08_hr=12_ [srcpart] - nullscan://null/default.srcpart/part_ds=2008-04-09_hr=11_ [srcpart] - nullscan://null/default.srcpart/part_ds=2008-04-09_hr=12_ [srcpart] + nullscan://null/default.srcpart/part_ds=2008-04-08_hr=11_ [b:srcpart] + nullscan://null/default.srcpart/part_ds=2008-04-08_hr=12_ [b:srcpart] + nullscan://null/default.srcpart/part_ds=2008-04-09_hr=11_ [b:srcpart] + nullscan://null/default.srcpart/part_ds=2008-04-09_hr=12_ [b:srcpart] Path -> Partition: nullscan://null/default.srcpart/part_ds=2008-04-08_hr=11_ Partition @@ -428,10 +428,10 @@ STAGE PLANS: name: default.srcpart name: default.srcpart Truncated Path -> Alias: - nullscan://null/default.srcpart/part_ds=2008-04-08_hr=11_ [srcpart] - nullscan://null/default.srcpart/part_ds=2008-04-08_hr=12_ [srcpart] - nullscan://null/default.srcpart/part_ds=2008-04-09_hr=11_ [srcpart] - nullscan://null/default.srcpart/part_ds=2008-04-09_hr=12_ [srcpart] + nullscan://null/default.srcpart/part_ds=2008-04-08_hr=11_ [b:srcpart] + nullscan://null/default.srcpart/part_ds=2008-04-08_hr=12_ [b:srcpart] + nullscan://null/default.srcpart/part_ds=2008-04-09_hr=11_ [b:srcpart] + nullscan://null/default.srcpart/part_ds=2008-04-09_hr=12_ [b:srcpart] Reducer 2 Needs Tagging: true Reduce Operator Tree: @@ -549,7 +549,7 @@ STAGE PLANS: value expressions: _col0 (type: bigint) auto parallelism: false Path -> Alias: - nullscan://null/default.src/part_ [src] + nullscan://null/default.src/part_ [null-subquery1:_u1-subquery1:src] Path -> Partition: nullscan://null/default.src/part_ Partition @@ -597,7 +597,7 @@ STAGE PLANS: name: default.src name: default.src Truncated Path -> Alias: - nullscan://null/default.src/part_ [src] + nullscan://null/default.src/part_ [null-subquery1:_u1-subquery1:src] Map 3 Map Operator Tree: TableScan @@ -816,10 +816,10 @@ STAGE PLANS: name: default.srcpart name: default.srcpart Truncated Path -> Alias: - /srcpart/ds=2008-04-08/hr=11 [srcpart] - /srcpart/ds=2008-04-08/hr=12 [srcpart] - /srcpart/ds=2008-04-09/hr=11 [srcpart] - /srcpart/ds=2008-04-09/hr=12 [srcpart] + /srcpart/ds=2008-04-08/hr=11 [null-subquery2:_u1-subquery2:srcpart] + /srcpart/ds=2008-04-08/hr=12 [null-subquery2:_u1-subquery2:srcpart] + /srcpart/ds=2008-04-09/hr=11 [null-subquery2:_u1-subquery2:srcpart] + /srcpart/ds=2008-04-09/hr=12 [null-subquery2:_u1-subquery2:srcpart] Reducer 2 Needs Tagging: false Reduce Operator Tree: @@ -948,7 +948,7 @@ STAGE PLANS: value expressions: _col0 (type: string) auto parallelism: false Path -> Alias: -
hive git commit: HIVE-16643: BeeLine tests output should keep the PREHOOK/POSTHOOK Input/Output orderdering (Peter Vary, reviewed by Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/master 1bdc5413d -> dec96ca6f HIVE-16643: BeeLine tests output should keep the PREHOOK/POSTHOOK Input/Output orderdering (Peter Vary, reviewed by Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/dec96ca6 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/dec96ca6 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/dec96ca6 Branch: refs/heads/master Commit: dec96ca6f6fb8de04b2847be2285792275ea9423 Parents: 1bdc541 Author: Yongzhi ChenAuthored: Wed May 24 10:30:13 2017 -0400 Committer: Yongzhi Chen Committed: Wed May 24 10:30:13 2017 -0400 -- .../test/resources/testconfiguration.properties | 3 +- .../java/org/apache/hive/beeline/QFile.java | 32 - .../beeline/create_merge_compressed.q.out | 138 +++ .../beeline/escape_comments.q.out | 2 - 4 files changed, 168 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/dec96ca6/itests/src/test/resources/testconfiguration.properties -- diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties index e23ef63..8e99b85 100644 --- a/itests/src/test/resources/testconfiguration.properties +++ b/itests/src/test/resources/testconfiguration.properties @@ -756,7 +756,8 @@ encrypted.query.files=encryption_join_unencrypted_tbl.q,\ encryption_auto_purge_tables.q \ encryption_drop_table_in_encrypted_db.q -beeline.positive.include=drop_with_concurrency.q,\ +beeline.positive.include=create_merge_compressed.q,\ + drop_with_concurrency.q,\ escape_comments.q,\ mapjoin2.q,\ materialized_view_create_rewrite.q,\ http://git-wip-us.apache.org/repos/asf/hive/blob/dec96ca6/itests/util/src/main/java/org/apache/hive/beeline/QFile.java -- diff --git a/itests/util/src/main/java/org/apache/hive/beeline/QFile.java b/itests/util/src/main/java/org/apache/hive/beeline/QFile.java index af464b9..0d00a9b 100644 --- a/itests/util/src/main/java/org/apache/hive/beeline/QFile.java +++ b/itests/util/src/main/java/org/apache/hive/beeline/QFile.java @@ -31,8 +31,11 @@ import java.io.IOException; import java.io.PrintStream; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Set; +import java.util.regex.Matcher; import java.util.regex.Pattern; /** @@ -59,6 +62,8 @@ public final class QFile { private static final Pattern USE_PATTERN = Pattern.compile("^\\s*use\\s.*", Pattern.CASE_INSENSITIVE); + private static final Pattern ENTITYLIST_PATTERN = + Pattern.compile("(((PREHOOK|POSTHOOK): (Input|Output): \\S+\n)+)", Pattern.MULTILINE); private static final String MASK_PATTERN = " A masked pattern was here \n"; @@ -174,13 +179,29 @@ public final class QFile { return source; } + /** + * The PREHOOK/POSTHOOK Input/Output lists should be sorted again after reverting the database + * name in those strings to match the original Cli output. + * @param source The original query output + * @return The query output where the input/output list are alphabetically ordered + */ + private String sortInputOutput(String source) { +Matcher matcher = ENTITYLIST_PATTERN.matcher(source); +while(matcher.find()) { + List lines = Arrays.asList(matcher.group(1).split("\n")); + Collections.sort(lines); + source = source.replaceAll(matcher.group(1), String.join("\n", lines) + "\n"); +} +return source; + } + public void filterOutput() throws IOException { -String rawOutput = FileUtils.readFileToString(rawOutputFile, "UTF-8"); +String output = FileUtils.readFileToString(rawOutputFile, "UTF-8"); +output = staticFilterSet.filter(specificFilterSet.filter(output)); if (rewriteSourceTables) { - rawOutput = revertReplaceTableNames(rawOutput); + output = sortInputOutput(revertReplaceTableNames(output)); } -String filteredOutput = staticFilterSet.filter(specificFilterSet.filter(rawOutput)); -FileUtils.writeStringToFile(outputFile, filteredOutput); +FileUtils.writeStringToFile(outputFile, output); } public QTestProcessExecResult compareResults() throws IOException, InterruptedException { @@ -285,6 +306,9 @@ public final class QFile { .addFilter(".*file\\..*\n", MASK_PATTERN) .addFilter(".*CreateTime.*\n", MASK_PATTERN) .addFilter(".*transient_lastDdlTime.*\n", MASK_PATTERN) +.addFilter(".*lastUpdateTime.*\n", MASK_PATTERN) +
hive git commit: HIVE-16660: Not able to add partition for views in hive when sentry is enabled (Yongzhi Chen, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/master 2bcbd29e8 -> 6e0c52e68 HIVE-16660: Not able to add partition for views in hive when sentry is enabled (Yongzhi Chen, reviewed by Aihua Xu) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/6e0c52e6 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/6e0c52e6 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/6e0c52e6 Branch: refs/heads/master Commit: 6e0c52e6883cb55de3b8557504c13f3c8dfb619f Parents: 2bcbd29 Author: Yongzhi ChenAuthored: Fri May 12 10:06:04 2017 -0400 Committer: Yongzhi Chen Committed: Tue May 16 13:53:33 2017 -0400 -- .../org/apache/hadoop/hive/ql/metadata/Partition.java | 10 -- .../apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java | 4 +++- 2 files changed, 11 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/6e0c52e6/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java index bff1688..75179ab 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java @@ -213,9 +213,15 @@ public class Partition implements Serializable { public Path getDataLocation() { if (table.isPartitioned()) { - return new Path(tPartition.getSd().getLocation()); + if (tPartition.getSd() == null) +return null; + else +return new Path(tPartition.getSd().getLocation()); } else { - return new Path(table.getTTable().getSd().getLocation()); + if (table.getTTable() == null || table.getTTable().getSd() == null) +return null; + else +return new Path(table.getTTable().getSd().getLocation()); } } http://git-wip-us.apache.org/repos/asf/hive/blob/6e0c52e6/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java index 77bc12c..7601267 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java @@ -2934,7 +2934,9 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer { } cmd.append(")"); } - Driver driver = new Driver(conf); + SessionState ss = SessionState.get(); + String uName = (ss == null? null: ss.getUserName()); + Driver driver = new Driver(conf, uName); int rc = driver.compile(cmd.toString(), false); if (rc != 0) { throw new SemanticException(ErrorMsg.NO_VALID_PARTN.getMsg());
hive git commit: HIVE-14389: Beeline should not output query and prompt to stdout (Vihang Karajgaonkar, reviewed by Peter Vary, Thejas M Nair and Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/branch-2 43f1d7791 -> 812e78956 HIVE-14389: Beeline should not output query and prompt to stdout (Vihang Karajgaonkar, reviewed by Peter Vary, Thejas M Nair and Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/812e7895 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/812e7895 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/812e7895 Branch: refs/heads/branch-2 Commit: 812e789565d0ad3699d086232c5d4dbc0e87a336 Parents: 43f1d77 Author: Yongzhi ChenAuthored: Fri May 12 14:10:10 2017 -0400 Committer: Yongzhi Chen Committed: Fri May 12 14:27:46 2017 -0400 -- .../src/java/org/apache/hive/beeline/BeeLine.java | 9 +++-- .../org/apache/hive/beeline/cli/TestHiveCli.java| 8 .../apache/hive/beeline/TestBeeLineWithArgs.java| 16 +++- 3 files changed, 22 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/812e7895/beeline/src/java/org/apache/hive/beeline/BeeLine.java -- diff --git a/beeline/src/java/org/apache/hive/beeline/BeeLine.java b/beeline/src/java/org/apache/hive/beeline/BeeLine.java index 11526a7..4090897 100644 --- a/beeline/src/java/org/apache/hive/beeline/BeeLine.java +++ b/beeline/src/java/org/apache/hive/beeline/BeeLine.java @@ -138,6 +138,7 @@ public class BeeLine implements Closeable { private OutputFile recordOutputFile = null; private PrintStream outputStream = new PrintStream(System.out, true); private PrintStream errorStream = new PrintStream(System.err, true); + private InputStream inputStream = System.in; private ConsoleReader consoleReader; private List batch = null; private final Reflector reflector = new Reflector(this); @@ -1216,10 +1217,10 @@ public class BeeLine implements Closeable { // by appending a newline to the end of inputstream InputStream inputStreamAppendedNewline = new SequenceInputStream(inputStream, new ByteArrayInputStream((new String("\n")).getBytes())); - consoleReader = new ConsoleReader(inputStreamAppendedNewline, getOutputStream()); + consoleReader = new ConsoleReader(inputStreamAppendedNewline, getErrorStream()); consoleReader.setCopyPasteDetection(true); // jline will detect if is regular character } else { - consoleReader = new ConsoleReader(); + consoleReader = new ConsoleReader(getInputStream(), getErrorStream()); } //disable the expandEvents for the purpose of backward compatibility @@ -2348,6 +2349,10 @@ public class BeeLine implements Closeable { return errorStream; } + InputStream getInputStream() { +return inputStream; + } + ConsoleReader getConsoleReader() { return consoleReader; } http://git-wip-us.apache.org/repos/asf/hive/blob/812e7895/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java -- diff --git a/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java b/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java index d306e29..8eaa683 100644 --- a/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java +++ b/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java @@ -112,7 +112,7 @@ public class TestHiveCli { @Test public void testSetPromptValue() { -verifyCMD("set hive.cli.prompt=MYCLI;SHOW\nTABLES;", "MYCLI> ", os, null, +verifyCMD("set hive.cli.prompt=MYCLI;SHOW\nTABLES;", "MYCLI> ", errS, null, ERRNO_OK, true); } @@ -213,21 +213,21 @@ public class TestHiveCli { public void testUseCurrentDB1() { verifyCMD( "create database if not exists testDB; set hive.cli.print.current.db=true;use testDB;\n" -+ "use default;drop if exists testDB;", "hive (testDB)>", os, null, ERRNO_OTHER, true); ++ "use default;drop if exists testDB;", "hive (testDB)>", errS, null, ERRNO_OTHER, true); } @Test public void testUseCurrentDB2() { verifyCMD( "create database if not exists testDB; set hive.cli.print.current.db=true;use\ntestDB;\nuse default;drop if exists testDB;", -"hive (testDB)>", os, null, ERRNO_OTHER, true); +"hive (testDB)>", errS, null, ERRNO_OTHER, true); } @Test public void testUseCurrentDB3() { verifyCMD( "create database if not exists testDB; set hive.cli.print.current.db=true;use testDB;\n" -+ "use default;drop if exists testDB;", "hive (testDB)>", os, null, ERRNO_OTHER, true); ++ "use default;drop if exists testDB;", "hive (testDB)>", errS, null, ERRNO_OTHER, true); } @Test
hive git commit: HIVE-16646: Alias in transform ... as clause shouldn't be case sensitive (Yibing Shi, reviewed by Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/branch-2 6e117f344 -> 43f1d7791 HIVE-16646: Alias in transform ... as clause shouldn't be case sensitive (Yibing Shi, reviewed by Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/43f1d779 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/43f1d779 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/43f1d779 Branch: refs/heads/branch-2 Commit: 43f1d7791ed22411c701f4973316f49eaf004bf8 Parents: 6e117f3 Author: Yongzhi ChenAuthored: Thu May 11 14:53:57 2017 -0400 Committer: Yongzhi Chen Committed: Fri May 12 14:18:27 2017 -0400 -- .../hadoop/hive/ql/parse/SemanticAnalyzer.java | 4 +-- ql/src/test/queries/clientpositive/transform3.q | 6 + .../results/clientpositive/transform3.q.out | 28 3 files changed, 36 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/43f1d779/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index d2aa47b..3a1e5a9 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -3762,7 +3762,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { if (outputColNames) { for (int i = 0; i < ccount; ++i) { String colAlias = unescapeIdentifier(((ASTNode) collist.getChild(i)) - .getText()); + .getText()).toLowerCase(); failIfColAliasExists(colAliasNamesDuplicateCheck, colAlias); String intName = getColumnInternalName(i); ColumnInfo colInfo = new ColumnInfo(intName, @@ -3775,7 +3775,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { ASTNode child = (ASTNode) collist.getChild(i); assert child.getType() == HiveParser.TOK_TABCOL; String colAlias = unescapeIdentifier(((ASTNode) child.getChild(0)) - .getText()); + .getText()).toLowerCase(); failIfColAliasExists(colAliasNamesDuplicateCheck, colAlias); String intName = getColumnInternalName(i); ColumnInfo colInfo = new ColumnInfo(intName, TypeInfoUtils http://git-wip-us.apache.org/repos/asf/hive/blob/43f1d779/ql/src/test/queries/clientpositive/transform3.q -- diff --git a/ql/src/test/queries/clientpositive/transform3.q b/ql/src/test/queries/clientpositive/transform3.q new file mode 100644 index 000..4a2a368 --- /dev/null +++ b/ql/src/test/queries/clientpositive/transform3.q @@ -0,0 +1,6 @@ +CREATE TABLE transform3_t1 (col string); +INSERT OVERWRITE TABLE transform3_t1 VALUES(''); + +SELECT t.newCol FROM ( + SELECT TRANSFORM(col) USING 'cat' AS (NewCol string) FROM transform3_t1 +) t; http://git-wip-us.apache.org/repos/asf/hive/blob/43f1d779/ql/src/test/results/clientpositive/transform3.q.out -- diff --git a/ql/src/test/results/clientpositive/transform3.q.out b/ql/src/test/results/clientpositive/transform3.q.out new file mode 100644 index 000..5f93ed8 --- /dev/null +++ b/ql/src/test/results/clientpositive/transform3.q.out @@ -0,0 +1,28 @@ +PREHOOK: query: CREATE TABLE transform3_t1 (col string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@transform3_t1 +POSTHOOK: query: CREATE TABLE transform3_t1 (col string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@transform3_t1 +PREHOOK: query: INSERT OVERWRITE TABLE transform3_t1 VALUES('') +PREHOOK: type: QUERY +PREHOOK: Output: default@transform3_t1 +POSTHOOK: query: INSERT OVERWRITE TABLE transform3_t1 VALUES('') +POSTHOOK: type: QUERY +POSTHOOK: Output: default@transform3_t1 +POSTHOOK: Lineage: transform3_t1.col SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +PREHOOK: query: SELECT t.newCol FROM ( + SELECT TRANSFORM(col) USING 'cat' AS (NewCol string) FROM transform3_t1 +) t +PREHOOK: type: QUERY +PREHOOK: Input: default@transform3_t1 + A masked pattern was here +POSTHOOK: query: SELECT t.newCol FROM ( + SELECT TRANSFORM(col) USING 'cat' AS (NewCol string) FROM transform3_t1 +) t +POSTHOOK: type: QUERY +POSTHOOK: Input: default@transform3_t1 + A masked pattern was here +
hive git commit: HIVE-14389: Beeline should not output query and prompt to stdout (Vihang Karajgaonkar, reviewed by Peter Vary, Thejas M Nair and Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/master c9cfb6019 -> dd2697c00 HIVE-14389: Beeline should not output query and prompt to stdout (Vihang Karajgaonkar, reviewed by Peter Vary, Thejas M Nair and Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/dd2697c0 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/dd2697c0 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/dd2697c0 Branch: refs/heads/master Commit: dd2697c00dffe17699f19f8accfbf5c14bd07219 Parents: c9cfb60 Author: Yongzhi ChenAuthored: Fri May 12 14:10:10 2017 -0400 Committer: Yongzhi Chen Committed: Fri May 12 14:10:10 2017 -0400 -- .../src/java/org/apache/hive/beeline/BeeLine.java | 9 +++-- .../org/apache/hive/beeline/cli/TestHiveCli.java| 8 .../apache/hive/beeline/TestBeeLineWithArgs.java| 16 +++- 3 files changed, 22 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/dd2697c0/beeline/src/java/org/apache/hive/beeline/BeeLine.java -- diff --git a/beeline/src/java/org/apache/hive/beeline/BeeLine.java b/beeline/src/java/org/apache/hive/beeline/BeeLine.java index 444b3cf..33a65d4 100644 --- a/beeline/src/java/org/apache/hive/beeline/BeeLine.java +++ b/beeline/src/java/org/apache/hive/beeline/BeeLine.java @@ -141,6 +141,7 @@ public class BeeLine implements Closeable { private OutputFile recordOutputFile = null; private PrintStream outputStream = new PrintStream(System.out, true); private PrintStream errorStream = new PrintStream(System.err, true); + private InputStream inputStream = System.in; private ConsoleReader consoleReader; private List batch = null; private final Reflector reflector = new Reflector(this); @@ -1225,10 +1226,10 @@ public class BeeLine implements Closeable { // by appending a newline to the end of inputstream InputStream inputStreamAppendedNewline = new SequenceInputStream(inputStream, new ByteArrayInputStream((new String("\n")).getBytes())); - consoleReader = new ConsoleReader(inputStreamAppendedNewline, getOutputStream()); + consoleReader = new ConsoleReader(inputStreamAppendedNewline, getErrorStream()); consoleReader.setCopyPasteDetection(true); // jline will detect if is regular character } else { - consoleReader = new ConsoleReader(); + consoleReader = new ConsoleReader(getInputStream(), getErrorStream()); } //disable the expandEvents for the purpose of backward compatibility @@ -2406,6 +2407,10 @@ public class BeeLine implements Closeable { return errorStream; } + InputStream getInputStream() { +return inputStream; + } + ConsoleReader getConsoleReader() { return consoleReader; } http://git-wip-us.apache.org/repos/asf/hive/blob/dd2697c0/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java -- diff --git a/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java b/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java index d306e29..8eaa683 100644 --- a/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java +++ b/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java @@ -112,7 +112,7 @@ public class TestHiveCli { @Test public void testSetPromptValue() { -verifyCMD("set hive.cli.prompt=MYCLI;SHOW\nTABLES;", "MYCLI> ", os, null, +verifyCMD("set hive.cli.prompt=MYCLI;SHOW\nTABLES;", "MYCLI> ", errS, null, ERRNO_OK, true); } @@ -213,21 +213,21 @@ public class TestHiveCli { public void testUseCurrentDB1() { verifyCMD( "create database if not exists testDB; set hive.cli.print.current.db=true;use testDB;\n" -+ "use default;drop if exists testDB;", "hive (testDB)>", os, null, ERRNO_OTHER, true); ++ "use default;drop if exists testDB;", "hive (testDB)>", errS, null, ERRNO_OTHER, true); } @Test public void testUseCurrentDB2() { verifyCMD( "create database if not exists testDB; set hive.cli.print.current.db=true;use\ntestDB;\nuse default;drop if exists testDB;", -"hive (testDB)>", os, null, ERRNO_OTHER, true); +"hive (testDB)>", errS, null, ERRNO_OTHER, true); } @Test public void testUseCurrentDB3() { verifyCMD( "create database if not exists testDB; set hive.cli.print.current.db=true;use testDB;\n" -+ "use default;drop if exists testDB;", "hive (testDB)>", os, null, ERRNO_OTHER, true); ++ "use default;drop if exists testDB;", "hive (testDB)>", errS, null, ERRNO_OTHER, true); } @Test
hive git commit: HIVE-16646: Alias in transform ... as clause shouldn't be case sensitive (Yibing Shi, reviewed by Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/master 538c0088a -> 603c5341e HIVE-16646: Alias in transform ... as clause shouldn't be case sensitive (Yibing Shi, reviewed by Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/603c5341 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/603c5341 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/603c5341 Branch: refs/heads/master Commit: 603c5341e63953310c613286e7263f4c833d732a Parents: 538c008 Author: Yongzhi ChenAuthored: Thu May 11 14:53:57 2017 -0400 Committer: Yongzhi Chen Committed: Fri May 12 08:49:12 2017 -0400 -- .../hadoop/hive/ql/parse/SemanticAnalyzer.java | 4 +-- ql/src/test/queries/clientpositive/transform3.q | 6 + .../results/clientpositive/transform3.q.out | 28 3 files changed, 36 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/603c5341/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index ddf74f2..c6b67d1 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -3781,7 +3781,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { if (outputColNames) { for (int i = 0; i < ccount; ++i) { String colAlias = unescapeIdentifier(((ASTNode) collist.getChild(i)) - .getText()); + .getText()).toLowerCase(); failIfColAliasExists(colAliasNamesDuplicateCheck, colAlias); String intName = getColumnInternalName(i); ColumnInfo colInfo = new ColumnInfo(intName, @@ -3794,7 +3794,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { ASTNode child = (ASTNode) collist.getChild(i); assert child.getType() == HiveParser.TOK_TABCOL; String colAlias = unescapeIdentifier(((ASTNode) child.getChild(0)) - .getText()); + .getText()).toLowerCase(); failIfColAliasExists(colAliasNamesDuplicateCheck, colAlias); String intName = getColumnInternalName(i); ColumnInfo colInfo = new ColumnInfo(intName, TypeInfoUtils http://git-wip-us.apache.org/repos/asf/hive/blob/603c5341/ql/src/test/queries/clientpositive/transform3.q -- diff --git a/ql/src/test/queries/clientpositive/transform3.q b/ql/src/test/queries/clientpositive/transform3.q new file mode 100644 index 000..4a2a368 --- /dev/null +++ b/ql/src/test/queries/clientpositive/transform3.q @@ -0,0 +1,6 @@ +CREATE TABLE transform3_t1 (col string); +INSERT OVERWRITE TABLE transform3_t1 VALUES(''); + +SELECT t.newCol FROM ( + SELECT TRANSFORM(col) USING 'cat' AS (NewCol string) FROM transform3_t1 +) t; http://git-wip-us.apache.org/repos/asf/hive/blob/603c5341/ql/src/test/results/clientpositive/transform3.q.out -- diff --git a/ql/src/test/results/clientpositive/transform3.q.out b/ql/src/test/results/clientpositive/transform3.q.out new file mode 100644 index 000..5f93ed8 --- /dev/null +++ b/ql/src/test/results/clientpositive/transform3.q.out @@ -0,0 +1,28 @@ +PREHOOK: query: CREATE TABLE transform3_t1 (col string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@transform3_t1 +POSTHOOK: query: CREATE TABLE transform3_t1 (col string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@transform3_t1 +PREHOOK: query: INSERT OVERWRITE TABLE transform3_t1 VALUES('') +PREHOOK: type: QUERY +PREHOOK: Output: default@transform3_t1 +POSTHOOK: query: INSERT OVERWRITE TABLE transform3_t1 VALUES('') +POSTHOOK: type: QUERY +POSTHOOK: Output: default@transform3_t1 +POSTHOOK: Lineage: transform3_t1.col SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +PREHOOK: query: SELECT t.newCol FROM ( + SELECT TRANSFORM(col) USING 'cat' AS (NewCol string) FROM transform3_t1 +) t +PREHOOK: type: QUERY +PREHOOK: Input: default@transform3_t1 + A masked pattern was here +POSTHOOK: query: SELECT t.newCol FROM ( + SELECT TRANSFORM(col) USING 'cat' AS (NewCol string) FROM transform3_t1 +) t +POSTHOOK: type: QUERY +POSTHOOK: Input: default@transform3_t1 + A masked pattern was here +
hive git commit: HIVE-16590: Make initializing dag names in SparkWork thread safe for parallel compilation (Sahil Takiar, reviewed by Aihua Xu and Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/master 691acd5a3 -> 15bfc0ebc HIVE-16590: Make initializing dag names in SparkWork thread safe for parallel compilation (Sahil Takiar, reviewed by Aihua Xu and Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/15bfc0eb Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/15bfc0eb Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/15bfc0eb Branch: refs/heads/master Commit: 15bfc0ebc011cc557026e6ffa920384b01b89b1c Parents: 691acd5 Author: Yongzhi ChenAuthored: Tue May 9 10:40:17 2017 -0400 Committer: Yongzhi Chen Committed: Tue May 9 10:40:17 2017 -0400 -- .../apache/hadoop/hive/ql/plan/SparkWork.java | 6 +- .../TestExecutionEngineWorkConcurrency.java | 119 +++ .../hive/ql/plan/TestTezWorkConcurrency.java| 82 - 3 files changed, 123 insertions(+), 84 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/15bfc0eb/ql/src/java/org/apache/hadoop/hive/ql/plan/SparkWork.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/SparkWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/SparkWork.java index 9d46cac..9ca5544 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/SparkWork.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/SparkWork.java @@ -30,6 +30,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; @@ -47,7 +48,8 @@ import com.google.common.base.Preconditions; @Explain(displayName = "Spark", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED }, vectorization = Vectorization.SUMMARY_PATH) public class SparkWork extends AbstractOperatorDesc { - private static int counter; + + private static final AtomicInteger counter = new AtomicInteger(1); private final String name; private final Set roots = new LinkedHashSet(); @@ -65,7 +67,7 @@ public class SparkWork extends AbstractOperatorDesc { private Map cloneToWork; public SparkWork(String name) { -this.name = name + ":" + (++counter); +this.name = name + ":" + counter.getAndIncrement(); cloneToWork = new HashMap (); } http://git-wip-us.apache.org/repos/asf/hive/blob/15bfc0eb/ql/src/test/org/apache/hadoop/hive/ql/plan/TestExecutionEngineWorkConcurrency.java -- diff --git a/ql/src/test/org/apache/hadoop/hive/ql/plan/TestExecutionEngineWorkConcurrency.java b/ql/src/test/org/apache/hadoop/hive/ql/plan/TestExecutionEngineWorkConcurrency.java new file mode 100644 index 000..a7fcad0 --- /dev/null +++ b/ql/src/test/org/apache/hadoop/hive/ql/plan/TestExecutionEngineWorkConcurrency.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.ql.plan; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.FutureTask; + +import static org.junit.Assert.assertEquals; + + +@RunWith(Parameterized.class) +public final class TestExecutionEngineWorkConcurrency { + + @Parameterized.Parameters + public static Collection
hive git commit: HIVE-16451: Race condition between HiveStatement.getQueryLog and HiveStatement.runAsyncOnServer (Peter Vary, reviewed by Anishek and Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/master f3310a37b -> 691acd5a3 HIVE-16451: Race condition between HiveStatement.getQueryLog and HiveStatement.runAsyncOnServer (Peter Vary, reviewed by Anishek and Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/691acd5a Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/691acd5a Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/691acd5a Branch: refs/heads/master Commit: 691acd5a3a0948c7c8fb956b6a72c9afb780e270 Parents: f3310a3 Author: Yongzhi ChenAuthored: Tue May 9 10:23:15 2017 -0400 Committer: Yongzhi Chen Committed: Tue May 9 10:23:15 2017 -0400 -- .../org/apache/hive/jdbc/HiveStatement.java | 21 1 file changed, 17 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/691acd5a/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java -- diff --git a/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java b/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java index c385e2c..b743b46 100644 --- a/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java +++ b/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java @@ -201,18 +201,27 @@ public class HiveStatement implements java.sql.Statement { warningChain = null; } - void closeClientOperation() throws SQLException { + /** + * Closes the statement if there is one running. Do not change the the flags. + * @throws SQLException If there is an error closing the statement + */ + private void closeStatementIfNeeded() throws SQLException { try { if (stmtHandle != null) { TCloseOperationReq closeReq = new TCloseOperationReq(stmtHandle); TCloseOperationResp closeResp = client.CloseOperation(closeReq); Utils.verifySuccessWithInfo(closeResp.getStatus()); +stmtHandle = null; } } catch (SQLException e) { throw e; } catch (Exception e) { throw new SQLException(e.toString(), "08S01", e); } + } + + void closeClientOperation() throws SQLException { +closeStatementIfNeeded(); isQueryClosed = true; isExecuteStatementFailed = false; stmtHandle = null; @@ -295,8 +304,7 @@ public class HiveStatement implements java.sql.Statement { private void runAsyncOnServer(String sql) throws SQLException { checkConnection("execute"); -closeClientOperation(); -initFlags(); +reInitState(); TExecuteStatementReq execReq = new TExecuteStatementReq(sessHandle, sql); /** @@ -413,7 +421,12 @@ public class HiveStatement implements java.sql.Statement { } } - private void initFlags() { + /** + * Close statement if needed, and reset the flags. + * @throws SQLException + */ + private void reInitState() throws SQLException { +closeStatementIfNeeded(); isCancelled = false; isQueryClosed = false; isLogBeingGenerated = true;
hive git commit: HIVE-16426: Query cancel: improve the way to handle files.
Repository: hive Updated Branches: refs/heads/master eaa439e39 -> 19a1a4b42 HIVE-16426: Query cancel: improve the way to handle files. Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/19a1a4b4 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/19a1a4b4 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/19a1a4b4 Branch: refs/heads/master Commit: 19a1a4b4280b017c20b5123232aefb1fb4fa6053 Parents: eaa439e Author: Yongzhi ChenAuthored: Thu Apr 13 16:25:24 2017 -0400 Committer: Yongzhi Chen Committed: Mon Apr 24 10:46:40 2017 -0400 -- .../java/org/apache/hadoop/hive/ql/Driver.java | 24 .../apache/hadoop/hive/ql/exec/Utilities.java | 11 + .../hive/ql/io/CombineHiveInputFormat.java | 7 ++ .../service/cli/operation/SQLOperation.java | 4 +++- 4 files changed, 45 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/19a1a4b4/ql/src/java/org/apache/hadoop/hive/ql/Driver.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java index 6a8cc60..16b8101 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java @@ -201,6 +201,25 @@ public class Driver implements CommandProcessor { // resource releases public final ReentrantLock stateLock = new ReentrantLock(); public DriverState driverState = DriverState.INITIALIZED; +private static ThreadLocal lds = new ThreadLocal() { + @Override + protected LockedDriverState initialValue() { +return new LockedDriverState(); + } +}; + +public static void setLockedDriverState(LockedDriverState lDrv) { + lds.set(lDrv); +} + +public static LockedDriverState getLockedDriverState() { + return lds.get(); +} + +public static void removeLockedDriverState() { + if (lds != null) +lds.remove(); +} } private boolean checkConcurrency() { @@ -429,6 +448,8 @@ public class Driver implements CommandProcessor { TaskFactory.resetId(); } +LockedDriverState.setLockedDriverState(lDrvState); + String queryId = conf.getVar(HiveConf.ConfVars.HIVEQUERYID); //save some info for webUI for use after plan is freed @@ -1407,6 +1428,8 @@ public class Driver implements CommandProcessor { errorMessage = null; SQLState = null; downstreamError = null; +LockedDriverState.setLockedDriverState(lDrvState); + lDrvState.stateLock.lock(); try { if (alreadyCompiled) { @@ -2337,6 +2360,7 @@ public class Driver implements CommandProcessor { lDrvState.driverState = DriverState.CLOSED; } finally { lDrvState.stateLock.unlock(); + LockedDriverState.removeLockedDriverState(); } if (SessionState.get() != null) { SessionState.get().getLineageState().clear(); http://git-wip-us.apache.org/repos/asf/hive/blob/19a1a4b4/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java index b0657f0..9036d9e 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java @@ -24,6 +24,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.google.common.util.concurrent.ThreadFactoryBuilder; + import org.apache.commons.codec.binary.Base64; import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.WordUtils; @@ -52,6 +53,8 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.Order; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.Driver.DriverState; +import org.apache.hadoop.hive.ql.Driver.LockedDriverState; import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.QueryPlan; import org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter; @@ -3018,6 +3021,7 @@ public final class Utilities { Set pathsProcessed = new HashSet(); List pathsToAdd = new LinkedList(); +LockedDriverState lDrvStat = LockedDriverState.getLockedDriverState(); // AliasToWork contains all the aliases for (String alias : work.getAliasToWork().keySet()) { LOG.info("Processing alias " + alias); @@ -3027,6 +3031,9 @@ public final class Utilities { boolean hasLogged =
[4/7] hive git commit: HIVE-16345: BeeLineDriver should be able to run qtest files which are using default database tables (Peter Vary via Yongzhi Chen)
http://git-wip-us.apache.org/repos/asf/hive/blob/a2ce7f3d/ql/src/test/results/clientpositive/beeline/smb_mapjoin_12.q.out -- diff --git a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_12.q.out b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_12.q.out new file mode 100644 index 000..98bf25e --- /dev/null +++ b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_12.q.out @@ -0,0 +1,822 @@ +>>> set hive.optimize.bucketmapjoin = true; +No rows affected +>>> set hive.optimize.bucketmapjoin.sortedmerge = true; +No rows affected +>>> set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +No rows affected +>>> +>>> +>>> set hive.exec.reducers.max = 1; +No rows affected +>>> set hive.merge.mapfiles=false; +No rows affected +>>> set hive.merge.mapredfiles=false; +No rows affected +>>> set hive.cbo.enable=false; +No rows affected +>>> -- This test verifies that the output of a sort merge join on 1 big partition with multiple small partitions is bucketed and sorted +>>> +>>> -- Create two bucketed and sorted tables +>>> CREATE TABLE test_table1 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS; +INFO : Compiling commandqueryId=(!!{queryId}!!): CREATE TABLE test_table1 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS +INFO : Semantic Analysis Completed +INFO : Returning Hive schema: Schema(fieldSchemas:null, properties:null) +INFO : Completed compiling commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : Executing commandqueryId=(!!{queryId}!!): CREATE TABLE test_table1 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS +INFO : PREHOOK: query: CREATE TABLE test_table1 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS +INFO : PREHOOK: type: CREATETABLE +INFO : PREHOOK: Output: database:smb_mapjoin_12 +INFO : PREHOOK: Output: smb_mapjoin_12@test_table1 +INFO : Starting task [Stage-0:DDL] in serial mode +INFO : POSTHOOK: query: CREATE TABLE test_table1 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS +INFO : POSTHOOK: type: CREATETABLE +INFO : POSTHOOK: Output: database:smb_mapjoin_12 +INFO : POSTHOOK: Output: smb_mapjoin_12@test_table1 +INFO : Completed executing commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : OK +DEBUG : Shutting down query CREATE TABLE test_table1 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS +No rows affected +>>> CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS; +INFO : Compiling commandqueryId=(!!{queryId}!!): CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS +INFO : Semantic Analysis Completed +INFO : Returning Hive schema: Schema(fieldSchemas:null, properties:null) +INFO : Completed compiling commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : Executing commandqueryId=(!!{queryId}!!): CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS +INFO : PREHOOK: query: CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS +INFO : PREHOOK: type: CREATETABLE +INFO : PREHOOK: Output: database:smb_mapjoin_12 +INFO : PREHOOK: Output: smb_mapjoin_12@test_table2 +INFO : Starting task [Stage-0:DDL] in serial mode +INFO : POSTHOOK: query: CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS +INFO : POSTHOOK: type: CREATETABLE +INFO : POSTHOOK: Output: database:smb_mapjoin_12 +INFO : POSTHOOK: Output: smb_mapjoin_12@test_table2 +INFO : Completed executing commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : OK +DEBUG : Shutting down query CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS +No rows affected +>>> +>>> FROM default.src +INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT * +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') SELECT * +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '2') SELECT * +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '3') SELECT *; +INFO : Compiling commandqueryId=(!!{queryId}!!): FROM default.src +INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT * +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') SELECT * +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '2') SELECT * +INSERT OVERWRITE TABLE
[5/7] hive git commit: HIVE-16345: BeeLineDriver should be able to run qtest files which are using default database tables (Peter Vary via Yongzhi Chen)
http://git-wip-us.apache.org/repos/asf/hive/blob/a2ce7f3d/ql/src/test/results/clientpositive/beeline/smb_mapjoin_11.q.out -- diff --git a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_11.q.out b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_11.q.out new file mode 100644 index 000..19c07a0 --- /dev/null +++ b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_11.q.out @@ -0,0 +1,2563 @@ +>>> set hive.mapred.mode=nonstrict; +No rows affected +>>> set hive.optimize.bucketmapjoin = true; +No rows affected +>>> set hive.optimize.bucketmapjoin.sortedmerge = true; +No rows affected +>>> set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +No rows affected +>>> +>>> set hive.cbo.enable=false; +No rows affected +>>> +>>> set hive.exec.reducers.max = 1; +No rows affected +>>> set hive.merge.mapfiles=false; +No rows affected +>>> set hive.merge.mapredfiles=false; +No rows affected +>>> +>>> -- This test verifies that the output of a sort merge join on 2 partitions (one on each side of the join) is bucketed +>>> +>>> -- Create two bucketed and sorted tables +>>> CREATE TABLE test_table1 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS; +INFO : Compiling commandqueryId=(!!{queryId}!!): CREATE TABLE test_table1 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS +INFO : Semantic Analysis Completed +INFO : Returning Hive schema: Schema(fieldSchemas:null, properties:null) +INFO : Completed compiling commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : Executing commandqueryId=(!!{queryId}!!): CREATE TABLE test_table1 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS +INFO : PREHOOK: query: CREATE TABLE test_table1 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS +INFO : PREHOOK: type: CREATETABLE +INFO : PREHOOK: Output: database:smb_mapjoin_11 +INFO : PREHOOK: Output: smb_mapjoin_11@test_table1 +INFO : Starting task [Stage-0:DDL] in serial mode +INFO : POSTHOOK: query: CREATE TABLE test_table1 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS +INFO : POSTHOOK: type: CREATETABLE +INFO : POSTHOOK: Output: database:smb_mapjoin_11 +INFO : POSTHOOK: Output: smb_mapjoin_11@test_table1 +INFO : Completed executing commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : OK +DEBUG : Shutting down query CREATE TABLE test_table1 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS +No rows affected +>>> CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS; +INFO : Compiling commandqueryId=(!!{queryId}!!): CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS +INFO : Semantic Analysis Completed +INFO : Returning Hive schema: Schema(fieldSchemas:null, properties:null) +INFO : Completed compiling commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : Executing commandqueryId=(!!{queryId}!!): CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS +INFO : PREHOOK: query: CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS +INFO : PREHOOK: type: CREATETABLE +INFO : PREHOOK: Output: database:smb_mapjoin_11 +INFO : PREHOOK: Output: smb_mapjoin_11@test_table2 +INFO : Starting task [Stage-0:DDL] in serial mode +INFO : POSTHOOK: query: CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS +INFO : POSTHOOK: type: CREATETABLE +INFO : POSTHOOK: Output: database:smb_mapjoin_11 +INFO : POSTHOOK: Output: smb_mapjoin_11@test_table2 +INFO : Completed executing commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : OK +DEBUG : Shutting down query CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS +No rows affected +>>> +>>> FROM default.src +INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT * +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') SELECT *; +INFO : Compiling commandqueryId=(!!{queryId}!!): FROM default.src +INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT * +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') SELECT * +INFO : Semantic Analysis Completed +INFO : Returning Hive schema: Schema(fieldSchemas:[FieldSchema(name:_col0, type:int, comment:null), FieldSchema(name:_col1, type:string,
[1/7] hive git commit: HIVE-16345: BeeLineDriver should be able to run qtest files which are using default database tables (Peter Vary via Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/master 392b6e320 -> a2ce7f3d2 http://git-wip-us.apache.org/repos/asf/hive/blob/a2ce7f3d/ql/src/test/results/clientpositive/beeline/smb_mapjoin_7.q.out -- diff --git a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_7.q.out b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_7.q.out new file mode 100644 index 000..b15c951 --- /dev/null +++ b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_7.q.out @@ -0,0 +1,1805 @@ +>>> set hive.strict.checks.bucketing=false; +No rows affected +>>> +>>> set hive.mapred.mode=nonstrict; +No rows affected +>>> set hive.exec.reducers.max = 1; +No rows affected +>>> +>>> +>>> CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +INFO : Compiling commandqueryId=(!!{queryId}!!): CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +INFO : Semantic Analysis Completed +INFO : Returning Hive schema: Schema(fieldSchemas:null, properties:null) +INFO : Completed compiling commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : Executing commandqueryId=(!!{queryId}!!): CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +INFO : PREHOOK: query: CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +INFO : PREHOOK: type: CREATETABLE +INFO : PREHOOK: Output: database:smb_mapjoin_7 +INFO : PREHOOK: Output: smb_mapjoin_7@smb_bucket4_1 +INFO : Starting task [Stage-0:DDL] in serial mode +INFO : POSTHOOK: query: CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +INFO : POSTHOOK: type: CREATETABLE +INFO : POSTHOOK: Output: database:smb_mapjoin_7 +INFO : POSTHOOK: Output: smb_mapjoin_7@smb_bucket4_1 +INFO : Completed executing commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : OK +DEBUG : Shutting down query CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +No rows affected +>>> +>>> +>>> CREATE TABLE smb_bucket4_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +INFO : Compiling commandqueryId=(!!{queryId}!!): CREATE TABLE smb_bucket4_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +INFO : Semantic Analysis Completed +INFO : Returning Hive schema: Schema(fieldSchemas:null, properties:null) +INFO : Completed compiling commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : Executing commandqueryId=(!!{queryId}!!): CREATE TABLE smb_bucket4_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +INFO : PREHOOK: query: CREATE TABLE smb_bucket4_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +INFO : PREHOOK: type: CREATETABLE +INFO : PREHOOK: Output: database:smb_mapjoin_7 +INFO : PREHOOK: Output: smb_mapjoin_7@smb_bucket4_2 +INFO : Starting task [Stage-0:DDL] in serial mode +INFO : POSTHOOK: query: CREATE TABLE smb_bucket4_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +INFO : POSTHOOK: type: CREATETABLE +INFO : POSTHOOK: Output: database:smb_mapjoin_7 +INFO : POSTHOOK: Output: smb_mapjoin_7@smb_bucket4_2 +INFO : Completed executing commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : OK +DEBUG : Shutting down query CREATE TABLE smb_bucket4_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +No rows affected +>>> +>>> +>>> +>>> +>>> create table smb_join_results(k1 int, v1 string, k2 int, v2 string); +INFO : Compiling commandqueryId=(!!{queryId}!!): create table smb_join_results(k1 int, v1 string, k2 int, v2 string) +INFO : Semantic Analysis Completed +INFO : Returning Hive schema: Schema(fieldSchemas:null, properties:null) +INFO : Completed compiling commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : Executing commandqueryId=(!!{queryId}!!): create table smb_join_results(k1 int, v1 string, k2 int, v2 string) +INFO : PREHOOK: query: create table smb_join_results(k1 int, v1 string, k2 int, v2 string) +INFO : PREHOOK: type: CREATETABLE +INFO : PREHOOK: Output: database:smb_mapjoin_7 +INFO : PREHOOK: Output: smb_mapjoin_7@smb_join_results +INFO : Starting task [Stage-0:DDL] in serial mode +INFO : POSTHOOK: query: create table smb_join_results(k1 int, v1 string, k2 int, v2 string) +INFO : POSTHOOK: type: CREATETABLE +INFO : POSTHOOK: Output: database:smb_mapjoin_7 +INFO : POSTHOOK: Output: smb_mapjoin_7@smb_join_results +INFO : Completed executing commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : OK +DEBUG : Shutting down query create table smb_join_results(k1 int, v1 string, k2 int, v2 string) +No rows affected +>>>
[6/7] hive git commit: HIVE-16345: BeeLineDriver should be able to run qtest files which are using default database tables (Peter Vary via Yongzhi Chen)
http://git-wip-us.apache.org/repos/asf/hive/blob/a2ce7f3d/ql/src/test/results/clientpositive/beeline/smb_mapjoin_1.q.out -- diff --git a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_1.q.out b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_1.q.out new file mode 100644 index 000..70a37ca --- /dev/null +++ b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_1.q.out @@ -0,0 +1,948 @@ +>>> set hive.strict.checks.bucketing=false; +No rows affected +>>> +>>> +>>> +>>> +>>> +>>> create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; +INFO : Compiling commandqueryId=(!!{queryId}!!): create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +INFO : Semantic Analysis Completed +INFO : Returning Hive schema: Schema(fieldSchemas:null, properties:null) +INFO : Completed compiling commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : Executing commandqueryId=(!!{queryId}!!): create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +INFO : PREHOOK: query: create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +INFO : PREHOOK: type: CREATETABLE +INFO : PREHOOK: Output: database:smb_mapjoin_1 +INFO : PREHOOK: Output: smb_mapjoin_1@smb_bucket_1 +INFO : Starting task [Stage-0:DDL] in serial mode +INFO : POSTHOOK: query: create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +INFO : POSTHOOK: type: CREATETABLE +INFO : POSTHOOK: Output: database:smb_mapjoin_1 +INFO : POSTHOOK: Output: smb_mapjoin_1@smb_bucket_1 +INFO : Completed executing commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : OK +DEBUG : Shutting down query create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +No rows affected +>>> create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; +INFO : Compiling commandqueryId=(!!{queryId}!!): create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +INFO : Semantic Analysis Completed +INFO : Returning Hive schema: Schema(fieldSchemas:null, properties:null) +INFO : Completed compiling commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : Executing commandqueryId=(!!{queryId}!!): create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +INFO : PREHOOK: query: create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +INFO : PREHOOK: type: CREATETABLE +INFO : PREHOOK: Output: database:smb_mapjoin_1 +INFO : PREHOOK: Output: smb_mapjoin_1@smb_bucket_2 +INFO : Starting task [Stage-0:DDL] in serial mode +INFO : POSTHOOK: query: create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +INFO : POSTHOOK: type: CREATETABLE +INFO : POSTHOOK: Output: database:smb_mapjoin_1 +INFO : POSTHOOK: Output: smb_mapjoin_1@smb_bucket_2 +INFO : Completed executing commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : OK +DEBUG : Shutting down query create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +No rows affected +>>> create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; +INFO : Compiling commandqueryId=(!!{queryId}!!): create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +INFO : Semantic Analysis Completed +INFO : Returning Hive schema: Schema(fieldSchemas:null, properties:null) +INFO : Completed compiling commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : Executing commandqueryId=(!!{queryId}!!): create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +INFO : PREHOOK: query: create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +INFO : PREHOOK: type: CREATETABLE +INFO : PREHOOK: Output: database:smb_mapjoin_1 +INFO : PREHOOK: Output: smb_mapjoin_1@smb_bucket_3 +INFO : Starting task [Stage-0:DDL] in serial mode +INFO : POSTHOOK: query: create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +INFO : POSTHOOK: type: CREATETABLE +INFO : POSTHOOK: Output: database:smb_mapjoin_1 +INFO : POSTHOOK: Output: smb_mapjoin_1@smb_bucket_3 +INFO : Completed executing
[2/7] hive git commit: HIVE-16345: BeeLineDriver should be able to run qtest files which are using default database tables (Peter Vary via Yongzhi Chen)
http://git-wip-us.apache.org/repos/asf/hive/blob/a2ce7f3d/ql/src/test/results/clientpositive/beeline/smb_mapjoin_3.q.out -- diff --git a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_3.q.out b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_3.q.out new file mode 100644 index 000..6c9b8e4 --- /dev/null +++ b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_3.q.out @@ -0,0 +1,950 @@ +>>> set hive.cbo.enable=false; +No rows affected +>>> set hive.strict.checks.bucketing=false; +No rows affected +>>> +>>> -- SORT_QUERY_RESULTS +>>> +>>> +>>> +>>> create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; +INFO : Compiling commandqueryId=(!!{queryId}!!): create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +INFO : Semantic Analysis Completed +INFO : Returning Hive schema: Schema(fieldSchemas:null, properties:null) +INFO : Completed compiling commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : Executing commandqueryId=(!!{queryId}!!): create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +INFO : PREHOOK: query: create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +INFO : PREHOOK: type: CREATETABLE +INFO : PREHOOK: Output: database:smb_mapjoin_3 +INFO : PREHOOK: Output: smb_mapjoin_3@smb_bucket_1 +INFO : Starting task [Stage-0:DDL] in serial mode +INFO : POSTHOOK: query: create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +INFO : POSTHOOK: type: CREATETABLE +INFO : POSTHOOK: Output: database:smb_mapjoin_3 +INFO : POSTHOOK: Output: smb_mapjoin_3@smb_bucket_1 +INFO : Completed executing commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : OK +DEBUG : Shutting down query create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +No rows affected +>>> create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; +INFO : Compiling commandqueryId=(!!{queryId}!!): create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +INFO : Semantic Analysis Completed +INFO : Returning Hive schema: Schema(fieldSchemas:null, properties:null) +INFO : Completed compiling commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : Executing commandqueryId=(!!{queryId}!!): create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +INFO : PREHOOK: query: create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +INFO : PREHOOK: type: CREATETABLE +INFO : PREHOOK: Output: database:smb_mapjoin_3 +INFO : PREHOOK: Output: smb_mapjoin_3@smb_bucket_2 +INFO : Starting task [Stage-0:DDL] in serial mode +INFO : POSTHOOK: query: create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +INFO : POSTHOOK: type: CREATETABLE +INFO : POSTHOOK: Output: database:smb_mapjoin_3 +INFO : POSTHOOK: Output: smb_mapjoin_3@smb_bucket_2 +INFO : Completed executing commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : OK +DEBUG : Shutting down query create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +No rows affected +>>> create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; +INFO : Compiling commandqueryId=(!!{queryId}!!): create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +INFO : Semantic Analysis Completed +INFO : Returning Hive schema: Schema(fieldSchemas:null, properties:null) +INFO : Completed compiling commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : Executing commandqueryId=(!!{queryId}!!): create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +INFO : PREHOOK: query: create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +INFO : PREHOOK: type: CREATETABLE +INFO : PREHOOK: Output: database:smb_mapjoin_3 +INFO : PREHOOK: Output: smb_mapjoin_3@smb_bucket_3 +INFO : Starting task [Stage-0:DDL] in serial mode +INFO : POSTHOOK: query: create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE +INFO : POSTHOOK: type: CREATETABLE +INFO : POSTHOOK: Output: database:smb_mapjoin_3 +INFO :
[7/7] hive git commit: HIVE-16345: BeeLineDriver should be able to run qtest files which are using default database tables (Peter Vary via Yongzhi Chen)
HIVE-16345: BeeLineDriver should be able to run qtest files which are using default database tables (Peter Vary via Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/a2ce7f3d Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/a2ce7f3d Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/a2ce7f3d Branch: refs/heads/master Commit: a2ce7f3d2edf404858b84f3d729a85a3660969b3 Parents: 392b6e3 Author: Yongzhi ChenAuthored: Mon Apr 10 10:38:52 2017 -0400 Committer: Yongzhi Chen Committed: Mon Apr 10 10:38:52 2017 -0400 -- .../java/org/apache/hive/beeline/BeeLine.java | 52 + .../java/org/apache/hive/beeline/Commands.java | 55 +- .../test/resources/testconfiguration.properties | 12 +- .../hive/cli/control/CoreBeeLineDriver.java | 17 +- .../org/apache/hadoop/hive/ql/QTestUtil.java|4 +- .../org/apache/hive/beeline/qfile/QFile.java| 124 +- .../hive/beeline/qfile/QFileBeeLineClient.java | 14 +- .../apache/hive/beeline/qfile/package-info.java |4 +- .../apache/hadoop/hive/ql/exec/Utilities.java |2 +- .../beeline/drop_with_concurrency.q.out | 47 +- .../beeline/escape_comments.q.out | 267 +- .../clientpositive/beeline/smb_mapjoin_1.q.out | 948 +++ .../clientpositive/beeline/smb_mapjoin_10.q.out | 248 ++ .../clientpositive/beeline/smb_mapjoin_11.q.out | 2563 ++ .../clientpositive/beeline/smb_mapjoin_12.q.out | 822 ++ .../clientpositive/beeline/smb_mapjoin_13.q.out | 687 + .../clientpositive/beeline/smb_mapjoin_16.q.out | 254 ++ .../clientpositive/beeline/smb_mapjoin_2.q.out | 955 +++ .../clientpositive/beeline/smb_mapjoin_3.q.out | 950 +++ .../clientpositive/beeline/smb_mapjoin_7.q.out | 1805 20 files changed, 9549 insertions(+), 281 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/a2ce7f3d/beeline/src/java/org/apache/hive/beeline/BeeLine.java -- diff --git a/beeline/src/java/org/apache/hive/beeline/BeeLine.java b/beeline/src/java/org/apache/hive/beeline/BeeLine.java index 11526a7..27b353c 100644 --- a/beeline/src/java/org/apache/hive/beeline/BeeLine.java +++ b/beeline/src/java/org/apache/hive/beeline/BeeLine.java @@ -22,6 +22,7 @@ */ package org.apache.hive.beeline; +import java.io.BufferedReader; import java.io.ByteArrayInputStream; import java.io.Closeable; import java.io.EOFException; @@ -29,6 +30,7 @@ import java.io.File; import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; +import java.io.InputStreamReader; import java.io.PrintStream; import java.io.SequenceInputStream; import java.lang.reflect.InvocationTargetException; @@ -59,6 +61,7 @@ import java.util.Enumeration; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedList; import java.util.List; import java.util.ListIterator; import java.util.Map; @@ -1380,6 +1383,55 @@ public class BeeLine implements Closeable { return lineTrimmed.startsWith("#") || lineTrimmed.startsWith("--"); } + String[] getCommands(File file) throws IOException { +List cmds = new LinkedList(); +try (BufferedReader reader = + new BufferedReader(new InputStreamReader(new FileInputStream(file), "UTF-8"))) { + StringBuilder cmd = null; + while (true) { +String scriptLine = reader.readLine(); + +if (scriptLine == null) { + break; +} + +String trimmedLine = scriptLine.trim(); +if (getOpts().getTrimScripts()) { + scriptLine = trimmedLine; +} + +if (cmd != null) { + // we're continuing an existing command + cmd.append("\n"); + cmd.append(scriptLine); + if (trimmedLine.endsWith(";")) { +// this command has terminated +cmds.add(cmd.toString()); +cmd = null; + } +} else { + // we're starting a new command + if (needsContinuation(scriptLine)) { +// multi-line +cmd = new StringBuilder(scriptLine); + } else { +// single-line +cmds.add(scriptLine); + } +} + } + + if (cmd != null) { +// ### REVIEW: oops, somebody left the last command +// unterminated; should we fix it for them or complain? +// For now be nice and fix it. +cmd.append(";"); +cmds.add(cmd.toString()); + } +} +return cmds.toArray(new String[0]); + } + /** * Print the specified message to the console *
[3/7] hive git commit: HIVE-16345: BeeLineDriver should be able to run qtest files which are using default database tables (Peter Vary via Yongzhi Chen)
http://git-wip-us.apache.org/repos/asf/hive/blob/a2ce7f3d/ql/src/test/results/clientpositive/beeline/smb_mapjoin_16.q.out -- diff --git a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_16.q.out b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_16.q.out new file mode 100644 index 000..b8a06dc --- /dev/null +++ b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_16.q.out @@ -0,0 +1,254 @@ +>>> set hive.optimize.bucketmapjoin = true; +No rows affected +>>> set hive.optimize.bucketmapjoin.sortedmerge = true; +No rows affected +>>> set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +No rows affected +>>> set hive.cbo.enable=false; +No rows affected +>>> +>>> set hive.exec.reducers.max = 1; +No rows affected +>>> set hive.merge.mapfiles=false; +No rows affected +>>> set hive.merge.mapredfiles=false; +No rows affected +>>> +>>> -- Create bucketed and sorted tables +>>> CREATE TABLE test_table1 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +INFO : Compiling commandqueryId=(!!{queryId}!!): CREATE TABLE test_table1 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +INFO : Semantic Analysis Completed +INFO : Returning Hive schema: Schema(fieldSchemas:null, properties:null) +INFO : Completed compiling commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : Executing commandqueryId=(!!{queryId}!!): CREATE TABLE test_table1 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +INFO : PREHOOK: query: CREATE TABLE test_table1 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +INFO : PREHOOK: type: CREATETABLE +INFO : PREHOOK: Output: database:smb_mapjoin_16 +INFO : PREHOOK: Output: smb_mapjoin_16@test_table1 +INFO : Starting task [Stage-0:DDL] in serial mode +INFO : POSTHOOK: query: CREATE TABLE test_table1 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +INFO : POSTHOOK: type: CREATETABLE +INFO : POSTHOOK: Output: database:smb_mapjoin_16 +INFO : POSTHOOK: Output: smb_mapjoin_16@test_table1 +INFO : Completed executing commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : OK +DEBUG : Shutting down query CREATE TABLE test_table1 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +No rows affected +>>> CREATE TABLE test_table2 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +INFO : Compiling commandqueryId=(!!{queryId}!!): CREATE TABLE test_table2 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +INFO : Semantic Analysis Completed +INFO : Returning Hive schema: Schema(fieldSchemas:null, properties:null) +INFO : Completed compiling commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : Executing commandqueryId=(!!{queryId}!!): CREATE TABLE test_table2 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +INFO : PREHOOK: query: CREATE TABLE test_table2 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +INFO : PREHOOK: type: CREATETABLE +INFO : PREHOOK: Output: database:smb_mapjoin_16 +INFO : PREHOOK: Output: smb_mapjoin_16@test_table2 +INFO : Starting task [Stage-0:DDL] in serial mode +INFO : POSTHOOK: query: CREATE TABLE test_table2 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +INFO : POSTHOOK: type: CREATETABLE +INFO : POSTHOOK: Output: database:smb_mapjoin_16 +INFO : POSTHOOK: Output: smb_mapjoin_16@test_table2 +INFO : Completed executing commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : OK +DEBUG : Shutting down query CREATE TABLE test_table2 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +No rows affected +>>> +>>> FROM default.src +INSERT OVERWRITE TABLE test_table1 SELECT * +INSERT OVERWRITE TABLE test_table2 SELECT *; +INFO : Compiling commandqueryId=(!!{queryId}!!): FROM default.src +INSERT OVERWRITE TABLE test_table1 SELECT * +INSERT OVERWRITE TABLE test_table2 SELECT * +INFO : Semantic Analysis Completed +INFO : Returning Hive schema: Schema(fieldSchemas:[FieldSchema(name:_col0, type:int, comment:null), FieldSchema(name:_col1, type:string, comment:null)], properties:null) +INFO : Completed compiling commandqueryId=(!!{queryId}!!); Time taken: !!ELIDED!! seconds +INFO : Executing commandqueryId=(!!{queryId}!!): FROM default.src +INSERT OVERWRITE TABLE test_table1 SELECT * +INSERT OVERWRITE TABLE test_table2 SELECT * +INFO : PREHOOK: query: FROM default.src +INSERT OVERWRITE TABLE test_table1 SELECT * +INSERT OVERWRITE TABLE test_table2 SELECT * +INFO : PREHOOK: type: QUERY +INFO : PREHOOK: Input: default@src +INFO : PREHOOK: Output: smb_mapjoin_16@test_table1 +INFO : PREHOOK: Output: smb_mapjoin_16@test_table2 +WARN : Hive-on-MR is deprecated in
hive git commit: HIVE-16313: Flaky test: TestBeeLineDriver[drop_with_concurrency] (Peter Vary via Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/master 6e4dc6712 -> b176d072e HIVE-16313: Flaky test: TestBeeLineDriver[drop_with_concurrency] (Peter Vary via Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/b176d072 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/b176d072 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/b176d072 Branch: refs/heads/master Commit: b176d072ed2ed33cd53f7e5189d93650246714d6 Parents: 6e4dc67 Author: Yongzhi ChenAuthored: Wed Mar 29 13:08:21 2017 -0400 Committer: Yongzhi Chen Committed: Wed Mar 29 13:13:35 2017 -0400 -- .../org/apache/hive/beeline/qfile/QFile.java| 23 +++- .../beeline/drop_with_concurrency.q.out | 3 --- .../beeline/escape_comments.q.out | 15 - 3 files changed, 17 insertions(+), 24 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/b176d072/itests/util/src/main/java/org/apache/hive/beeline/qfile/QFile.java -- diff --git a/itests/util/src/main/java/org/apache/hive/beeline/qfile/QFile.java b/itests/util/src/main/java/org/apache/hive/beeline/qfile/QFile.java index 890a167..ae5a349 100644 --- a/itests/util/src/main/java/org/apache/hive/beeline/qfile/QFile.java +++ b/itests/util/src/main/java/org/apache/hive/beeline/qfile/QFile.java @@ -170,17 +170,27 @@ public final class QFile { return Shell.WINDOWS ? String.format("\"%s\"", file.getAbsolutePath()) : file.getAbsolutePath(); } + private static class Filter { +private final Pattern pattern; +private final String replacement; + +public Filter(Pattern pattern, String replacement) { + this.pattern = pattern; + this.replacement = replacement; +} + } + private static class RegexFilterSet { -private final Map regexFilters = new LinkedHashMap (); +private final List regexFilters = new ArrayList(); public RegexFilterSet addFilter(String regex, String replacement) { - regexFilters.put(Pattern.compile(regex), replacement); + regexFilters.add(new Filter(Pattern.compile(regex), replacement)); return this; } public String filter(String input) { - for (Pattern pattern : regexFilters.keySet()) { -input = pattern.matcher(input).replaceAll(regexFilters.get(pattern)); + for (Filter filter : regexFilters) { +input = filter.pattern.matcher(input).replaceAll(filter.replacement); } return input; } @@ -209,8 +219,9 @@ public final class QFile { return new RegexFilterSet() .addFilter(logPattern, "") -.addFilter("(?s)\nWaiting to acquire compile lock:.*?Acquired the compile lock.\n", -"\nAcquired the compile lock.\n") +.addFilter("(?s)\n[^\n]*Waiting to acquire compile lock.*?Acquired the compile lock.\n", +"\n") +.addFilter("Acquired the compile lock.\n","") .addFilter("Getting log thread is interrupted, since query is done!\n", "") .addFilter("going to print operations logs\n", "") .addFilter("printed operations logs\n", "") http://git-wip-us.apache.org/repos/asf/hive/blob/b176d072/ql/src/test/results/clientpositive/beeline/drop_with_concurrency.q.out -- diff --git a/ql/src/test/results/clientpositive/beeline/drop_with_concurrency.q.out b/ql/src/test/results/clientpositive/beeline/drop_with_concurrency.q.out index d22c9ec..993329e 100644 --- a/ql/src/test/results/clientpositive/beeline/drop_with_concurrency.q.out +++ b/ql/src/test/results/clientpositive/beeline/drop_with_concurrency.q.out @@ -9,7 +9,6 @@ No rows affected No rows affected >>> >>> drop table if exists drop_with_concurrency_1; -Acquired the compile lock. Compiling commandqueryId=(!!{queryId}!!): drop table if exists drop_with_concurrency_1 Semantic Analysis Completed Returning Hive schema: Schema(fieldSchemas:null, properties:null) @@ -25,7 +24,6 @@ OK Shutting down query drop table if exists drop_with_concurrency_1 No rows affected >>> create table drop_with_concurrency_1 (c1 int); -Acquired the compile lock. Compiling commandqueryId=(!!{queryId}!!): create table drop_with_concurrency_1 (c1 int) Semantic Analysis Completed Returning Hive schema: Schema(fieldSchemas:null, properties:null) @@ -45,7 +43,6 @@ OK Shutting down query create table drop_with_concurrency_1 (c1 int) No rows affected >>> drop table drop_with_concurrency_1; -Acquired the compile lock. Compiling commandqueryId=(!!{queryId}!!): drop table drop_with_concurrency_1 Semantic Analysis Completed Returning Hive schema: Schema(fieldSchemas:null,
hive git commit: HIVE-16152: TestBeeLineDriver logging improvements (Peter Vary, reviewed by Vihang Karajgaonkar and Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/master 5eb02b939 -> bec38dea5 HIVE-16152: TestBeeLineDriver logging improvements (Peter Vary, reviewed by Vihang Karajgaonkar and Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/bec38dea Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/bec38dea Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/bec38dea Branch: refs/heads/master Commit: bec38dea558030506e54b52f2dd8cc564f2eda6c Parents: 5eb02b9 Author: Yongzhi ChenAuthored: Fri Mar 24 09:59:12 2017 -0400 Committer: Yongzhi Chen Committed: Fri Mar 24 09:59:12 2017 -0400 -- .../hive/cli/control/CoreBeeLineDriver.java | 38 --- .../org/apache/hive/beeline/qfile/QFile.java| 66 +--- .../hive/beeline/qfile/QFileBeeLineClient.java | 27 3 files changed, 85 insertions(+), 46 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/bec38dea/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreBeeLineDriver.java -- diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreBeeLineDriver.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreBeeLineDriver.java index acc02eb..0d63f5d 100644 --- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreBeeLineDriver.java +++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreBeeLineDriver.java @@ -17,10 +17,11 @@ */ package org.apache.hadoop.hive.cli.control; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.google.common.base.Strings; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.QTestProcessExecResult; import org.apache.hadoop.hive.ql.hooks.PreExecutePrinter; import org.apache.hive.beeline.qfile.QFile; import org.apache.hive.beeline.qfile.QFile.QFileBuilder; @@ -32,6 +33,7 @@ import org.junit.BeforeClass; import java.io.File; import java.io.IOException; +import java.sql.SQLException; import java.util.HashMap; public class CoreBeeLineDriver extends CliAdapter { @@ -104,7 +106,7 @@ public class CoreBeeLineDriver extends CliAdapter { } protected void runInfraScript(File script, File beeLineOutput, File log) - throws IOException { + throws IOException, SQLException { try (QFileBeeLineClient beeLineClient = clientBuilder.getClient(beeLineOutput)) { beeLineClient.execute( new String[]{ @@ -114,6 +116,9 @@ public class CoreBeeLineDriver extends CliAdapter { "!run " + script, }, log); +} catch (Exception e) { + throw new SQLException("Error running infra script: " + script + + "\nCheck the following logs for details:\n - " + beeLineOutput + "\n - " + log, e); } } @@ -134,28 +139,41 @@ public class CoreBeeLineDriver extends CliAdapter { try (QFileBeeLineClient beeLineClient = clientBuilder.getClient(qFile.getLogFile())) { long startTime = System.currentTimeMillis(); System.err.println(">>> STARTED " + qFile.getName()); - assertTrue("QFile execution failed, see logs for details", beeLineClient.execute(qFile)); - long endTime = System.currentTimeMillis(); - System.err.println(">>> EXECUTED " + qFile.getName() + ":" + (endTime - startTime) / 1000 - + "s"); + beeLineClient.execute(qFile); + + long queryEndTime = System.currentTimeMillis(); + System.err.println(">>> EXECUTED " + qFile.getName() + ": " + (queryEndTime - startTime) + + "ms"); qFile.filterOutput(); long filterEndTime = System.currentTimeMillis(); - System.err.println(">>> FILTERED " + qFile.getName() + ":" + (filterEndTime - endTime) / 1000 - + "s"); + System.err.println(">>> FILTERED " + qFile.getName() + ": " + (filterEndTime - queryEndTime) + + "ms"); if (!overwrite) { -if (qFile.compareResults()) { +QTestProcessExecResult result = qFile.compareResults(); + +long compareEndTime = System.currentTimeMillis(); +System.err.println(">>> COMPARED " + qFile.getName() + ": " ++ (compareEndTime - filterEndTime) + "ms"); +if (result.getReturnCode() == 0) { System.err.println(">>> PASSED " + qFile.getName()); } else { System.err.println(">>> FAILED " + qFile.getName()); - fail("Failed diff"); + String messageText = "Client result comparison failed with error code = " + + result.getReturnCode() + " while executing fname=" + qFile.getName() + "\n"; + String messageBody =
hive git commit: HIVE-16242: Run BeeLine tests parallel (Peter Vary via Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/master 440c9c723 -> 5eb02b939 HIVE-16242: Run BeeLine tests parallel (Peter Vary via Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/5eb02b93 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/5eb02b93 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/5eb02b93 Branch: refs/heads/master Commit: 5eb02b9392eeb8309d0e071a22d0319507bb63b3 Parents: 440c9c7 Author: Yongzhi ChenAuthored: Fri Mar 24 09:42:13 2017 -0400 Committer: Yongzhi Chen Committed: Fri Mar 24 09:42:13 2017 -0400 -- .../hadoop/hive/cli/TestBeeLineDriver.java | 4 +- .../org/apache/hive/beeline/Parallelized.java | 64 .../org/apache/hive/beeline/qfile/QFile.java| 2 + 3 files changed, 68 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/5eb02b93/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestBeeLineDriver.java -- diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestBeeLineDriver.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestBeeLineDriver.java index 24eeb9d..4dd17c1e 100644 --- a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestBeeLineDriver.java +++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestBeeLineDriver.java @@ -22,15 +22,15 @@ import java.util.List; import org.apache.hadoop.hive.cli.control.CliAdapter; import org.apache.hadoop.hive.cli.control.CliConfigs; +import org.apache.hive.beeline.Parallelized; import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestRule; import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; -@RunWith(Parameterized.class) +@RunWith(Parallelized.class) public class TestBeeLineDriver { static CliAdapter adapter = new CliConfigs.BeeLineConfig().getCliAdapter(); http://git-wip-us.apache.org/repos/asf/hive/blob/5eb02b93/itests/util/src/main/java/org/apache/hive/beeline/Parallelized.java -- diff --git a/itests/util/src/main/java/org/apache/hive/beeline/Parallelized.java b/itests/util/src/main/java/org/apache/hive/beeline/Parallelized.java new file mode 100644 index 000..a819e26 --- /dev/null +++ b/itests/util/src/main/java/org/apache/hive/beeline/Parallelized.java @@ -0,0 +1,64 @@ +/** + * 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.hive.beeline; + +import org.junit.runners.Parameterized; +import org.junit.runners.model.RunnerScheduler; + +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +/** + * Class to run Parameterized test in parallel. + * Source: http://hwellmann.blogspot.hu/2009/12/running-parameterized-junit-tests-in.html + */ +public class Parallelized extends Parameterized { + private static class ThreadPoolScheduler implements RunnerScheduler { +private ExecutorService executor; + +public ThreadPoolScheduler() { + String threads = System.getProperty("junit.parallel.threads"); + int numThreads = Runtime.getRuntime().availableProcessors(); + if (threads != null) { +numThreads = Integer.parseInt(threads); + } + executor = Executors.newFixedThreadPool(numThreads); +} + +@Override +public void finished() { + executor.shutdown(); + try { +executor.awaitTermination(10, TimeUnit.MINUTES); + } catch (InterruptedException exc) { +throw new RuntimeException(exc); + } +} + +@Override +public void schedule(Runnable childStatement) { + executor.submit(childStatement); +} + } + + public Parallelized(Class klass) throws Throwable { +super(klass); +setScheduler(new ThreadPoolScheduler()); + } +}
hive git commit: HIVE-15751: Make it possible to run findbugs for itest modules as well (Peter Vary via Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/master ea0bc430c -> 9b28f33da HIVE-15751: Make it possible to run findbugs for itest modules as well (Peter Vary via Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/9b28f33d Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/9b28f33d Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/9b28f33d Branch: refs/heads/master Commit: 9b28f33dacab24fb8df54087b092231eab27495d Parents: ea0bc43 Author: Yongzhi ChenAuthored: Thu Mar 9 11:42:41 2017 -0500 Committer: Yongzhi Chen Committed: Thu Mar 9 11:46:55 2017 -0500 -- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/9b28f33d/pom.xml -- diff --git a/pom.xml b/pom.xml index d8b4cf3..3ea3c77 100644 --- a/pom.xml +++ b/pom.xml @@ -1266,7 +1266,7 @@ true 2048 -Djava.awt.headless=true -Xmx2048m -Xms512m - ${project.parent.basedir}/findbugs/findbugs-exclude.xml + ${basedir}/${hive.path.to.root}/findbugs/findbugs-exclude.xml
hive git commit: HIVE-15997: Resource leaks when query is cancelled (Yongzhi Chen, reviewed by Chaoyu Tang)
Repository: hive Updated Branches: refs/heads/master f72ea223a -> 35d707950 HIVE-15997: Resource leaks when query is cancelled (Yongzhi Chen, reviewed by Chaoyu Tang) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/35d70795 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/35d70795 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/35d70795 Branch: refs/heads/master Commit: 35d707950ddd210c37533be3da51cea730bac881 Parents: f72ea22 Author: Yongzhi ChenAuthored: Wed Mar 8 12:46:43 2017 -0500 Committer: Yongzhi Chen Committed: Wed Mar 8 12:49:20 2017 -0500 -- .../java/org/apache/hadoop/hive/ql/Driver.java | 1 - .../hadoop/hive/ql/exec/mr/ExecDriver.java | 20 .../ql/exec/spark/LocalHiveSparkClient.java | 5 + .../ql/exec/spark/RemoteHiveSparkClient.java| 5 + .../zookeeper/ZooKeeperHiveLockManager.java | 20 +--- 5 files changed, 47 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/35d70795/ql/src/java/org/apache/hadoop/hive/ql/Driver.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java index cdf24d4..d981119 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java @@ -655,7 +655,6 @@ public class Driver implements CommandProcessor { lDrvState.stateLock.lock(); try { if (lDrvState.driverState == DriverState.INTERRUPT) { -Thread.currentThread().interrupt(); return true; } else { return false; http://git-wip-us.apache.org/repos/asf/hive/blob/35d70795/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java index 34b683c..1945163 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java @@ -225,6 +225,11 @@ public class ExecDriver extends Task implements Serializable, Hadoop Path emptyScratchDir; JobClient jc = null; +if (driverContext.isShutdown()) { + LOG.warn("Task was cancelled"); + return 5; +} + MapWork mWork = work.getMapWork(); ReduceWork rWork = work.getReduceWork(); @@ -398,7 +403,22 @@ public class ExecDriver extends Task implements Serializable, Hadoop HiveConfUtil.updateJobCredentialProviders(job); // Finally SUBMIT the JOB! + if (driverContext.isShutdown()) { +LOG.warn("Task was cancelled"); +return 5; + } + rj = jc.submitJob(job); + + if (driverContext.isShutdown()) { +LOG.warn("Task was cancelled"); +if (rj != null) { + rj.killJob(); + rj = null; +} +return 5; + } + this.jobID = rj.getJobID(); updateStatusInQueryDisplay(); returnVal = jobExecHelper.progress(rj, jc, ctx); http://git-wip-us.apache.org/repos/asf/hive/blob/35d70795/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java index f5d9c4c..beeafd0 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hive.ql.exec.spark.status.impl.JobMetricsListener; import org.apache.hadoop.hive.ql.exec.spark.status.impl.LocalSparkJobRef; import org.apache.hadoop.hive.ql.exec.spark.status.impl.LocalSparkJobStatus; import org.apache.hadoop.hive.ql.io.HiveKey; +import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.plan.BaseWork; import org.apache.hadoop.hive.ql.plan.SparkWork; import org.apache.hadoop.hive.ql.session.SessionState; @@ -135,6 +136,10 @@ public class LocalHiveSparkClient implements HiveSparkClient { new SparkPlanGenerator(sc, ctx, jobConf, emptyScratchDir, sparkReporter); SparkPlan plan = gen.generate(sparkWork); +if (driverContext.isShutdown()) { + throw new HiveException("Operation is cancelled."); +} + // Execute generated plan. JavaPairRDD finalRDD = plan.generateGraph(); // We use Spark RDD async action to submit job as it's the only way to get
hive git commit: HIVE-15864: Fix typo introduced in HIVE-14754. (Zsombor Klara via Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/master 48596c33b -> a633d38d6 HIVE-15864: Fix typo introduced in HIVE-14754. (Zsombor Klara via Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/a633d38d Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/a633d38d Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/a633d38d Branch: refs/heads/master Commit: a633d38d62afdc01267fa57a279e9c42ff15788f Parents: 48596c3 Author: Yongzhi ChenAuthored: Fri Mar 3 09:26:12 2017 -0500 Committer: Yongzhi Chen Committed: Fri Mar 3 09:26:12 2017 -0500 -- .../hive/common/metrics/common/MetricsConstant.java | 2 +- .../apache/hive/service/cli/operation/SQLOperation.java | 2 +- .../service/cli/operation/TestSQLOperationMetrics.java| 10 +- 3 files changed, 7 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/a633d38d/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java -- diff --git a/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java b/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java index 4c53297..b2e78c3 100644 --- a/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java +++ b/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java @@ -80,5 +80,5 @@ public class MetricsConstant { public static final String HS2_COMPILING_QUERIES = "hs2_compiling_queries"; public static final String HS2_EXECUTING_QUERIES = "hs2_executing_queries"; public static final String HS2_FAILED_QUERIES = "hs2_failed_queries"; - public static final String HS2_SUCEEDED_QUERIES = "hs2_suceeded_queries"; + public static final String HS2_SUCCEEDED_QUERIES = "hs2_succeeded_queries"; } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/hive/blob/a633d38d/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java -- diff --git a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java index 7dde7bf..ff389ac 100644 --- a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java +++ b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java @@ -666,7 +666,7 @@ public class SQLOperation extends ExecuteStatementOperation { markQueryMetric(MetricsFactory.getInstance(), MetricsConstant.HS2_FAILED_QUERIES); } if (state == OperationState.FINISHED) { - markQueryMetric(MetricsFactory.getInstance(), MetricsConstant.HS2_SUCEEDED_QUERIES); + markQueryMetric(MetricsFactory.getInstance(), MetricsConstant.HS2_SUCCEEDED_QUERIES); } } http://git-wip-us.apache.org/repos/asf/hive/blob/a633d38d/service/src/test/org/apache/hive/service/cli/operation/TestSQLOperationMetrics.java -- diff --git a/service/src/test/org/apache/hive/service/cli/operation/TestSQLOperationMetrics.java b/service/src/test/org/apache/hive/service/cli/operation/TestSQLOperationMetrics.java index 5a648c0..d39f040 100644 --- a/service/src/test/org/apache/hive/service/cli/operation/TestSQLOperationMetrics.java +++ b/service/src/test/org/apache/hive/service/cli/operation/TestSQLOperationMetrics.java @@ -101,27 +101,27 @@ public class TestSQLOperationMetrics { String json = ((CodahaleMetrics) metrics).dumpJson(); MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.METER, -MetricsConstant.HS2_SUCEEDED_QUERIES, ""); +MetricsConstant.HS2_SUCCEEDED_QUERIES, ""); operation.onNewState(OperationState.FINISHED, OperationState.RUNNING); json = ((CodahaleMetrics) metrics).dumpJson(); MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.METER, -MetricsConstant.HS2_SUCEEDED_QUERIES, "1"); +MetricsConstant.HS2_SUCCEEDED_QUERIES, "1"); operation.onNewState(OperationState.ERROR, OperationState.RUNNING); json = ((CodahaleMetrics) metrics).dumpJson(); MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.METER, -MetricsConstant.HS2_SUCEEDED_QUERIES, "1"); +MetricsConstant.HS2_SUCCEEDED_QUERIES, "1"); operation.onNewState(OperationState.CANCELED, OperationState.RUNNING); json = ((CodahaleMetrics) metrics).dumpJson(); MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.METER, -MetricsConstant.HS2_SUCEEDED_QUERIES, "1"); +MetricsConstant.HS2_SUCCEEDED_QUERIES, "1");
hive git commit: HIVE-13864: Beeline ignores the command that follows a semicolon and comment (Reuben Kuhnert and Yongzhi Chen, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/master 3e77bbcc2 -> 9b4d72ead HIVE-13864: Beeline ignores the command that follows a semicolon and comment (Reuben Kuhnert and Yongzhi Chen, reviewed by Aihua Xu) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/9b4d72ea Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/9b4d72ea Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/9b4d72ea Branch: refs/heads/master Commit: 9b4d72ead9f3262a03ddd7730591729434064821 Parents: 3e77bbc Author: Yongzhi ChenAuthored: Wed Feb 22 12:46:31 2017 -0500 Committer: Yongzhi Chen Committed: Tue Feb 28 09:30:18 2017 -0500 -- .../java/org/apache/hive/beeline/Commands.java | 33 - .../org/apache/hive/beeline/TestCommands.java | 48 + .../queries/clientpositive/cmdwithcomments.q| 13 .../clientpositive/cmdwithcomments.q.out| 74 4 files changed, 167 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/9b4d72ea/beeline/src/java/org/apache/hive/beeline/Commands.java -- diff --git a/beeline/src/java/org/apache/hive/beeline/Commands.java b/beeline/src/java/org/apache/hive/beeline/Commands.java index 6a3ad42..962ddf7 100644 --- a/beeline/src/java/org/apache/hive/beeline/Commands.java +++ b/beeline/src/java/org/apache/hive/beeline/Commands.java @@ -65,6 +65,8 @@ import org.apache.hive.jdbc.HiveStatement; import org.apache.hive.jdbc.Utils; import org.apache.hive.jdbc.Utils.JdbcConnectionParams; +import com.google.common.annotations.VisibleForTesting; + public class Commands { private final BeeLine beeLine; @@ -1042,11 +1044,39 @@ public class Commands { return true; } + //startQuote use array type in order to pass int type as input/output parameter. + //This method remove comment from current line of a query. + //It does not remove comment like strings inside quotes. + @VisibleForTesting + String removeComments(String line, int[] startQuote) { +if (line == null || line.isEmpty()) return line; +if (startQuote[0] == -1 && beeLine.isComment(line)) return ""; //assume # can only be used at the beginning of line. +StringBuilder builder = new StringBuilder(); +for (int index = 0; index < line.length(); index++) { + if (startQuote[0] == -1 && index < line.length() - 1 && line.charAt(index) == '-' && line.charAt(index + 1) =='-') { +return builder.toString().trim(); + } + + char letter = line.charAt(index); + if (startQuote[0] == letter && (index == 0 || line.charAt(index -1) != '\\') ) { +startQuote[0] = -1; // Turn escape off. + } else if (startQuote[0] == -1 && (letter == '\'' || letter == '"') && (index == 0 || line.charAt(index -1) != '\\')) { +startQuote[0] = letter; // Turn escape on. + } + + builder.append(letter); +} + +return builder.toString().trim(); + } + /* * Check if the input line is a multi-line command which needs to read further */ public String handleMultiLineCmd(String line) throws IOException { //When using -e, console reader is not initialized and command is always a single line +int[] startQuote = {-1}; +line = removeComments(line,startQuote); while (isMultiLine(line) && beeLine.getOpts().isAllowMultiLineCommand()) { StringBuilder prompt = new StringBuilder(beeLine.getPrompt()); if (!beeLine.getOpts().isSilent()) { @@ -1071,7 +1101,8 @@ public class Commands { if (extra == null) { //it happens when using -f and the line of cmds does not end with ; break; } - if (!beeLine.isComment(extra)) { + extra = removeComments(extra,startQuote); + if (extra != null && !extra.isEmpty()) { line += "\n" + extra; } } http://git-wip-us.apache.org/repos/asf/hive/blob/9b4d72ea/beeline/src/test/org/apache/hive/beeline/TestCommands.java -- diff --git a/beeline/src/test/org/apache/hive/beeline/TestCommands.java b/beeline/src/test/org/apache/hive/beeline/TestCommands.java new file mode 100644 index 000..04c939a --- /dev/null +++ b/beeline/src/test/org/apache/hive/beeline/TestCommands.java @@ -0,0 +1,48 @@ +/** + * 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 + * + *
hive git commit: HIVE-15735: In some cases, view objects inside a view do not have parents. (Yongzhi Chen, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/master 1c2c4858a -> 77dfbe0b5 HIVE-15735: In some cases, view objects inside a view do not have parents. (Yongzhi Chen, reviewed by Aihua Xu) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/77dfbe0b Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/77dfbe0b Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/77dfbe0b Branch: refs/heads/master Commit: 77dfbe0b52ed7098347d734fd5e7f42b3354c67a Parents: 1c2c485 Author: Yongzhi ChenAuthored: Thu Jan 26 16:43:52 2017 -0500 Committer: Yongzhi Chen Committed: Tue Jan 31 09:07:39 2017 -0500 -- .../hadoop/hive/ql/parse/SemanticAnalyzer.java | 4 +++ .../apache/hadoop/hive/ql/plan/PlanUtils.java | 3 +- .../hadoop/hive/ql/plan/TestViewEntity.java | 37 3 files changed, 43 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/77dfbe0b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index 6c0f300..248dd63 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -2005,6 +2005,10 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { replaceViewReferenceWithDefinition(qb, tab, tabName, alias); // This is the last time we'll see the Table objects for views, so add it to the inputs // now. isInsideView will tell if this view is embedded in another view. +// If the view is Inside another view, it should have at least one parent +if (qb.isInsideView() && parentInput == null) { + parentInput = PlanUtils.getParentViewInfo(getAliasId(alias, qb), viewAliasToInput); +} ReadEntity viewInput = new ReadEntity(tab, parentInput, !qb.isInsideView()); viewInput = PlanUtils.addInput(inputs, viewInput); aliasToViewInfo.put(alias, new ObjectPair (fullViewName, viewInput)); http://git-wip-us.apache.org/repos/asf/hive/blob/77dfbe0b/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java index b2c5865..05d2c81 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java @@ -1127,7 +1127,8 @@ public final class PlanUtils { // For eg: for a query like 'select * from V3', where V3 -> V2, V2 -> V1, V1 -> T // -> implies depends on. // T's parent would be V1 -for (int pos = 0; pos < aliases.length; pos++) { +// do not check last alias in the array for parent can not be itself. +for (int pos = 0; pos < aliases.length -1; pos++) { currentAlias = currentAlias == null ? aliases[pos] : currentAlias + ":" + aliases[pos]; currentAlias = currentAlias.replace(SemanticAnalyzer.SUBQUERY_TAG_1, "") http://git-wip-us.apache.org/repos/asf/hive/blob/77dfbe0b/ql/src/test/org/apache/hadoop/hive/ql/plan/TestViewEntity.java -- diff --git a/ql/src/test/org/apache/hadoop/hive/ql/plan/TestViewEntity.java b/ql/src/test/org/apache/hadoop/hive/ql/plan/TestViewEntity.java index fa01416..4cc9d72 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/plan/TestViewEntity.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/plan/TestViewEntity.java @@ -168,8 +168,45 @@ public class TestViewEntity { // table1 and view1 as second read entity assertEquals("default@" + view1, CheckInputReadEntity.readEntities[1].getName()); assertFalse("Table is not direct input", CheckInputReadEntity.readEntities[1].isDirect()); +Set parents = CheckInputReadEntity.readEntities[1].getParents(); +assertTrue("Table does not have parent", parents != null && parents.size() > 0); assertEquals("default@" + tab1, CheckInputReadEntity.readEntities[2].getName()); assertFalse("Table is not direct input", CheckInputReadEntity.readEntities[2].isDirect()); } + + /** + * Verify that the the query with the subquery inside a view will have the correct + * direct and indirect inputs. + * @throws Exception + */ + @Test + public void testUnionAllInSubView() throws Exception { +String prefix = "tvunionallinsubview" + NAME_PREFIX; +final String tab1 = prefix + "t"; +final String view1 = prefix + "v"; +final
hive git commit: HIVE-15572: Improve the response time for query canceling when it happens during acquiring locks (Yongzhi Chen, reviewed by Chaoyu Tang)
Repository: hive Updated Branches: refs/heads/master 92090823d -> dbc2dffcd HIVE-15572: Improve the response time for query canceling when it happens during acquiring locks (Yongzhi Chen, reviewed by Chaoyu Tang) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/dbc2dffc Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/dbc2dffc Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/dbc2dffc Branch: refs/heads/master Commit: dbc2dffcd212a023ed43a932a716a177e5b466ef Parents: 9209082 Author: Yongzhi ChenAuthored: Tue Jan 10 21:48:34 2017 -0500 Committer: Yongzhi Chen Committed: Thu Jan 19 21:25:52 2017 -0500 -- .../java/org/apache/hadoop/hive/ql/Driver.java | 113 ++- .../org/apache/hadoop/hive/ql/ErrorMsg.java | 1 + .../hadoop/hive/ql/lockmgr/DbLockManager.java | 3 +- .../hadoop/hive/ql/lockmgr/DummyTxnManager.java | 9 +- .../hive/ql/lockmgr/EmbeddedLockManager.java| 3 +- .../hadoop/hive/ql/lockmgr/HiveLockManager.java | 3 +- .../hadoop/hive/ql/lockmgr/HiveTxnManager.java | 15 +++ .../hive/ql/lockmgr/HiveTxnManagerImpl.java | 8 ++ .../zookeeper/ZooKeeperHiveLockManager.java | 29 +++-- .../hive/ql/lockmgr/TestDummyTxnManager.java| 24 +++- 10 files changed, 138 insertions(+), 70 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/dbc2dffc/ql/src/java/org/apache/hadoop/hive/ql/Driver.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java index fd6020b..efa2bdc 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java @@ -172,6 +172,7 @@ public class Driver implements CommandProcessor { // For WebUI. Kept alive after queryPlan is freed. private final QueryDisplay queryDisplay = new QueryDisplay(); + private LockedDriverState lDrvState = new LockedDriverState(); // Query specific info private QueryState queryState; @@ -179,12 +180,7 @@ public class Driver implements CommandProcessor { // Query hooks that execute before compilation and after execution List queryHooks; - // a lock is used for synchronizing the state transition and its associated - // resource releases - private final ReentrantLock stateLock = new ReentrantLock(); - private DriverState driverState = DriverState.INITIALIZED; - - private enum DriverState { + public enum DriverState { INITIALIZED, COMPILING, COMPILED, @@ -201,6 +197,13 @@ public class Driver implements CommandProcessor { ERROR } + public static class LockedDriverState { +// a lock is used for synchronizing the state transition and its associated +// resource releases +public final ReentrantLock stateLock = new ReentrantLock(); +public DriverState driverState = DriverState.INITIALIZED; + } + private boolean checkConcurrency() { boolean supportConcurrency = conf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY); if (!supportConcurrency) { @@ -381,11 +384,11 @@ public class Driver implements CommandProcessor { PerfLogger perfLogger = SessionState.getPerfLogger(true); perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.DRIVER_RUN); perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.COMPILE); -stateLock.lock(); +lDrvState.stateLock.lock(); try { - driverState = DriverState.COMPILING; + lDrvState.driverState = DriverState.COMPILING; } finally { - stateLock.unlock(); + lDrvState.stateLock.unlock(); } command = new VariableSubstitution(new HiveVariableSource() { @@ -623,15 +626,15 @@ public class Driver implements CommandProcessor { if (isInterrupted && !deferClose) { closeInProcess(true); } - stateLock.lock(); + lDrvState.stateLock.lock(); try { if (isInterrupted) { - driverState = deferClose ? DriverState.EXECUTING : DriverState.ERROR; + lDrvState.driverState = deferClose ? DriverState.EXECUTING : DriverState.ERROR; } else { - driverState = compileError ? DriverState.ERROR : DriverState.COMPILED; + lDrvState.driverState = compileError ? DriverState.ERROR : DriverState.COMPILED; } } finally { -stateLock.unlock(); +lDrvState.stateLock.unlock(); } if (isInterrupted) { @@ -650,16 +653,16 @@ public class Driver implements CommandProcessor { } private boolean isInterrupted() { -stateLock.lock(); +lDrvState.stateLock.lock(); try { - if (driverState == DriverState.INTERRUPT) { + if (lDrvState.driverState == DriverState.INTERRUPT) {
hive git commit: HIVE-15437: avro tables join fails when - tbl join tbl_postfix (Yongzhi Chen, reviewed by Chaoyu Tang)
Repository: hive Updated Branches: refs/heads/master 29cce163d -> 24577b6a3 HIVE-15437: avro tables join fails when - tbl join tbl_postfix (Yongzhi Chen, reviewed by Chaoyu Tang) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/24577b6a Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/24577b6a Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/24577b6a Branch: refs/heads/master Commit: 24577b6a34da7b955a0b94e508a4b6fedad856ff Parents: 29cce16 Author: Yongzhi ChenAuthored: Thu Dec 15 11:00:02 2016 -0500 Committer: Yongzhi Chen Committed: Mon Dec 19 11:26:03 2016 -0500 -- data/files/table1.avsc | 25 ++ data/files/table1_1.avsc| 19 + .../ql/io/avro/AvroGenericRecordReader.java | 7 +- .../test/queries/clientpositive/avrotblsjoin.q | 28 +++ .../results/clientpositive/avrotblsjoin.q.out | 82 5 files changed, 158 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/24577b6a/data/files/table1.avsc -- diff --git a/data/files/table1.avsc b/data/files/table1.avsc new file mode 100644 index 000..2c96ad2 --- /dev/null +++ b/data/files/table1.avsc @@ -0,0 +1,25 @@ +{ + "type" : "record", + "name" : "table1", + "doc" : "Sqoop import of table1", + "fields" : [ { +"name" : "col1", +"type" : [ "null", "string" ], +"default" : null, +"columnName" : "col1", +"sqlType" : "12" + }, { +"name" : "col2", +"type" : [ "null", "long" ], +"default" : null, +"columnName" : "col2", +"sqlType" : "13" + }, { +"name" : "col3", +"type" : [ "null", "string" ], +"default" : null, +"columnName" : "col3", +"sqlType" : "12" + } ], + "tableName" : "table1" +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/hive/blob/24577b6a/data/files/table1_1.avsc -- diff --git a/data/files/table1_1.avsc b/data/files/table1_1.avsc new file mode 100644 index 000..1a7e518 --- /dev/null +++ b/data/files/table1_1.avsc @@ -0,0 +1,19 @@ +{ + "type" : "record", + "name" : "table1_1", + "doc" : "Sqoop import of table1_1", + "fields" : [ { +"name" : "col1", +"type" : [ "null", "long" ], +"default" : null, +"columnName" : "col1", +"sqlType" : "13" + }, { +"name" : "col2", +"type" : [ "null", "string" ], +"default" : null, +"columnName" : "col2", +"sqlType" : "12" + }], + "tableName" : "table1_1" +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/hive/blob/24577b6a/ql/src/java/org/apache/hadoop/hive/ql/io/avro/AvroGenericRecordReader.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/avro/AvroGenericRecordReader.java b/ql/src/java/org/apache/hadoop/hive/ql/io/avro/AvroGenericRecordReader.java index 4fccfc1..68138c8 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/avro/AvroGenericRecordReader.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/avro/AvroGenericRecordReader.java @@ -32,6 +32,7 @@ import org.apache.avro.mapred.FsInput; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.FileUtils; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.PartitionDesc; @@ -146,10 +147,10 @@ public class AvroGenericRecordReader implements private boolean pathIsInPartition(Path split, Path partitionPath) { boolean schemeless = split.toUri().getScheme() == null; if (schemeless) { - String schemelessPartitionPath = partitionPath.toUri().getPath(); - return split.toString().startsWith(schemelessPartitionPath); + Path pathNoSchema = Path.getPathWithoutSchemeAndAuthority(partitionPath); + return FileUtils.isPathWithinSubtree(split,pathNoSchema); } else { - return split.toString().startsWith(partitionPath.toString()); + return FileUtils.isPathWithinSubtree(split,partitionPath); } } http://git-wip-us.apache.org/repos/asf/hive/blob/24577b6a/ql/src/test/queries/clientpositive/avrotblsjoin.q -- diff --git a/ql/src/test/queries/clientpositive/avrotblsjoin.q b/ql/src/test/queries/clientpositive/avrotblsjoin.q new file mode 100644 index 000..8c1f084 --- /dev/null +++ b/ql/src/test/queries/clientpositive/avrotblsjoin.q @@ -0,0 +1,28 @@ +drop table if exists table1; +drop table if exists table1_1; + +dfs -cp
hive git commit: HIVE-15391: Location validation for table should ignore the values for view. (Yongzhi Chen, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/master 6fb6e565e -> 950ed9db8 HIVE-15391: Location validation for table should ignore the values for view. (Yongzhi Chen, reviewed by Aihua Xu) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/950ed9db Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/950ed9db Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/950ed9db Branch: refs/heads/master Commit: 950ed9db894f35942c1d4e80cd3e4032a7f54304 Parents: 6fb6e56 Author: Yongzhi ChenAuthored: Thu Dec 8 23:40:55 2016 -0500 Committer: Yongzhi Chen Committed: Tue Dec 13 09:06:35 2016 -0500 -- beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java | 7 --- .../src/test/java/org/apache/hive/beeline/TestSchemaTool.java | 4 +++- 2 files changed, 7 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/950ed9db/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java -- diff --git a/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java b/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java index 32a7b32..2172906 100644 --- a/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java +++ b/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java @@ -264,10 +264,11 @@ public class HiveSchemaTool { if (getDbCommandParser(dbType).needsQuotedIdentifier()) { tabLoc = "select tbl.\"TBL_ID\", tbl.\"TBL_NAME\", sd.\"LOCATION\", dbt.\"DB_ID\", dbt.\"NAME\" from \"TBLS\" tbl inner join " + -"\"SDS\" sd on tbl.\"SD_ID\" = sd.\"SD_ID\" and tbl.\"TBL_ID\" >= ? and tbl.\"TBL_ID\"<= ? " + -"inner join \"DBS\" dbt on tbl.\"DB_ID\" = dbt.\"DB_ID\" "; +"\"SDS\" sd on tbl.\"SD_ID\" = sd.\"SD_ID\" and tbl.\"TBL_TYPE\" != '" + TableType.VIRTUAL_VIEW + +"' and tbl.\"TBL_ID\" >= ? and tbl.\"TBL_ID\"<= ? " + "inner join \"DBS\" dbt on tbl.\"DB_ID\" = dbt.\"DB_ID\" "; } else { - tabLoc = "select tbl.TBL_ID, tbl.TBL_NAME, sd.LOCATION, dbt.DB_ID, dbt.NAME from TBLS tbl join SDS sd on tbl.SD_ID = sd.SD_ID and tbl.TBL_ID >= ? and tbl.TBL_ID <= ? inner join DBS dbt on tbl.DB_ID = dbt.DB_ID"; + tabLoc = "select tbl.TBL_ID, tbl.TBL_NAME, sd.LOCATION, dbt.DB_ID, dbt.NAME from TBLS tbl join SDS sd on tbl.SD_ID = sd.SD_ID and tbl.TBL_TYPE !='" + + TableType.VIRTUAL_VIEW + "' and tbl.TBL_ID >= ? and tbl.TBL_ID <= ? inner join DBS dbt on tbl.DB_ID = dbt.DB_ID"; } long maxID = 0, minID = 0; http://git-wip-us.apache.org/repos/asf/hive/blob/950ed9db/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java -- diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java index 3d585ac..17a4bd9 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java @@ -612,8 +612,10 @@ public class TestSchemaTool extends TestCase { "insert into DBS values(2, 'my db', 'hdfs://myhost.com:8020/user/hive/warehouse/mydb', 'mydb', 'public', 'role')", "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (1,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)", "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (2,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/admin/2015_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)", + "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (3,null,'org.apache.hadoop.mapred.TextInputFormat','N','N',null,-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)", "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL)", - "insert into PARTITiONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(1, 1441402388,0, 'd1=1/d2=1',2,2)" + "insert into PARTITiONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(1, 1441402388,0, 'd1=1/d2=1',2,2)", + "insert into
hive git commit: HIVE-15359: skip.footer.line.count doesnt work properly for certain situations (Yongzhi Chen, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/master b920fa735 -> b4980afd3 HIVE-15359: skip.footer.line.count doesnt work properly for certain situations (Yongzhi Chen, reviewed by Aihua Xu) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/b4980afd Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/b4980afd Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/b4980afd Branch: refs/heads/master Commit: b4980afd3500b7367a82dc1fc5cd01fe942c53d1 Parents: b920fa7 Author: Yongzhi ChenAuthored: Mon Dec 5 17:45:41 2016 -0500 Committer: Yongzhi Chen Committed: Thu Dec 8 09:50:45 2016 -0500 -- .../hadoop/hive/ql/io/CombineHiveRecordReader.java | 12 +++- .../hadoop/hive/ql/io/HiveContextAwareRecordReader.java | 10 ++ .../org/apache/hadoop/hive/shims/HadoopShimsSecure.java | 6 -- 3 files changed, 25 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/b4980afd/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveRecordReader.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveRecordReader.java b/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveRecordReader.java index aa607cc..ba25573 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveRecordReader.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveRecordReader.java @@ -31,6 +31,7 @@ import org.apache.hadoop.mapred.InputFormat; import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapred.lib.CombineFileSplit; /** @@ -43,7 +44,7 @@ public class CombineHiveRecordReader { public CombineHiveRecordReader(InputSplit split, Configuration conf, - Reporter reporter, Integer partition) throws IOException { + Reporter reporter, Integer partition, RecordReader preReader) throws IOException { super((JobConf)conf); CombineHiveInputSplit hsplit = split instanceof CombineHiveInputSplit ? (CombineHiveInputSplit) split : @@ -67,6 +68,15 @@ public class CombineHiveRecordReaderhttp://git-wip-us.apache.org/repos/asf/hive/blob/b4980afd/ql/src/java/org/apache/hadoop/hive/ql/io/HiveContextAwareRecordReader.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveContextAwareRecordReader.java b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveContextAwareRecordReader.java index d602c76..46f9970 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveContextAwareRecordReader.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveContextAwareRecordReader.java @@ -244,6 +244,15 @@ public abstract class HiveContextAwareRecordReader implements RecordReader private int headerCount = 0; private int footerCount = 0; + protected FooterBuffer getFooterBuffer() { + return footerBuffer; + } + + protected void setFooterBuffer( FooterBuffer buf) { +footerBuffer = buf; + } + + public boolean doNext(K key, V value) throws IOException { if (this.isSorted) { if (this.getIOContext().shouldEndBinarySearch() || @@ -308,6 +317,7 @@ public abstract class HiveContextAwareRecordReader implements RecordReader if (this.ioCxtRef.getCurrentBlockStart() == 0) { // Check if the table file has header to skip. +footerBuffer = null; Path filePath = this.ioCxtRef.getInputPath(); PartitionDesc part = null; try { http://git-wip-us.apache.org/repos/asf/hive/blob/b4980afd/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java -- diff --git a/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java b/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java index 224ce3b..018cb9f 100644 --- a/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java +++ b/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java @@ -118,7 +118,8 @@ public abstract class HadoopShimsSecure implements HadoopShims { InputSplit.class, Configuration.class, Reporter.class, -Integer.class +Integer.class, +RecordReader.class }; protected CombineFileSplit split; @@ -237,6 +238,7 @@ public abstract class HadoopShimsSecure implements HadoopShims { */ protected boolean initNextRecordReader(K key) throws IOException { + RecordReader preReader = curReader; //it is OK, curReader is closed, for we only
hive git commit: HIVE-15073: Schematool should detect malformed URIs (Yongzhi Chen, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/master 4a235fc55 -> a6c4004a9 HIVE-15073: Schematool should detect malformed URIs (Yongzhi Chen, reviewed by Aihua Xu) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/a6c4004a Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/a6c4004a Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/a6c4004a Branch: refs/heads/master Commit: a6c4004a97209a7b208037fe2de2d8dc4562e4ed Parents: 4a235fc Author: Yongzhi ChenAuthored: Thu Nov 17 17:30:52 2016 -0500 Committer: Yongzhi Chen Committed: Tue Nov 22 09:55:38 2016 -0500 -- .../org/apache/hive/beeline/HiveSchemaTool.java | 273 ++- .../org/apache/hive/beeline/TestSchemaTool.java | 52 2 files changed, 324 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/a6c4004a/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java -- diff --git a/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java b/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java index 3402470..a2ab3e0 100644 --- a/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java +++ b/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java @@ -29,6 +29,7 @@ import org.apache.commons.cli.ParseException; import org.apache.commons.io.output.NullOutputStream; import org.apache.commons.lang.StringUtils; import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.metastore.HiveMetaException; @@ -48,8 +49,10 @@ import java.io.FileReader; import java.io.FileWriter; import java.io.IOException; import java.io.PrintStream; +import java.net.URI; import java.sql.Connection; import java.sql.DatabaseMetaData; +import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; @@ -177,6 +180,274 @@ public class HiveSchemaTool { } } + boolean validateLocations(String defaultLocPrefix) throws HiveMetaException { +boolean rtn; +rtn = checkMetaStoreDBLocation(defaultLocPrefix); +rtn = checkMetaStoreTableLocation(defaultLocPrefix) && rtn; +rtn = checkMetaStorePartitionLocation(defaultLocPrefix) && rtn; +return rtn; + } + + private String getNameOrID(ResultSet res, int nameInx, int idInx) throws SQLException { +String itemName = res.getString(nameInx); +return (itemName == null || itemName.isEmpty()) ? "ID: " + res.getString(idInx) : "Name: " + itemName; + } + + // read schema version from metastore + private boolean checkMetaStoreDBLocation(String locHeader) + throws HiveMetaException { +String defaultPrefix = locHeader; +String dbLoc; +boolean isValid = true; +int numOfInvalid = 0; +Connection metastoreConn = getConnectionToMetastore(true); +if (getDbCommandParser(dbType).needsQuotedIdentifier()) { + dbLoc = "select dbt.\"DB_ID\", dbt.\"NAME\", dbt.\"DB_LOCATION_URI\" from \"DBS\" dbt"; +} else { + dbLoc = "select dbt.DB_ID, dbt.NAME, dbt.DB_LOCATION_URI from DBS dbt"; +} +String locValue; +String dbName; +try(Statement stmt = metastoreConn.createStatement(); +ResultSet res = stmt.executeQuery(dbLoc)) { + while (res.next()) { +locValue = res.getString(3); +if (locValue == null) { + System.err.println("NULL Location for DB with " + getNameOrID(res,2,1)); + numOfInvalid++; +} else { + URI currentUri = null; + try { +currentUri = new Path(locValue).toUri(); + } catch (Exception pe) { +System.err.println("Invalid Location for DB with " + getNameOrID(res,2,1)); +System.err.println(pe.getMessage()); +numOfInvalid++; +continue; + } + + if (currentUri.getScheme() == null || currentUri.getScheme().isEmpty()) { +System.err.println("Missing Location scheme for DB with " + getNameOrID(res,2,1)); +System.err.println("The Location is: " + locValue); +numOfInvalid++; + } else if (defaultPrefix != null && !defaultPrefix.isEmpty() && locValue.substring(0,defaultPrefix.length()) + .compareToIgnoreCase(defaultPrefix) != 0) { +System.err.println("Mismatch root Location for DB with " + getNameOrID(res,2,1)); +System.err.println("The Location is: " + locValue); +numOfInvalid++; + } +} + } + +} catch (SQLException e) { + throw new HiveMetaException("Failed to get DB Location Info.", e); +
hive git commit: HIVE-14784: Operation logs are disabled automatically if the parent directory does not exist. (Naveen Gangam via Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/master 74a6ff678 -> 0562efce6 HIVE-14784: Operation logs are disabled automatically if the parent directory does not exist. (Naveen Gangam via Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/0562efce Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/0562efce Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/0562efce Branch: refs/heads/master Commit: 0562efce642e70f1ac69eae6cca8c0a63230bafd Parents: 74a6ff6 Author: Yongzhi ChenAuthored: Fri Sep 30 10:39:11 2016 -0400 Committer: Yongzhi Chen Committed: Fri Sep 30 10:41:37 2016 -0400 -- .../org/apache/hive/service/cli/operation/Operation.java | 11 +++ 1 file changed, 11 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/0562efce/service/src/java/org/apache/hive/service/cli/operation/Operation.java -- diff --git a/service/src/java/org/apache/hive/service/cli/operation/Operation.java b/service/src/java/org/apache/hive/service/cli/operation/Operation.java index 90fe76d..6a656f9 100644 --- a/service/src/java/org/apache/hive/service/cli/operation/Operation.java +++ b/service/src/java/org/apache/hive/service/cli/operation/Operation.java @@ -234,6 +234,17 @@ public abstract class Operation { operationLogFile.getAbsolutePath()); operationLogFile.delete(); } +if (!operationLogFile.getParentFile().exists()) { + LOG.warn("Operations log directory for this session does not exist, it could have been deleted " + + "externally. Recreating the directory for future queries in this session but the older operation " + + "logs for this session are no longer available"); + if (!operationLogFile.getParentFile().mkdir()) { +LOG.warn("Log directory for this session could not be created, disabling " + +"operation logs: " + operationLogFile.getParentFile().getAbsolutePath()); +isOperationLogEnabled = false; +return; + } +} if (!operationLogFile.createNewFile()) { // the log file already exists and cannot be deleted. // If it can be read/written, keep its contents and use it.
hive git commit: HIVE-14358: Add metrics for number of queries executed for each execution engine (Barna Zsombor Klara, reviewed by Gabor Szadovszky, Yongzhi Chen)
Repository: hive Updated Branches: refs/heads/master 737fd09a2 -> 0c55d46f2 HIVE-14358: Add metrics for number of queries executed for each execution engine (Barna Zsombor Klara, reviewed by Gabor Szadovszky, Yongzhi Chen) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/0c55d46f Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/0c55d46f Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/0c55d46f Branch: refs/heads/master Commit: 0c55d46f2afdc7c282304839a10ac39221520316 Parents: 737fd09 Author: Yongzhi ChenAuthored: Mon Sep 26 13:55:28 2016 -0400 Committer: Yongzhi Chen Committed: Tue Sep 27 09:23:16 2016 -0400 -- .../common/metrics/common/MetricsConstant.java | 7 +++ .../java/org/apache/hadoop/hive/ql/Driver.java | 5 +++ .../org/apache/hadoop/hive/ql/exec/Task.java| 9 .../hadoop/hive/ql/exec/mr/MapRedTask.java | 11 + .../hadoop/hive/ql/exec/mr/MapredLocalTask.java | 11 + .../hadoop/hive/ql/exec/spark/SparkTask.java| 11 + .../apache/hadoop/hive/ql/exec/tez/TezTask.java | 11 + .../hadoop/hive/ql/exec/mr/TestMapRedTask.java | 47 .../hive/ql/exec/mr/TestMapredLocalTask.java| 46 +++ .../hive/ql/exec/spark/TestSparkTask.java | 46 +++ .../hadoop/hive/ql/exec/tez/TestTezTask.java| 17 +++ 11 files changed, 221 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/0c55d46f/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java -- diff --git a/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java b/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java index 9dc96f9..c9d4087 100644 --- a/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java +++ b/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java @@ -61,4 +61,11 @@ public class MetricsConstant { // The number of Hive operations that are waiting to enter the compile block public static final String WAITING_COMPILE_OPS = "waiting_compile_ops"; + // The number of map reduce tasks executed by the HiveServer2 since the last restart + public static final String HIVE_MR_TASKS = "hive_mapred_tasks"; + // The number of spark tasks executed by the HiveServer2 since the last restart + public static final String HIVE_SPARK_TASKS = "hive_spark_tasks"; + // The number of tez tasks executed by the HiveServer2 since the last restart + public static final String HIVE_TEZ_TASKS = "hive_tez_tasks"; + } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/hive/blob/0c55d46f/ql/src/java/org/apache/hadoop/hive/ql/Driver.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java index 42d398d..03c56e1 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java @@ -1671,6 +1671,11 @@ public class Driver implements CommandProcessor { // incorrect results. assert tsk.getParentTasks() == null || tsk.getParentTasks().isEmpty(); driverCxt.addToRunnable(tsk); + +Metrics metrics = MetricsFactory.getInstance(); +if (metrics != null) { + tsk.updateTaskMetrics(metrics); +} } perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.RUN_TASKS); http://git-wip-us.apache.org/repos/asf/hive/blob/0c55d46f/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java index eeaa543..e1bd291 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java @@ -27,6 +27,8 @@ import java.util.HashMap; import java.util.LinkedList; import java.util.List; +import org.apache.hadoop.hive.common.metrics.common.Metrics; +import org.apache.hadoop.hive.common.metrics.common.MetricsConstant; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.CompilationOpContext; import org.apache.hadoop.hive.ql.DriverContext; @@ -534,6 +536,13 @@ public abstract class Task implements Serializable, Node } } + /** + * Provide metrics on the type and number of tasks executed by the HiveServer + * @param metrics + */ + public void updateTaskMetrics(Metrics metrics) { +// no metrics gathered by default + } public int getTaskTag() { return