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/ea95b712
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/ea95b712
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/ea95b712

Branch: refs/heads/hive-14535
Commit: ea95b712618bc922c028e2e67c45739fd38fd28a
Parents: 5242f71 be47d9e
Author: Sergey Shelukhin <ser...@apache.org>
Authored: Mon Mar 13 12:35:44 2017 -0700
Committer: Sergey Shelukhin <ser...@apache.org>
Committed: Mon Mar 13 12:35:44 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hive/beeline/BeeLineOpts.java    |    9 -
 beeline/src/main/resources/BeeLine.properties   |    2 -
 bin/hive                                        |    5 +
 .../apache/hadoop/hive/common/FileUtils.java    |   42 +-
 .../org/apache/hadoop/hive/conf/Constants.java  |    1 +
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   11 +-
 .../hadoop/hive/common/TestFileUtils.java       |   32 +
 .../typedbytes/TypedBytesWritableOutput.java    |    6 +-
 data/files/test_data                            |    3 +
 druid-handler/pom.xml                           |    8 +-
 .../hadoop/hive/druid/io/DruidOutputFormat.java |   27 +-
 .../hadoop/hive/druid/io/DruidRecordWriter.java |    8 +-
 .../hadoop/hive/druid/io/HiveDruidSplit.java    |   19 +-
 .../hive/druid/io/TestHiveDruidSplit.java       |   46 +
 files/test_data                                 |    3 -
 .../test/queries/clientpositive/create_like.q   |   30 -
 .../results/clientpositive/create_like.q.out    |  189 ---
 .../test/queries/clientpositive/create_like.q   |   30 +
 .../results/clientpositive/create_like.q.out    |  189 +++
 itests/hive-unit-hadoop2/pom.xml                |    5 +
 .../hadoop/hive/common/TestFileUtils.java       |  103 ++
 itests/hive-unit/pom.xml                        |   12 -
 .../hadoop/hive/ql/TestLocationQueries.java     |    8 +-
 .../hive/ql/TestReplicationScenarios.java       |   77 +-
 .../hive/beeline/TestBeeLineWithArgs.java       |  313 ++--
 .../test/resources/testconfiguration.properties |    2 +
 .../control/AbstractCoreBlobstoreCliDriver.java |   11 +-
 .../hive/cli/control/CoreAccumuloCliDriver.java |   10 +-
 .../hadoop/hive/cli/control/CoreCliDriver.java  |   12 +-
 .../hive/cli/control/CoreCompareCliDriver.java  |   12 +-
 .../hive/cli/control/CoreHBaseCliDriver.java    |    9 +-
 .../cli/control/CoreHBaseNegativeCliDriver.java |    9 +-
 .../hive/cli/control/CoreNegativeCliDriver.java |   12 +-
 .../hive/cli/control/CorePerfCliDriver.java     |   12 +-
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |  126 +-
 .../hadoop/hive/ql/parse/CoreParseNegative.java |   19 +-
 jdbc/pom.xml                                    |    5 +-
 .../org/apache/hive/jdbc/HiveConnection.java    |    9 +-
 .../impl/LlapZookeeperRegistryImpl.java         |    3 +-
 .../hive/llap/cli/LlapOptionsProcessor.java     |   22 +-
 .../hadoop/hive/llap/cli/LlapServiceDriver.java |   24 +-
 .../hive/llap/daemon/impl/LlapDaemon.java       |    7 +-
 .../hive/llap/metrics/LlapDaemonJvmInfo.java    |   60 +
 .../hive/llap/metrics/LlapDaemonJvmMetrics.java |  116 ++
 .../tezplugins/LlapTaskSchedulerService.java    |    6 +-
 pom.xml                                         |    2 +-
 .../java/org/apache/hadoop/hive/ql/Driver.java  |    1 -
 .../hive/ql/exec/CommonMergeJoinOperator.java   |   11 +
 .../apache/hadoop/hive/ql/exec/ExplainTask.java |    9 +-
 .../hadoop/hive/ql/exec/FileSinkOperator.java   |   16 +-
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |    1 +
 .../hadoop/hive/ql/exec/mr/ExecDriver.java      |   20 +
 .../ql/exec/spark/LocalHiveSparkClient.java     |    5 +
 .../ql/exec/spark/RemoteHiveSparkClient.java    |    5 +
 .../ql/exec/tez/InterruptibleProcessing.java    |   79 +
 .../hive/ql/exec/tez/MapRecordProcessor.java    |   18 +-
 .../ql/exec/tez/MergeFileRecordProcessor.java   |   16 +-
 .../hive/ql/exec/tez/RecordProcessor.java       |   18 +-
 .../hive/ql/exec/tez/ReduceRecordProcessor.java |   27 +-
 .../hive/ql/exec/tez/TezSessionPoolManager.java |    2 +-
 .../apache/hadoop/hive/ql/exec/tez/TezTask.java |    3 +
 .../ql/exec/tez/monitoring/RenderStrategy.java  |   22 +-
 .../ql/exec/vector/VectorHashKeyWrapper.java    |   37 +-
 .../exec/vector/VectorHashKeyWrapperBatch.java  |    2 +-
 .../ql/exec/vector/VectorizationContext.java    |    5 +-
 .../VectorMapJoinGenerateResultOperator.java    |   29 +
 .../hadoop/hive/ql/io/HiveInputFormat.java      |   24 +-
 .../hadoop/hive/ql/io/orc/LocalCache.java       |   56 +-
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |   15 +-
 .../zookeeper/ZooKeeperHiveLockManager.java     |   20 +-
 .../hive/ql/metadata/HiveMetaStoreChecker.java  |   90 +-
 .../DynamicPartitionPruningOptimization.java    |   57 +-
 .../physical/CommonJoinTaskDispatcher.java      |   15 +-
 .../hive/ql/parse/BaseSemanticAnalyzer.java     |   19 +-
 .../hadoop/hive/ql/parse/GenTezUtils.java       |    4 +
 .../hadoop/hive/ql/parse/IdentifiersParser.g    |    8 +-
 .../hive/ql/parse/ImportSemanticAnalyzer.java   |    5 +-
 .../hive/ql/parse/ProcessAnalyzeTable.java      |  147 +-
 .../ql/parse/ReplicationSemanticAnalyzer.java   |   13 +-
 .../hadoop/hive/ql/parse/ReplicationSpec.java   |   19 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   33 +-
 .../hive/ql/exec/tez/SampleTezSessionState.java |    1 +
 .../hive/ql/io/orc/TestInputOutputFormat.java   |    9 +-
 .../ql/metadata/TestHiveMetaStoreChecker.java   |   39 +-
 .../queries/clientpositive/column_table_stats.q |   88 ++
 .../clientpositive/column_table_stats_orc.q     |   57 +
 .../clientpositive/dynamic_semijoin_reduction.q |   14 +-
 .../test/queries/clientpositive/groupby_empty.q |    9 +
 .../test/queries/clientpositive/perf/query14.q  |    1 +
 .../test/queries/clientpositive/perf/query23.q  |    1 +
 ql/src/test/queries/clientpositive/udf_modulo.q |    7 +
 .../test/queries/clientpositive/vector_date_1.q |    5 +
 .../clientpositive/vectorized_timestamp.q       |   10 +
 .../results/clientpositive/groupby_empty.q.out  |   38 +
 .../alter_table_invalidate_column_stats.q.out   |   16 +
 .../columnStatsUpdateForStatsOptimizer_1.q.out  |   10 +
 .../llap/column_table_stats.q.out               | 1421 ++++++++++++++++++
 .../llap/column_table_stats_orc.q.out           |  989 ++++++++++++
 .../llap/columnstats_part_coltype.q.out         |   42 +
 .../clientpositive/llap/deleteAnalyze.q.out     |    4 +
 .../llap/drop_partition_with_stats.q.out        |   80 +
 .../llap/dynamic_semijoin_reduction.q.out       |  870 ++++++-----
 .../clientpositive/llap/explainuser_2.q.out     |    6 +
 .../extrapolate_part_stats_partial_ndv.q.out    |   24 +
 .../clientpositive/llap/llap_stats.q.out        |   26 +
 .../clientpositive/llap/llapdecider.q.out       |    2 +
 .../results/clientpositive/llap/mergejoin.q.out |    2 +-
 .../llap/metadata_only_queries.q.out            |   14 +
 .../metadata_only_queries_with_filters.q.out    |    8 +
 .../clientpositive/llap/schema_evol_stats.q.out |   12 +
 .../llap/schema_evol_text_vec_table.q.out       |    2 +-
 .../llap/special_character_in_tabnames_1.q.out  |   16 +
 .../clientpositive/llap/stats_only_null.q.out   |   16 +
 .../clientpositive/llap/union_remove_26.q.out   |    6 +
 .../clientpositive/llap/vector_date_1.q.out     |   78 +
 .../llap/vector_outer_join1.q.out               |    2 +
 .../llap/vector_outer_join2.q.out               |    2 +
 .../llap/vector_outer_join3.q.out               |    2 +
 .../llap/vector_outer_join4.q.out               |    2 +
 .../llap/vector_outer_join5.q.out               |    8 +
 .../vectorized_dynamic_semijoin_reduction.q.out |    2 +
 ...vectorized_dynamic_semijoin_reduction2.q.out |    4 +
 .../llap/vectorized_timestamp.q.out             |   60 +
 .../results/clientpositive/perf/query14.q.out   |    4 +-
 .../results/clientpositive/show_functions.q.out |    2 +
 .../spark/vector_between_in.q.out               |   32 +-
 .../clientpositive/tez/explainanalyze_3.q.out   |   28 +-
 .../clientpositive/tez/explainanalyze_5.q.out   |   28 +-
 .../clientpositive/tez/explainuser_3.q.out      |   38 +-
 .../results/clientpositive/udf_modulo.q.out     |   23 +
 .../results/clientpositive/vector_date_1.q.out  |   74 +
 .../clientpositive/vectorized_timestamp.q.out   |  125 ++
 .../hadoop/hive/serde2/avro/InstanceCache.java  |    2 +-
 .../hive/serde2/lazy/fast/StringToDouble.java   |    3 +
 .../org/apache/hive/service/cli/CLIService.java |    8 +-
 .../hive/service/server/TestHS2HttpServer.java  |    5 +-
 .../apache/hadoop/hive/shims/Hadoop23Shims.java |    7 +-
 137 files changed, 5682 insertions(+), 1219 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/ea95b712/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/ea95b712/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/ea95b712/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/ea95b712/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
