You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2016/09/19 22:42:18 UTC

[32/34] hive git commit: HIVE-14671 : merge master into hive-14535 (Sergey Shelukhin)

HIVE-14671 : merge master into hive-14535 (Sergey Shelukhin)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/87083986
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/87083986
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/87083986

Branch: refs/heads/hive-14535
Commit: 870839862b500ec2dd6902573dc37bcacc665add
Parents: 3e481b4 4340d46
Author: Sergey Shelukhin <se...@apache.org>
Authored: Mon Sep 19 15:26:45 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Mon Sep 19 15:26:45 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/hive/beeline/BeeLine.java   |   5 +-
 .../org/apache/hadoop/hive/cli/CliDriver.java   |   5 +-
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  13 +-
 .../hive/common/util/ShutdownHookManager.java   | 171 +---
 .../common/util/TestShutdownHookManager.java    |  35 -
 dev-support/jenkins-common.sh                   |  63 ++
 dev-support/jenkins-execute-build.sh            | 157 ++--
 .../hive/hbase/HiveHBaseTableInputFormat.java   |   5 +-
 .../src/test/queries/positive/hbase_viewjoins.q |  56 ++
 .../test/results/positive/hbase_viewjoins.q.out | 172 ++++
 .../hive/hcatalog/common/HiveClientCache.java   |   3 +-
 .../org/apache/hive/jdbc/TestJdbcDriver2.java   | 806 ++++++++-----------
 .../test/resources/testconfiguration.properties |  87 +-
 .../apache/hive/jdbc/HiveDatabaseMetaData.java  |   2 +
 .../hive/llap/security/SecretManager.java       |   9 +-
 .../llap/security/SigningSecretManager.java     |   4 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |  53 +-
 .../hadoop/hive/metastore/MetaStoreUtils.java   |   7 +
 .../apache/hadoop/hive/metastore/TableType.java |   2 +-
 .../org/apache/hadoop/hive/ql/ErrorMsg.java     |   4 +
 .../apache/hadoop/hive/ql/QueryProperties.java  |  15 +
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java | 102 ++-
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |  56 +-
 .../hive/ql/exec/mr/HadoopJobExecHelper.java    |   3 +-
 .../spark/session/SparkSessionManagerImpl.java  |   3 +-
 .../hadoop/hive/ql/exec/tez/TezJobMonitor.java  |   3 +-
 .../ql/exec/vector/VectorDeserializeRow.java    | 238 +++---
 .../hive/ql/exec/vector/VectorMapOperator.java  |  22 +-
 .../fast/VectorMapJoinFastLongHashTable.java    |   2 +-
 .../fast/VectorMapJoinFastStringCommon.java     |   2 +-
 .../VectorMapJoinOptimizedLongCommon.java       |  56 --
 .../VectorMapJoinOptimizedStringCommon.java     |  26 -
 .../apache/hadoop/hive/ql/hooks/ATSHook.java    |   3 +-
 .../zookeeper/CuratorFrameworkSingleton.java    |   3 +-
 .../apache/hadoop/hive/ql/metadata/Table.java   |   9 +-
 .../optimizer/SortedDynPartitionOptimizer.java  |   9 +-
 .../hive/ql/optimizer/physical/Vectorizer.java  |  25 +-
 .../hive/ql/parse/BaseSemanticAnalyzer.java     |  16 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |  63 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |  16 +-
 .../apache/hadoop/hive/ql/parse/EximUtil.java   |   2 +-
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |  29 +
 .../hive/ql/parse/ImportSemanticAnalyzer.java   |   2 +-
 .../hive/ql/parse/LoadSemanticAnalyzer.java     |   2 +-
 .../hadoop/hive/ql/parse/ParseContext.java      |  27 +-
 .../org/apache/hadoop/hive/ql/parse/QB.java     |  16 +
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  | 231 ++++--
 .../hive/ql/parse/SemanticAnalyzerFactory.java  |   3 +
 .../hadoop/hive/ql/parse/StorageFormat.java     |  17 +-
 .../hadoop/hive/ql/parse/TaskCompiler.java      | 106 ++-
 .../ql/parse/UpdateDeleteSemanticAnalyzer.java  |   7 +
 .../hadoop/hive/ql/plan/CreateViewDesc.java     |  93 ++-
 .../hadoop/hive/ql/plan/DropTableDesc.java      |  27 +-
 .../hadoop/hive/ql/plan/HiveOperation.java      |   3 +
 .../hadoop/hive/ql/plan/LoadFileDesc.java       |  10 +-
 .../apache/hadoop/hive/ql/plan/PlanUtils.java   |  41 +
 .../AuthorizationPreEventListener.java          |   2 +
 .../authorization/plugin/HiveOperationType.java |   2 +
 .../plugin/sqlstd/Operation2Privilege.java      |   6 +
 .../hive/ql/exec/TestFunctionRegistry.java      |  15 +-
 .../ql/exec/vector/TestVectorRowObject.java     |   2 +-
 .../hive/ql/exec/vector/TestVectorSerDeRow.java |  14 +-
 .../mapjoin/fast/CheckFastRowHashMap.java       |  10 +-
 .../exec/vector/mapjoin/fast/VerifyFastRow.java |   2 +-
 ...ialized_view_authorization_create_no_grant.q |  15 +
 ...d_view_authorization_create_no_select_perm.q |  14 +
 ...materialized_view_authorization_drop_other.q |  14 +
 ...rialized_view_authorization_no_select_perm.q |  14 +
 .../clientnegative/materialized_view_delete.q   |  10 +
 .../clientnegative/materialized_view_drop.q     |   9 +
 .../clientnegative/materialized_view_drop2.q    |   7 +
 .../clientnegative/materialized_view_insert.q   |   6 +
 .../clientnegative/materialized_view_load.q     |   7 +
 .../materialized_view_replace_with_view.q       |   8 +
 .../clientnegative/materialized_view_update.q   |  10 +
 .../test/queries/clientnegative/view_delete.q   |   9 +
 .../test/queries/clientnegative/view_update.q   |   9 +
 .../clientpositive/alter_partition_change_col.q |   2 +-
 .../clientpositive/alter_table_cascade.q        |   4 +-
 .../clientpositive/authorization_view_sqlstd.q  |  86 --
 ql/src/test/queries/clientpositive/bucket5.q    |   1 +
 .../dynpart_sort_optimization_acid.q            |  13 +
 .../queries/clientpositive/groupby_sort_1_23.q  |   4 +-
 .../clientpositive/groupby_sort_skew_1_23.q     |   8 +-
 .../materialized_view_authorization_sqlstd.q    |  56 ++
 .../clientpositive/materialized_view_create.q   |  31 +
 .../clientpositive/materialized_view_describe.q |  58 ++
 .../clientpositive/materialized_view_drop.q     |   7 +
 ql/src/test/queries/clientpositive/union32.q    |   8 +-
 ql/src/test/queries/clientpositive/union33.q    |  10 +-
 ql/src/test/queries/clientpositive/union36.q    |   4 +-
 .../queries/clientpositive/unionDistinct_1.q    |  16 +-
 .../queries/clientpositive/union_date_trim.q    |   2 +-
 ql/src/test/queries/clientpositive/union_null.q |   8 +-
 .../queries/clientpositive/union_remove_12.q    |   4 +-
 .../queries/clientpositive/union_remove_13.q    |   4 +-
 .../queries/clientpositive/union_remove_14.q    |   4 +-
 .../queries/clientpositive/union_type_chk.q     |   7 -
 .../clientpositive/unionall_join_nullconstant.q |   2 +-
 .../clientpositive/view_authorization_sqlstd.q  |  89 ++
 ...zed_view_authorization_create_no_grant.q.out |  26 +
 ...ew_authorization_create_no_select_perm.q.out |  20 +
 ...rialized_view_authorization_drop_other.q.out |  30 +
 ...ized_view_authorization_no_select_perm.q.out |  30 +
 .../materialized_view_delete.q.out              |  19 +
 .../clientnegative/materialized_view_drop.q.out |  42 +
 .../materialized_view_drop2.q.out               |  34 +
 .../materialized_view_insert.q.out              |  19 +
 .../clientnegative/materialized_view_load.q.out |  19 +
 .../materialized_view_replace_with_view.q.out   |  30 +
 .../materialized_view_update.q.out              |  19 +
 .../results/clientnegative/view_delete.q.out    |  19 +
 .../results/clientnegative/view_update.q.out    |  19 +
 .../alter_partition_change_col.q.out            |   4 +-
 .../clientpositive/alter_table_cascade.q.out    |   8 +-
 .../authorization_view_sqlstd.q.out             | 269 -------
 .../dynpart_sort_optimization_acid.q.out        | 267 +++++-
 .../clientpositive/groupby_sort_1_23.q.out      | 122 +--
 .../clientpositive/groupby_sort_skew_1_23.q.out | 132 +--
 .../results/clientpositive/llap/bucket5.q.out   | 586 ++++++++++++++
 .../results/clientpositive/llap/bucket6.q.out   | 203 +++++
 .../clientpositive/llap/bucket_many.q.out       | 223 +++++
 .../llap/bucketizedhiveinputformat.q.out        | 112 +++
 .../clientpositive/llap/bucketmapjoin6.q.out    | 146 ++++
 .../clientpositive/llap/bucketmapjoin7.q.out    | 315 ++++++++
 .../llap/empty_dir_in_table.q.out               |  46 ++
 .../llap/exchgpartition2lel.q.out               | 203 +++++
 ...rnal_table_with_space_in_location_path.q.out |  88 ++
 .../llap/file_with_header_footer.q.out          | 184 +++++
 .../llap/import_exported_table.q.out            |  28 +
 .../llap/infer_bucket_sort_bucketed_table.q.out | 121 +++
 .../clientpositive/llap/input16_cc.q.out        | 534 ++++++++++++
 .../clientpositive/llap/insert_dir_distcp.q.out |  14 +
 .../llap/join_acid_non_acid.q.out               |  58 ++
 .../clientpositive/llap/leftsemijoin_mr.q.out   |  98 +++
 .../llap/list_bucket_dml_10.q.out               | 256 ++++++
 .../results/clientpositive/llap/load_fs2.q.out  | 121 +++
 .../load_hdfs_file_with_space_in_the_name.q.out |  25 +
 .../llap/non_native_window_udf.q.out            |  52 ++
 .../clientpositive/llap/quotedid_smb.q.out      |  81 ++
 .../llap/reduce_deduplicate.q.out               | 387 +++++++++
 .../clientpositive/llap/remote_script.q.out     |  49 ++
 .../clientpositive/llap/schemeAuthority.q.out   |  84 ++
 .../clientpositive/llap/schemeAuthority2.q.out  |  53 ++
 .../llap/table_nonprintable.q.out               |  72 ++
 .../llap/temp_table_external.q.out              |  34 +
 .../clientpositive/llap/uber_reduce.q.out       |  31 +
 .../clientpositive/llap/unionDistinct_1.q.out   |  70 +-
 .../clientpositive/llap/union_type_chk.q.out    |  30 -
 ...materialized_view_authorization_sqlstd.q.out | 185 +++++
 .../materialized_view_create.q.out              | 170 ++++
 .../materialized_view_describe.q.out            | 335 ++++++++
 .../clientpositive/materialized_view_drop.q.out |  39 +
 .../spark/groupby_sort_1_23.q.out               | 134 +--
 .../spark/groupby_sort_skew_1_23.q.out          | 144 ++--
 .../results/clientpositive/spark/union32.q.out  | 130 +--
 .../results/clientpositive/spark/union33.q.out  |  16 +-
 .../clientpositive/spark/union_date_trim.q.out  |   4 +-
 .../clientpositive/spark/union_null.q.out       |  12 +-
 .../clientpositive/spark/union_remove_12.q.out  |  48 +-
 .../clientpositive/spark/union_remove_13.q.out  |  52 +-
 .../clientpositive/spark/union_remove_14.q.out  |  48 +-
 .../test/results/clientpositive/union32.q.out   | 130 +--
 .../test/results/clientpositive/union33.q.out   |  16 +-
 .../test/results/clientpositive/union36.q.out   |  28 +-
 .../clientpositive/unionDistinct_1.q.out        |  70 +-
 .../clientpositive/union_date_trim.q.out        |   4 +-
 .../results/clientpositive/union_null.q.out     |  12 +-
 .../clientpositive/union_remove_12.q.out        |   8 +-
 .../clientpositive/union_remove_13.q.out        |  26 +-
 .../clientpositive/union_remove_14.q.out        |   8 +-
 .../results/clientpositive/union_type_chk.q.out |  30 -
 .../unionall_join_nullconstant.q.out            |   4 +-
 .../view_authorization_sqlstd.q.out             | 295 +++++++
 .../fast/BinarySortableDeserializeRead.java     | 132 ++-
 .../hive/serde2/fast/DeserializeRead.java       |  71 +-
 .../lazy/fast/LazySimpleDeserializeRead.java    | 770 +++++++++---------
 .../fast/LazyBinaryDeserializeRead.java         | 119 +--
 .../hive/serde2/typeinfo/TypeInfoUtils.java     |   3 +
 .../apache/hadoop/hive/serde2/VerifyFast.java   |   2 +-
 .../binarysortable/TestBinarySortableFast.java  |  35 +-
 .../hive/serde2/lazy/TestLazySimpleFast.java    |  31 +-
 .../serde2/lazybinary/TestLazyBinaryFast.java   |  32 +-
 .../cli/operation/ClassicTableTypeMapping.java  |   5 +
 .../apache/hive/service/server/HiveServer2.java |   3 +-
 testutils/ptest2/pom.xml                        |  41 +-
 .../hive/ptest/execution/ExecutionPhase.java    |   8 +
 .../hive/ptest/execution/HostExecutor.java      |   7 +-
 .../org/apache/hive/ptest/execution/PTest.java  |   3 +-
 .../org/apache/hive/ptest/execution/Phase.java  |   2 +-
 .../ptest/execution/conf/FileListProvider.java  |  31 +
 .../ptest/execution/conf/QFileTestBatch.java    |  20 +-
 .../hive/ptest/execution/conf/TestBatch.java    |  30 +-
 .../ptest/execution/conf/TestConfiguration.java |   4 +
 .../hive/ptest/execution/conf/TestParser.java   |  71 +-
 .../ptest/execution/conf/UnitTestBatch.java     |  91 ++-
 .../conf/UnitTestPropertiesParser.java          | 666 +++++++++++++++
 .../ptest2/src/main/resources/batch-exec.vm     |  10 +-
 .../ptest2/src/main/resources/source-prep.vm    |  20 +-
 .../execution/MockRSyncCommandExecutor.java     |   7 +
 .../ptest/execution/MockSSHCommandExecutor.java |   7 +
 .../ptest/execution/TestExecutionPhase.java     |  10 +-
 ...ExecutionPhase.testFailingQFile.approved.txt |   4 +-
 ...cutionPhase.testFailingUnitTest.approved.txt |  10 +-
 ...utionPhase.testPassingQFileTest.approved.txt |   4 +-
 ...cutionPhase.testPassingUnitTest.approved.txt |  10 +-
 .../hive/ptest/execution/TestHostExecutor.java  |  52 +-
 .../TestHostExecutor.testBasic.approved.txt     |  24 +-
 ...xecutor.testIsolatedFailsOnExec.approved.txt |  10 +-
 ...tor.testIsolatedFailsOnRsyncOne.approved.txt |   8 +-
 ...testIsolatedFailsOnRsyncUnknown.approved.txt |   8 +-
 ...xecutor.testParallelFailsOnExec.approved.txt |  10 +-
 ...ecutor.testParallelFailsOnRsync.approved.txt |   8 +-
 ...tScripts.testAlternativeTestJVM.approved.txt |  16 +-
 .../TestScripts.testBatch.approved.txt          |  16 +-
 .../TestScripts.testPrepGit.approved.txt        |  13 +-
 .../TestScripts.testPrepHadoop1.approved.txt    |  20 +-
 .../TestScripts.testPrepNone.approved.txt       |  13 +-
 .../TestScripts.testPrepSvn.approved.txt        |  13 +-
 .../execution/conf/TestQFileTestBatch.java      |   9 +-
 .../ptest/execution/conf/TestTestParser.java    |  10 +-
 .../conf/TestUnitTestPropertiesParser.java      | 672 ++++++++++++++++
 .../ptest2/src/test/resources/log4j2.properties |  62 ++
 .../resources/test-configuration2.properties    | 154 ++++
 224 files changed, 11098 insertions(+), 3219 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/87083986/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/87083986/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/87083986/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/87083986/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/87083986/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/87083986/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/87083986/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/87083986/ql/src/java/org/apache/hadoop/hive/ql/plan/LoadFileDesc.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/87083986/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
