diff --git hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java index da677c7977..569de706df 100644 --- hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java +++ hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java @@ -392,7 +392,7 @@ public void testNoBuckets() throws Exception { rs = queryTable(driver,"select ROW__ID, a, b, INPUT__FILE__NAME from default.streamingnobuckets order by ROW__ID"); Assert.assertTrue(rs.get(0), rs.get(0).startsWith("{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\tfoo\tbar")); - Assert.assertTrue(rs.get(0), rs.get(0).endsWith("streamingnobuckets/delta_0000001_0000001_0000/bucket_00000")); + Assert.assertTrue(rs.get(0), rs.get(0).endsWith("streamingnobuckets/delta_0000001_0000001_0000/bucket_00000_0")); Assert.assertTrue(rs.get(1), rs.get(1).startsWith("{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\ta1\tb2")); Assert.assertTrue(rs.get(1), rs.get(1).endsWith("streamingnobuckets/delta_0000002_0000003/bucket_00000")); Assert.assertTrue(rs.get(2), rs.get(2).startsWith("{\"writeid\":2,\"bucketid\":536870912,\"rowid\":1}\ta3\tb4")); diff --git itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestAcidOnTez.java itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestAcidOnTez.java index 056cd27496..01b7a361b6 100644 --- itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestAcidOnTez.java +++ itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestAcidOnTez.java @@ -681,11 +681,11 @@ public void testAcidInsertWithRemoveUnion() throws Exception { } String[][] expected2 = { - {"{\"writeid\":1,\"bucketid\":536870913,\"rowid\":0}\t1\t2", "warehouse/t/delta_0000001_0000001_0001/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536870913,\"rowid\":1}\t3\t4", "warehouse/t/delta_0000001_0000001_0001/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536870914,\"rowid\":0}\t5\t6", "warehouse/t/delta_0000001_0000001_0002/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536870914,\"rowid\":1}\t7\t8", "warehouse/t/delta_0000001_0000001_0002/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536870915,\"rowid\":0}\t9\t10", "warehouse/t/delta_0000001_0000001_0003/bucket_00000"} + {"{\"writeid\":1,\"bucketid\":536870913,\"rowid\":0}\t1\t2", "warehouse/t/delta_0000001_0000001_0001/bucket_00000_0"}, + {"{\"writeid\":1,\"bucketid\":536870913,\"rowid\":1}\t3\t4", "warehouse/t/delta_0000001_0000001_0001/bucket_00000_0"}, + {"{\"writeid\":1,\"bucketid\":536870914,\"rowid\":0}\t5\t6", "warehouse/t/delta_0000001_0000001_0002/bucket_00000_0"}, + {"{\"writeid\":1,\"bucketid\":536870914,\"rowid\":1}\t7\t8", "warehouse/t/delta_0000001_0000001_0002/bucket_00000_0"}, + {"{\"writeid\":1,\"bucketid\":536870915,\"rowid\":0}\t9\t10", "warehouse/t/delta_0000001_0000001_0003/bucket_00000_0"} }; Assert.assertEquals("Unexpected row count", expected2.length, rs.size()); for(int i = 0; i < expected2.length; i++) { @@ -727,11 +727,11 @@ public void testBucketedAcidInsertWithRemoveUnion() throws Exception { LOG.warn(s); } String[][] expected2 = { - {"{\"writeid\":1,\"bucketid\":536936448,\"rowid\":0}\t1\t2", "warehouse/t/delta_0000001_0000001_0000/bucket_00001"}, - {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t2\t4", "warehouse/t/delta_0000001_0000001_0000/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536936448,\"rowid\":1}\t5\t6", "warehouse/t/delta_0000001_0000001_0000/bucket_00001"}, - {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t6\t8", "warehouse/t/delta_0000001_0000001_0000/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536936448,\"rowid\":2}\t9\t10", "warehouse/t/delta_0000001_0000001_0000/bucket_00001"} + {"{\"writeid\":1,\"bucketid\":536936448,\"rowid\":0}\t1\t2", "warehouse/t/delta_0000001_0000001_0000/bucket_00001_0"}, + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t2\t4", "warehouse/t/delta_0000001_0000001_0000/bucket_00000_0"}, + {"{\"writeid\":1,\"bucketid\":536936448,\"rowid\":1}\t5\t6", "warehouse/t/delta_0000001_0000001_0000/bucket_00001_0"}, + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t6\t8", "warehouse/t/delta_0000001_0000001_0000/bucket_00000_0"}, + {"{\"writeid\":1,\"bucketid\":536936448,\"rowid\":2}\t9\t10", "warehouse/t/delta_0000001_0000001_0000/bucket_00001_0"} }; Assert.assertEquals("Unexpected row count", expected2.length, rs.size()); for(int i = 0; i < expected2.length; i++) { diff --git itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/history/TestHiveHistory.java itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/history/TestHiveHistory.java index 31d15fdef9..127de2301b 100644 --- itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/history/TestHiveHistory.java +++ itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/history/TestHiveHistory.java @@ -112,7 +112,7 @@ public void setUp() { db.createTable(src, cols, null, TextInputFormat.class, IgnoreKeyTextOutputFormat.class); db.loadTable(hadoopDataFile[i], src, - LoadFileType.KEEP_EXISTING, false, false, false, false, null, 0, false); + LoadFileType.KEEP_EXISTING, false, false, false, false, null, 0, false, false); i++; } diff --git itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorTestUtil.java itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorTestUtil.java index c2aa73b5f1..62c088da34 100644 --- itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorTestUtil.java +++ itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorTestUtil.java @@ -50,7 +50,9 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; @@ -346,7 +348,7 @@ public RangeResponse isWriteIdRangeAborted(long minWriteId, long maxWriteId) { conf.setBoolean("orc.schema.evolution.case.sensitive", false); HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVE_TRANSACTIONAL_TABLE_SCAN, true); AcidInputFormat.RawReader reader = - aif.getRawReader(conf, true, bucket, writeIdList, base, deltas); + aif.getRawReader(conf, true, bucket, writeIdList, base, deltas, new HashMap()); RecordIdentifier identifier = reader.createKey(); OrcStruct value = reader.createValue(); long currentTxn = min; diff --git itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java index 4c01311117..c8add42d64 100644 --- itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java +++ itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java @@ -92,6 +92,8 @@ public void setup() throws Exception { hiveConf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE, TEST_WAREHOUSE_DIR); hiveConf.setVar(HiveConf.ConfVars.HIVEINPUTFORMAT, HiveInputFormat.class.getName()); hiveConf.setVar(HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "none"); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVESTATSAUTOGATHER, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVESTATSCOLAUTOGATHER, false); TxnDbUtil.setConfValues(hiveConf); TxnDbUtil.cleanDb(hiveConf); TxnDbUtil.prepDb(hiveConf); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractFileMergeOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractFileMergeOperator.java index 9a3258115b..d68d8f9409 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractFileMergeOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractFileMergeOperator.java @@ -257,7 +257,7 @@ public void closeOp(boolean abort) throws HiveException { assert finalPath.equals(outPath); // There's always just one file that we have merged. // The union/DP/etc. should already be account for in the path. - Utilities.writeMmCommitManifest(Lists.newArrayList(outPath), + Utilities.writeCommitManifest(Lists.newArrayList(outPath), tmpPath.getParent(), fs, taskId, conf.getWriteId(), conf.getStmtId(), null, false); LOG.info("Merged into " + finalPath + "(" + fss.getLen() + " bytes)."); } @@ -337,8 +337,8 @@ public void jobCloseOp(Configuration hconf, boolean success) lbLevels = conf.getListBucketingDepth(); // We don't expect missing buckets from mere (actually there should be no buckets), // so just pass null as bucketing context. Union suffix should also be accounted for. - Utilities.handleMmTableFinalPath(outputDir.getParent(), null, hconf, success, - dpLevels, lbLevels, null, mmWriteId, stmtId, reporter, isMmTable, false, false); + Utilities.handleDirectInsertTableFinalPath(outputDir.getParent(), null, hconf, success, + dpLevels, lbLevels, null, mmWriteId, stmtId, reporter, isMmTable, false, false, false); } } catch (IOException e) { diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java index 9ad4e71482..5c5972600d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java @@ -161,6 +161,8 @@ private final String subdirForTxn; private Path taskOutputTempPathRoot; Path[] outPaths; + // The bucket files we successfully wrote to in this writer + Path[] outPathsCommitted; Path[] finalPaths; RecordWriter[] outWriters; RecordUpdater[] updaters; @@ -168,19 +170,31 @@ int acidLastBucket = -1; int acidFileOffset = -1; private boolean isMmTable; + private boolean isDirectInsert; + private boolean isInsertOverwrite; String dpDirForCounters; - public FSPaths(Path specPath, boolean isMmTable) { + public FSPaths(Path specPath, boolean isMmTable, boolean isDirectInsert, boolean isInsertOverwrite) { this.isMmTable = isMmTable; - if (!isMmTable) { + this.isDirectInsert = isDirectInsert; + this.isInsertOverwrite = isInsertOverwrite; + if (!isMmTable && !isDirectInsert) { tmpPathRoot = Utilities.toTempPath(specPath); taskOutputTempPathRoot = Utilities.toTaskTempPath(specPath); subdirForTxn = null; } else { tmpPathRoot = specPath; taskOutputTempPathRoot = null; // Should not be used. - subdirForTxn = AcidUtils.baseOrDeltaSubdir(conf.getInsertOverwrite(), - conf.getTableWriteId(), conf.getTableWriteId(), conf.getStatementId()); + if (isMmTable) { + subdirForTxn = AcidUtils.baseOrDeltaSubdir(conf.getInsertOverwrite(), + conf.getTableWriteId(), conf.getTableWriteId(), conf.getStatementId()); + } else { + /** + * For direct write to final path during ACID insert, we create the delta directories + * later when we create the RecordUpdater using AcidOutputFormat.Options + */ + subdirForTxn = null; + } } if (Utilities.FILE_OP_LOGGER.isTraceEnabled()) { Utilities.FILE_OP_LOGGER.trace("new FSPaths for " + numFiles @@ -188,6 +202,7 @@ public FSPaths(Path specPath, boolean isMmTable) { } outPaths = new Path[numFiles]; + outPathsCommitted = new Path[numFiles]; finalPaths = new Path[numFiles]; outWriters = new RecordWriter[numFiles]; updaters = new RecordUpdater[numFiles]; @@ -211,6 +226,11 @@ public void closeWriters(boolean abort) throws HiveException { try { for (int i = 0; i < updaters.length; i++) { if (updaters[i] != null) { + SerDeStats stats = updaters[i].getStats(); + // Ignore 0 row files + if (isDirectInsert && (stats.getRowCount() > 0 || isInsertOverwrite)) { + outPathsCommitted[i] = updaters[i].getUpdatedFilePath(); + } updaters[i].close(abort); } } @@ -249,7 +269,9 @@ private void commitOneOutPath(int idx, FileSystem fs, List commitPaths) if (isMmTable) { assert outPaths[idx].equals(finalPaths[idx]); commitPaths.add(outPaths[idx]); - } else if (!fs.rename(outPaths[idx], finalPaths[idx])) { + } else if (isDirectInsert && (outPathsCommitted[idx] != null)) { + commitPaths.add(outPathsCommitted[idx]); + } else if (!isDirectInsert && !fs.rename(outPaths[idx], 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."); @@ -264,15 +286,14 @@ private void commitOneOutPath(int idx, FileSystem fs, List commitPaths) } } } - updateProgress(); } - public void abortWriters(FileSystem fs, boolean abort, boolean delete) throws HiveException { - //should this close updaters[]? + public void abortWritersAndUpdaters(FileSystem fs, boolean abort, boolean delete) throws HiveException { for (int idx = 0; idx < outWriters.length; idx++) { if (outWriters[idx] != null) { try { + LOG.debug("Aborted: closing: " + outWriters[idx].toString()); outWriters[idx].close(abort); if (delete) { fs.delete(outPaths[idx], true); @@ -283,6 +304,20 @@ public void abortWriters(FileSystem fs, boolean abort, boolean delete) throws Hi } } } + for (int idx = 0; idx < updaters.length; idx++) { + if (updaters[idx] != null) { + try { + LOG.debug("Aborted: closing: " + updaters[idx].toString()); + updaters[idx].close(abort); + if (delete) { + fs.delete(outPaths[idx], true); + } + updateProgress(); + } catch (IOException e) { + throw new HiveException(e); + } + } + } } public void initializeBucketPaths(int filesIdx, String taskId, boolean isNativeTable, @@ -290,7 +325,7 @@ public void initializeBucketPaths(int filesIdx, String taskId, boolean isNativeT if (isNativeTable) { String extension = Utilities.getFileExtension(jc, isCompressed, hiveOutputFormat); String taskWithExt = extension == null ? taskId : taskId + extension; - if (!isMmTable) { + if (!isMmTable && !isDirectInsert) { if (!bDynParts && !isSkewedStoredAsSubDirectories) { finalPaths[filesIdx] = new Path(parent, taskWithExt); } else { @@ -308,7 +343,12 @@ public void initializeBucketPaths(int filesIdx, String taskId, boolean isNativeT taskIdPath += extension; } - Path finalPath = new Path(buildTmpPath(), taskIdPath); + Path finalPath; + if (isDirectInsert) { + finalPath = buildTmpPath(); + } else { + finalPath = new Path(buildTmpPath(), taskIdPath); + } // In the cases that have multi-stage insert, e.g. a "hive.skewjoin.key"-based skew join, // it can happen that we want multiple commits into the same directory from different @@ -319,7 +359,7 @@ public void initializeBucketPaths(int filesIdx, String taskId, boolean isNativeT // affects some less obscure scenario. try { FileSystem fpfs = finalPath.getFileSystem(hconf); - if (fpfs.exists(finalPath)) { + if ((!isDirectInsert) && fpfs.exists(finalPath)) { throw new RuntimeException(finalPath + " already exists"); } } catch (IOException e) { @@ -330,7 +370,7 @@ public void initializeBucketPaths(int filesIdx, String taskId, boolean isNativeT } if (LOG.isInfoEnabled()) { LOG.info("Final Path: FS " + finalPaths[filesIdx]); - if (LOG.isInfoEnabled() && !isMmTable) { + if (LOG.isInfoEnabled() && (!isMmTable && !isDirectInsert)) { LOG.info("Writing to temp file: FS " + outPaths[filesIdx]); } } @@ -472,6 +512,9 @@ private void initializeSpecPath() { // MM tables need custom handling for union suffix; DP tables use parent too. specPath = conf.getParentDir(); unionPath = conf.getDirName().getName(); + } else if (conf.isDirectInsert()) { + specPath = conf.getParentDir(); + unionPath = null; } else { // For now, keep the old logic for non-MM non-DP union case. Should probably be unified. specPath = conf.getDirName(); @@ -526,7 +569,11 @@ protected void initializeOp(Configuration hconf) throws HiveException { throw ex; } isCompressed = conf.getCompressed(); - parent = Utilities.toTempPath(conf.getDirName()); + if (conf.isLinkedFileSink() && conf.isDirectInsert()) { + parent = Utilities.toTempPath(conf.getFinalDirName()); + } else { + parent = Utilities.toTempPath(conf.getDirName()); + } statsFromRecordWriter = new boolean[numFiles]; serializer = (Serializer) conf.getTableInfo().getDeserializerClass().newInstance(); serializer.initialize(unsetNestedColumnPaths(hconf), conf.getTableInfo().getProperties()); @@ -565,7 +612,7 @@ protected void initializeOp(Configuration hconf) throws HiveException { } if (!bDynParts) { - fsp = new FSPaths(specPath, conf.isMmTable()); + fsp = new FSPaths(specPath, conf.isMmTable(), conf.isDirectInsert(), conf.getInsertOverwrite()); fsp.subdirAfterTxn = combinePathFragments(generateListBucketingDirName(null), unionPath); if (Utilities.FILE_OP_LOGGER.isTraceEnabled()) { Utilities.FILE_OP_LOGGER.trace("creating new paths " + System.identityHashCode(fsp) @@ -740,7 +787,7 @@ protected void createBucketFiles(FSPaths fsp) throws HiveException { assert filesIdx == numFiles; // in recent hadoop versions, use deleteOnExit to clean tmp files. - if (isNativeTable() && fs != null && fsp != null && !conf.isMmTable()) { + if (isNativeTable() && fs != null && fsp != null && !conf.isMmTable() && !conf.isDirectInsert()) { autoDelete = fs.deleteOnExit(fsp.outPaths[0]); } } catch (Exception e) { @@ -764,7 +811,7 @@ protected void createBucketForFileIdx(FSPaths fsp, int filesIdx) LOG.info("New Final Path: FS " + fsp.finalPaths[filesIdx]); } - if (isNativeTable() && !conf.isMmTable()) { + if (isNativeTable() && !conf.isMmTable() && !conf.isDirectInsert()) { // in recent hadoop versions, use deleteOnExit to clean tmp files. autoDelete = fs.deleteOnExit(fsp.outPaths[filesIdx]); } @@ -790,12 +837,21 @@ protected void createBucketForFileIdx(FSPaths fsp, int filesIdx) StatsProvidingRecordWriter; // increment the CREATED_FILES counter } else if (conf.getWriteType() == AcidUtils.Operation.INSERT) { + Path outPath = fsp.outPaths[filesIdx]; + if (conf.isDirectInsert() + && !FileUtils.mkdir(fs, outPath.getParent(), hconf)) { + LOG.warn("Unable to create directory with inheritPerms: " + outPath); + } // Only set up the updater for insert. For update and delete we don't know unitl we see // the row. ObjectInspector inspector = bDynParts ? subSetOI : outputObjInspector; int acidBucketNum = Integer.parseInt(Utilities.getTaskIdFromFilename(taskId)); + String attemptId = null; + if (conf.isDirectInsert()) { + attemptId = taskId.split("_")[1]; + } fsp.updaters[filesIdx] = HiveFileFormatUtils.getAcidRecordUpdater(jc, conf.getTableInfo(), - acidBucketNum, conf, fsp.outPaths[filesIdx], inspector, reporter, -1); + acidBucketNum, conf, fsp.outPaths[filesIdx], inspector, reporter, -1, attemptId); // outPath.getParent() } if (reporter != null) { @@ -824,9 +880,9 @@ private void updateDPCounters(final FSPaths fsp, final int filesIdx) { // /// // for non-MM tables, the final destination partition directory is created during move task via rename - // for MM tables, the final destination partition directory is created by the tasks themselves + // for MM tables and ACID insert, the final destination partition directory is created by the tasks themselves try { - if (conf.isMmTable()) { + if (conf.isMmTable() || conf.isDirectInsert()) { createDpDir(destPartPath); } else { // outPath will be @@ -1086,7 +1142,7 @@ assert getConf().getWriteType() != AcidUtils.Operation.DELETE && * @throws HiveException */ private FSPaths createNewPaths(String dpDir, String lbDir) throws HiveException { - FSPaths fsp2 = new FSPaths(specPath, conf.isMmTable()); + FSPaths fsp2 = new FSPaths(specPath, conf.isMmTable(), conf.isDirectInsert(), conf.getInsertOverwrite()); fsp2.subdirAfterTxn = combinePathFragments(lbDir, unionPath); fsp2.subdirBeforeTxn = dpDir; String pathKey = combinePathFragments(dpDir, lbDir); @@ -1337,9 +1393,9 @@ public void closeOp(boolean abort) throws HiveException { SparkMetricUtils.updateSparkBytesWrittenMetrics(LOG, fs, fsp.finalPaths); } } - if (conf.isMmTable()) { - Utilities.writeMmCommitManifest(commitPaths, specPath, fs, originalTaskId, - conf.getTableWriteId(), conf.getStatementId(), unionPath, conf.getInsertOverwrite()); + if (conf.isMmTable() || conf.isDirectInsert()) { + Utilities.writeCommitManifest(commitPaths, specPath, fs, originalTaskId, conf.getTableWriteId(), conf + .getStatementId(), unionPath, conf.getInsertOverwrite()); } // Only publish stats if this operator's flag was set to gather stats if (conf.isGatherStats()) { @@ -1350,7 +1406,7 @@ public void closeOp(boolean abort) throws HiveException { // Hadoop always call close() even if an Exception was thrown in map() or // reduce(). for (FSPaths fsp : valToPaths.values()) { - fsp.abortWriters(fs, abort, !autoDelete && isNativeTable() && !conf.isMmTable()); + fsp.abortWritersAndUpdaters(fs, abort, !autoDelete && isNativeTable() && !conf.isMmTable()); } } fsp = prevFsp = null; @@ -1383,10 +1439,14 @@ public void jobCloseOp(Configuration hconf, boolean success) specPath = conf.getParentDir(); unionSuffix = conf.getDirName().getName(); } + if (conf.isLinkedFileSink() && conf.isDirectInsert()) { + specPath = conf.getParentDir(); + unionSuffix = null; + } if (Utilities.FILE_OP_LOGGER.isTraceEnabled()) { Utilities.FILE_OP_LOGGER.trace("jobCloseOp using specPath " + specPath); } - if (!conf.isMmTable()) { + if (!conf.isMmTable() && !conf.isDirectInsert()) { Utilities.mvFileToFinalPath(specPath, hconf, success, LOG, dpCtx, conf, reporter); } else { int dpLevels = dpCtx == null ? 0 : dpCtx.getNumDPCols(), @@ -1396,9 +1456,9 @@ public void jobCloseOp(Configuration hconf, boolean success) : (dpCtx != null ? dpCtx.getNumBuckets() : 0); MissingBucketsContext mbc = new MissingBucketsContext( conf.getTableInfo(), numBuckets, conf.getCompressed()); - Utilities.handleMmTableFinalPath(specPath, unionSuffix, hconf, success, - dpLevels, lbLevels, mbc, conf.getTableWriteId(), conf.getStatementId(), reporter, - conf.isMmTable(), conf.isMmCtas(), conf.getInsertOverwrite()); + Utilities.handleDirectInsertTableFinalPath(specPath, unionSuffix, hconf, success, dpLevels, lbLevels, mbc, + conf.getTableWriteId(), conf.getStatementId(), reporter, conf.isMmTable(), conf.isMmCtas(), conf + .getInsertOverwrite(), conf.isDirectInsert()); } } } catch (IOException e) { diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java index 06e4ebee82..51de87f2fd 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java @@ -419,7 +419,7 @@ public int execute() { db.loadTable(tbd.getSourcePath(), tbd.getTable().getTableName(), tbd.getLoadFileType(), work.isSrcLocal(), isSkewedStoredAsDirs(tbd), isFullAcidOp, resetStatisticsProps(table), tbd.getWriteId(), tbd.getStmtId(), - tbd.isInsertOverwrite()); + tbd.isInsertOverwrite(), tbd.isDirectInsert()); if (work.getOutputs() != null) { DDLUtils.addIfAbsentByName(new WriteEntity(table, getWriteType(tbd, work.getLoadTableWork().getWriteType())), work.getOutputs()); @@ -521,7 +521,7 @@ private DataContainer handleStaticParts(Hive db, Table table, LoadTableDesc tbd, work.getLoadTableWork().getWriteType() != AcidUtils.Operation.NOT_ACID && !tbd.isMmTable(), resetStatisticsProps(table), tbd.getWriteId(), tbd.getStmtId(), - tbd.isInsertOverwrite()); + tbd.isInsertOverwrite(), tbd.isDirectInsert()); Partition partn = db.getPartition(table, tbd.getPartitionSpec(), false); // See the comment inside updatePartitionBucketSortColumns. @@ -568,7 +568,9 @@ private DataContainer handleDynParts(Hive db, Table table, LoadTableDesc tbd, tbd.getStmtId(), resetStatisticsProps(table), work.getLoadTableWork().getWriteType(), - tbd.isInsertOverwrite()); + tbd.isInsertOverwrite(), + tbd.isDirectInsert() + ); // publish DP columns to its subscribers if (dps != null && dps.size() > 0) { diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java index 6c67bc7dd8..59e3347d1d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java @@ -1600,7 +1600,7 @@ public static void removeTempOrDuplicateFiles(FileSystem fs, Path path, boolean int dpLevels = dpCtx == null ? 0 : dpCtx.getNumDPCols(), numBuckets = (conf != null && conf.getTable() != null) ? conf.getTable().getNumBuckets() : 0; return removeTempOrDuplicateFiles( - fs, fileStats, null, dpLevels, numBuckets, hconf, null, 0, false, filesKept, isBaseDir); + fs, fileStats, null, dpLevels, numBuckets, hconf, null, 0, false, filesKept, isBaseDir, false); } private static FileStatus[] removeEmptyDpDirectory(FileSystem fs, Path path) throws IOException { @@ -1664,9 +1664,9 @@ public static void removeTempOrDuplicateFiles(FileSystem fs, Path path, boolean return partStatusMap; } - public static List removeTempOrDuplicateFiles(FileSystem fs, FileStatus[] fileStats, - String unionSuffix, int dpLevels, int numBuckets, Configuration hconf, Long writeId, - int stmtId, boolean isMmTable, Set filesKept, boolean isBaseDir) throws IOException { + public static List removeTempOrDuplicateFiles(FileSystem fs, FileStatus[] fileStats, String unionSuffix, + int dpLevels, int numBuckets, Configuration hconf, Long writeId, int stmtId, boolean isMmTable, + Set filesKept, boolean isBaseDir, boolean isDirectInsert) throws IOException { if (fileStats == null) { return null; } @@ -1683,12 +1683,12 @@ public static void removeTempOrDuplicateFiles(FileSystem fs, Path path, boolean continue; } - if (isMmTable) { + if (isMmTable || isDirectInsert) { if (!path.getName().equals(AcidUtils.baseOrDeltaSubdir(isBaseDir, writeId, writeId, stmtId))) { - throw new IOException("Unexpected non-MM directory name " + path); + throw new IOException("Unexpected direct insert directory name " + path); } - Utilities.FILE_OP_LOGGER.trace("removeTempOrDuplicateFiles processing files in MM directory {}", path); + Utilities.FILE_OP_LOGGER.trace("removeTempOrDuplicateFiles processing files in direct insert directory {}", path); if (!StringUtils.isEmpty(unionSuffix)) { try { @@ -1699,100 +1699,114 @@ public static void removeTempOrDuplicateFiles(FileSystem fs, Path path, boolean } } - taskIDToFile = removeTempOrDuplicateFilesNonMm(items, fs); + taskIDToFile = removeTempOrDuplicateFilesNonDirectInsert(items, fs); if (filesKept != null && taskIDToFile != null) { addFilesToPathSet(taskIDToFile.values(), filesKept); } - addBucketFileToResults(taskIDToFile, numBuckets, hconf, result); + addBucketFileToResults(taskIDToFile, numBuckets, hconf, result, isDirectInsert); } - } else if (isMmTable && !StringUtils.isEmpty(unionSuffix)) { + } else if ((isMmTable || isDirectInsert) && !StringUtils.isEmpty(unionSuffix)) { if (fileStats.length == 0) { return result; } - Path mmDir = extractNonDpMmDir(writeId, stmtId, fileStats, isBaseDir); - taskIDToFile = removeTempOrDuplicateFilesNonMm( - fs.listStatus(new Path(mmDir, unionSuffix)), fs); + Path dir = extractNonDpDirectInsertDir(writeId, stmtId, fileStats, isBaseDir); + taskIDToFile = removeTempOrDuplicateFilesNonDirectInsert( + fs.listStatus(new Path(dir, unionSuffix)), fs); if (filesKept != null && taskIDToFile != null) { addFilesToPathSet(taskIDToFile.values(), filesKept); } - addBucketFileToResults2(taskIDToFile, numBuckets, hconf, result); + addBucketFileToResults2(taskIDToFile, numBuckets, hconf, result, isDirectInsert); } else { if (fileStats.length == 0) { return result; } - if (!isMmTable) { - taskIDToFile = removeTempOrDuplicateFilesNonMm(fileStats, fs); + if (!isMmTable && !isDirectInsert) { + taskIDToFile = removeTempOrDuplicateFilesNonDirectInsert(fileStats, fs); if (filesKept != null && taskIDToFile != null) { addFilesToPathSet(taskIDToFile.values(), filesKept); } } else { - Path mmDir = extractNonDpMmDir(writeId, stmtId, fileStats, isBaseDir); - taskIDToFile = removeTempOrDuplicateFilesNonMm(fs.listStatus(mmDir), fs); + Path dir = extractNonDpDirectInsertDir(writeId, stmtId, fileStats, isBaseDir); + PathFilter orcAcidVersionFilter = new PathFilter() { + @Override + public boolean accept(Path p) { + String name = p.getName(); + return !name.equals(AcidUtils.OrcAcidVersion.ACID_FORMAT); + } + }; + taskIDToFile = removeTempOrDuplicateFilesNonDirectInsert(fs.listStatus(dir, orcAcidVersionFilter), fs); if (filesKept != null && taskIDToFile != null) { addFilesToPathSet(taskIDToFile.values(), filesKept); } } - addBucketFileToResults2(taskIDToFile, numBuckets, hconf, result); + addBucketFileToResults2(taskIDToFile, numBuckets, hconf, result, isDirectInsert); } return result; } - private static Path extractNonDpMmDir(Long writeId, int stmtId, FileStatus[] items, boolean isBaseDir) throws IOException { + private static Path extractNonDpDirectInsertDir(Long writeId, int stmtId, FileStatus[] items, boolean isBaseDir) + throws IOException { if (items.length > 1) { - throw new IOException("Unexpected directories for non-DP MM: " + Arrays.toString(items)); + throw new IOException("Unexpected directories for non-DP direct insert: " + Arrays.toString(items)); } - Path mmDir = items[0].getPath(); - if (!mmDir.getName().equals(AcidUtils.baseOrDeltaSubdir(isBaseDir, writeId, writeId, stmtId))) { - throw new IOException("Unexpected non-MM directory " + mmDir); + Path dir = items[0].getPath(); + if (!dir.getName().equals(AcidUtils.baseOrDeltaSubdir(isBaseDir, writeId, writeId, stmtId))) { + throw new IOException("Unexpected non direct insert directory " + dir); } - Utilities.FILE_OP_LOGGER.trace("removeTempOrDuplicateFiles processing files in MM directory {}", mmDir); - return mmDir; + Utilities.FILE_OP_LOGGER.trace("removeTempOrDuplicateFiles processing files in direct insert directory {}", dir); + return dir; } // TODO: not clear why two if conditions are different. Preserve the existing logic for now. private static void addBucketFileToResults2(HashMap taskIDToFile, - int numBuckets, Configuration hconf, List result) { + int numBuckets, Configuration hconf, List result, boolean isDirectInsert) { if (MapUtils.isNotEmpty(taskIDToFile) && (numBuckets > taskIDToFile.size()) && !"tez".equalsIgnoreCase(hconf.get(ConfVars.HIVE_EXECUTION_ENGINE.varname))) { - addBucketsToResultsCommon(taskIDToFile, numBuckets, result); + addBucketsToResultsCommon(taskIDToFile, numBuckets, result, isDirectInsert); } } // TODO: not clear why two if conditions are different. Preserve the existing logic for now. private static void addBucketFileToResults(HashMap taskIDToFile, - int numBuckets, Configuration hconf, List result) { + int numBuckets, Configuration hconf, List result, boolean isDirectInsert) { // if the table is bucketed and enforce bucketing, we should check and generate all buckets if (numBuckets > 0 && taskIDToFile != null && !"tez".equalsIgnoreCase(hconf.get(ConfVars.HIVE_EXECUTION_ENGINE.varname))) { - addBucketsToResultsCommon(taskIDToFile, numBuckets, result); + addBucketsToResultsCommon(taskIDToFile, numBuckets, result, isDirectInsert); } } - private static void addBucketsToResultsCommon( - HashMap taskIDToFile, int numBuckets, List result) { + private static void addBucketsToResultsCommon(HashMap taskIDToFile, int numBuckets, + List result, boolean isDirectInsert) { String taskID1 = taskIDToFile.keySet().iterator().next(); Path bucketPath = taskIDToFile.values().iterator().next().getPath(); for (int j = 0; j < numBuckets; ++j) { - addBucketFileIfMissing(result, taskIDToFile, taskID1, bucketPath, j); + addBucketFileIfMissing(result, taskIDToFile, taskID1, bucketPath, j, isDirectInsert); } } - private static void addBucketFileIfMissing(List result, - HashMap taskIDToFile, String taskID1, Path bucketPath, int j) { + private static void addBucketFileIfMissing(List result, HashMap taskIDToFile, + String taskID1, Path bucketPath, int j, boolean isDirectInsert) { String taskID2 = replaceTaskId(taskID1, j); if (!taskIDToFile.containsKey(taskID2)) { // create empty bucket, file name should be derived from taskID2 URI bucketUri = bucketPath.toUri(); - String path2 = replaceTaskIdFromFilename(bucketUri.getPath().toString(), j); - Utilities.FILE_OP_LOGGER.trace("Creating an empty bucket file {}", path2); - result.add(new Path(bucketUri.getScheme(), bucketUri.getAuthority(), path2)); + if (isDirectInsert) { + Path missingBucketPath = AcidUtils.createBucketFile(bucketPath.getParent(), j); + Utilities.FILE_OP_LOGGER.trace("Creating an empty bucket file {}", missingBucketPath.toString()); + result.add(missingBucketPath); + } else { + String path2 = replaceTaskIdFromFilename(bucketUri.getPath().toString(), j); + Utilities.FILE_OP_LOGGER.trace("Creating an empty bucket file {}", path2); + result.add(new Path(bucketUri.getScheme(), bucketUri.getAuthority(), path2)); + } } } - private static HashMap removeTempOrDuplicateFilesNonMm( + private static HashMap removeTempOrDuplicateFilesNonDirectInsert( FileStatus[] files, FileSystem fs) throws IOException { if (files == null || fs == null) { return null; @@ -1963,9 +1977,9 @@ public static int getBucketIdFromFile(String bucketName) { // This can happen in ACID cases when we have splits on delta files, where the filenames // are of the form delta_x_y/bucket_a. if (bucketName.startsWith(AcidUtils.BUCKET_PREFIX)) { - m = AcidUtils.BUCKET_DIGIT_PATTERN.matcher(bucketName); + m = AcidUtils.BUCKET_PATTERN.matcher(bucketName); if (m.find()) { - return Integer.parseInt(m.group()); + return Integer.parseInt(m.group(1)); } // Note that legacy bucket digit pattern are being ignored here. } @@ -3694,8 +3708,9 @@ private static void createTmpDirs(Configuration conf, if (op instanceof FileSinkOperator) { FileSinkDesc fdesc = ((FileSinkOperator) op).getConf(); - if (fdesc.isMmTable()) { - continue; // No need to create for MM tables + if (fdesc.isMmTable() || fdesc.isDirectInsert()) { + // No need to create for MM tables, or ACID insert + continue; } Path tempDir = fdesc.getDirName(); if (tempDir != null) { @@ -4108,7 +4123,7 @@ private static void tryDelete(FileSystem fs, Path path) { } } - public static Path[] getMmDirectoryCandidates(FileSystem fs, Path path, int dpLevels, + public static Path[] getDirectInsertDirectoryCandidates(FileSystem fs, Path path, int dpLevels, PathFilter filter, long writeId, int stmtId, Configuration conf, Boolean isBaseDir) throws IOException { int skipLevels = dpLevels; @@ -4124,9 +4139,9 @@ private static void tryDelete(FileSystem fs, Path path) { // /want/ to know isBaseDir because that is error prone; so, it ends up never being used. if (stmtId < 0 || isBaseDir == null || (HiveConf.getBoolVar(conf, ConfVars.HIVE_MM_AVOID_GLOBSTATUS_ON_S3) && isS3(fs))) { - return getMmDirectoryCandidatesRecursive(fs, path, skipLevels, filter); + return getDirectInsertDirectoryCandidatesRecursive(fs, path, skipLevels, filter); } - return getMmDirectoryCandidatesGlobStatus(fs, path, skipLevels, filter, writeId, stmtId, isBaseDir); + return getDirectInsertDirectoryCandidatesGlobStatus(fs, path, skipLevels, filter, writeId, stmtId, isBaseDir); } private static boolean isS3(FileSystem fs) { @@ -4149,7 +4164,7 @@ private static boolean isS3(FileSystem fs) { return paths; } - private static Path[] getMmDirectoryCandidatesRecursive(FileSystem fs, + private static Path[] getDirectInsertDirectoryCandidatesRecursive(FileSystem fs, Path path, int skipLevels, PathFilter filter) throws IOException { String lastRelDir = null; HashSet results = new HashSet(); @@ -4202,7 +4217,7 @@ private static boolean isS3(FileSystem fs) { return results.toArray(new Path[results.size()]); } - private static Path[] getMmDirectoryCandidatesGlobStatus(FileSystem fs, Path path, int skipLevels, + private static Path[] getDirectInsertDirectoryCandidatesGlobStatus(FileSystem fs, Path path, int skipLevels, PathFilter filter, long writeId, int stmtId, boolean isBaseDir) throws IOException { StringBuilder sb = new StringBuilder(path.toUri().getPath()); for (int i = 0; i < skipLevels; i++) { @@ -4219,10 +4234,10 @@ private static boolean isS3(FileSystem fs) { return statusToPath(fs.globStatus(pathPattern, filter)); } - private static void tryDeleteAllMmFiles(FileSystem fs, Path specPath, Path manifestDir, + private static void tryDeleteAllDirectInsertFiles(FileSystem fs, Path specPath, Path manifestDir, int dpLevels, int lbLevels, AcidUtils.IdPathFilter filter, long writeId, int stmtId, Configuration conf) throws IOException { - Path[] files = getMmDirectoryCandidates( + Path[] files = getDirectInsertDirectoryCandidates( fs, specPath, dpLevels, filter, writeId, stmtId, conf, null); if (files != null) { for (Path path : files) { @@ -4235,7 +4250,7 @@ private static void tryDeleteAllMmFiles(FileSystem fs, Path specPath, Path manif } - public static void writeMmCommitManifest(List commitPaths, Path specPath, FileSystem fs, + public static void writeCommitManifest(List commitPaths, Path specPath, FileSystem fs, String taskId, Long writeId, int stmtId, String unionSuffix, boolean isInsertOverwrite) throws HiveException { if (commitPaths.isEmpty()) { return; @@ -4280,15 +4295,15 @@ public MissingBucketsContext(TableDesc tableInfo, int numBuckets, boolean isComp } } - public static void handleMmTableFinalPath(Path specPath, String unionSuffix, Configuration hconf, + public static void handleDirectInsertTableFinalPath(Path specPath, String unionSuffix, Configuration hconf, boolean success, int dpLevels, int lbLevels, MissingBucketsContext mbc, long writeId, int stmtId, - Reporter reporter, boolean isMmTable, boolean isMmCtas, boolean isInsertOverwrite) - throws IOException, HiveException { + Reporter reporter, boolean isMmTable, boolean isMmCtas, boolean isInsertOverwrite, boolean isDirectInsert) + throws IOException, HiveException { FileSystem fs = specPath.getFileSystem(hconf); Path manifestDir = getManifestDir(specPath, writeId, stmtId, unionSuffix, isInsertOverwrite); if (!success) { AcidUtils.IdPathFilter filter = new AcidUtils.IdPathFilter(writeId, stmtId); - tryDeleteAllMmFiles(fs, specPath, manifestDir, dpLevels, lbLevels, + tryDeleteAllDirectInsertFiles(fs, specPath, manifestDir, dpLevels, lbLevels, filter, writeId, stmtId, hconf); return; } @@ -4317,13 +4332,13 @@ public static void handleMmTableFinalPath(Path specPath, String unionSuffix, Con Utilities.FILE_OP_LOGGER.info("Creating directory with no output at {}", specPath); FileUtils.mkdir(fs, specPath, hconf); } - Path[] files = getMmDirectoryCandidates( + Path[] files = getDirectInsertDirectoryCandidates( fs, specPath, dpLevels, filter, writeId, stmtId, hconf, isInsertOverwrite); - ArrayList mmDirectories = new ArrayList<>(); + ArrayList directInsertDirectories = new ArrayList<>(); if (files != null) { for (Path path : files) { Utilities.FILE_OP_LOGGER.trace("Looking at path: {}", path); - mmDirectories.add(path); + directInsertDirectories.add(path); } } @@ -4356,15 +4371,15 @@ public static void handleMmTableFinalPath(Path specPath, String unionSuffix, Con } } - for (Path path : mmDirectories) { - cleanMmDirectory(path, fs, unionSuffix, lbLevels, committed); + for (Path path : directInsertDirectories) { + cleanDirectInsertDirectory(path, fs, unionSuffix, lbLevels, committed); } if (!committed.isEmpty()) { throw new HiveException("The following files were committed but not found: " + committed); } - if (mmDirectories.isEmpty()) { + if (directInsertDirectories.isEmpty()) { return; } @@ -4375,15 +4390,15 @@ public static void handleMmTableFinalPath(Path specPath, String unionSuffix, Con } // Create fake file statuses to avoid querying the file system. removeTempOrDuplicateFiles // doesn't need tocheck anything except path and directory status for MM directories. - FileStatus[] finalResults = new FileStatus[mmDirectories.size()]; - for (int i = 0; i < mmDirectories.size(); ++i) { - finalResults[i] = new PathOnlyFileStatus(mmDirectories.get(i)); + FileStatus[] finalResults = new FileStatus[directInsertDirectories.size()]; + for (int i = 0; i < directInsertDirectories.size(); ++i) { + finalResults[i] = new PathOnlyFileStatus(directInsertDirectories.get(i)); } List emptyBuckets = Utilities.removeTempOrDuplicateFiles(fs, finalResults, unionSuffix, dpLevels, mbc == null ? 0 : mbc.numBuckets, hconf, writeId, stmtId, - isMmTable, null, isInsertOverwrite); + isMmTable, null, isInsertOverwrite, isDirectInsert); // create empty buckets if necessary - if (!emptyBuckets.isEmpty()) { + if (!emptyBuckets.isEmpty() && !isDirectInsert) { assert mbc != null; Utilities.createEmptyBuckets(hconf, emptyBuckets, mbc.isCompressed, mbc.tableInfo, reporter); } @@ -4395,7 +4410,7 @@ public PathOnlyFileStatus(Path path) { } } - private static void cleanMmDirectory(Path dir, FileSystem fs, String unionSuffix, + private static void cleanDirectInsertDirectory(Path dir, FileSystem fs, String unionSuffix, int lbLevels, HashSet committed) throws IOException, HiveException { for (FileStatus child : fs.listStatus(dir)) { Path childPath = child.getPath(); @@ -4406,7 +4421,7 @@ private static void cleanMmDirectory(Path dir, FileSystem fs, String unionSuffix if (child.isDirectory()) { Utilities.FILE_OP_LOGGER.trace( "Recursion into LB directory {}; levels remaining ", childPath, lbLevels - 1); - cleanMmDirectory(childPath, fs, unionSuffix, lbLevels - 1, committed); + cleanDirectInsertDirectory(childPath, fs, unionSuffix, lbLevels - 1, committed); } else { if (committed.contains(childPath)) { throw new HiveException("LB FSOP has commited " @@ -4421,7 +4436,9 @@ private static void cleanMmDirectory(Path dir, FileSystem fs, String unionSuffix if (committed.remove(childPath)) { continue; // A good file. } - deleteUncommitedFile(childPath, fs); + if (!childPath.getName().equals(AcidUtils.OrcAcidVersion.ACID_FORMAT)) { + deleteUncommitedFile(childPath, fs); + } } else if (!child.isDirectory()) { if (committed.contains(childPath)) { throw new HiveException("Union FSOP has commited " @@ -4429,8 +4446,8 @@ private static void cleanMmDirectory(Path dir, FileSystem fs, String unionSuffix } deleteUncommitedFile(childPath, fs); } else if (childPath.getName().equals(unionSuffix)) { - // Found the right union directory; treat it as "our" MM directory. - cleanMmDirectory(childPath, fs, null, 0, committed); + // Found the right union directory; treat it as "our" directory. + cleanDirectInsertDirectory(childPath, fs, null, 0, committed); } else { String childName = childPath.getName(); if (!childName.startsWith(AbstractFileMergeOperator.UNION_SUDBIR_PREFIX) diff --git ql/src/java/org/apache/hadoop/hive/ql/io/AcidInputFormat.java ql/src/java/org/apache/hadoop/hive/ql/io/AcidInputFormat.java index bba3960102..034d2d3320 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/AcidInputFormat.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/AcidInputFormat.java @@ -34,6 +34,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Map; /** * The interface required for input formats that what to support ACID @@ -258,7 +259,8 @@ public Reporter getReporter() { int bucket, ValidWriteIdList validWriteIdList, Path baseDirectory, - Path[] deltaDirectory + Path[] deltaDirectory, + Map deltasToAttemptId ) throws IOException; /** diff --git ql/src/java/org/apache/hadoop/hive/ql/io/AcidOutputFormat.java ql/src/java/org/apache/hadoop/hive/ql/io/AcidOutputFormat.java index 1e8bb223f2..57428c1ecc 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/AcidOutputFormat.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/AcidOutputFormat.java @@ -51,6 +51,7 @@ private Reporter reporter; private long minimumWriteId; private long maximumWriteId; + private String attemptId; /** * actual bucketId (as opposed to bucket property via BucketCodec) */ @@ -240,6 +241,11 @@ public Options useDummy(PrintStream stream) { return this; } + public Options attemptId(String attemptId) { + this.attemptId = attemptId; + return this; + } + /** * @since 1.3.0 * This can be set to -1 to make the system generate old style (delta_xxxx_yyyy) file names. @@ -313,6 +319,10 @@ public int getBucketId() { return bucketId; } + public String getAttemptId() { + return attemptId; + } + public int getRecordIdColumn() { return recIdCol; } diff --git ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java index 2f5ec5270c..9cbc34dd1c 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java @@ -35,6 +35,7 @@ import java.util.Optional; import java.util.Properties; import java.util.Set; +import java.util.regex.Matcher; import java.util.regex.Pattern; import com.google.common.base.Strings; @@ -160,8 +161,8 @@ public boolean accept(Path path) { * This must be in sync with {@link #STATEMENT_DIGITS} */ public static final int MAX_STATEMENTS_PER_TXN = 10000; - public static final Pattern BUCKET_DIGIT_PATTERN = Pattern.compile("[0-9]{5}$"); - public static final Pattern LEGACY_BUCKET_DIGIT_PATTERN = Pattern.compile("^[0-9]{6}"); + public static final Pattern LEGACY_BUCKET_DIGIT_PATTERN = Pattern.compile("^[0-9]{6}"); + public static final Pattern BUCKET_PATTERN = Pattern.compile("bucket_([0-9]+)(_[0-9]+)?$"); /** * A write into a non-aicd table produces files like 0000_0 or 0000_0_copy_1 @@ -180,7 +181,6 @@ private AcidUtils() { } private static final Logger LOG = LoggerFactory.getLogger(AcidUtils.class); - public static final Pattern BUCKET_PATTERN = Pattern.compile(BUCKET_PREFIX + "_[0-9]{5}$"); public static final Pattern ORIGINAL_PATTERN = Pattern.compile("[0-9]+_[0-9]+"); /** @@ -241,7 +241,11 @@ public boolean accept(Path dirPath) { * @return the filename */ public static Path createBucketFile(Path subdir, int bucket) { - return createBucketFile(subdir, bucket, true); + return createBucketFile(subdir, bucket, null, true); + } + + public static Path createBucketFile(Path subdir, int bucket, String attemptId) { + return createBucketFile(subdir, bucket, attemptId, true); } /** @@ -250,10 +254,13 @@ public static Path createBucketFile(Path subdir, int bucket) { * @param bucket the bucket number * @return the filename */ - private static Path createBucketFile(Path subdir, int bucket, boolean isAcidSchema) { + private static Path createBucketFile(Path subdir, int bucket, String attemptId, boolean isAcidSchema) { if(isAcidSchema) { - return new Path(subdir, - BUCKET_PREFIX + String.format(BUCKET_DIGITS, bucket)); + String fileName = BUCKET_PREFIX + String.format(BUCKET_DIGITS, bucket); + if (attemptId != null) { + fileName = fileName + "_" + attemptId; + } + return new Path(subdir, fileName); } else { return new Path(subdir, @@ -353,7 +360,7 @@ public static Path createFilename(Path directory, return new Path(directory, String.format(LEGACY_FILE_BUCKET_DIGITS, options.getBucketId()) + "_0"); } else { - return createBucketFile(baseOrDeltaSubdirPath(directory, options), options.getBucketId()); + return createBucketFile(baseOrDeltaSubdirPath(directory, options), options.getBucketId(), options.getAttemptId()); } } @@ -421,11 +428,26 @@ public static int parseBucketId(Path bucketFile) { if (ORIGINAL_PATTERN.matcher(filename).matches() || ORIGINAL_PATTERN_COPY.matcher(filename).matches()) { return Integer.parseInt(filename.substring(0, filename.indexOf('_'))); } else if (filename.startsWith(BUCKET_PREFIX)) { - return Integer.parseInt(filename.substring(filename.indexOf('_') + 1)); + Matcher matcher = BUCKET_PATTERN.matcher(filename); + if (matcher.matches()) { + String bucketId = matcher.group(1); + filename = filename.substring(0,matcher.end(1)); + return Integer.parseInt(bucketId); + } } return -1; } + public static String parseAttemptId(Path bucketFile) { + String filename = bucketFile.getName(); + Matcher matcher = BUCKET_PATTERN.matcher(filename); + String attemptId = null; + if (matcher.matches()) { + attemptId = matcher.group(2) != null ? matcher.group(2).substring(1) : null; + } + return attemptId; + } + /** * Read the first row of an ORC file and determine the bucket ID based on the bucket column. This only works with * files with ACID schema. @@ -460,6 +482,7 @@ public static int parseBucketId(Path bucketFile) { AcidOutputFormat.Options result = new AcidOutputFormat.Options(conf); String filename = bucketFile.getName(); int bucket = parseBucketId(bucketFile); + String attemptId = parseAttemptId(bucketFile); if (ORIGINAL_PATTERN.matcher(filename).matches()) { result .setOldStyle(true) @@ -494,7 +517,8 @@ else if (filename.startsWith(BUCKET_PREFIX)) { .setOldStyle(false) .minimumWriteId(parsedDelta.minWriteId) .maximumWriteId(parsedDelta.maxWriteId) - .bucket(bucket); + .bucket(bucket) + .attemptId(attemptId); } else if (bucketFile.getParent().getName().startsWith(DELETE_DELTA_PREFIX)) { ParsedDelta parsedDelta = parsedDelta(bucketFile.getParent(), DELETE_DELTA_PREFIX, bucketFile.getFileSystem(conf), null); @@ -2559,7 +2583,7 @@ public static boolean isRawFormatFile(Path dataFile, FileSystem fs) throws IOExc */ public static final class OrcAcidVersion { private static final String ACID_VERSION_KEY = "hive.acid.version"; - private static final String ACID_FORMAT = "_orc_acid_version"; + public static final String ACID_FORMAT = "_orc_acid_version"; private static final Charset UTF8 = Charset.forName("UTF-8"); public static final int ORC_ACID_VERSION_DEFAULT = 0; /** diff --git ql/src/java/org/apache/hadoop/hive/ql/io/HiveFileFormatUtils.java ql/src/java/org/apache/hadoop/hive/ql/io/HiveFileFormatUtils.java index 8980a6292a..22099e08c3 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/HiveFileFormatUtils.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/HiveFileFormatUtils.java @@ -306,10 +306,15 @@ public static RecordWriter getHiveRecordWriter(JobConf jc, return (HiveOutputFormat) outputFormat; } + public static RecordUpdater getAcidRecordUpdater(JobConf jc, TableDesc tableInfo, int bucket, FileSinkDesc conf, + Path outPath, ObjectInspector inspector, Reporter reporter, int rowIdColNum) throws HiveException, IOException { + return getAcidRecordUpdater(jc, tableInfo, bucket, conf, outPath, inspector, reporter, rowIdColNum, null); + } + public static RecordUpdater getAcidRecordUpdater(JobConf jc, TableDesc tableInfo, int bucket, FileSinkDesc conf, Path outPath, ObjectInspector inspector, - Reporter reporter, int rowIdColNum) + Reporter reporter, int rowIdColNum, String attemptId) throws HiveException, IOException { HiveOutputFormat hiveOutputFormat = getHiveOutputFormat(jc, tableInfo); AcidOutputFormat acidOutputFormat = null; @@ -323,10 +328,9 @@ public static RecordUpdater getAcidRecordUpdater(JobConf jc, TableDesc tableInfo // file the way getHiveRecordWriter does, as ORC appears to read the value for itself. Not // sure if this is correct or not. return getRecordUpdater(jc, acidOutputFormat, - bucket, inspector, tableInfo.getProperties(), outPath, reporter, rowIdColNum, conf); + bucket, inspector, tableInfo.getProperties(), outPath, reporter, rowIdColNum, conf, attemptId); } - private static RecordUpdater getRecordUpdater(JobConf jc, AcidOutputFormat acidOutputFormat, int bucket, @@ -335,7 +339,8 @@ private static RecordUpdater getRecordUpdater(JobConf jc, Path outPath, Reporter reporter, int rowIdColNum, - FileSinkDesc conf) throws IOException { + FileSinkDesc conf, + String attemptId) throws IOException { return acidOutputFormat.getRecordUpdater(outPath, new AcidOutputFormat.Options(jc) .isCompressed(conf.getCompressed()) .tableProperties(tableProp) @@ -348,6 +353,7 @@ private static RecordUpdater getRecordUpdater(JobConf jc, .recordIdColumn(rowIdColNum) .statementId(conf.getStatementId()) .finalDestination(conf.getDestPath()) + .attemptId(attemptId) .temporary(conf.isTemporary())); } diff --git ql/src/java/org/apache/hadoop/hive/ql/io/RecordUpdater.java ql/src/java/org/apache/hadoop/hive/ql/io/RecordUpdater.java index 737e6774b7..bb257d1128 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/RecordUpdater.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/RecordUpdater.java @@ -20,6 +20,7 @@ import java.io.IOException; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.serde2.SerDeStats; /** @@ -79,4 +80,9 @@ * @return - buffered row count */ long getBufferedRowCount(); + + /** + * Returns the path of the file this updater wrote to + */ + public Path getUpdatedFilePath(); } diff --git ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java index 76984abd0a..de726af66f 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java @@ -2454,7 +2454,8 @@ private static boolean isStripeSatisfyPredicate( int bucket, ValidWriteIdList validWriteIdList, Path baseDirectory, - Path[] deltaDirectory + Path[] deltaDirectory, + Map deltasToAttemptId ) throws IOException { boolean isOriginal = false; OrcRawRecordMerger.Options mergerOptions = new OrcRawRecordMerger.Options().isCompacting(true) @@ -2476,7 +2477,7 @@ private static boolean isStripeSatisfyPredicate( mergerOptions.rootPath(deltaDirectory[0].getParent()); } return new OrcRawRecordMerger(conf, collapseEvents, null, isOriginal, - bucket, validWriteIdList, new Reader.Options(conf), deltaDirectory, mergerOptions); + bucket, validWriteIdList, new Reader.Options(conf), deltaDirectory, mergerOptions, deltasToAttemptId); } /** diff --git ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcOutputFormat.java ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcOutputFormat.java index c4c56f8477..202f78b81c 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcOutputFormat.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcOutputFormat.java @@ -268,6 +268,11 @@ private String stringifyObject(Object obj, stringifyObject(buffer, obj, inspector); return buffer.toString(); } + + @Override + public Path getUpdatedFilePath() { + return null; + } } @Override diff --git ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java index b8a0f0465c..f543418179 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java @@ -23,7 +23,6 @@ import java.util.Objects; import java.util.TreeMap; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator; import org.apache.hadoop.hive.ql.exec.Utilities; @@ -928,6 +927,20 @@ public Options clone() { } } } + + OrcRawRecordMerger(Configuration conf, + boolean collapseEvents, + Reader reader, + boolean isOriginal, + int bucket, + ValidWriteIdList validWriteIdList, + Reader.Options options, + Path[] deltaDirectory, + Options mergerOptions) throws IOException { + this(conf, collapseEvents, reader, isOriginal, bucket, validWriteIdList, options, deltaDirectory, mergerOptions, + null); + } + /** * Create a reader that merge sorts the ACID events together. This handles * 1. 'normal' reads on behalf of a query (non vectorized) @@ -952,7 +965,9 @@ public Options clone() { int bucket, ValidWriteIdList validWriteIdList, Reader.Options options, - Path[] deltaDirectory, Options mergerOptions) throws IOException { + Path[] deltaDirectory, + Options mergerOptions, + Map deltasToAttemptId) throws IOException { this.collapse = collapseEvents; this.offset = options.getOffset(); this.length = options.getLength(); @@ -1126,7 +1141,13 @@ public Options clone() { } continue; } - for (Path deltaFile : getDeltaFiles(delta, bucket, mergerOptions)) { + + String attemptId = null; + if (deltasToAttemptId != null) { + attemptId = deltasToAttemptId.get(delta.toString()); + } + + for (Path deltaFile : getDeltaFiles(delta, bucket, mergerOptions, attemptId)) { FileSystem fs = deltaFile.getFileSystem(conf); if(!fs.exists(deltaFile)) { /** @@ -1264,12 +1285,12 @@ private Options modifyForNonAcidSchemaRead(Options baseOptions, long writeId, Pa * This determines the set of {@link ReaderPairAcid} to create for a given delta/. * For unbucketed tables {@code bucket} can be thought of as a write tranche. */ - static Path[] getDeltaFiles(Path deltaDirectory, int bucket, Options mergerOptions) { + static Path[] getDeltaFiles(Path deltaDirectory, int bucket, Options mergerOptions, String attemptId) { assert (!mergerOptions.isCompacting && deltaDirectory.getName().startsWith(AcidUtils.DELETE_DELTA_PREFIX) ) || mergerOptions.isCompacting : "Unexpected delta: " + deltaDirectory + "(isCompacting=" + mergerOptions.isCompacting() + ")"; - return new Path[] {AcidUtils.createBucketFile(deltaDirectory, bucket)}; + return new Path[] {AcidUtils.createBucketFile(deltaDirectory, bucket, attemptId)}; } @VisibleForTesting diff --git ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java index 398698ec06..7998a276dc 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java @@ -32,6 +32,7 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.io.AcidOutputFormat; import org.apache.hadoop.hive.ql.io.AcidUtils; @@ -617,11 +618,19 @@ public void close(boolean abort) throws IOException { deleteEventWriter = null; writerClosed = true; } + private void initWriter() throws IOException { if (writer == null) { writer = OrcFile.createWriter(path, writerOptions); AcidUtils.OrcAcidVersion.setAcidVersionInDataFile(writer); - AcidUtils.OrcAcidVersion.writeVersionFile(path.getParent(), fs); + try { + AcidUtils.OrcAcidVersion.writeVersionFile(path.getParent(), fs); + } catch (Exception e) { + e.printStackTrace(); + // Ignore; might have been created by another concurrent writer, writing to a different bucket + // within this delta/base directory + LOG.trace(e.fillInStackTrace().toString()); + } } } @@ -806,4 +815,9 @@ private int setBucket(int bucketProperty, int operation) { bucket.set(bucketProperty); return currentBucketProperty; } + + @Override + public Path getUpdatedFilePath() { + return path; + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/io/orc/VectorizedOrcAcidRowBatchReader.java ql/src/java/org/apache/hadoop/hive/ql/io/orc/VectorizedOrcAcidRowBatchReader.java index 2543dc6fc4..2c116216fc 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/orc/VectorizedOrcAcidRowBatchReader.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/orc/VectorizedOrcAcidRowBatchReader.java @@ -1133,7 +1133,7 @@ DeleteEventRegistry getDeleteEventRegistry() { assert !orcSplit.isOriginal() : "If this now supports Original splits, set up mergeOptions properly"; this.deleteRecords = new OrcRawRecordMerger(conf, true, null, false, bucket, validWriteIdList, readerOptions, deleteDeltas, - mergerOptions); + mergerOptions, null); this.deleteRecordKey = new OrcRawRecordMerger.ReaderKey(); this.deleteRecordValue = this.deleteRecords.createValue(); // Initialize the first value in the delete reader. @@ -1564,7 +1564,7 @@ public int compareTo(CompressedOwid other) { for (Path deleteDeltaDir : deleteDeltaDirs) { FileSystem fs = deleteDeltaDir.getFileSystem(conf); Path[] deleteDeltaFiles = OrcRawRecordMerger.getDeltaFiles(deleteDeltaDir, bucket, - new OrcRawRecordMerger.Options().isCompacting(false)); + new OrcRawRecordMerger.Options().isCompacting(false), null); for (Path deleteDeltaFile : deleteDeltaFiles) { // NOTE: Calling last flush length below is more for future-proofing when we have // streaming deletes. But currently we don't support streaming deletes, and this can diff --git ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java index 7f061d4a6b..b2c42fdcdc 100644 --- ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java +++ ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java @@ -2197,7 +2197,7 @@ public Partition loadPartition(Path loadPath, Table tbl, Map par boolean isSkewedStoreAsSubdir, boolean isSrcLocal, boolean isAcidIUDoperation, boolean resetStatistics, Long writeId, - int stmtId, boolean isInsertOverwrite) throws HiveException { + int stmtId, boolean isInsertOverwrite, boolean isDirectInsert) throws HiveException { PerfLogger perfLogger = SessionState.getPerfLogger(); perfLogger.PerfLogBegin("MoveTask", PerfLogger.LOAD_PARTITION); @@ -2214,7 +2214,7 @@ public Partition loadPartition(Path loadPath, Table tbl, Map par Partition newTPart = loadPartitionInternal(loadPath, tbl, partSpec, oldPart, loadFileType, inheritTableSpecs, inheritLocation, isSkewedStoreAsSubdir, isSrcLocal, isAcidIUDoperation, - resetStatistics, writeId, stmtId, isInsertOverwrite, isTxnTable, newFiles); + resetStatistics, writeId, stmtId, isInsertOverwrite, isTxnTable, newFiles, isDirectInsert); AcidUtils.TableSnapshot tableSnapshot = isTxnTable ? getTableSnapshot(tbl, writeId) : null; if (tableSnapshot != null) { @@ -2285,7 +2285,7 @@ private Partition loadPartitionInternal(Path loadPath, Table tbl, Map newFiles) throws HiveException { + boolean isTxnTable, List newFiles, boolean isDirectInsert) throws HiveException { Path tblDataLocationPath = tbl.getDataLocation(); boolean isMmTableWrite = AcidUtils.isInsertOnlyTable(tbl.getParameters()); assert tbl.getPath() != null : "null==getPath() for " + tbl.getTableName(); @@ -2332,15 +2332,17 @@ private Partition loadPartitionInternal(Path loadPath, Table tbl, Map getValidPartitionsInPath( + private Set getValidPartitionsInPath( // pass isDirectInsert from tbd int numDP, int numLB, Path loadPath, Long writeId, int stmtId, - boolean isMmTable, boolean isInsertOverwrite) throws HiveException { + boolean isMmTable, boolean isInsertOverwrite, boolean isDirectInsert) throws HiveException { Set validPartitions = new HashSet(); try { FileSystem fs = loadPath.getFileSystem(conf); - if (!isMmTable) { + if (!isMmTable || !isDirectInsert) { List leafStatus = HiveStatsUtils.getFileStatusRecurse(loadPath, numDP, fs); // Check for empty partitions for (FileStatus s : leafStatus) { @@ -2791,7 +2789,7 @@ private void constructOneLBLocationMap(FileStatus fSta, // we have multiple statements anyway is union. Utilities.FILE_OP_LOGGER.trace( "Looking for dynamic partitions in {} ({} levels)", loadPath, numDP); - Path[] leafStatus = Utilities.getMmDirectoryCandidates( + Path[] leafStatus = Utilities.getDirectInsertDirectoryCandidates( fs, loadPath, numDP, null, writeId, -1, conf, isInsertOverwrite); for (Path p : leafStatus) { Path dpPath = p.getParent(); // Skip the MM directory that we have found. @@ -2839,7 +2837,7 @@ private void constructOneLBLocationMap(FileStatus fSta, final String tableName, final Map partSpec, final LoadFileType loadFileType, final int numDP, final int numLB, final boolean isAcid, final long writeId, final int stmtId, final boolean resetStatistics, final AcidUtils.Operation operation, - boolean isInsertOverwrite) throws HiveException { + boolean isInsertOverwrite, boolean isDirectInsert) throws HiveException { PerfLogger perfLogger = SessionState.getPerfLogger(); perfLogger.PerfLogBegin("MoveTask", PerfLogger.LOAD_DYNAMIC_PARTITIONS); @@ -2847,7 +2845,7 @@ private void constructOneLBLocationMap(FileStatus fSta, // Get all valid partition paths and existing partitions for them (if any) final Table tbl = getTable(tableName); final Set validPartitions = getValidPartitionsInPath(numDP, numLB, loadPath, writeId, stmtId, - AcidUtils.isInsertOnlyTable(tbl.getParameters()), isInsertOverwrite); + AcidUtils.isInsertOnlyTable(tbl.getParameters()), isInsertOverwrite, isDirectInsert); final int partsToLoad = validPartitions.size(); final AtomicInteger partitionsLoaded = new AtomicInteger(0); @@ -2915,7 +2913,7 @@ private void constructOneLBLocationMap(FileStatus fSta, // load the partition Partition partition = loadPartitionInternal(entry.getKey(), tbl, fullPartSpec, oldPartition, loadFileType, true, false, numLB > 0, false, isAcid, - resetStatistics, writeId, stmtId, isInsertOverwrite, isTxnTable, newFiles); + resetStatistics, writeId, stmtId, isInsertOverwrite, isTxnTable, newFiles, isDirectInsert); // if the partition already existed before the loading, no need to add it again to the // metastore @@ -3067,7 +3065,7 @@ private void constructOneLBLocationMap(FileStatus fSta, */ public void loadTable(Path loadPath, String tableName, LoadFileType loadFileType, boolean isSrcLocal, boolean isSkewedStoreAsSubdir, boolean isAcidIUDoperation, boolean resetStatistics, - Long writeId, int stmtId, boolean isInsertOverwrite) throws HiveException { + Long writeId, int stmtId, boolean isInsertOverwrite, boolean isDirectInsert) throws HiveException { PerfLogger perfLogger = SessionState.getPerfLogger(); perfLogger.PerfLogBegin("MoveTask", PerfLogger.LOAD_TABLE); @@ -3084,7 +3082,7 @@ public void loadTable(Path loadPath, String tableName, LoadFileType loadFileType } // Note: this assumes both paths are qualified; which they are, currently. - if (((isMmTable || isFullAcidTable) && loadPath.equals(tbl.getPath())) || (loadFileType == LoadFileType.IGNORE)) { + if (((isMmTable || isDirectInsert || isFullAcidTable) && loadPath.equals(tbl.getPath())) || (loadFileType == LoadFileType.IGNORE)) { /** * some operations on Transactional tables (e.g. Import) write directly to the final location * and avoid the 'move' operation. Since MoveTask does other things, setting 'loadPath' to be @@ -3097,14 +3095,16 @@ public void loadTable(Path loadPath, String tableName, LoadFileType loadFileType //new files list is required only for event notification. if (newFiles != null) { - listFilesCreatedByQuery(loadPath, writeId, stmtId, isMmTable ? isInsertOverwrite : false, newFiles); + if (!isMmTable && !isDirectInsert) { + isInsertOverwrite = false; + } + listFilesCreatedByQuery(loadPath, writeId, stmtId, isInsertOverwrite, newFiles); } } else { // Either a non-MM query, or a load into MM table from an external source. Path tblPath = tbl.getPath(); Path destPath = tblPath; if (isMmTable) { - assert !isAcidIUDoperation; // We will load into MM directory, and hide previous directories if needed. destPath = new Path(destPath, isInsertOverwrite ? AcidUtils.baseDir(writeId) : AcidUtils.deltaSubdir(writeId, writeId, stmtId)); diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java index 73ca658d9c..aa4fd295a0 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java @@ -68,6 +68,8 @@ import org.apache.hadoop.hive.ql.exec.spark.SparkTask; import org.apache.hadoop.hive.ql.exec.tez.TezTask; import org.apache.hadoop.hive.ql.hooks.ReadEntity; +import org.apache.hadoop.hive.ql.io.AcidUtils; +import org.apache.hadoop.hive.ql.io.AcidUtils.Operation; import org.apache.hadoop.hive.ql.io.RCFileInputFormat; import org.apache.hadoop.hive.ql.io.merge.MergeFileWork; import org.apache.hadoop.hive.ql.io.orc.OrcFileStripeMergeInputFormat; @@ -1382,7 +1384,7 @@ public static void createMRWorkForMergingFiles(FileSinkOperator fsInput, Path fsopPath = srcMmWriteId != null ? fsInputDesc.getFinalDirName() : finalName; Task mvTask = GenMapRedUtils.findMoveTaskForFsopOutput( - mvTasks, fsopPath, fsInputDesc.isMmTable()); + mvTasks, fsopPath, fsInputDesc.isMmTable(), fsInputDesc.isDirectInsert()); ConditionalTask cndTsk = GenMapRedUtils.createCondTask(conf, currTask, dummyMv, work, fsInputDesc.getMergeInputDirName(), finalName, mvTask, dependencyTask, lineageState); @@ -1869,8 +1871,8 @@ public static boolean isSkewedStoredAsDirs(FileSinkDesc fsInputDesc) { .isSkewedStoredAsDir(); } - public static Task findMoveTaskForFsopOutput( - List> mvTasks, Path fsopFinalDir, boolean isMmFsop) { + public static Task findMoveTaskForFsopOutput(List> mvTasks, Path fsopFinalDir, + boolean isMmFsop, boolean isDirectInsert) { // find the move task for (Task mvTsk : mvTasks) { MoveWork mvWork = mvTsk.getWork(); @@ -1879,7 +1881,7 @@ public static boolean isSkewedStoredAsDirs(FileSinkDesc fsInputDesc) { if (mvWork.getLoadFileWork() != null) { srcDir = mvWork.getLoadFileWork().getSourcePath(); isLfd = true; - if (isMmFsop) { + if (isMmFsop || isDirectInsert) { srcDir = srcDir.getParent(); } } else if (mvWork.getLoadTableWork() != null) { @@ -1910,8 +1912,8 @@ public static boolean isMergeRequired(List> mvTasks, HiveConf hco // no need of merging if the move is to a local file system // We are looking based on the original FSOP, so use the original path as is. - MoveTask mvTask = (MoveTask) GenMapRedUtils.findMoveTaskForFsopOutput( - mvTasks, fsOp.getConf().getFinalDirName(), fsOp.getConf().isMmTable()); + MoveTask mvTask = (MoveTask) GenMapRedUtils.findMoveTaskForFsopOutput(mvTasks, fsOp.getConf().getFinalDirName(), + fsOp.getConf().isMmTable(), fsOp.getConf().isDirectInsert()); // TODO: wtf?!! why is this in this method? This has nothing to do with anything. if (isInsertTable && hconf.getBoolVar(ConfVars.HIVESTATSAUTOGATHER) @@ -1985,9 +1987,15 @@ public static Path createMoveTask(Task currTask, boolean chDir, FileSinkDesc fileSinkDesc = fsOp.getConf(); boolean isMmTable = fileSinkDesc.isMmTable(); + boolean isDirectInsert = fileSinkDesc.isDirectInsert(); if (chDir) { dest = fileSinkDesc.getMergeInputDirName(); - if (!isMmTable) { + /** + * Skip temporary file generation for: + * 1. MM Tables + * 2. INSERT operation on full ACID table + */ + if ((!isMmTable) && (!isDirectInsert)) { // generate the temporary file // it must be on the same file system as the current destination Context baseCtx = parseCtx.getContext(); @@ -2018,8 +2026,8 @@ public static Path createMoveTask(Task currTask, boolean chDir, Task mvTask = null; if (!chDir) { - mvTask = GenMapRedUtils.findMoveTaskForFsopOutput( - mvTasks, fsOp.getConf().getFinalDirName(), fsOp.getConf().isMmTable()); + mvTask = GenMapRedUtils.findMoveTaskForFsopOutput(mvTasks, fsOp.getConf().getFinalDirName(), isMmTable, + isDirectInsert); } // Set the move task to be dependent on the current task diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index 5fcc367cc9..b832d79522 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -7307,6 +7307,7 @@ protected Operator genFileSinkPlan(String dest, QB qb, Operator input) Table destinationTable = null; // destination table if any boolean destTableIsTransactional; // true for full ACID table and MM table boolean destTableIsFullAcid; // should the destination table be written to using ACID + boolean isDirectInsert = false; // should we add files directly to the final path boolean destTableIsTemporary = false; boolean destTableIsMaterialization = false; Partition destinationPartition = null;// destination partition if any @@ -7320,7 +7321,7 @@ protected Operator genFileSinkPlan(String dest, QB qb, Operator input) LoadTableDesc ltd = null; ListBucketingCtx lbCtx = null; Map partSpec = null; - boolean isMmTable = false, isMmCreate = false; + boolean isMmTable = false, isMmCreate = false, isNonNativeTable = false; Long writeId = null; HiveTxnManager txnMgr = getTxnMgr(); @@ -7364,9 +7365,25 @@ protected Operator genFileSinkPlan(String dest, QB qb, Operator input) destinationPath = new Path(destinationTable.getPath(), dpCtx.getSPPath()); } - boolean isNonNativeTable = destinationTable.isNonNative(); + isNonNativeTable = destinationTable.isNonNative(); isMmTable = AcidUtils.isInsertOnlyTable(destinationTable.getParameters()); - if (isNonNativeTable || isMmTable) { + AcidUtils.Operation acidOp = AcidUtils.Operation.NOT_ACID; + // this table_desc does not contain the partitioning columns + tableDescriptor = Utilities.getTableDesc(destinationTable); + + if (!isNonNativeTable) { + if (destTableIsTransactional) { + acidOp = getAcidType(tableDescriptor.getOutputFileFormatClass(), dest, isMmTable); + } + } + isDirectInsert = destTableIsFullAcid && (acidOp == AcidUtils.Operation.INSERT); + /** + * We will directly insert to the final destination in the following cases: + * 1. Non native table + * 2. Micro-managed (insert only table) + * 3. Full ACID table and operation type is INSERT + */ + if (isNonNativeTable || isMmTable || isDirectInsert) { queryTmpdir = destinationPath; } else { queryTmpdir = ctx.getTempDirForFinalJobPath(destinationPath); @@ -7379,8 +7396,6 @@ protected Operator genFileSinkPlan(String dest, QB qb, Operator input) // set the root of the temporary path where dynamic partition columns will populate dpCtx.setRootPath(queryTmpdir); } - // this table_desc does not contain the partitioning columns - tableDescriptor = Utilities.getTableDesc(destinationTable); // Add NOT NULL constraint check input = genConstraintsPlan(dest, qb, input); @@ -7410,7 +7425,6 @@ protected Operator genFileSinkPlan(String dest, QB qb, Operator input) // Create the work for moving the table // NOTE: specify Dynamic partitions in dest_tab for WriteEntity if (!isNonNativeTable) { - AcidUtils.Operation acidOp = AcidUtils.Operation.NOT_ACID; if (destTableIsTransactional) { acidOp = getAcidType(tableDescriptor.getOutputFileFormatClass(), dest, isMmTable); checkAcidConstraints(); @@ -7441,8 +7455,12 @@ protected Operator genFileSinkPlan(String dest, QB qb, Operator input) destinationTable.getDbName(), destinationTable.getTableName()); LoadFileType loadType = (!isInsertInto && !destTableIsTransactional) ? LoadFileType.REPLACE_ALL : LoadFileType.KEEP_EXISTING; + if (isDirectInsert) { + loadType = LoadFileType.IGNORE; + } ltd.setLoadFileType(loadType); ltd.setInsertOverwrite(!isInsertInto); + ltd.setIsDirectInsert(isDirectInsert); ltd.setLbCtx(lbCtx); loadTableWork.add(ltd); } else { @@ -7499,13 +7517,34 @@ protected Operator genFileSinkPlan(String dest, QB qb, Operator input) } } + isNonNativeTable = destinationTable.isNonNative(); isMmTable = AcidUtils.isInsertOnlyTable(destinationTable.getParameters()); - queryTmpdir = isMmTable ? destinationPath : ctx.getTempDirForFinalJobPath(destinationPath); + AcidUtils.Operation acidOp = AcidUtils.Operation.NOT_ACID; + // this table_desc does not contain the partitioning columns + tableDescriptor = Utilities.getTableDesc(destinationTable); + + if (!isNonNativeTable) { + if (destTableIsTransactional) { + acidOp = getAcidType(tableDescriptor.getOutputFileFormatClass(), dest, isMmTable); + } + } + isDirectInsert = destTableIsFullAcid && (acidOp == AcidUtils.Operation.INSERT); + + /** + * We will directly insert to the final destination in the following cases: + * 1. Non native table + * 2. Micro-managed (insert only table) + * 3. Full ACID table and operation type is INSERT + */ + if (isNonNativeTable || isMmTable || isDirectInsert) { + queryTmpdir = destinationPath; + } else { + queryTmpdir = ctx.getTempDirForFinalJobPath(destinationPath); + } if (Utilities.FILE_OP_LOGGER.isTraceEnabled()) { Utilities.FILE_OP_LOGGER.trace("create filesink w/DEST_PARTITION specifying " + queryTmpdir + " from " + destinationPath); } - tableDescriptor = Utilities.getTableDesc(destinationTable); // Add NOT NULL constraint check input = genConstraintsPlan(dest, qb, input); @@ -7531,7 +7570,6 @@ protected Operator genFileSinkPlan(String dest, QB qb, Operator input) lbCtx = constructListBucketingCtx(destinationPartition.getSkewedColNames(), destinationPartition.getSkewedColValues(), destinationPartition.getSkewedColValueLocationMaps(), destinationPartition.isStoredAsSubDirectories()); - AcidUtils.Operation acidOp = AcidUtils.Operation.NOT_ACID; if (destTableIsTransactional) { acidOp = getAcidType(tableDescriptor.getOutputFileFormatClass(), dest, isMmTable); checkAcidConstraints(); @@ -7561,8 +7599,12 @@ protected Operator genFileSinkPlan(String dest, QB qb, Operator input) // deltas and base and leave them up to the cleaner to clean up LoadFileType loadType = (!isInsertInto && !destTableIsTransactional) ? LoadFileType.REPLACE_ALL : LoadFileType.KEEP_EXISTING; + if (isDirectInsert) { + loadType = LoadFileType.IGNORE; + } ltd.setLoadFileType(loadType); ltd.setInsertOverwrite(!isInsertInto); + ltd.setIsDirectInsert(isDirectInsert); ltd.setLbCtx(lbCtx); loadTableWork.add(ltd); @@ -7788,7 +7830,7 @@ protected Operator genFileSinkPlan(String dest, QB qb, Operator input) // the metastore table, then we move and commit the partitions. At least for the time being, // this order needs to be enforced because metastore expects a table to exist before we can // add any partitions to it. - boolean isNonNativeTable = tableDescriptor.isNonNative(); + isNonNativeTable = tableDescriptor.isNonNative(); if (!isNonNativeTable) { AcidUtils.Operation acidOp = AcidUtils.Operation.NOT_ACID; if (destTableIsTransactional) { @@ -7878,7 +7920,7 @@ protected Operator genFileSinkPlan(String dest, QB qb, Operator input) FileSinkDesc fileSinkDesc = createFileSinkDesc(dest, tableDescriptor, destinationPartition, destinationPath, currentTableId, destTableIsFullAcid, destTableIsTemporary,//this was 1/4 acid destTableIsMaterialization, queryTmpdir, rsCtx, dpCtx, lbCtx, fsRS, - canBeMerged, destinationTable, writeId, isMmCreate, destType, qb); + canBeMerged, destinationTable, writeId, isMmCreate, destType, qb, isDirectInsert); if (isMmCreate) { // Add FSD so that the LoadTask compilation could fix up its path to avoid the move. if (tableDesc != null) { @@ -8088,7 +8130,7 @@ private FileSinkDesc createFileSinkDesc(String dest, TableDesc table_desc, boolean destTableIsMaterialization, Path queryTmpdir, SortBucketRSCtx rsCtx, DynamicPartitionCtx dpCtx, ListBucketingCtx lbCtx, RowSchema fsRS, boolean canBeMerged, Table dest_tab, Long mmWriteId, boolean isMmCtas, - Integer dest_type, QB qb) throws SemanticException { + Integer dest_type, QB qb, boolean isDirectInsert) throws SemanticException { boolean isInsertOverwrite = false; switch (dest_type) { case QBMetaData.DEST_PARTITION: @@ -8113,7 +8155,7 @@ private FileSinkDesc createFileSinkDesc(String dest, TableDesc table_desc, conf.getBoolVar(HiveConf.ConfVars.COMPRESSRESULT), currentTableId, rsCtx.isMultiFileSpray(), canBeMerged, rsCtx.getNumFiles(), rsCtx.getTotalFiles(), rsCtx.getPartnCols(), dpCtx, dest_path, mmWriteId, isMmCtas, isInsertOverwrite, qb.getIsQuery(), - qb.isCTAS() || qb.isMaterializedView()); + qb.isCTAS() || qb.isMaterializedView(), isDirectInsert); boolean isHiveServerQuery = SessionState.get().isHiveServerQuery(); fileSinkDesc.setHiveServerQuery(isHiveServerQuery); diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java index c102a69f8f..5fb5fd308a 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java @@ -433,7 +433,7 @@ public static Path createMoveTask(Task currTask, boolean chDir, Task mvTask = null; if (!chDir) { - mvTask = GenMapRedUtils.findMoveTaskForFsopOutput(mvTasks, fileSinkDesc.getFinalDirName(), false); + mvTask = GenMapRedUtils.findMoveTaskForFsopOutput(mvTasks, fileSinkDesc.getFinalDirName(), false, false); } // Set the move task to be dependent on the current task diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java index ecc7bdee4d..bbf73cb828 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java @@ -112,6 +112,8 @@ private boolean isInsertOverwrite = false; + private boolean isDirectInsert = false; + private boolean isQuery = false; private boolean isCTASorCM = false; @@ -122,12 +124,10 @@ public FileSinkDesc() { /** * @param destPath - the final destination for data */ - public FileSinkDesc(final Path dirName, final TableDesc tableInfo, - final boolean compressed, final int destTableId, final boolean multiFileSpray, - final boolean canBeMerged, final int numFiles, final int totalFiles, - final List partitionCols, final DynamicPartitionCtx dpCtx, Path destPath, - Long mmWriteId, boolean isMmCtas, boolean isInsertOverwrite, boolean isQuery, boolean isCTASorCM) { - + public FileSinkDesc(final Path dirName, final TableDesc tableInfo, final boolean compressed, final int destTableId, + final boolean multiFileSpray, final boolean canBeMerged, final int numFiles, final int totalFiles, + final List partitionCols, final DynamicPartitionCtx dpCtx, Path destPath, Long mmWriteId, + boolean isMmCtas, boolean isInsertOverwrite, boolean isQuery, boolean isCTASorCM, boolean isDirectInsert) { this.dirName = dirName; this.tableInfo = tableInfo; this.compressed = compressed; @@ -145,6 +145,7 @@ public FileSinkDesc(final Path dirName, final TableDesc tableInfo, this.isInsertOverwrite = isInsertOverwrite; this.isQuery = isQuery; this.isCTASorCM = isCTASorCM; + this.isDirectInsert = isDirectInsert; } public FileSinkDesc(final Path dirName, final TableDesc tableInfo, @@ -164,9 +165,9 @@ public FileSinkDesc(final Path dirName, final TableDesc tableInfo, @Override public Object clone() throws CloneNotSupportedException { - FileSinkDesc ret = new FileSinkDesc(dirName, tableInfo, compressed, - destTableId, multiFileSpray, canBeMerged, numFiles, totalFiles, - partitionCols, dpCtx, destPath, mmWriteId, isMmCtas, isInsertOverwrite, isQuery, isCTASorCM); + FileSinkDesc ret = new FileSinkDesc(dirName, tableInfo, compressed, destTableId, multiFileSpray, canBeMerged, + numFiles, totalFiles, partitionCols, dpCtx, destPath, mmWriteId, isMmCtas, isInsertOverwrite, isQuery, + isCTASorCM, isDirectInsert); ret.setCompressCodec(compressCodec); ret.setCompressType(compressType); ret.setGatherStats(gatherStats); @@ -184,6 +185,7 @@ public Object clone() throws CloneNotSupportedException { ret.setFilesToFetch(filesToFetch); ret.setIsQuery(isQuery); ret.setIsCTASorCM(isCTASorCM); + ret.setIsDirectInsert(isDirectInsert); return ret; } @@ -223,6 +225,14 @@ public void setIsUsingBatchingSerDe(boolean isUsingBatchingSerDe) { this.isUsingBatchingSerDe = isUsingBatchingSerDe; } + public void setIsDirectInsert(boolean isDirectInsert) { + this.isDirectInsert = isDirectInsert; + } + + public boolean isDirectInsert() { + return this.isDirectInsert; + } + @Explain(displayName = "directory", explainLevels = { Level.EXTENDED }) public Path getDirName() { return dirName; diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/LoadTableDesc.java ql/src/java/org/apache/hadoop/hive/ql/plan/LoadTableDesc.java index bed05819b5..a62b3cc850 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/LoadTableDesc.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/LoadTableDesc.java @@ -43,6 +43,7 @@ private int stmtId; private Long currentWriteId; private boolean isInsertOverwrite; + private boolean isDirectInsert; // TODO: the below seem like they should just be combined into partitionDesc private Table mdTable; @@ -235,6 +236,14 @@ public void setInsertOverwrite(boolean v) { this.isInsertOverwrite = v; } + public void setIsDirectInsert(boolean isDirectInsert) { + this.isDirectInsert = isDirectInsert; + } + + public boolean isDirectInsert() { + return this.isDirectInsert; + } + /** * @return the lbCtx */ diff --git ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java index bb70db4524..a720320b38 100644 --- ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java +++ ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java @@ -419,6 +419,7 @@ public JobConf getMrJob() { private int bucketNum; private Path base; private Path[] deltas; + private Map deltasToAttemptId; public CompactorInputSplit() { } @@ -435,12 +436,13 @@ public CompactorInputSplit() { * @throws IOException */ CompactorInputSplit(Configuration hadoopConf, int bucket, List files, Path base, - Path[] deltas) + Path[] deltas, Map deltasToAttemptId) throws IOException { bucketNum = bucket; this.base = base; this.deltas = deltas; locations = new ArrayList(); + this.deltasToAttemptId = deltasToAttemptId; for (Path path : files) { FileSystem fs = path.getFileSystem(hadoopConf); @@ -485,6 +487,14 @@ public void write(DataOutput dataOutput) throws IOException { for (int i = 0; i < deltas.length; i++) { dataOutput.writeInt(deltas[i].toString().length()); dataOutput.writeBytes(deltas[i].toString()); + String attemptId = deltasToAttemptId.get(deltas[i].toString()); + if (attemptId == null) { + dataOutput.writeInt(0); + } + else { + dataOutput.writeInt(attemptId.length()); + dataOutput.writeBytes(attemptId.toString()); + } } } @@ -517,11 +527,20 @@ public void readFields(DataInput dataInput) throws IOException { } numElements = dataInput.readInt(); deltas = new Path[numElements]; + deltasToAttemptId = new HashMap<>(); for (int i = 0; i < numElements; i++) { len = dataInput.readInt(); buf = new byte[len]; dataInput.readFully(buf); deltas[i] = new Path(new String(buf)); + len = dataInput.readInt(); + String attemptId = null; + if (len > 0) { + buf = new byte[len]; + dataInput.readFully(buf); + attemptId = new String(buf); + } + deltasToAttemptId.put(deltas[i].toString(), attemptId); } } @@ -531,6 +550,7 @@ public void set(CompactorInputSplit other) { bucketNum = other.bucketNum; base = other.base; deltas = other.deltas; + deltasToAttemptId = other.deltasToAttemptId; } int getBucket() { @@ -545,6 +565,10 @@ Path getBaseDir() { return deltas; } + Map getDeltasToAttemptId() { + return deltasToAttemptId; + } + @Override public String toString() { StringBuilder builder = new StringBuilder(); @@ -602,7 +626,7 @@ public String toString() { // For each file, figure out which bucket it is. Matcher matcher = isRawFormat ? AcidUtils.LEGACY_BUCKET_DIGIT_PATTERN.matcher(f.getPath().getName()) - : AcidUtils.BUCKET_DIGIT_PATTERN.matcher(f.getPath().getName()); + : AcidUtils.BUCKET_PATTERN.matcher(f.getPath().getName()); addFileToMap(matcher, f.getPath(), sawBase, splitToBucketMap); } } else { @@ -621,7 +645,7 @@ public String toString() { // multiple ingestions of various sizes. Path[] deltasForSplit = isTableBucketed ? deltaDirs : getDeltaDirsFromBucketTracker(bt); splits.add(new CompactorInputSplit(entries, e.getKey(), bt.buckets, - bt.sawBase ? baseDir : null, deltasForSplit)); + bt.sawBase ? baseDir : null, deltasForSplit, bt.deltasToAttemptId)); } LOG.debug("Returning " + splits.size() + " splits"); @@ -658,7 +682,16 @@ private void addFileToMap(Matcher matcher, Path file, boolean sawBase, //may be a data loss scenario throw new IllegalArgumentException(msg); } - int bucketNum = Integer.parseInt(matcher.group()); + int bucketNum = -1; + String attemptId = null; + if (matcher.groupCount() > 0) { + bucketNum = Integer.parseInt(matcher.group(1)); + attemptId = matcher.group(2) != null ? matcher.group(2).substring(1) : null; + } + else { + bucketNum = Integer.parseInt(matcher.group()); + } + BucketTracker bt = splitToBucketMap.get(bucketNum); if (bt == null) { bt = new BucketTracker(); @@ -667,16 +700,19 @@ private void addFileToMap(Matcher matcher, Path file, boolean sawBase, LOG.debug("Adding " + file.toString() + " to list of files for splits"); bt.buckets.add(file); bt.sawBase |= sawBase; + bt.deltasToAttemptId.put(file.getParent().toString(), attemptId); } private static class BucketTracker { BucketTracker() { sawBase = false; buckets = new ArrayList(); + deltasToAttemptId = new HashMap<>(); } boolean sawBase; List buckets; + Map deltasToAttemptId; } } @@ -749,7 +785,7 @@ public void map(WritableComparable key, CompactorInputSplit split, boolean isMajor = jobConf.getBoolean(IS_MAJOR, false); AcidInputFormat.RawReader reader = aif.getRawReader(jobConf, isMajor, split.getBucket(), - writeIdList, split.getBaseDir(), split.getDeltaDirs()); + writeIdList, split.getBaseDir(), split.getDeltaDirs(), split.getDeltasToAttemptId()); RecordIdentifier identifier = reader.createKey(); V value = reader.createValue(); getWriter(reporter, reader.getObjectInspector(), split.getBucket()); diff --git ql/src/java/org/apache/hadoop/hive/ql/util/UpgradeTool.java ql/src/java/org/apache/hadoop/hive/ql/util/UpgradeTool.java index 58e6289583..e9e49a6270 100644 --- ql/src/java/org/apache/hadoop/hive/ql/util/UpgradeTool.java +++ ql/src/java/org/apache/hadoop/hive/ql/util/UpgradeTool.java @@ -246,7 +246,6 @@ private static void alterTable(Table t, Hive db, boolean isMM) */ static void handleRenameFiles(Table t, Path p, boolean execute, Configuration conf, boolean isBucketed, PrintWriter pw) throws IOException { - AcidUtils.BUCKET_DIGIT_PATTERN.matcher("foo"); if (isBucketed) { /* For bucketed tables we assume that Hive wrote them and 0000M_0 and 0000M_0_copy_8 are the only possibilities. Since we can't move files across buckets the only thing we diff --git ql/src/test/org/apache/hadoop/hive/ql/TestTxnAddPartition.java ql/src/test/org/apache/hadoop/hive/ql/TestTxnAddPartition.java index c9cb6692df..fa15e2876a 100644 --- ql/src/test/org/apache/hadoop/hive/ql/TestTxnAddPartition.java +++ ql/src/test/org/apache/hadoop/hive/ql/TestTxnAddPartition.java @@ -294,6 +294,6 @@ public void addPartitionTransactional() throws Exception { runStatementOnDriver("insert into Tstage partition(p=1) values(0,2),(1,4)"); runStatementOnDriver("ALTER TABLE T ADD PARTITION (p=0) location '" - + getWarehouseDir() + "/tstage/p=1/delta_0000001_0000001_0000/bucket_00001'"); + + getWarehouseDir() + "/tstage/p=1/delta_0000001_0000001_0000/bucket_00001_0'"); } } diff --git ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java index 842140815d..a1f59a8927 100644 --- ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java +++ ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java @@ -1167,7 +1167,7 @@ public void testNonAcidToAcidConversion01() throws Exception { {"{\"writeid\":0,\"bucketid\":536936448,\"rowid\":0}\t1\t2", "nonacidorctbl/000001_0"}, {"{\"writeid\":0,\"bucketid\":536936448,\"rowid\":1}\t0\t12", "nonacidorctbl/000001_0_copy_1"}, {"{\"writeid\":0,\"bucketid\":536936448,\"rowid\":2}\t1\t5", "nonacidorctbl/000001_0_copy_1"}, - {"{\"writeid\":10000001,\"bucketid\":536936448,\"rowid\":0}\t1\t17", "nonacidorctbl/delta_10000001_10000001_0000/bucket_00001"} + {"{\"writeid\":10000001,\"bucketid\":536936448,\"rowid\":0}\t1\t17", "nonacidorctbl/delta_10000001_10000001_0000/bucket_00001_0"} }; checkResult(expected, query, isVectorized, "before compact", LOG); diff --git ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java index 88ca683173..9ab00ded10 100644 --- ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java +++ ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java @@ -474,7 +474,7 @@ public void testNonAcidToAcidConversion1() throws Exception { sawNewDelta = true; FileStatus[] buckets = fs.listStatus(status[i].getPath(), FileUtils.HIDDEN_FILES_PATH_FILTER); Assert.assertEquals(1, buckets.length); // only one bucket file - Assert.assertTrue(buckets[0].getPath().getName().matches("bucket_00000")); + Assert.assertTrue(buckets[0].getPath().getName().matches("bucket_00000_0")); } else { Assert.assertTrue(status[i].getPath().getName().matches("00000[01]_0")); } @@ -776,7 +776,7 @@ public void testNonAcidToAcidConversion3() throws Exception { } else if (numDelta == 2) { Assert.assertEquals("delta_10000002_10000002_0000", status[i].getPath().getName()); Assert.assertEquals(1, buckets.length); - Assert.assertEquals("bucket_00000", buckets[0].getPath().getName()); + Assert.assertEquals("bucket_00000_0", buckets[0].getPath().getName()); } } else if (status[i].getPath().getName().matches("delete_delta_.*")) { numDeleteDelta++; diff --git ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java index 908ceb43fc..4f4bca229c 100644 --- ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java +++ ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java @@ -76,7 +76,7 @@ public void testRenameTable() throws Exception { String testQuery = "select ROW__ID, a, b, INPUT__FILE__NAME from mydb1.S"; String[][] expected = new String[][] { {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t1\t2", - "s/delta_0000001_0000001_0000/bucket_00000"}, + "s/delta_0000001_0000001_0000/bucket_00000_0"}, {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t4\t6", "s/delta_0000002_0000002_0000/bucket_00000"}}; checkResult(expected, testQuery, false, "check data", LOG); @@ -273,14 +273,14 @@ private void testSdpoBucketed(boolean isVectorized, boolean isSdpo, int bucketin "select ROW__ID, a, b, ds, INPUT__FILE__NAME from acid_uap order by ds, a, b"; String[][] expected = new String[][]{ {"{\"writeid\":2,\"bucketid\":536936448,\"rowid\":0}\t1\tbah\ttoday", - "warehouse/acid_uap/ds=today/delta_0000002_0000002_0000/bucket_00001"}, + "warehouse/acid_uap/ds=today/delta_0000002_0000002_0000/bucket_00001_0"}, {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t2\tyah\ttoday", - "warehouse/acid_uap/ds=today/delta_0000002_0000002_0000/bucket_00000"}, + "warehouse/acid_uap/ds=today/delta_0000002_0000002_0000/bucket_00000_0"}, {"{\"writeid\":1,\"bucketid\":536936448,\"rowid\":0}\t1\tbah\ttomorrow", - "warehouse/acid_uap/ds=tomorrow/delta_0000001_0000001_0000/bucket_00001"}, + "warehouse/acid_uap/ds=tomorrow/delta_0000001_0000001_0000/bucket_00001_0"}, {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t2\tyah\ttomorrow", - "warehouse/acid_uap/ds=tomorrow/delta_0000001_0000001_0000/bucket_00000"}}; + "warehouse/acid_uap/ds=tomorrow/delta_0000001_0000001_0000/bucket_00000_0"}}; checkResult(expected, testQuery, isVectorized, "after insert", LOG); runStatementOnDriver("update acid_uap set b = 'fred'"); @@ -324,9 +324,9 @@ public void testCleaner2() throws Exception { String testQuery = "select ROW__ID, a, b, INPUT__FILE__NAME from T"; String[][] expected = new String[][] { {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t0\t2", - "t/delta_0000001_0000001_0000/bucket_00000"}, + "t/delta_0000001_0000001_0000/bucket_00000_0"}, {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t1\t4", - "t/delta_0000002_0000002_0000/bucket_00000"}}; + "t/delta_0000002_0000002_0000/bucket_00000_0"}}; checkResult(expected, testQuery, false, "check data", LOG); diff --git ql/src/test/org/apache/hadoop/hive/ql/TestTxnConcatenate.java ql/src/test/org/apache/hadoop/hive/ql/TestTxnConcatenate.java index 8676e0db11..63944293ce 100644 --- ql/src/test/org/apache/hadoop/hive/ql/TestTxnConcatenate.java +++ ql/src/test/org/apache/hadoop/hive/ql/TestTxnConcatenate.java @@ -61,9 +61,9 @@ public void testConcatenate() throws Exception { {"{\"writeid\":2,\"bucketid\":536936448,\"rowid\":1}\t4\t4", "acidtbl/delta_0000002_0000002_0000/bucket_00001"}, {"{\"writeid\":3,\"bucketid\":536936448,\"rowid\":0}\t5\t6", - "acidtbl/delta_0000003_0000003_0000/bucket_00001"}, + "acidtbl/delta_0000003_0000003_0000/bucket_00001_0"}, {"{\"writeid\":3,\"bucketid\":536936448,\"rowid\":1}\t8\t8", - "acidtbl/delta_0000003_0000003_0000/bucket_00001"}}; + "acidtbl/delta_0000003_0000003_0000/bucket_00001_0"}}; checkResult(expected, testQuery, false, "check data", LOG); /*in UTs, there is no standalone HMS running to kick off compaction so it's done via runWorker() @@ -100,11 +100,11 @@ public void testConcatenatePart() throws Exception { {"{\"writeid\":2,\"bucketid\":536936448,\"rowid\":0}\t1\t4", "acidtblpart/p=p1/delta_0000002_0000002_0000/bucket_00001"}, {"{\"writeid\":1,\"bucketid\":536936448,\"rowid\":0}\t4\t5", - "acidtblpart/p=p2/delta_0000001_0000001_0000/bucket_00001"}, + "acidtblpart/p=p2/delta_0000001_0000001_0000/bucket_00001_0"}, {"{\"writeid\":3,\"bucketid\":536936448,\"rowid\":0}\t5\t6", - "acidtblpart/p=p1/delta_0000003_0000003_0000/bucket_00001"}, + "acidtblpart/p=p1/delta_0000003_0000003_0000/bucket_00001_0"}, {"{\"writeid\":3,\"bucketid\":536936448,\"rowid\":0}\t8\t8", - "acidtblpart/p=p2/delta_0000003_0000003_0000/bucket_00001"}}; + "acidtblpart/p=p2/delta_0000003_0000003_0000/bucket_00001_0"}}; checkResult(expected, testQuery, false, "check data", LOG); /*in UTs, there is no standalone HMS running to kick off compaction so it's done via runWorker() @@ -124,11 +124,11 @@ public void testConcatenatePart() throws Exception { {"{\"writeid\":2,\"bucketid\":536936448,\"rowid\":0}\t1\t4", "acidtblpart/p=p1/base_0000003_v0000019/bucket_00001"}, {"{\"writeid\":1,\"bucketid\":536936448,\"rowid\":0}\t4\t5", - "acidtblpart/p=p2/delta_0000001_0000001_0000/bucket_00001"}, + "acidtblpart/p=p2/delta_0000001_0000001_0000/bucket_00001_0"}, {"{\"writeid\":3,\"bucketid\":536936448,\"rowid\":0}\t5\t6", "acidtblpart/p=p1/base_0000003_v0000019/bucket_00001"}, {"{\"writeid\":3,\"bucketid\":536936448,\"rowid\":0}\t8\t8", - "acidtblpart/p=p2/delta_0000003_0000003_0000/bucket_00001"}}; + "acidtblpart/p=p2/delta_0000003_0000003_0000/bucket_00001_0"}}; checkResult(expected2, testQuery, false, "check data after concatenate", LOG); } diff --git ql/src/test/org/apache/hadoop/hive/ql/TestTxnExIm.java ql/src/test/org/apache/hadoop/hive/ql/TestTxnExIm.java index 66b2b2768b..ba5341778c 100644 --- ql/src/test/org/apache/hadoop/hive/ql/TestTxnExIm.java +++ ql/src/test/org/apache/hadoop/hive/ql/TestTxnExIm.java @@ -379,7 +379,7 @@ public void testImportPartitioned() throws Exception { "select ROW__ID, a, b, INPUT__FILE__NAME from T order by ROW__ID"; String[][] expected = new String[][] { {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t0\t0", - "t/p=10/delta_0000001_0000001_0000/bucket_00000"}, + "t/p=10/delta_0000001_0000001_0000/bucket_00000_0"}, {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t3\t4", "t/p=11/delta_0000002_0000002_0000/000000_0"}, {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t5\t6", diff --git ql/src/test/org/apache/hadoop/hive/ql/TestTxnLoadData.java ql/src/test/org/apache/hadoop/hive/ql/TestTxnLoadData.java index bb55d9fd79..125c76ab5a 100644 --- ql/src/test/org/apache/hadoop/hive/ql/TestTxnLoadData.java +++ ql/src/test/org/apache/hadoop/hive/ql/TestTxnLoadData.java @@ -126,13 +126,13 @@ private void loadDataUpdate(boolean isVectorized) throws Exception { String[][] expectedInter2 = new String[][] { {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t3\t4", "t/delta_0000001_0000001_0000/000000_0"}, {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t1\t17", "t/delta_0000002_0000002_0000/bucket_00000"}, - {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t2\t2", "t/delta_0000003_0000003_0000/bucket_00000"} + {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t2\t2", "t/delta_0000003_0000003_0000/bucket_00000_0"} }; checkResult(expectedInter2, testQuery, isVectorized, "insert"); runStatementOnDriver("delete from T where a = 3"); String[][] expectedInter3 = new String[][] { {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t1\t17", "t/delta_0000002_0000002_0000/bucket_00000"}, - {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t2\t2", "t/delta_0000003_0000003_0000/bucket_00000"} + {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t2\t2", "t/delta_0000003_0000003_0000/bucket_00000_0"} }; checkResult(expectedInter3, testQuery, isVectorized, "delete"); //test minor compaction @@ -165,7 +165,7 @@ private void loadDataUpdate(boolean isVectorized) throws Exception { String[][] expected5 = new String[][]{ {"{\"writeid\":7,\"bucketid\":536870912,\"rowid\":0}\t1\t17", "t/delta_0000007_0000007_0000/bucket_00000"}, {"{\"writeid\":7,\"bucketid\":536870912,\"rowid\":1}\t1\t17", "t/delta_0000007_0000007_0000/bucket_00000"}, - {"{\"writeid\":9,\"bucketid\":536870912,\"rowid\":0}\t2\t2", "t/delta_0000009_0000009_0000/bucket_00000"} + {"{\"writeid\":9,\"bucketid\":536870912,\"rowid\":0}\t2\t2", "t/delta_0000009_0000009_0000/bucket_00000_0"} }; checkResult(expected5, testQuery, isVectorized, "load data inpath overwrite update"); @@ -199,8 +199,8 @@ private void loadData(boolean isVectorized) throws Exception { "select ROW__ID, a, b, INPUT__FILE__NAME from T order by ROW__ID"; String[][] expected = new String[][] { //normal insert - {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t0\t2", "t/delta_0000001_0000001_0000/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t0\t4", "t/delta_0000001_0000001_0000/bucket_00000"}, + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t0\t2", "t/delta_0000001_0000001_0000/bucket_00000_0"}, + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t0\t4", "t/delta_0000001_0000001_0000/bucket_00000_0"}, //Load Data {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t1\t2", "t/delta_0000002_0000002_0000/000000_0"}, {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":1}\t3\t4", "t/delta_0000002_0000002_0000/000000_0"}}; @@ -444,8 +444,8 @@ private void testMultiStatement(boolean isVectorized) throws Exception { String testQuery = isVectorized ? "select ROW__ID, a, b from T order by ROW__ID" : "select ROW__ID, a, b, INPUT__FILE__NAME from T order by ROW__ID"; String[][] expected = new String[][] { - {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t1\t2", "t/delta_0000001_0000001_0000/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t3\t4", "t/delta_0000001_0000001_0000/bucket_00000"}, + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t1\t2", "t/delta_0000001_0000001_0000/bucket_00000_0"}, + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t3\t4", "t/delta_0000001_0000001_0000/bucket_00000_0"}, {"{\"writeid\":1,\"bucketid\":536870913,\"rowid\":0}\t5\t5", "t/delta_0000001_0000001_0001/000000_0"}, {"{\"writeid\":1,\"bucketid\":536870913,\"rowid\":1}\t6\t6", "t/delta_0000001_0000001_0001/000000_0"} }; @@ -483,8 +483,8 @@ public void testAbort() throws Exception { String testQuery = isVectorized ? "select ROW__ID, a, b from T order by ROW__ID" : "select ROW__ID, a, b, INPUT__FILE__NAME from T order by ROW__ID"; String[][] expected = new String[][] { - {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t1\t2", "t/delta_0000001_0000001_0000/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t3\t4", "t/delta_0000001_0000001_0000/bucket_00000"} + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t1\t2", "t/delta_0000001_0000001_0000/bucket_00000_0"}, + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t3\t4", "t/delta_0000001_0000001_0000/bucket_00000_0"} }; checkResult(expected, testQuery, isVectorized, "load data inpath"); } @@ -505,7 +505,7 @@ public void testLoadAcidFile() throws Exception { List rs = runStatementOnDriver("select INPUT__FILE__NAME from T"); Assert.assertEquals(1, rs.size()); Assert.assertTrue("Unexpcted file name", rs.get(0) - .endsWith("t/delta_0000001_0000001_0000/bucket_00000")); + .endsWith("t/delta_0000001_0000001_0000/bucket_00000_0")); //T2 is an acid table so this should fail CommandProcessorException e = runStatementOnDriverNegative("load data local inpath '" + rs.get(0) + "' into table T2"); diff --git ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java index ea6b1d9bec..7d3b8c7424 100644 --- ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java +++ ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java @@ -74,6 +74,7 @@ private boolean shouldVectorize() { */ @Test public void testNoBuckets() throws Exception { + String tableName = "nobuckets"; int[][] sourceVals1 = {{0,0,0},{3,3,3}}; int[][] sourceVals2 = {{1,1,1},{2,2,2}}; runStatementOnDriver("drop table if exists tmp"); @@ -81,7 +82,7 @@ public void testNoBuckets() throws Exception { runStatementOnDriver("insert into tmp " + makeValuesClause(sourceVals1)); runStatementOnDriver("insert into tmp " + makeValuesClause(sourceVals2)); runStatementOnDriver("drop table if exists nobuckets"); - runStatementOnDriver("create table nobuckets (c1 integer, c2 integer, c3 integer) stored " + + runStatementOnDriver("create table " + tableName + " (c1 integer, c2 integer, c3 integer) stored " + "as orc tblproperties('transactional'='true', 'transactional_properties'='default')"); String stmt = "insert into nobuckets select * from tmp"; runStatementOnDriver(stmt); @@ -96,13 +97,13 @@ public void testNoBuckets() throws Exception { * The number in the file name is writerId. This is the number encoded in ROW__ID.bucketId - * see {@link org.apache.hadoop.hive.ql.io.BucketCodec}*/ Assert.assertTrue(rs.get(0), rs.get(0).startsWith("{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t0\t0\t0\t")); - Assert.assertTrue(rs.get(0), rs.get(0).endsWith("nobuckets/delta_0000001_0000001_0000/bucket_00000")); + Assert.assertTrue(rs.get(0), rs.get(0).endsWith("nobuckets/delta_0000001_0000001_0000/bucket_00000_0")); Assert.assertTrue(rs.get(1), rs.get(1).startsWith("{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t3\t3\t3\t")); - Assert.assertTrue(rs.get(1), rs.get(1).endsWith("nobuckets/delta_0000001_0000001_0000/bucket_00000")); + Assert.assertTrue(rs.get(1), rs.get(1).endsWith("nobuckets/delta_0000001_0000001_0000/bucket_00000_0")); Assert.assertTrue(rs.get(2), rs.get(2).startsWith("{\"writeid\":1,\"bucketid\":536936448,\"rowid\":0}\t1\t1\t1\t")); - Assert.assertTrue(rs.get(2), rs.get(2).endsWith("nobuckets/delta_0000001_0000001_0000/bucket_00001")); + Assert.assertTrue(rs.get(2), rs.get(2).endsWith("nobuckets/delta_0000001_0000001_0000/bucket_00001_0")); Assert.assertTrue(rs.get(3), rs.get(3).startsWith("{\"writeid\":1,\"bucketid\":536936448,\"rowid\":1}\t2\t2\t2\t")); - Assert.assertTrue(rs.get(3), rs.get(3).endsWith("nobuckets/delta_0000001_0000001_0000/bucket_00001")); + Assert.assertTrue(rs.get(3), rs.get(3).endsWith("nobuckets/delta_0000001_0000001_0000/bucket_00001_0")); hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_EXPLAIN_USER, false); rs = runStatementOnDriver("explain update nobuckets set c3 = 17 where c3 in(0,1)"); @@ -118,9 +119,9 @@ public void testNoBuckets() throws Exception { LOG.warn(s); } Assert.assertTrue(rs.get(0), rs.get(0).startsWith("{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t3\t3\t3\t")); - Assert.assertTrue(rs.get(0), rs.get(0).endsWith("nobuckets/delta_0000001_0000001_0000/bucket_00000")); + Assert.assertTrue(rs.get(0), rs.get(0).endsWith("nobuckets/delta_0000001_0000001_0000/bucket_00000_0")); Assert.assertTrue(rs.get(1), rs.get(1).startsWith("{\"writeid\":1,\"bucketid\":536936448,\"rowid\":1}\t2\t2\t2\t")); - Assert.assertTrue(rs.get(1), rs.get(1).endsWith("nobuckets/delta_0000001_0000001_0000/bucket_00001")); + Assert.assertTrue(rs.get(1), rs.get(1).endsWith("nobuckets/delta_0000001_0000001_0000/bucket_00001_0")); //so update has 1 writer, but which creates buckets where the new rows land Assert.assertTrue(rs.get(2), rs.get(2).startsWith("{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t0\t0\t17\t")); Assert.assertTrue(rs.get(2), rs.get(2).endsWith("nobuckets/delta_0000002_0000002_0000/bucket_00000")); @@ -130,14 +131,14 @@ public void testNoBuckets() throws Exception { Set expectedFiles = new HashSet<>(); //both delete events land in corresponding buckets to the original row-ids - expectedFiles.add("ts/delete_delta_0000002_0000002_0000/bucket_00000"); - expectedFiles.add("ts/delete_delta_0000002_0000002_0000/bucket_00001"); - expectedFiles.add("nobuckets/delta_0000001_0000001_0000/bucket_00000"); - expectedFiles.add("nobuckets/delta_0000001_0000001_0000/bucket_00001"); + expectedFiles.add("nobuckets/delete_delta_0000002_0000002_0000/bucket_00000"); + expectedFiles.add("nobuckets/delete_delta_0000002_0000002_0000/bucket_00001"); + expectedFiles.add("nobuckets/delta_0000001_0000001_0000/bucket_00000_0"); + expectedFiles.add("nobuckets/delta_0000001_0000001_0000/bucket_00001_0"); expectedFiles.add("nobuckets/delta_0000002_0000002_0000/bucket_00000"); expectedFiles.add("nobuckets/delta_0000002_0000002_0000/bucket_00001"); //check that we get the right files on disk - assertExpectedFileSet(expectedFiles, getWarehouseDir() + "/nobuckets"); + assertExpectedFileSet(expectedFiles, getWarehouseDir() + "/" + tableName, tableName); //todo: it would be nice to check the contents of the files... could use orc.FileDump - it has // methods to print to a supplied stream but those are package private @@ -175,15 +176,15 @@ public void testNoBuckets() throws Exception { "After Major Compaction", LOG); expectedFiles.clear(); - expectedFiles.add("obuckets/delete_delta_0000002_0000002_0000/bucket_00000"); - expectedFiles.add("obuckets/delete_delta_0000002_0000002_0000/bucket_00001"); - expectedFiles.add("house/nobuckets/delta_0000001_0000001_0000/bucket_00000"); - expectedFiles.add("house/nobuckets/delta_0000001_0000001_0000/bucket_00001"); - expectedFiles.add("house/nobuckets/delta_0000002_0000002_0000/bucket_00000"); - expectedFiles.add("house/nobuckets/delta_0000002_0000002_0000/bucket_00001"); - expectedFiles.add("/warehouse/nobuckets/base_0000002_v0000025/bucket_00000"); - expectedFiles.add("/warehouse/nobuckets/base_0000002_v0000025/bucket_00001"); - assertExpectedFileSet(expectedFiles, getWarehouseDir() + "/nobuckets"); + expectedFiles.add("nobuckets/delete_delta_0000002_0000002_0000/bucket_00000"); + expectedFiles.add("nobuckets/delete_delta_0000002_0000002_0000/bucket_00001"); + expectedFiles.add("nobuckets/delta_0000001_0000001_0000/bucket_00000_0"); + expectedFiles.add("nobuckets/delta_0000001_0000001_0000/bucket_00001_0"); + expectedFiles.add("nobuckets/delta_0000002_0000002_0000/bucket_00000"); + expectedFiles.add("nobuckets/delta_0000002_0000002_0000/bucket_00001"); + expectedFiles.add("nobuckets/base_0000002_v0000025/bucket_00000"); + expectedFiles.add("nobuckets/base_0000002_v0000025/bucket_00001"); + assertExpectedFileSet(expectedFiles, getWarehouseDir() + "/" + tableName, tableName); TestTxnCommands2.runCleaner(hiveConf); rs = runStatementOnDriver("select c1, c2, c3 from nobuckets order by c1, c2, c3"); @@ -191,9 +192,9 @@ public void testNoBuckets() throws Exception { Assert.assertEquals("Unexpected result after clean", stringifyValues(result), rs); expectedFiles.clear(); - expectedFiles.add("/warehouse/nobuckets/base_0000002_v0000025/bucket_00000"); - expectedFiles.add("/warehouse/nobuckets/base_0000002_v0000025/bucket_00001"); - assertExpectedFileSet(expectedFiles, getWarehouseDir() + "/nobuckets"); + expectedFiles.add("nobuckets/base_0000002_v0000025/bucket_00000"); + expectedFiles.add("nobuckets/base_0000002_v0000025/bucket_00001"); + assertExpectedFileSet(expectedFiles, getWarehouseDir() + "/" + tableName, tableName); } @Test @@ -332,11 +333,11 @@ public void testInsertToAcidWithUnionRemove() throws Exception { List rs = runStatementOnDriver("select ROW__ID, a, b, INPUT__FILE__NAME from T order by ROW__ID"); String expected[][] = { - {"{\"writeid\":1,\"bucketid\":536870913,\"rowid\":0}\t1\t2", "/delta_0000001_0000001_0001/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536870913,\"rowid\":1}\t3\t4", "/delta_0000001_0000001_0001/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536870914,\"rowid\":0}\t5\t6", "/delta_0000001_0000001_0002/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536870915,\"rowid\":0}\t9\t10", "/delta_0000001_0000001_0003/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536936450,\"rowid\":0}\t7\t8", "/delta_0000001_0000001_0002/bucket_00001"}, + {"{\"writeid\":1,\"bucketid\":536870913,\"rowid\":0}\t1\t2", "/delta_0000001_0000001_0001/bucket_00000_0"}, + {"{\"writeid\":1,\"bucketid\":536870913,\"rowid\":1}\t3\t4", "/delta_0000001_0000001_0001/bucket_00000_0"}, + {"{\"writeid\":1,\"bucketid\":536870914,\"rowid\":0}\t5\t6", "/delta_0000001_0000001_0002/bucket_00000_0"}, + {"{\"writeid\":1,\"bucketid\":536870915,\"rowid\":0}\t9\t10", "/delta_0000001_0000001_0003/bucket_00000_0"}, + {"{\"writeid\":1,\"bucketid\":536936450,\"rowid\":0}\t7\t8", "/delta_0000001_0000001_0002/bucket_00001_0"}, }; checkExpected(rs, expected, "Unexpected row count after ctas"); } @@ -563,9 +564,9 @@ public void testToAcidConversion02() throws Exception { {"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":3}\t0\t13", "bucket_00000", "000000_0_copy_1"}, {"{\"writeid\":10000001,\"bucketid\":536870912,\"rowid\":0}\t0\t15", - "bucket_00000", "bucket_00000"}, + "bucket_00000", "bucket_00000_0"}, {"{\"writeid\":10000003,\"bucketid\":536870912,\"rowid\":0}\t0\t17", - "bucket_00000", "bucket_00000"}, + "bucket_00000", "bucket_00000_0"}, {"{\"writeid\":10000002,\"bucketid\":536870912,\"rowid\":0}\t0\t120", "bucket_00000", "bucket_00000"}, {"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":0}\t1\t2", @@ -577,7 +578,7 @@ public void testToAcidConversion02() throws Exception { {"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":6}\t1\t6", "bucket_00000", "000000_0_copy_2"}, {"{\"writeid\":10000001,\"bucketid\":536870912,\"rowid\":1}\t1\t16", - "bucket_00000", "bucket_00000"} + "bucket_00000", "bucket_00000_0"} }; Assert.assertEquals("Unexpected row count before compaction", expected.length, rs.size()); for(int i = 0; i < expected.length; i++) { @@ -792,14 +793,14 @@ public void testCompactStatsGather() throws Exception { String query = "select ROW__ID, p, q, a, b, INPUT__FILE__NAME from T order by p, q, a, b"; List rs = runStatementOnDriver(query); String[][] expected = { - {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t1\t1\t4\t1", "t/p=1/q=1/delta_0000001_0000001_0000/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t1\t1\t4\t3", "t/p=1/q=1/delta_0000001_0000001_0000/bucket_00000"}, - {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t1\t1\t5\t1", "t/p=1/q=1/delta_0000003_0000003_0000/bucket_00000"}, - {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":1}\t1\t1\t5\t3", "t/p=1/q=1/delta_0000003_0000003_0000/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t1\t2\t4\t2", "t/p=1/q=2/delta_0000001_0000001_0000/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t1\t2\t4\t4", "t/p=1/q=2/delta_0000001_0000001_0000/bucket_00000"}, - {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t1\t2\t5\t2", "t/p=1/q=2/delta_0000003_0000003_0000/bucket_00000"}, - {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":1}\t1\t2\t5\t4", "t/p=1/q=2/delta_0000003_0000003_0000/bucket_00000"} + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t1\t1\t4\t1", "t/p=1/q=1/delta_0000001_0000001_0000/bucket_00000_0"}, + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t1\t1\t4\t3", "t/p=1/q=1/delta_0000001_0000001_0000/bucket_00000_0"}, + {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t1\t1\t5\t1", "t/p=1/q=1/delta_0000003_0000003_0000/bucket_00000_0"}, + {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":1}\t1\t1\t5\t3", "t/p=1/q=1/delta_0000003_0000003_0000/bucket_00000_0"}, + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t1\t2\t4\t2", "t/p=1/q=2/delta_0000001_0000001_0000/bucket_00000_0"}, + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t1\t2\t4\t4", "t/p=1/q=2/delta_0000001_0000001_0000/bucket_00000_0"}, + {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t1\t2\t5\t2", "t/p=1/q=2/delta_0000003_0000003_0000/bucket_00000_0"}, + {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":1}\t1\t2\t5\t4", "t/p=1/q=2/delta_0000003_0000003_0000/bucket_00000_0"} }; checkExpected(rs, expected, "insert data"); @@ -810,10 +811,10 @@ public void testCompactStatsGather() throws Exception { query = "select ROW__ID, p, q, a, b, INPUT__FILE__NAME from T order by p, q, a, b"; rs = runStatementOnDriver(query); String[][] expected2 = { - {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t1\t1\t4\t1", "t/p=1/q=1/delta_0000001_0000001_0000/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t1\t1\t4\t3", "t/p=1/q=1/delta_0000001_0000001_0000/bucket_00000"}, - {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t1\t1\t5\t1", "t/p=1/q=1/delta_0000003_0000003_0000/bucket_00000"}, - {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":1}\t1\t1\t5\t3", "t/p=1/q=1/delta_0000003_0000003_0000/bucket_00000"}, + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t1\t1\t4\t1", "t/p=1/q=1/delta_0000001_0000001_0000/bucket_00000_0"}, + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t1\t1\t4\t3", "t/p=1/q=1/delta_0000001_0000001_0000/bucket_00000_0"}, + {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t1\t1\t5\t1", "t/p=1/q=1/delta_0000003_0000003_0000/bucket_00000_0"}, + {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":1}\t1\t1\t5\t3", "t/p=1/q=1/delta_0000003_0000003_0000/bucket_00000_0"}, {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t1\t2\t4\t2", "t/p=1/q=2/base_0000003_v0000020/bucket_00000"}, {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t1\t2\t4\t4", "t/p=1/q=2/base_0000003_v0000020/bucket_00000"}, {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t1\t2\t5\t2", "t/p=1/q=2/base_0000003_v0000020/bucket_00000"}, @@ -842,8 +843,8 @@ public void testDefault() throws Exception { List rs = runStatementOnDriver(query); String[][] expected = { //this proves data is written in Acid layout so T was made Acid - {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t1\t2", "t/delta_0000001_0000001_0000/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t3\t4", "t/delta_0000001_0000001_0000/bucket_00000"} + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t1\t2", "t/delta_0000001_0000001_0000/bucket_00000_0"}, + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t3\t4", "t/delta_0000001_0000001_0000/bucket_00000_0"} }; checkExpected(rs, expected, "insert data"); } diff --git ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java index af14e628b3..1435269ed3 100644 --- ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java +++ ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java @@ -23,6 +23,8 @@ import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import org.apache.commons.io.FileUtils; import org.apache.hadoop.fs.FileSystem; @@ -231,14 +233,8 @@ void assertVectorized(boolean vectorized, String query) throws Exception { * @param expectedFiles - suffixes of expected Paths. Must be the same length * @param rootPath - table or partition root where to start looking for actual files, recursively */ - void assertExpectedFileSet(Set expectedFiles, String rootPath) throws Exception { - int suffixLength = 0; - for(String s : expectedFiles) { - if(suffixLength > 0) { - assert suffixLength == s.length() : "all entries must be the same length. current: " + s; - } - suffixLength = s.length(); - } + void assertExpectedFileSet(Set expectedFiles, String rootPath, String tableName) throws Exception { + Pattern pattern = Pattern.compile("(.+)/(" + tableName + "/[delete_delta|delta|base].+)"); FileSystem fs = FileSystem.get(hiveConf); Set actualFiles = new HashSet<>(); RemoteIterator remoteIterator = fs.listFiles(new Path(rootPath), true); @@ -246,7 +242,10 @@ void assertExpectedFileSet(Set expectedFiles, String rootPath) throws Ex LocatedFileStatus lfs = remoteIterator.next(); if(!lfs.isDirectory() && org.apache.hadoop.hive.common.FileUtils.HIDDEN_FILES_PATH_FILTER.accept(lfs.getPath())) { String p = lfs.getPath().toString(); - actualFiles.add(p.substring(p.length() - suffixLength, p.length())); + Matcher matcher = pattern.matcher(p); + if (matcher.matches()) { + actualFiles.add(matcher.group(2)); + } } } Assert.assertEquals("Unexpected file list", expectedFiles, actualFiles); diff --git ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java index 83db48e758..801133d85c 100644 --- ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java +++ ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java @@ -146,7 +146,7 @@ db.createTable(src, cols, null, TextInputFormat.class, HiveIgnoreKeyTextOutputFormat.class); db.loadTable(hadoopDataFile[i], src, LoadFileType.KEEP_EXISTING, - true, false, false, true, null, 0, false); + true, false, false, true, null, 0, false, false); i++; } diff --git ql/src/test/org/apache/hadoop/hive/ql/exec/TestFileSinkOperator.java ql/src/test/org/apache/hadoop/hive/ql/exec/TestFileSinkOperator.java index 2c4b69b2fe..ebb51c447b 100644 --- ql/src/test/org/apache/hadoop/hive/ql/exec/TestFileSinkOperator.java +++ ql/src/test/org/apache/hadoop/hive/ql/exec/TestFileSinkOperator.java @@ -286,7 +286,7 @@ private FileSinkOperator getFileSink(AcidUtils.Operation writeType, DynamicPartitionCtx dpCtx = new DynamicPartitionCtx(partColMap, "Sunday", 100); //todo: does this need the finalDestination? desc = new FileSinkDesc(basePath, tableDesc, false, 1, false, - false, 1, 1, partCols, dpCtx, null, null, false, false, false, false); + false, 1, 1, partCols, dpCtx, null, null, false, false, false, false, false); } else { desc = new FileSinkDesc(basePath, tableDesc, false); } @@ -705,7 +705,8 @@ public float getProgress() throws IOException { int bucket, ValidWriteIdList validWriteIdList, Path baseDirectory, - Path[] deltaDirectory) throws + Path[] deltaDirectory, + Map deltaToAttemptId) throws IOException { return null; } @@ -778,6 +779,11 @@ public SerDeStats getStats() { public long getBufferedRowCount() { return records.size(); } + + @Override + public Path getUpdatedFilePath() { + return null; + } }; } diff --git ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java index 48e9afc496..393b6070d1 100644 --- ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java +++ ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java @@ -1243,6 +1243,7 @@ public void testWriteSetTracking7() throws Exception { //================ //test with predicates such that partition pruning doesn't kick in + driver.run("drop table tab1"); driver.run("create table if not exists tab1 (a int, b int) partitioned by (p string) " + "clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')"); driver.run("insert into tab1 partition(p)(a,b,p) values(1,1,'one'),(2,2,'two')"); //txnid:4 diff --git ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java index cfd7290762..9a9ab53fcc 100644 --- ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java +++ ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java @@ -383,7 +383,7 @@ private void addFile(Table t, Partition p, long minTxn, long maxTxn, @Override public RawReader getRawReader(Configuration conf, boolean collapseEvents, int bucket, ValidWriteIdList validWriteIdList, - Path baseDirectory, Path... deltaDirectory) throws IOException { + Path baseDirectory, Path[] deltaDirectory, Map deltaToAttemptId) throws IOException { List filesToRead = new ArrayList(); if (baseDirectory != null) { diff --git ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java index 70ae85c458..443f982d66 100644 --- ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java +++ ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java @@ -162,7 +162,7 @@ public void inputSplit() throws Exception { deltas[1] = new Path(delta2); CompactorMR.CompactorInputSplit split = - new CompactorMR.CompactorInputSplit(conf, 3, files, new Path(basename), deltas); + new CompactorMR.CompactorInputSplit(conf, 3, files, new Path(basename), deltas, new HashMap()); Assert.assertEquals(520L, split.getLength()); String[] locations = split.getLocations(); @@ -207,7 +207,7 @@ public void inputSplitNullBase() throws Exception { deltas[1] = new Path(delta2); CompactorMR.CompactorInputSplit split = - new CompactorMR.CompactorInputSplit(conf, 3, files, null, deltas); + new CompactorMR.CompactorInputSplit(conf, 3, files, null, deltas, new HashMap()); ByteArrayOutputStream buf = new ByteArrayOutputStream(); DataOutput out = new DataOutputStream(buf); diff --git ql/src/test/results/clientpositive/acid_subquery.q.out ql/src/test/results/clientpositive/acid_subquery.q.out index 1dc1775557..4ca2e5a09d 100644 --- ql/src/test/results/clientpositive/acid_subquery.q.out +++ ql/src/test/results/clientpositive/acid_subquery.q.out @@ -95,8 +95,17 @@ POSTHOOK: Input: default@target@p=2/q=2 POSTHOOK: Output: default@merge_tmp_table POSTHOOK: Output: default@target@p=1/q=2 POSTHOOK: Output: default@target@p=1/q=2 +POSTHOOK: Output: default@target@p=1/q=2 +POSTHOOK: Output: default@target@p=1/q=3 POSTHOOK: Output: default@target@p=1/q=3 POSTHOOK: Output: default@target@p=1/q=3 POSTHOOK: Output: default@target@p=2/q=2 POSTHOOK: Output: default@target@p=2/q=2 +POSTHOOK: Output: default@target@p=2/q=2 POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(target)t.FieldSchema(name:ROW__ID, type:struct, comment:), (target)t.FieldSchema(name:p, type:int, comment:null), (target)t.FieldSchema(name:q, type:int, comment:null), ] +POSTHOOK: Lineage: target PARTITION(p=1,q=2).a SIMPLE [(source)s.FieldSchema(name:a1, type:int, comment:null), ] +POSTHOOK: Lineage: target PARTITION(p=1,q=2).b SIMPLE [(source)s.FieldSchema(name:b1, type:int, comment:null), ] +POSTHOOK: Lineage: target PARTITION(p=1,q=3).a SIMPLE [(source)s.FieldSchema(name:a1, type:int, comment:null), ] +POSTHOOK: Lineage: target PARTITION(p=1,q=3).b SIMPLE [(source)s.FieldSchema(name:b1, type:int, comment:null), ] +POSTHOOK: Lineage: target PARTITION(p=2,q=2).a SIMPLE [(source)s.FieldSchema(name:a1, type:int, comment:null), ] +POSTHOOK: Lineage: target PARTITION(p=2,q=2).b SIMPLE [(source)s.FieldSchema(name:b1, type:int, comment:null), ] diff --git ql/src/test/results/clientpositive/create_transactional_full_acid.q.out ql/src/test/results/clientpositive/create_transactional_full_acid.q.out index e324d5ec43..04b16a06ac 100644 --- ql/src/test/results/clientpositive/create_transactional_full_acid.q.out +++ ql/src/test/results/clientpositive/create_transactional_full_acid.q.out @@ -190,8 +190,17 @@ POSTHOOK: Input: default@target@p=2/q=2 POSTHOOK: Output: default@merge_tmp_table POSTHOOK: Output: default@target@p=1/q=2 POSTHOOK: Output: default@target@p=1/q=2 +POSTHOOK: Output: default@target@p=1/q=2 +POSTHOOK: Output: default@target@p=1/q=3 POSTHOOK: Output: default@target@p=1/q=3 POSTHOOK: Output: default@target@p=1/q=3 POSTHOOK: Output: default@target@p=2/q=2 POSTHOOK: Output: default@target@p=2/q=2 +POSTHOOK: Output: default@target@p=2/q=2 POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(target)t.FieldSchema(name:ROW__ID, type:struct, comment:), (target)t.FieldSchema(name:p, type:int, comment:null), (target)t.FieldSchema(name:q, type:int, comment:null), ] +POSTHOOK: Lineage: target PARTITION(p=1,q=2).a SIMPLE [(source)s.FieldSchema(name:a1, type:int, comment:null), ] +POSTHOOK: Lineage: target PARTITION(p=1,q=2).b SIMPLE [(source)s.FieldSchema(name:b1, type:int, comment:null), ] +POSTHOOK: Lineage: target PARTITION(p=1,q=3).a SIMPLE [(source)s.FieldSchema(name:a1, type:int, comment:null), ] +POSTHOOK: Lineage: target PARTITION(p=1,q=3).b SIMPLE [(source)s.FieldSchema(name:b1, type:int, comment:null), ] +POSTHOOK: Lineage: target PARTITION(p=2,q=2).a SIMPLE [(source)s.FieldSchema(name:a1, type:int, comment:null), ] +POSTHOOK: Lineage: target PARTITION(p=2,q=2).b SIMPLE [(source)s.FieldSchema(name:b1, type:int, comment:null), ] diff --git ql/src/test/results/clientpositive/encrypted/encryption_insert_partition_dynamic.q.out ql/src/test/results/clientpositive/encrypted/encryption_insert_partition_dynamic.q.out index 61b0057adb..f9c7060315 100644 --- ql/src/test/results/clientpositive/encrypted/encryption_insert_partition_dynamic.q.out +++ ql/src/test/results/clientpositive/encrypted/encryption_insert_partition_dynamic.q.out @@ -73,8 +73,12 @@ insert into table encryptedTable_n0 partition (key) POSTHOOK: type: QUERY POSTHOOK: Input: default@src POSTHOOK: Output: default@encryptedtable_n0@key=238 +POSTHOOK: Output: default@encryptedtable_n0@key=501 +POSTHOOK: Output: default@encryptedtable_n0@key=502 POSTHOOK: Output: default@encryptedtable_n0@key=86 POSTHOOK: Lineage: encryptedtable_n0 PARTITION(key=238).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: encryptedtable_n0 PARTITION(key=501).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: encryptedtable_n0 PARTITION(key=502).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] POSTHOOK: Lineage: encryptedtable_n0 PARTITION(key=86).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] PREHOOK: query: select * from encryptedTable_n0 order by key PREHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out index fbf4e481f1..6ffbc4ea24 100644 --- ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out +++ ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out @@ -611,13 +611,25 @@ POSTHOOK: Input: default@srcpart_acid@ds=2008-04-09/hr=12 POSTHOOK: Output: default@merge_tmp_table POSTHOOK: Output: default@srcpart_acid@ds=2008-04-08/hr=11 POSTHOOK: Output: default@srcpart_acid@ds=2008-04-08/hr=11 +POSTHOOK: Output: default@srcpart_acid@ds=2008-04-08/hr=11 +POSTHOOK: Output: default@srcpart_acid@ds=2008-04-08/hr=12 POSTHOOK: Output: default@srcpart_acid@ds=2008-04-08/hr=12 POSTHOOK: Output: default@srcpart_acid@ds=2008-04-08/hr=12 POSTHOOK: Output: default@srcpart_acid@ds=2008-04-09/hr=11 POSTHOOK: Output: default@srcpart_acid@ds=2008-04-09/hr=11 +POSTHOOK: Output: default@srcpart_acid@ds=2008-04-09/hr=11 +POSTHOOK: Output: default@srcpart_acid@ds=2008-04-09/hr=12 POSTHOOK: Output: default@srcpart_acid@ds=2008-04-09/hr=12 POSTHOOK: Output: default@srcpart_acid@ds=2008-04-09/hr=12 POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(srcpart_acid)t.FieldSchema(name:ROW__ID, type:struct, comment:), (srcpart_acid)t.FieldSchema(name:ds, type:string, comment:null), (srcpart_acid)t.FieldSchema(name:hr, type:string, comment:null), ] +POSTHOOK: Lineage: srcpart_acid PARTITION(ds=2008-04-08,hr=11).key SIMPLE [] +POSTHOOK: Lineage: srcpart_acid PARTITION(ds=2008-04-08,hr=11).value SIMPLE [] +POSTHOOK: Lineage: srcpart_acid PARTITION(ds=2008-04-08,hr=12).key SIMPLE [] +POSTHOOK: Lineage: srcpart_acid PARTITION(ds=2008-04-08,hr=12).value SIMPLE [] +POSTHOOK: Lineage: srcpart_acid PARTITION(ds=2008-04-09,hr=11).key SIMPLE [] +POSTHOOK: Lineage: srcpart_acid PARTITION(ds=2008-04-09,hr=11).value SIMPLE [] +POSTHOOK: Lineage: srcpart_acid PARTITION(ds=2008-04-09,hr=12).key SIMPLE [] +POSTHOOK: Lineage: srcpart_acid PARTITION(ds=2008-04-09,hr=12).value SIMPLE [] PREHOOK: query: select count(*) from srcpart_acid where ds='2008-04-08' and hr=='12' PREHOOK: type: QUERY PREHOOK: Input: default@srcpart_acid @@ -626,7 +638,7 @@ POSTHOOK: query: select count(*) from srcpart_acid where ds='2008-04-08' and hr= POSTHOOK: type: QUERY POSTHOOK: Input: default@srcpart_acid #### A masked pattern was here #### -0 +497 PREHOOK: query: select ds, hr, key, value from srcpart_acid where value like '%updated by merge' PREHOOK: type: QUERY PREHOOK: Input: default@srcpart_acid @@ -1150,13 +1162,25 @@ POSTHOOK: Input: default@srcpart_acidb@ds=2008-04-09/hr=12 POSTHOOK: Output: default@merge_tmp_table POSTHOOK: Output: default@srcpart_acidb@ds=2008-04-08/hr=11 POSTHOOK: Output: default@srcpart_acidb@ds=2008-04-08/hr=11 +POSTHOOK: Output: default@srcpart_acidb@ds=2008-04-08/hr=11 +POSTHOOK: Output: default@srcpart_acidb@ds=2008-04-08/hr=12 POSTHOOK: Output: default@srcpart_acidb@ds=2008-04-08/hr=12 POSTHOOK: Output: default@srcpart_acidb@ds=2008-04-08/hr=12 POSTHOOK: Output: default@srcpart_acidb@ds=2008-04-09/hr=11 POSTHOOK: Output: default@srcpart_acidb@ds=2008-04-09/hr=11 +POSTHOOK: Output: default@srcpart_acidb@ds=2008-04-09/hr=11 +POSTHOOK: Output: default@srcpart_acidb@ds=2008-04-09/hr=12 POSTHOOK: Output: default@srcpart_acidb@ds=2008-04-09/hr=12 POSTHOOK: Output: default@srcpart_acidb@ds=2008-04-09/hr=12 POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(srcpart_acidb)t.FieldSchema(name:ROW__ID, type:struct, comment:), (srcpart_acidb)t.FieldSchema(name:ds, type:string, comment:null), (srcpart_acidb)t.FieldSchema(name:hr, type:string, comment:null), ] +POSTHOOK: Lineage: srcpart_acidb PARTITION(ds=2008-04-08,hr=11).key SIMPLE [] +POSTHOOK: Lineage: srcpart_acidb PARTITION(ds=2008-04-08,hr=11).value SIMPLE [] +POSTHOOK: Lineage: srcpart_acidb PARTITION(ds=2008-04-08,hr=12).key SIMPLE [] +POSTHOOK: Lineage: srcpart_acidb PARTITION(ds=2008-04-08,hr=12).value SIMPLE [] +POSTHOOK: Lineage: srcpart_acidb PARTITION(ds=2008-04-09,hr=11).key SIMPLE [] +POSTHOOK: Lineage: srcpart_acidb PARTITION(ds=2008-04-09,hr=11).value SIMPLE [] +POSTHOOK: Lineage: srcpart_acidb PARTITION(ds=2008-04-09,hr=12).key SIMPLE [] +POSTHOOK: Lineage: srcpart_acidb PARTITION(ds=2008-04-09,hr=12).value SIMPLE [] PREHOOK: query: select count(*) from srcpart_acidb where ds='2008-04-08' and hr=='12' PREHOOK: type: QUERY PREHOOK: Input: default@srcpart_acidb @@ -1165,7 +1189,7 @@ POSTHOOK: query: select count(*) from srcpart_acidb where ds='2008-04-08' and hr POSTHOOK: type: QUERY POSTHOOK: Input: default@srcpart_acidb #### A masked pattern was here #### -0 +497 PREHOOK: query: select ds, hr, key, value from srcpart_acidb where value like '%updated by merge' PREHOOK: type: QUERY PREHOOK: Input: default@srcpart_acidb @@ -1997,13 +2021,25 @@ POSTHOOK: Input: default@srcpart_acidv@ds=2008-04-09/hr=12 POSTHOOK: Output: default@merge_tmp_table POSTHOOK: Output: default@srcpart_acidv@ds=2008-04-08/hr=11 POSTHOOK: Output: default@srcpart_acidv@ds=2008-04-08/hr=11 +POSTHOOK: Output: default@srcpart_acidv@ds=2008-04-08/hr=11 +POSTHOOK: Output: default@srcpart_acidv@ds=2008-04-08/hr=12 POSTHOOK: Output: default@srcpart_acidv@ds=2008-04-08/hr=12 POSTHOOK: Output: default@srcpart_acidv@ds=2008-04-08/hr=12 POSTHOOK: Output: default@srcpart_acidv@ds=2008-04-09/hr=11 POSTHOOK: Output: default@srcpart_acidv@ds=2008-04-09/hr=11 +POSTHOOK: Output: default@srcpart_acidv@ds=2008-04-09/hr=11 +POSTHOOK: Output: default@srcpart_acidv@ds=2008-04-09/hr=12 POSTHOOK: Output: default@srcpart_acidv@ds=2008-04-09/hr=12 POSTHOOK: Output: default@srcpart_acidv@ds=2008-04-09/hr=12 POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(srcpart_acidv)t.FieldSchema(name:ROW__ID, type:struct, comment:), (srcpart_acidv)t.FieldSchema(name:ds, type:string, comment:null), (srcpart_acidv)t.FieldSchema(name:hr, type:string, comment:null), ] +POSTHOOK: Lineage: srcpart_acidv PARTITION(ds=2008-04-08,hr=11).key SIMPLE [] +POSTHOOK: Lineage: srcpart_acidv PARTITION(ds=2008-04-08,hr=11).value SIMPLE [] +POSTHOOK: Lineage: srcpart_acidv PARTITION(ds=2008-04-08,hr=12).key SIMPLE [] +POSTHOOK: Lineage: srcpart_acidv PARTITION(ds=2008-04-08,hr=12).value SIMPLE [] +POSTHOOK: Lineage: srcpart_acidv PARTITION(ds=2008-04-09,hr=11).key SIMPLE [] +POSTHOOK: Lineage: srcpart_acidv PARTITION(ds=2008-04-09,hr=11).value SIMPLE [] +POSTHOOK: Lineage: srcpart_acidv PARTITION(ds=2008-04-09,hr=12).key SIMPLE [] +POSTHOOK: Lineage: srcpart_acidv PARTITION(ds=2008-04-09,hr=12).value SIMPLE [] PREHOOK: query: select count(*) from srcpart_acidv where ds='2008-04-08' and hr=='12' PREHOOK: type: QUERY PREHOOK: Input: default@srcpart_acidv @@ -2012,7 +2048,7 @@ POSTHOOK: query: select count(*) from srcpart_acidv where ds='2008-04-08' and hr POSTHOOK: type: QUERY POSTHOOK: Input: default@srcpart_acidv #### A masked pattern was here #### -0 +497 PREHOOK: query: select ds, hr, key, value from srcpart_acidv where value like '%updated by merge' PREHOOK: type: QUERY PREHOOK: Input: default@srcpart_acidv @@ -2853,13 +2889,25 @@ POSTHOOK: Input: default@srcpart_acidvb@ds=2008-04-09/hr=12 POSTHOOK: Output: default@merge_tmp_table POSTHOOK: Output: default@srcpart_acidvb@ds=2008-04-08/hr=11 POSTHOOK: Output: default@srcpart_acidvb@ds=2008-04-08/hr=11 +POSTHOOK: Output: default@srcpart_acidvb@ds=2008-04-08/hr=11 +POSTHOOK: Output: default@srcpart_acidvb@ds=2008-04-08/hr=12 POSTHOOK: Output: default@srcpart_acidvb@ds=2008-04-08/hr=12 POSTHOOK: Output: default@srcpart_acidvb@ds=2008-04-08/hr=12 POSTHOOK: Output: default@srcpart_acidvb@ds=2008-04-09/hr=11 POSTHOOK: Output: default@srcpart_acidvb@ds=2008-04-09/hr=11 +POSTHOOK: Output: default@srcpart_acidvb@ds=2008-04-09/hr=11 +POSTHOOK: Output: default@srcpart_acidvb@ds=2008-04-09/hr=12 POSTHOOK: Output: default@srcpart_acidvb@ds=2008-04-09/hr=12 POSTHOOK: Output: default@srcpart_acidvb@ds=2008-04-09/hr=12 POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(srcpart_acidvb)t.FieldSchema(name:ROW__ID, type:struct, comment:), (srcpart_acidvb)t.FieldSchema(name:ds, type:string, comment:null), (srcpart_acidvb)t.FieldSchema(name:hr, type:string, comment:null), ] +POSTHOOK: Lineage: srcpart_acidvb PARTITION(ds=2008-04-08,hr=11).key SIMPLE [] +POSTHOOK: Lineage: srcpart_acidvb PARTITION(ds=2008-04-08,hr=11).value SIMPLE [] +POSTHOOK: Lineage: srcpart_acidvb PARTITION(ds=2008-04-08,hr=12).key SIMPLE [] +POSTHOOK: Lineage: srcpart_acidvb PARTITION(ds=2008-04-08,hr=12).value SIMPLE [] +POSTHOOK: Lineage: srcpart_acidvb PARTITION(ds=2008-04-09,hr=11).key SIMPLE [] +POSTHOOK: Lineage: srcpart_acidvb PARTITION(ds=2008-04-09,hr=11).value SIMPLE [] +POSTHOOK: Lineage: srcpart_acidvb PARTITION(ds=2008-04-09,hr=12).key SIMPLE [] +POSTHOOK: Lineage: srcpart_acidvb PARTITION(ds=2008-04-09,hr=12).value SIMPLE [] PREHOOK: query: select count(*) from srcpart_acidvb where ds='2008-04-08' and hr=='12' PREHOOK: type: QUERY PREHOOK: Input: default@srcpart_acidvb @@ -2868,7 +2916,7 @@ POSTHOOK: query: select count(*) from srcpart_acidvb where ds='2008-04-08' and h POSTHOOK: type: QUERY POSTHOOK: Input: default@srcpart_acidvb #### A masked pattern was here #### -0 +497 PREHOOK: query: select ds, hr, key, value from srcpart_acidvb where value like '%updated by merge' PREHOOK: type: QUERY PREHOOK: Input: default@srcpart_acidvb diff --git ql/src/test/results/clientpositive/llap/insert_overwrite.q.out ql/src/test/results/clientpositive/llap/insert_overwrite.q.out index fbc3326b39..1ee38d371c 100644 --- ql/src/test/results/clientpositive/llap/insert_overwrite.q.out +++ ql/src/test/results/clientpositive/llap/insert_overwrite.q.out @@ -372,6 +372,12 @@ PREHOOK: Output: default@int_part POSTHOOK: query: INSERT OVERWRITE TABLE int_part PARTITION (par) SELECT * FROM b POSTHOOK: type: QUERY POSTHOOK: Input: default@b +POSTHOOK: Output: default@int_part@par=1 +POSTHOOK: Output: default@int_part@par=2 +POSTHOOK: Output: default@int_part@par=3 +POSTHOOK: Lineage: int_part PARTITION(par=1).col SIMPLE [(b)b.FieldSchema(name:par, type:string, comment:null), ] +POSTHOOK: Lineage: int_part PARTITION(par=2).col SIMPLE [(b)b.FieldSchema(name:par, type:string, comment:null), ] +POSTHOOK: Lineage: int_part PARTITION(par=3).col SIMPLE [(b)b.FieldSchema(name:par, type:string, comment:null), ] PREHOOK: query: SELECT count(*) FROM int_part PREHOOK: type: QUERY PREHOOK: Input: default@int_part @@ -429,7 +435,11 @@ POSTHOOK: query: INSERT OVERWRITE TABLE int_part PARTITION (par) SELECT * FROM b POSTHOOK: type: QUERY POSTHOOK: Input: default@b POSTHOOK: Output: default@int_part@par=1 +POSTHOOK: Output: default@int_part@par=2 +POSTHOOK: Output: default@int_part@par=3 POSTHOOK: Lineage: int_part PARTITION(par=1).col SIMPLE [(b)b.FieldSchema(name:par, type:string, comment:null), ] +POSTHOOK: Lineage: int_part PARTITION(par=2).col SIMPLE [(b)b.FieldSchema(name:par, type:string, comment:null), ] +POSTHOOK: Lineage: int_part PARTITION(par=3).col SIMPLE [(b)b.FieldSchema(name:par, type:string, comment:null), ] PREHOOK: query: SELECT count(*) FROM int_part PREHOOK: type: QUERY PREHOOK: Input: default@int_part diff --git ql/src/test/results/clientpositive/llap/mm_all.q.out ql/src/test/results/clientpositive/llap/mm_all.q.out index 226f2a9374..fd28d39af7 100644 --- ql/src/test/results/clientpositive/llap/mm_all.q.out +++ ql/src/test/results/clientpositive/llap/mm_all.q.out @@ -1633,6 +1633,7 @@ POSTHOOK: Input: default@intermediate_n0@p=455 POSTHOOK: Input: default@intermediate_n0@p=456 POSTHOOK: Input: default@intermediate_n0@p=457 POSTHOOK: Output: default@multi1_mm@p=1 +POSTHOOK: Output: default@multi1_mm@p=2 POSTHOOK: Output: default@multi1_mm@p=455 POSTHOOK: Output: default@multi1_mm@p=456 POSTHOOK: Output: default@multi1_mm@p=457 @@ -1646,6 +1647,8 @@ POSTHOOK: Lineage: ###Masked### POSTHOOK: Lineage: ###Masked### POSTHOOK: Lineage: ###Masked### POSTHOOK: Lineage: ###Masked### +POSTHOOK: Lineage: ###Masked### +POSTHOOK: Lineage: ###Masked### PREHOOK: query: select key, key2, p from multi1_mm order by key, key2, p PREHOOK: type: QUERY PREHOOK: Input: default@multi1_mm @@ -1713,6 +1716,18 @@ POSTHOOK: Input: default@intermediate_n0@p=455 POSTHOOK: Input: default@intermediate_n0@p=456 POSTHOOK: Input: default@intermediate_n0@p=457 POSTHOOK: Output: default@multi1_mm@p=1 +POSTHOOK: Output: default@multi1_mm@p=2 +POSTHOOK: Output: default@multi1_mm@p=455 +POSTHOOK: Output: default@multi1_mm@p=456 +POSTHOOK: Output: default@multi1_mm@p=457 +POSTHOOK: Lineage: ###Masked### +POSTHOOK: Lineage: ###Masked### +POSTHOOK: Lineage: ###Masked### +POSTHOOK: Lineage: ###Masked### +POSTHOOK: Lineage: ###Masked### +POSTHOOK: Lineage: ###Masked### +POSTHOOK: Lineage: ###Masked### +POSTHOOK: Lineage: ###Masked### POSTHOOK: Lineage: ###Masked### POSTHOOK: Lineage: ###Masked### POSTHOOK: Lineage: ###Masked### diff --git ql/src/test/results/clientpositive/mm_all.q.out ql/src/test/results/clientpositive/mm_all.q.out index 143ebd69f9..16ba2f2e86 100644 --- ql/src/test/results/clientpositive/mm_all.q.out +++ ql/src/test/results/clientpositive/mm_all.q.out @@ -1647,6 +1647,7 @@ POSTHOOK: Input: default@intermediate_n0@p=455 POSTHOOK: Input: default@intermediate_n0@p=456 POSTHOOK: Input: default@intermediate_n0@p=457 POSTHOOK: Output: default@multi1_mm@p=1 +POSTHOOK: Output: default@multi1_mm@p=2 POSTHOOK: Output: default@multi1_mm@p=455 POSTHOOK: Output: default@multi1_mm@p=456 POSTHOOK: Output: default@multi1_mm@p=457 @@ -1660,6 +1661,8 @@ POSTHOOK: Lineage: ###Masked### POSTHOOK: Lineage: ###Masked### POSTHOOK: Lineage: ###Masked### POSTHOOK: Lineage: ###Masked### +POSTHOOK: Lineage: ###Masked### +POSTHOOK: Lineage: ###Masked### PREHOOK: query: select key, key2, p from multi1_mm order by key, key2, p PREHOOK: type: QUERY PREHOOK: Input: default@multi1_mm @@ -1727,6 +1730,18 @@ POSTHOOK: Input: default@intermediate_n0@p=455 POSTHOOK: Input: default@intermediate_n0@p=456 POSTHOOK: Input: default@intermediate_n0@p=457 POSTHOOK: Output: default@multi1_mm@p=1 +POSTHOOK: Output: default@multi1_mm@p=2 +POSTHOOK: Output: default@multi1_mm@p=455 +POSTHOOK: Output: default@multi1_mm@p=456 +POSTHOOK: Output: default@multi1_mm@p=457 +POSTHOOK: Lineage: ###Masked### +POSTHOOK: Lineage: ###Masked### +POSTHOOK: Lineage: ###Masked### +POSTHOOK: Lineage: ###Masked### +POSTHOOK: Lineage: ###Masked### +POSTHOOK: Lineage: ###Masked### +POSTHOOK: Lineage: ###Masked### +POSTHOOK: Lineage: ###Masked### POSTHOOK: Lineage: ###Masked### POSTHOOK: Lineage: ###Masked### POSTHOOK: Lineage: ###Masked### diff --git streaming/src/test/org/apache/hive/streaming/TestStreaming.java streaming/src/test/org/apache/hive/streaming/TestStreaming.java index 35a220facd..6101caac66 100644 --- streaming/src/test/org/apache/hive/streaming/TestStreaming.java +++ streaming/src/test/org/apache/hive/streaming/TestStreaming.java @@ -391,7 +391,7 @@ public void testNoBuckets() throws Exception { rs = queryTable(driver, "select ROW__ID, a, b, INPUT__FILE__NAME from default.streamingnobuckets order by ROW__ID"); Assert.assertTrue(rs.get(0), rs.get(0).startsWith("{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\tfoo\tbar")); - Assert.assertTrue(rs.get(0), rs.get(0).endsWith("streamingnobuckets/delta_0000001_0000001_0000/bucket_00000")); + Assert.assertTrue(rs.get(0), rs.get(0).endsWith("streamingnobuckets/delta_0000001_0000001_0000/bucket_00000_0")); Assert.assertTrue(rs.get(1), rs.get(1).startsWith("{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\ta1\tb2")); Assert.assertTrue(rs.get(1), rs.get(1).endsWith("streamingnobuckets/delta_0000002_0000003/bucket_00000")); Assert.assertTrue(rs.get(2), rs.get(2).startsWith("{\"writeid\":2,\"bucketid\":536870912,\"rowid\":1}\ta3\tb4")); @@ -569,7 +569,7 @@ public void testConnectionWithWriteId() throws Exception { + "INPUT__FILE__NAME from default.writeidconnection order by a"); Assert.assertEquals(4, rs.size()); Assert.assertTrue(rs.get(0), rs.get(0).startsWith("{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\ta0\tbar")); - Assert.assertTrue(rs.get(0), rs.get(0).endsWith("bucket_00000")); + Assert.assertTrue(rs.get(0), rs.get(0).endsWith("bucket_00000_0")); Assert.assertTrue(rs.get(1), rs.get(1).contains("\"rowid\":0}\ta1\tb2")); Assert.assertTrue(rs.get(1), rs.get(1).endsWith("bucket_00000")); Assert.assertTrue(rs.get(2), rs.get(2).contains("\"rowid\":1}\ta3\tb4")); @@ -727,7 +727,7 @@ public void testAutoRollTransactionBatch() throws Exception { rs = queryTable(driver, "select ROW__ID, a, b, INPUT__FILE__NAME from default.streamingnobuckets order by ROW__ID"); Assert.assertTrue(rs.get(0), rs.get(0).startsWith("{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\tfoo\tbar")); - Assert.assertTrue(rs.get(0), rs.get(0).endsWith("streamingnobuckets/delta_0000001_0000001_0000/bucket_00000")); + Assert.assertTrue(rs.get(0), rs.get(0).endsWith("streamingnobuckets/delta_0000001_0000001_0000/bucket_00000_0")); Assert.assertTrue(rs.get(1), rs.get(1).startsWith("{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\ta1\tb2")); Assert.assertTrue(rs.get(1), rs.get(1).endsWith("streamingnobuckets/delta_0000002_0000003/bucket_00000")); Assert.assertTrue(rs.get(2), rs.get(2).startsWith("{\"writeid\":2,\"bucketid\":536870912,\"rowid\":1}\ta3\tb4"));