----------------------------------------------------------------------
diff --cc ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
index 140ac29,a9d03d0..fda3e2a
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
@@@ -239,43 -205,49 +239,53 @@@ public class FileSinkOperator extends T
        }
      }
  
 -    private void commit(FileSystem fs) throws HiveException {
 +    private void commit(FileSystem fs, List<Path> commitPaths) throws 
HiveException {
        for (int idx = 0; idx < outPaths.length; ++idx) {
          try {
 -          if ((bDynParts || isSkewedStoredAsSubDirectories)
 -              && !fs.exists(finalPaths[idx].getParent())) {
 -            fs.mkdirs(finalPaths[idx].getParent());
 -          }
 -          boolean needToRename = true;
 -          if (conf.getWriteType() == AcidUtils.Operation.UPDATE ||
 -              conf.getWriteType() == AcidUtils.Operation.DELETE) {
 -            // If we're updating or deleting there may be no file to close.  
This can happen
 -            // because the where clause strained out all of the records for a 
given bucket.  So
 -            // before attempting the rename below, check if our file exists.  
If it doesn't,
 -            // then skip the rename.  If it does try it.  We could just 
blindly try the rename
 -            // and avoid the extra stat, but that would mask other errors.
 -            try {
 -              if (outPaths[idx] != null) {
 -                FileStatus stat = fs.getFileStatus(outPaths[idx]);
 -              }
 -            } catch (FileNotFoundException fnfe) {
 -              needToRename = false;
 -            }
 -          }
 -          if (needToRename && outPaths[idx] != null && 
!fs.rename(outPaths[idx], finalPaths[idx])) {
 +          commitOneOutPath(idx, fs, commitPaths);
 +        } catch (IOException e) {
-           throw new HiveException("Unable to rename output from: " +
++          throw new HiveException("Unable to commit output from: " +
 +              outPaths[idx] + " to: " + finalPaths[idx], e);
 +        }
 +      }
 +    }
 +
 +    private void commitOneOutPath(int idx, FileSystem fs, List<Path> 
commitPaths)
 +        throws IOException, HiveException {
 +      if ((bDynParts || isSkewedStoredAsSubDirectories)
 +          && !fs.exists(finalPaths[idx].getParent())) {
 +        Utilities.LOG14535.info("commit making path for dyn/skew: " + 
finalPaths[idx].getParent());
 +        FileUtils.mkdir(fs, finalPaths[idx].getParent(), inheritPerms, hconf);
 +      }
 +      // If we're updating or deleting there may be no file to close.  This 
can happen
 +      // because the where clause strained out all of the records for a given 
bucket.  So
 +      // before attempting the rename below, check if our file exists.  If it 
doesn't,
 +      // then skip the rename.  If it does try it.  We could just blindly try 
the rename
 +      // and avoid the extra stat, but that would mask other errors.
 +      boolean needToRename = (conf.getWriteType() != 
AcidUtils.Operation.UPDATE &&
 +          conf.getWriteType() != AcidUtils.Operation.DELETE) || 
fs.exists(outPaths[idx]);
 +      if (needToRename && outPaths[idx] != null) {
 +        Utilities.LOG14535.info("committing " + outPaths[idx] + " to " + 
finalPaths[idx] + " (" + isMmTable + ")");
 +        if (isMmTable) {
 +          assert outPaths[idx].equals(finalPaths[idx]);
 +          commitPaths.add(outPaths[idx]);
 +        } else if (!fs.rename(outPaths[idx], finalPaths[idx])) {
-           throw new HiveException("Unable to rename output from: "
-               + outPaths[idx] + " to: " + finalPaths[idx]);
+             FileStatus fileStatus = FileUtils.getFileStatusOrNull(fs, 
finalPaths[idx]);
+             if (fileStatus != null) {
+               LOG.warn("Target path " + finalPaths[idx] + " with a size " + 
fileStatus.getLen() + " exists. Trying to delete it.");
+               if (!fs.delete(finalPaths[idx], true)) {
+                 throw new HiveException("Unable to delete existing target 
output: " + finalPaths[idx]);
+               }
+             }
+ 
+             if (!fs.rename(outPaths[idx], finalPaths[idx])) {
 -              throw new HiveException("Unable to rename output from: " +
 -                outPaths[idx] + " to: " + finalPaths[idx]);
++              throw new HiveException("Unable to rename output from: "
++                + outPaths[idx] + " to: " + finalPaths[idx]);
+             }
 -          }
 -          updateProgress();
 -        } catch (IOException e) {
 -          throw new HiveException("Unable to rename output from: " +
 -              outPaths[idx] + " to: " + finalPaths[idx], e);
          }
        }
 +
 +      updateProgress();
      }
  
      public void abortWriters(FileSystem fs, boolean abort, boolean delete) 
throws HiveException {

http://git-wip-us.apache.org/repos/asf/hive/blob/ea95b712/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
----------------------------------------------------------------------
diff --cc ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
index c46e323,4995bdf..c697407
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
@@@ -348,9 -352,11 +358,11 @@@ public class HiveInputFormat<K extends 
      }
  
      boolean nonNative = false;
 -    PartitionDesc part = 
HiveFileFormatUtils.getPartitionDescFromPathRecursively(
 +    PartitionDesc part = HiveFileFormatUtils.getFromPathRecursively(
          pathToPartitionInfo, hsplit.getPath(), null);
-     LOG.debug("Found spec for " + hsplit.getPath() + " " + part + " from " + 
pathToPartitionInfo);
+     if (LOG.isDebugEnabled()) {
+       LOG.debug("Found spec for " + hsplit.getPath() + " " + part + " from " 
+ pathToPartitionInfo);
+     }
  
      if ((part != null) && (part.getTableDesc() != null)) {
        Utilities.copyTableJobPropertiesToConf(part.getTableDesc(), job);

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

http://git-wip-us.apache.org/repos/asf/hive/blob/ea95b712/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
----------------------------------------------------------------------
diff --cc 
ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
index 6df9ad5,c398792..f4fe6ac
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
@@@ -1025,8 -949,8 +1025,9 @@@ public class ImportSemanticAnalyzer ext
            return; // silently return, table is newer than our replacement.
          }
          if (!replicationSpec.isMetadataOnly()) {
-           // repl-imports are replace-into
-           loadTable(fromURI, table, true, new Path(fromURI), replicationSpec, 
x, mmWriteId, isSourceMm);
+           // repl-imports are replace-into unless the event is insert-into
 -          loadTable(fromURI, table, !replicationSpec.isInsert(), new 
Path(fromURI), replicationSpec, x);
++          loadTable(fromURI, table, !replicationSpec.isInsert(), new 
Path(fromURI),
++            replicationSpec, x, mmWriteId, isSourceMm);
          } else {
            x.getTasks().add(alterTableTask(tblDesc, x, replicationSpec));
          }

http://git-wip-us.apache.org/repos/asf/hive/blob/ea95b712/ql/src/java/org/apache/hadoop/hive/ql/parse/ProcessAnalyzeTable.java
----------------------------------------------------------------------
diff --cc ql/src/java/org/apache/hadoop/hive/ql/parse/ProcessAnalyzeTable.java
index 41f471d,46c24e3..4648c64
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ProcessAnalyzeTable.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ProcessAnalyzeTable.java
@@@ -30,7 -32,7 +32,8 @@@ import org.apache.hadoop.hive.ql.Driver
  import org.apache.hadoop.hive.ql.exec.TableScanOperator;
  import org.apache.hadoop.hive.ql.exec.Task;
  import org.apache.hadoop.hive.ql.exec.TaskFactory;
 +import org.apache.hadoop.hive.ql.exec.Utilities;
+ import org.apache.hadoop.hive.ql.hooks.WriteEntity;
  import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
  import org.apache.hadoop.hive.ql.io.rcfile.stats.PartialScanWork;
  import org.apache.hadoop.hive.ql.lib.Node;

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

http://git-wip-us.apache.org/repos/asf/hive/blob/ea95b712/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
----------------------------------------------------------------------

Reply via email to