----------------------------------------------------------------------
diff --cc ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
index f055cde,b15ad34..a07237e
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
@@@ -380,7 -380,62 +380,48 @@@ public final class PlanUtils 
      return ret;
    }
  
+  /**
+    * Generate a table descriptor from a createViewDesc.
+    */
+   public static TableDesc getTableDesc(CreateViewDesc crtViewDesc, String cols, String colTypes) {
+     TableDesc ret;
+ 
+     try {
+       Class serdeClass = JavaUtils.loadClass(crtViewDesc.getSerde());
+       ret = getTableDesc(serdeClass, new String(LazySerDeParameters.DefaultSeparators), cols,
+           colTypes, false,  false);
+ 
+       // set other table properties
+       /*
+       TODO - I don't think I need any of this
+       Properties properties = ret.getProperties();
+ 
+       if (crtTblDesc.getTableName() != null && crtTblDesc.getDatabaseName() != null) {
+         properties.setProperty(org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_NAME,
+             crtTblDesc.getTableName());
+       }
+ 
+       if (crtTblDesc.getTblProps() != null) {
+         properties.putAll(crtTblDesc.getTblProps());
+       }
+        */
+ 
+       // replace the default input & output file format with those found in
+       // crtTblDesc
+       Class<? extends InputFormat> inClass =
+           (Class<? extends InputFormat>)JavaUtils.loadClass(crtViewDesc.getInputFormat());
+       Class<? extends HiveOutputFormat> outClass =
+           (Class<? extends HiveOutputFormat>)JavaUtils.loadClass(crtViewDesc.getOutputFormat());
+ 
+       ret.setInputFileFormatClass(inClass);
+       ret.setOutputFileFormatClass(outClass);
+     } catch (ClassNotFoundException e) {
+       throw new RuntimeException("Unable to find class in getTableDesc: " + e.getMessage(), e);
+     }
+     return ret;
+   }
+ 
    /**
 -   * Generate the table descriptor of MetadataTypedColumnsetSerDe with the
 -   * separatorCode. MetaDataTypedColumnsetSerDe is used because LazySimpleSerDe
 -   * does not support a table with a single column "col" with type
 -   * "array<string>".
 -   */
 -  public static TableDesc getDefaultTableDesc(String separatorCode) {
 -    return new TableDesc(
 -        TextInputFormat.class, IgnoreKeyTextOutputFormat.class, Utilities
 -        .makeProperties(
 -            org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT,separatorCode,
 -            serdeConstants.SERIALIZATION_LIB,MetadataTypedColumnsetSerDe.class.getName()));
 -  }
 -
 -  /**
     * Generate the table descriptor for reduce key.
     */
    public static TableDesc getReduceKeyTableDesc(List<FieldSchema> fieldSchemas,