From 2358521fab12f6e12f3fc1ff606b83408b413337 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Tue, 1 Aug 2017 15:31:11 -0500 Subject: [PATCH 001/512] TEZ-3807. InMemoryWriter is not tested with RLE enabled. Contributed by Muhammad Samir Khan --- .../orderedgrouped/InMemoryWriter.java | 9 +++-- .../library/common/sort/impl/IFile.java | 6 ++-- .../library/common/sort/impl/TestIFile.java | 35 ++++++++++++------- 3 files changed, 32 insertions(+), 18 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/InMemoryWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/InMemoryWriter.java index 46dc72ed5d..c5db7c9ade 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/InMemoryWriter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/InMemoryWriter.java @@ -48,9 +48,12 @@ public InMemoryWriter(byte[] array) { } public InMemoryWriter(BoundedByteArrayOutputStream arrayStream) { - super(null, null); - this.out = - new NonSyncDataOutputStream(new IFileOutputStream(arrayStream)); + this(arrayStream, false); + } + + public InMemoryWriter(BoundedByteArrayOutputStream arrayStream, boolean rle) { + super(null, null, rle); + this.out = new NonSyncDataOutputStream(new IFileOutputStream(arrayStream)); } public void append(Object key, Object value) throws IOException { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java index b502fc95f5..e460859582 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java @@ -127,11 +127,11 @@ public Writer(Configuration conf, FileSystem fs, Path file, writesCounter, serializedBytesCounter); ownOutputStream = true; } - - protected Writer(TezCounter writesCounter, TezCounter serializedBytesCounter) { + + protected Writer(TezCounter writesCounter, TezCounter serializedBytesCounter, boolean rle) { writtenRecordsCounter = writesCounter; serializedUncompressedBytes = serializedBytesCounter; - this.rle = false; + this.rle = rle; } public Writer(Configuration conf, FSDataOutputStream outputStream, diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java index f06fda3873..90f5374590 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java @@ -313,6 +313,9 @@ public void testWithRLEMarker() throws IOException { writer = new IFile.Writer(defaultConf, out, Text.class, IntWritable.class, codec, null, null, true); + BoundedByteArrayOutputStream boundedOut = new BoundedByteArrayOutputStream(1024*1024); + Writer inMemWriter = new InMemoryWriter(boundedOut, true); + DataInputBuffer kin = new DataInputBuffer(); kin.reset(kvbuffer, pos, keyLength); @@ -324,6 +327,8 @@ public void testWithRLEMarker() throws IOException { //Write initial KV pair writer.append(kin, vin); assertFalse(writer.sameKey); + inMemWriter.append(kin, vin); + assertFalse(inMemWriter.sameKey); pos += (keyLength + valueLength); //Second key is similar to key1 (RLE should kick in) @@ -332,6 +337,8 @@ public void testWithRLEMarker() throws IOException { vin.reset(vout.getData(), vout.getLength()); writer.append(kin, vin); assertTrue(writer.sameKey); + inMemWriter.append(kin, vin); + assertTrue(inMemWriter.sameKey); pos += (keyLength + valueLength); //Next key (key3) is different (RLE should not kick in) @@ -340,9 +347,13 @@ public void testWithRLEMarker() throws IOException { vin.reset(vout.getData(), vout.getLength()); writer.append(kin, vin); assertFalse(writer.sameKey); + inMemWriter.append(kin, vin); + assertFalse(inMemWriter.sameKey); writer.close(); out.close(); + inMemWriter.close(); + boundedOut.close(); } @Test(timeout = 5000) @@ -416,25 +427,25 @@ public void testInMemoryWriter() throws IOException { //No RLE, No RepeatKeys, no compression writer = new InMemoryWriter(bout); - writeTestFileUsingDataBuffer(writer, false, false, data, null); + writeTestFileUsingDataBuffer(writer, false, data); readUsingInMemoryReader(bout.getBuffer(), data); //No RLE, RepeatKeys, no compression bout.reset(); writer = new InMemoryWriter(bout); - writeTestFileUsingDataBuffer(writer, false, true, data, null); + writeTestFileUsingDataBuffer(writer, true, data); readUsingInMemoryReader(bout.getBuffer(), data); //RLE, No RepeatKeys, no compression bout.reset(); - writer = new InMemoryWriter(bout); - writeTestFileUsingDataBuffer(writer, true, false, data, null); + writer = new InMemoryWriter(bout, true); + writeTestFileUsingDataBuffer(writer, false, data); readUsingInMemoryReader(bout.getBuffer(), data); //RLE, RepeatKeys, no compression bout.reset(); - writer = new InMemoryWriter(bout); - writeTestFileUsingDataBuffer(writer, true, true, data, null); + writer = new InMemoryWriter(bout, true); + writeTestFileUsingDataBuffer(writer, true, data); readUsingInMemoryReader(bout.getBuffer(), data); } @@ -753,13 +764,13 @@ private Writer writeTestFile(boolean rle, boolean repeatKeys, FSDataOutputStream out = localFs.create(outputPath); IFile.Writer writer = new IFile.Writer(defaultConf, out, Text.class, IntWritable.class, codec, null, null, rle); - writeTestFile(writer, rle, repeatKeys, data, codec); + writeTestFile(writer, repeatKeys, data); out.close(); return writer; } - private Writer writeTestFile(IFile.Writer writer, boolean rle, boolean repeatKeys, - List data, CompressionCodec codec) throws IOException { + private Writer writeTestFile(IFile.Writer writer, boolean repeatKeys, + List data) throws IOException { assertNotNull(writer); Text previousKey = null; @@ -786,13 +797,13 @@ private Writer writeTestFileUsingDataBuffer(boolean rle, boolean repeatKeys, FSDataOutputStream out = localFs.create(outputPath); IFile.Writer writer = new IFile.Writer(defaultConf, out, Text.class, IntWritable.class, codec, null, null, rle); - writeTestFileUsingDataBuffer(writer, rle, repeatKeys, data, codec); + writeTestFileUsingDataBuffer(writer, repeatKeys, data); out.close(); return writer; } - private Writer writeTestFileUsingDataBuffer(IFile.Writer writer, boolean rle, boolean repeatKeys, - List data, CompressionCodec codec) throws IOException { + private Writer writeTestFileUsingDataBuffer(Writer writer, boolean repeatKeys, + List data) throws IOException { DataInputBuffer previousKey = new DataInputBuffer(); DataInputBuffer key = new DataInputBuffer(); DataInputBuffer value = new DataInputBuffer(); From 614937c5df88b79c85ae9fc6394652fb65d98081 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Tue, 1 Aug 2017 16:58:42 -0500 Subject: [PATCH 002/512] TEZ-3804. FetcherOrderedGrouped#setupLocalDiskFetch should ignore empty partition records (Kuhu Shukla via jeagles) --- .../orderedgrouped/FetcherOrderedGrouped.java | 3 + .../shuffle/orderedgrouped/TestFetcher.java | 83 ++++++++++++++++++- 2 files changed, 82 insertions(+), 4 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java index 2c3aac3520..68a54e911f 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java @@ -715,6 +715,9 @@ protected void setupLocalDiskFetch(MapHost host) throws InterruptedException { srcAttemptId = scheduler.getIdentifierForFetchedOutput(srcAttemptId.getPathComponent(), reduceId); Path filename = getShuffleInputFileName(srcAttemptId.getPathComponent(), null); TezIndexRecord indexRecord = getIndexRecord(srcAttemptId.getPathComponent(), reduceId); + if(!indexRecord.hasData()) { + continue; + } mapOutput = getMapOutputForDirectDiskFetch(srcAttemptId, filename, indexRecord); long endTime = System.currentTimeMillis(); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java index ef371c200c..6d304480b8 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java @@ -299,7 +299,7 @@ public TezIndexRecord answer(InvocationOnMock invocation) throws Throwable { throw new IOException("failing to simulate failure case"); } // match with params for copySucceeded below. - return new TezIndexRecord(p * 10, p * 1000, p * 100); + return new TezIndexRecord(p * 10, (p+1) * 1000, (p+2) * 100); } }).when(spyFetcher).getIndexRecord(anyString(), eq(host.getPartitionId() + i)); } @@ -327,6 +327,81 @@ public TezIndexRecord answer(InvocationOnMock invocation) throws Throwable { verify(scheduler).putBackKnownMapOutput(host, srcAttempts.get(SECOND_FAILED_ATTEMPT_IDX)); } + @Test (timeout = 5000) + public void testSetupLocalDiskFetchEmptyPartitions() throws Exception { + Configuration conf = new TezConfiguration(); + ShuffleScheduler scheduler = mock(ShuffleScheduler.class); + MergeManager merger = mock(MergeManager.class); + Shuffle shuffle = mock(Shuffle.class); + InputContext inputContext = mock(InputContext.class); + when(inputContext.getCounters()).thenReturn(new TezCounters()); + when(inputContext.getSourceVertexName()).thenReturn(""); + + MapHost host = new MapHost(HOST, PORT, 1, 1); + FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, + null, conf, true, HOST, PORT, "src vertex", host, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, + wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, + false, false, true, false); + FetcherOrderedGrouped spyFetcher = spy(fetcher); + + final List srcAttempts = Arrays.asList( + new CompositeInputAttemptIdentifier(0, 1, InputAttemptIdentifier.PATH_PREFIX + "pathComponent_0", 1), + new CompositeInputAttemptIdentifier(1, 2, InputAttemptIdentifier.PATH_PREFIX + "pathComponent_1", 1), + new CompositeInputAttemptIdentifier(2, 3, InputAttemptIdentifier.PATH_PREFIX + "pathComponent_2", 1), + new CompositeInputAttemptIdentifier(3, 4, InputAttemptIdentifier.PATH_PREFIX + "pathComponent_3", 1), + new CompositeInputAttemptIdentifier(4, 4, InputAttemptIdentifier.PATH_PREFIX + "pathComponent_4", 1) + ); + + doReturn(srcAttempts).when(scheduler).getMapsForHost(host); + + final ConcurrentMap pathToIdentifierMap = new ConcurrentHashMap(); + for (CompositeInputAttemptIdentifier srcAttempt : srcAttempts) { + for (int i = 0; i < srcAttempt.getInputIdentifierCount(); i++) { + ShuffleScheduler.PathPartition pathPartition = new ShuffleScheduler.PathPartition(srcAttempt.getPathComponent(), host.getPartitionId() + i); + pathToIdentifierMap.put(pathPartition, srcAttempt.expand(i)); + } + } + doAnswer(new Answer() { + @Override + public InputAttemptIdentifier answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + String path = (String) args[0]; + int reduceId = (int) args[1]; + return pathToIdentifierMap.get(new ShuffleScheduler.PathPartition(path, reduceId)); + } + }).when(scheduler) + .getIdentifierForFetchedOutput(any(String.class), any(int.class)); + + doAnswer(new Answer() { + @Override + public Path answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + return new Path(SHUFFLE_INPUT_FILE_PREFIX + args[0]); + } + }).when(spyFetcher).getShuffleInputFileName(anyString(), anyString()); + + for (int i = 0; i < host.getPartitionCount(); i++) { + doAnswer(new Answer() { + @Override + public TezIndexRecord answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + String pathComponent = (String) args[0]; + int len = pathComponent.length(); + long p = Long.valueOf(pathComponent.substring(len - 1, len)); + // match with params for copySucceeded below. + return new TezIndexRecord(p * 10, 0, 0); + } + }).when(spyFetcher).getIndexRecord(anyString(), eq(host.getPartitionId() + i)); + } + + doNothing().when(scheduler).copySucceeded(any(InputAttemptIdentifier.class), any(MapHost.class), + anyLong(), anyLong(), anyLong(), any(MapOutput.class), anyBoolean()); + spyFetcher.setupLocalDiskFetch(host); + verify(scheduler, times(0)).copySucceeded(any(InputAttemptIdentifier.class), any(MapHost.class), + anyLong(), anyLong(), anyLong(), any(MapOutput.class), anyBoolean()); + verify(spyFetcher).putBackRemainingMapOutputs(host); + } + @Test(timeout = 5000) public void testSetupLocalDiskFetchAutoReduce() throws Exception { Configuration conf = new TezConfiguration(); @@ -412,7 +487,7 @@ public TezIndexRecord answer(InvocationOnMock invocation) throws Throwable { throw new IOException("Thowing exception to simulate failure case"); } // match with params for copySucceeded below. - return new TezIndexRecord(p * 10, p * 1000, p * 100); + return new TezIndexRecord(p * 10, (p + 1) * 1000, (p + 2) * 100); } }).when(spyFetcher).getIndexRecord(anyString(), eq(host.getPartitionId() + i)); } @@ -455,8 +530,8 @@ private void verifyCopySucceeded(ShuffleScheduler scheduler, MapHost host, InputAttemptIdentifier srcAttemptToMatch = srcAttempts.get((int) p).expand(j); String filenameToMatch = SHUFFLE_INPUT_FILE_PREFIX + srcAttemptToMatch.getPathComponent(); ArgumentCaptor captureMapOutput = ArgumentCaptor.forClass(MapOutput.class); - verify(scheduler).copySucceeded(eq(srcAttemptToMatch), eq(host), eq(p * 100), - eq(p * 1000), anyLong(), captureMapOutput.capture(), anyBoolean()); + verify(scheduler).copySucceeded(eq(srcAttemptToMatch), eq(host), eq((p+2) * 100), + eq((p+1) * 1000), anyLong(), captureMapOutput.capture(), anyBoolean()); // cannot use the equals of MapOutput as it compares id which is private. so doing it manually MapOutput m = captureMapOutput.getAllValues().get(0); From 8dcf8a121f5961e2974ef1121ec9d0200cbdc0ae Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Fri, 4 Aug 2017 15:21:54 -0500 Subject: [PATCH 003/512] TEZ-3803. Tasks can get killed due to insufficient progress while waiting for shuffle inputs to complete. Contributed by Kuhu Shukla --- .../common/shuffle/impl/ShuffleManager.java | 9 ++--- .../orderedgrouped/ShuffleScheduler.java | 33 +++++++++++-------- .../shuffle/impl/TestShuffleManager.java | 21 ++++++++++++ .../orderedgrouped/TestShuffleScheduler.java | 24 ++++++++++++++ 4 files changed, 69 insertions(+), 18 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java index e1b7f99008..24fb12befb 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java @@ -37,6 +37,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Condition; @@ -320,10 +321,10 @@ protected Void callInternal() throws Exception { while (!isShutdown.get() && numCompletedInputs.get() < numInputs) { lock.lock(); try { - if (runningFetchers.size() >= numFetchers || pendingHosts.isEmpty()) { - if (numCompletedInputs.get() < numInputs) { - wakeLoop.await(); - } + while ((runningFetchers.size() >= numFetchers || pendingHosts.isEmpty()) + && numCompletedInputs.get() < numInputs) { + inputContext.notifyProgress(); + boolean ret = wakeLoop.await(1000, TimeUnit.MILLISECONDS); } } finally { lock.unlock(); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java index b223c1a6cd..981e2246f7 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java @@ -1120,7 +1120,7 @@ public synchronized MapHost getHost() throws InterruptedException { if (LOG.isDebugEnabled()) { LOG.debug("PendingHosts=" + pendingHosts); } - wait(); + waitAndNotifyProgress(); } if (!pendingHosts.isEmpty()) { @@ -1360,19 +1360,19 @@ private class ShuffleSchedulerCallable extends CallableWithNdc { protected Void callInternal() throws InterruptedException { while (!isShutdown.get() && remainingMaps.get() > 0) { synchronized (ShuffleScheduler.this) { - if (runningFetchers.size() >= numFetchers || pendingHosts.isEmpty()) { - if (remainingMaps.get() > 0) { - try { - ShuffleScheduler.this.wait(); - } catch (InterruptedException e) { - if (isShutdown.get()) { - LOG.info(srcNameTrimmed + ": " + - "Interrupted while waiting for fetchers to complete and hasBeenShutdown. Breaking out of ShuffleSchedulerCallable loop"); - Thread.currentThread().interrupt(); - break; - } else { - throw e; - } + while ((runningFetchers.size() >= numFetchers || pendingHosts.isEmpty()) + && remainingMaps.get() > 0) { + try { + waitAndNotifyProgress(); + } catch (InterruptedException e) { + if (isShutdown.get()) { + LOG.info(srcNameTrimmed + ": " + + "Interrupted while waiting for fetchers to complete" + + "and hasBeenShutdown. Breaking out of ShuffleSchedulerCallable loop"); + Thread.currentThread().interrupt(); + break; + } else { + throw e; } } } @@ -1446,6 +1446,11 @@ protected Void callInternal() throws InterruptedException { } } + private synchronized void waitAndNotifyProgress() throws InterruptedException { + inputContext.notifyProgress(); + wait(1000); + } + @VisibleForTesting FetcherOrderedGrouped constructFetcherForHost(MapHost mapHost) { return new FetcherOrderedGrouped(httpConnectionParams, ShuffleScheduler.this, allocator, diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java index f361dc767b..23248edf46 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.anyInt; import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -193,6 +194,26 @@ public void testUseSharedExecutor() throws Exception { verify(inputContext).createTezFrameworkExecutorService(anyInt(), anyString()); } + @Test (timeout = 20000) + public void testProgressWithEmptyPendingHosts() throws Exception { + InputContext inputContext = createInputContext(); + final ShuffleManager shuffleManager = spy(createShuffleManager(inputContext, 1)); + Thread schedulerGetHostThread = new Thread(new Runnable() { + @Override + public void run() { + try { + shuffleManager.run(); + } catch (Exception e) { + e.printStackTrace(); + } + } + }); + schedulerGetHostThread.start(); + Thread.currentThread().sleep(1000 * 3 + 1000); + schedulerGetHostThread.interrupt(); + verify(inputContext, atLeast(3)).notifyProgress(); + } + private ShuffleManagerForTest createShuffleManager( InputContext inputContext, int expectedNumOfPhysicalInputs) throws IOException { diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java index c61391ccfb..381ad85abe 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java @@ -812,6 +812,28 @@ public void testPenalty() throws IOException, InterruptedException { assertFalse("Host identifier mismatch", (host.getHost() + ":" + host.getPort() + ":" + host.getPartitionId()).equalsIgnoreCase("host0:10000")); } + @Test (timeout = 20000) + public void testProgressDuringGetHostWait() throws IOException, InterruptedException { + long startTime = System.currentTimeMillis(); + Configuration conf = new TezConfiguration(); + Shuffle shuffle = mock(Shuffle.class); + final ShuffleSchedulerForTest scheduler = createScheduler(startTime, 1, shuffle, conf); + Thread schedulerGetHostThread = new Thread(new Runnable() { + @Override + public void run() { + try { + scheduler.getHost(); + } catch (Exception e) { + e.printStackTrace(); + } + } + }); + schedulerGetHostThread.start(); + Thread.currentThread().sleep(1000 * 3 + 1000); + schedulerGetHostThread.interrupt(); + verify(scheduler.inputContext, atLeast(3)).notifyProgress(); + } + @Test(timeout = 5000) public void testShutdown() throws Exception { InputContext inputContext = createTezInputContext(); @@ -964,6 +986,7 @@ private static class ShuffleSchedulerForTest extends ShuffleScheduler { private final AtomicInteger numFetchersCreated = new AtomicInteger(0); private final boolean fetcherShouldWait; private final ExceptionReporter reporter; + private final InputContext inputContext; public ShuffleSchedulerForTest(InputContext inputContext, Configuration conf, int numberOfInputs, @@ -989,6 +1012,7 @@ public ShuffleSchedulerForTest(InputContext inputContext, Configuration conf, ifileReadAhead, ifileReadAheadLength, srcNameTrimmed); this.fetcherShouldWait = fetcherShouldWait; this.reporter = shuffle; + this.inputContext = inputContext; } @Override From 1061cf5c310f954ffbf96a3295325b5e2eeb5d9f Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Wed, 9 Aug 2017 17:40:09 -0500 Subject: [PATCH 004/512] TEZ-3813. Reduce Object size of MemoryFetchedInput for large jobs (Muhammad Samir Khan via jeagles) --- tez-runtime-library/findbugs-exclude.xml | 6 ++ .../common/readers/UnorderedKVReader.java | 4 +- .../common/shuffle/DiskFetchedInput.java | 44 ++++++++------ .../library/common/shuffle/FetchedInput.java | 56 +++++++++++------- .../library/common/shuffle/Fetcher.java | 2 +- .../common/shuffle/LocalDiskFetchedInput.java | 43 ++++++++------ .../common/shuffle/MemoryFetchedInput.java | 57 +++++++++++++------ .../common/shuffle/impl/ShuffleManager.java | 12 +++- .../impl/SimpleFetchedInputAllocator.java | 8 +-- .../common/readers/TestUnorderedKVReader.java | 2 +- .../library/common/shuffle/TestFetcher.java | 3 +- .../shuffle/impl/TestShuffleManager.java | 12 +++- 12 files changed, 166 insertions(+), 83 deletions(-) diff --git a/tez-runtime-library/findbugs-exclude.xml b/tez-runtime-library/findbugs-exclude.xml index 24fe93e2e2..9e55221983 100644 --- a/tez-runtime-library/findbugs-exclude.xml +++ b/tez-runtime-library/findbugs-exclude.xml @@ -207,4 +207,10 @@ + + + + + + diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/readers/UnorderedKVReader.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/readers/UnorderedKVReader.java index f4400dbcc7..60f70ac271 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/readers/UnorderedKVReader.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/readers/UnorderedKVReader.java @@ -221,10 +221,10 @@ public IFile.Reader openIFileReader(FetchedInput fetchedInput) MemoryFetchedInput mfi = (MemoryFetchedInput) fetchedInput; return new InMemoryReader(null, mfi.getInputAttemptIdentifier(), - mfi.getBytes(), 0, (int) mfi.getActualSize()); + mfi.getBytes(), 0, (int) mfi.getSize()); } else { return new IFile.Reader(fetchedInput.getInputStream(), - fetchedInput.getCompressedSize(), codec, null, null, ifileReadAhead, + fetchedInput.getSize(), codec, null, null, ifileReadAhead, ifileReadAheadLength, ifileBufferSize); } } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/DiskFetchedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/DiskFetchedInput.java index c873af7140..5d1c037030 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/DiskFetchedInput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/DiskFetchedInput.java @@ -40,21 +40,33 @@ public class DiskFetchedInput extends FetchedInput { private final FileSystem localFS; private final Path tmpOutputPath; private final Path outputPath; + private final long size; - public DiskFetchedInput(long actualSize, long compressedSize, + public DiskFetchedInput(long compressedSize, InputAttemptIdentifier inputAttemptIdentifier, FetchedInputCallback callbackHandler, Configuration conf, LocalDirAllocator localDirAllocator, TezTaskOutputFiles filenameAllocator) throws IOException { - super(Type.DISK, actualSize, compressedSize, inputAttemptIdentifier, callbackHandler); + super(inputAttemptIdentifier, callbackHandler); + this.size = compressedSize; this.localFS = FileSystem.getLocal(conf).getRaw(); this.outputPath = filenameAllocator.getInputFileForWrite( - this.inputAttemptIdentifier.getInputIdentifier(), this - .inputAttemptIdentifier.getSpillEventId(), actualSize); + this.getInputAttemptIdentifier().getInputIdentifier(), this + .getInputAttemptIdentifier().getSpillEventId(), this.size); // Files are not clobbered due to the id being appended to the outputPath in the tmpPath, // otherwise fetches for the same task but from different attempts would clobber each other. - this.tmpOutputPath = outputPath.suffix(String.valueOf(id)); + this.tmpOutputPath = outputPath.suffix(String.valueOf(getId())); + } + + @Override + public Type getType() { + return Type.DISK; + } + + @Override + public long getSize() { + return size; } @Override @@ -68,7 +80,7 @@ public InputStream getInputStream() throws IOException { } public final Path getInputPath() { - if (state == State.COMMITTED) { + if (isState(State.COMMITTED)) { return this.outputPath; } return this.tmpOutputPath; @@ -76,8 +88,8 @@ public final Path getInputPath() { @Override public void commit() throws IOException { - if (state == State.PENDING) { - state = State.COMMITTED; + if (isState(State.PENDING)) { + setState(State.COMMITTED); localFS.rename(tmpOutputPath, outputPath); notifyFetchComplete(); } @@ -85,8 +97,8 @@ public void commit() throws IOException { @Override public void abort() throws IOException { - if (state == State.PENDING) { - state = State.ABORTED; + if (isState(State.PENDING)) { + setState(State.ABORTED); // TODO NEWTEZ Maybe defer this to container cleanup localFS.delete(tmpOutputPath, false); notifyFetchFailure(); @@ -96,10 +108,10 @@ public void abort() throws IOException { @Override public void free() { Preconditions.checkState( - state == State.COMMITTED || state == State.ABORTED, + isState(State.COMMITTED) || isState(State.ABORTED), "FetchedInput can only be freed after it is committed or aborted"); - if (state == State.COMMITTED) { - state = State.FREED; + if (isState(State.COMMITTED)) { + setState(State.FREED); try { // TODO NEWTEZ Maybe defer this to container cleanup localFS.delete(outputPath, false); @@ -115,8 +127,8 @@ public void free() { @Override public String toString() { return "DiskFetchedInput [outputPath=" + outputPath - + ", inputAttemptIdentifier=" + inputAttemptIdentifier - + ", actualSize=" + actualSize + ",compressedSize=" + compressedSize - + ", type=" + type + ", id=" + id + ", state=" + state + "]"; + + ", inputAttemptIdentifier=" + getInputAttemptIdentifier() + + ", actualSize=" + getSize() + + ", type=" + getType() + ", id=" + getId() + ", state=" + getState() + "]"; } } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetchedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetchedInput.java index 3e740a020e..8982c27cad 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetchedInput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetchedInput.java @@ -42,38 +42,52 @@ protected static enum State { private static AtomicInteger ID_GEN = new AtomicInteger(0); - protected InputAttemptIdentifier inputAttemptIdentifier; - protected final long actualSize; - protected final long compressedSize; - protected final Type type; - protected final FetchedInputCallback callback; - protected final int id; - protected State state; - - public FetchedInput(Type type, long actualSize, long compressedSize, - InputAttemptIdentifier inputAttemptIdentifier, + private InputAttemptIdentifier inputAttemptIdentifier; + private final FetchedInputCallback callback; + private final int id; + private byte state; + + protected FetchedInput(InputAttemptIdentifier inputAttemptIdentifier, FetchedInputCallback callbackHandler) { - this.type = type; - this.actualSize = actualSize; - this.compressedSize = compressedSize; this.inputAttemptIdentifier = inputAttemptIdentifier; this.callback = callbackHandler; this.id = ID_GEN.getAndIncrement(); - this.state = State.PENDING; + this.state = (byte) State.PENDING.ordinal(); } - public Type getType() { - return this.type; + public abstract Type getType(); + + protected boolean isState(State state) { + return this.state == (byte) state.ordinal(); } - public long getActualSize() { - return this.actualSize; + protected void setState(State state) { + this.state = (byte) state.ordinal(); } - - public long getCompressedSize() { - return this.compressedSize; + + protected State getState() { + if (isState(State.PENDING)) { + return State.PENDING; + } + if (isState(State.COMMITTED)) { + return State.COMMITTED; + } + if (isState(State.ABORTED)) { + return State.ABORTED; + } + if (isState(State.FREED)) { + return State.FREED; + } + // Should not get here + return null; } + protected int getId() { + return this.id; + } + + public abstract long getSize(); + public InputAttemptIdentifier getInputAttemptIdentifier() { return this.inputAttemptIdentifier; } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java index bf8c83b98e..9f657e5e9f 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java @@ -658,7 +658,7 @@ private HostFetchResult doLocalDiskFetch(boolean failMissing) { idxRecord = getTezIndexRecord(srcAttemptId, reduceId); fetchedInput = new LocalDiskFetchedInput(idxRecord.getStartOffset(), - idxRecord.getRawLength(), idxRecord.getPartLength(), srcAttemptId, + idxRecord.getPartLength(), srcAttemptId, getShuffleInputFileName(srcAttemptId.getPathComponent(), null), conf, new FetchedInputCallback() { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/LocalDiskFetchedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/LocalDiskFetchedInput.java index 5c6396156b..0ae8f08c19 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/LocalDiskFetchedInput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/LocalDiskFetchedInput.java @@ -39,17 +39,29 @@ public class LocalDiskFetchedInput extends FetchedInput { private final Path inputFile; private final FileSystem localFS; private final long startOffset; + private final long size; - public LocalDiskFetchedInput(long startOffset, long actualSize, long compressedSize, + public LocalDiskFetchedInput(long startOffset, long compressedSize, InputAttemptIdentifier inputAttemptIdentifier, Path inputFile, Configuration conf, FetchedInputCallback callbackHandler) throws IOException { - super(Type.DISK_DIRECT, actualSize, compressedSize, inputAttemptIdentifier, callbackHandler); + super(inputAttemptIdentifier, callbackHandler); + this.size = compressedSize; this.startOffset = startOffset; this.inputFile = inputFile; localFS = FileSystem.getLocal(conf); } + @Override + public Type getType() { + return Type.DISK_DIRECT; + } + + @Override + public long getSize() { + return size; + } + @Override public OutputStream getOutputStream() throws IOException { throw new IOException("Output Stream is not supported for " + this.toString()); @@ -59,21 +71,21 @@ public OutputStream getOutputStream() throws IOException { public InputStream getInputStream() throws IOException { FSDataInputStream inputStream = localFS.open(inputFile); inputStream.seek(startOffset); - return new BoundedInputStream(inputStream, compressedSize); + return new BoundedInputStream(inputStream, getSize()); } @Override public void commit() { - if (state == State.PENDING) { - state = State.COMMITTED; + if (isState(State.PENDING)) { + setState(State.COMMITTED); notifyFetchComplete(); } } @Override public void abort() { - if (state == State.PENDING) { - state = State.ABORTED; + if (isState(State.PENDING)) { + setState(State.ABORTED); notifyFetchFailure(); } } @@ -81,10 +93,10 @@ public void abort() { @Override public void free() { Preconditions.checkState( - state == State.COMMITTED || state == State.ABORTED, + isState(State.COMMITTED) || isState(State.ABORTED), "FetchedInput can only be freed after it is committed or aborted"); - if (state == State.COMMITTED) { // ABORTED would have already called cleanup - state = State.FREED; + if (isState(State.COMMITTED)) { // ABORTED would have already called cleanup + setState(State.FREED); notifyFreedResource(); } } @@ -93,12 +105,11 @@ public void free() { public String toString() { return "LocalDiskFetchedInput [inputFile path =" + inputFile + ", offset" + startOffset + - ", actualSize=" + actualSize + - ", compressedSize=" + compressedSize + - ", inputAttemptIdentifier=" + inputAttemptIdentifier + - ", type=" + type + - ", id=" + id + - ", state=" + state + "]"; + ", compressedSize=" + getSize() + + ", inputAttemptIdentifier=" + getInputAttemptIdentifier() + + ", type=" + getType() + + ", id=" + getId() + + ", state=" + getState() + "]"; } @VisibleForTesting diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/MemoryFetchedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/MemoryFetchedInput.java index 78f1f3b596..d0c3e771b7 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/MemoryFetchedInput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/MemoryFetchedInput.java @@ -29,41 +29,54 @@ public class MemoryFetchedInput extends FetchedInput { - private BoundedByteArrayOutputStream byteStream; + private byte[] byteArray; - public MemoryFetchedInput(long actualSize, long compressedSize, + public MemoryFetchedInput(long actualSize, InputAttemptIdentifier inputAttemptIdentifier, FetchedInputCallback callbackHandler) { - super(Type.MEMORY, actualSize, compressedSize, inputAttemptIdentifier, callbackHandler); - this.byteStream = new BoundedByteArrayOutputStream((int) actualSize); + super(inputAttemptIdentifier, callbackHandler); + this.byteArray = new byte[(int) actualSize]; + } + + @Override + public Type getType() { + return Type.MEMORY; + } + + @Override + public long getSize() { + if (this.byteArray == null) { + return 0; + } + return this.byteArray.length; } @Override public OutputStream getOutputStream() { - return byteStream; + return new InMemoryBoundedByteArrayOutputStream(byteArray); } @Override public InputStream getInputStream() { - return new NonSyncByteArrayInputStream(byteStream.getBuffer()); + return new NonSyncByteArrayInputStream(byteArray); } public byte[] getBytes() { - return byteStream.getBuffer(); + return byteArray; } @Override public void commit() { - if (state == State.PENDING) { - state = State.COMMITTED; + if (isState(State.PENDING)) { + setState(State.COMMITTED); notifyFetchComplete(); } } @Override public void abort() { - if (state == State.PENDING) { - state = State.ABORTED; + if (isState(State.PENDING)) { + setState(State.ABORTED); notifyFetchFailure(); } } @@ -71,20 +84,28 @@ public void abort() { @Override public void free() { Preconditions.checkState( - state == State.COMMITTED || state == State.ABORTED, + isState(State.COMMITTED) || isState(State.ABORTED), "FetchedInput can only be freed after it is committed or aborted"); - if (state == State.COMMITTED) { // ABORTED would have already called cleanup - state = State.FREED; - this.byteStream = null; + if (isState(State.COMMITTED)) { // ABORTED would have already called cleanup + setState(State.FREED); notifyFreedResource(); + // Set this to null AFTER notifyFreedResource() so that getSize() + // returns the correct size + this.byteArray = null; } } @Override public String toString() { return "MemoryFetchedInput [inputAttemptIdentifier=" - + inputAttemptIdentifier + ", actualSize=" + actualSize - + ", compressedSize=" + compressedSize + ", type=" + type + ", id=" - + id + ", state=" + state + "]"; + + getInputAttemptIdentifier() + ", size=" + getSize() + + ", type=" + getType() + ", id=" + + getId() + ", state=" + getState() + "]"; + } + + private static class InMemoryBoundedByteArrayOutputStream extends BoundedByteArrayOutputStream { + InMemoryBoundedByteArrayOutputStream(byte[] array) { + super(array, 0, array.length); + } } } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java index 24fb12befb..e142228546 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java @@ -914,7 +914,17 @@ public float getNumCompletedInputsFloat() { static class NullFetchedInput extends FetchedInput { public NullFetchedInput(InputAttemptIdentifier inputAttemptIdentifier) { - super(Type.MEMORY, -1, -1, inputAttemptIdentifier, null); + super(inputAttemptIdentifier, null); + } + + @Override + public Type getType() { + return Type.MEMORY; + } + + @Override + public long getSize() { + return -1; } @Override diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/SimpleFetchedInputAllocator.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/SimpleFetchedInputAllocator.java index f939cd1674..6072c039c1 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/SimpleFetchedInputAllocator.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/SimpleFetchedInputAllocator.java @@ -140,7 +140,7 @@ public synchronized FetchedInput allocate(long actualSize, long compressedSize, InputAttemptIdentifier inputAttemptIdentifier) throws IOException { if (actualSize > maxSingleShuffleLimit || this.usedMemory + actualSize > this.memoryLimit) { - return new DiskFetchedInput(actualSize, compressedSize, + return new DiskFetchedInput(compressedSize, inputAttemptIdentifier, this, conf, localDirAllocator, fileNameAllocator); } else { @@ -149,7 +149,7 @@ public synchronized FetchedInput allocate(long actualSize, long compressedSize, LOG.info(srcNameTrimmed + ": " + "Used memory after allocating " + actualSize + " : " + usedMemory); } - return new MemoryFetchedInput(actualSize, compressedSize, inputAttemptIdentifier, this); + return new MemoryFetchedInput(actualSize, inputAttemptIdentifier, this); } } @@ -160,7 +160,7 @@ public synchronized FetchedInput allocateType(Type type, long actualSize, switch (type) { case DISK: - return new DiskFetchedInput(actualSize, compressedSize, + return new DiskFetchedInput(compressedSize, inputAttemptIdentifier, this, conf, localDirAllocator, fileNameAllocator); default: @@ -197,7 +197,7 @@ private void cleanup(FetchedInput fetchedInput) { case DISK: break; case MEMORY: - unreserve(fetchedInput.getActualSize()); + unreserve(((MemoryFetchedInput) fetchedInput).getSize()); break; default: throw new TezUncheckedException("InputType: " + fetchedInput.getType() diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/readers/TestUnorderedKVReader.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/readers/TestUnorderedKVReader.java index c49a423883..6fef9440dd 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/readers/TestUnorderedKVReader.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/readers/TestUnorderedKVReader.java @@ -96,7 +96,7 @@ private void setupReader() throws IOException, InterruptedException { createIFile(outputPath, 1); final LinkedList inputs = new LinkedList(); - LocalDiskFetchedInput realFetchedInput = new LocalDiskFetchedInput(0, rawLen, compLen, new + LocalDiskFetchedInput realFetchedInput = new LocalDiskFetchedInput(0, compLen, new InputAttemptIdentifier(0, 0), outputPath, defaultConf, new FetchedInputCallback() { @Override public void fetchComplete(FetchedInput fetchedInput) { diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java index b0311541ac..db9c7afad0 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java @@ -246,8 +246,7 @@ protected void verifyFetchSucceeded(FetcherCallback callback, CompositeInputAtte SHUFFLE_INPUT_FILE_PREFIX + pathComponent); Assert.assertTrue("success callback fs", f.getLocalFS() instanceof LocalFileSystem); Assert.assertEquals("success callback filesystem", f.getStartOffset(), p * 10); - Assert.assertEquals("success callback raw size", f.getActualSize(), p * 1000); - Assert.assertEquals("success callback compressed size", f.getCompressedSize(), p * 100); + Assert.assertEquals("success callback compressed size", f.getSize(), p * 100); Assert.assertEquals("success callback input id", f.getInputAttemptIdentifier(), srcAttempId.expand(0)); Assert.assertEquals("success callback type", f.getType(), FetchedInput.Type.DISK_DIRECT); } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java index 23248edf46..103f83d3cc 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java @@ -298,7 +298,17 @@ boolean isFetcherExecutorShutdown() { static class TestFetchedInput extends FetchedInput { public TestFetchedInput(InputAttemptIdentifier inputAttemptIdentifier) { - super(Type.MEMORY, -1, -1, inputAttemptIdentifier, null); + super(inputAttemptIdentifier, null); + } + + @Override + public long getSize() { + return -1; + } + + @Override + public Type getType() { + return Type.MEMORY; } @Override From 823b1bb3b3ab034639bfb693ef83baa18dfde34b Mon Sep 17 00:00:00 2001 From: Siddharth Seth Date: Mon, 14 Aug 2017 14:21:11 -0700 Subject: [PATCH 005/512] TEZ-3816. Add ability to automatically speculate single-task vertices. Contributed by Muhammad Samir Khan. --- .../apache/tez/dag/api/TezConfiguration.java | 11 ++ .../speculation/legacy/LegacySpeculator.java | 100 +++++++++++------- .../apache/tez/dag/app/TestSpeculation.java | 51 ++++++++- 3 files changed, 124 insertions(+), 38 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 39688d68bc..5df5259910 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -526,6 +526,17 @@ public TezConfiguration(boolean loadDefaults) { public static final String TEZ_AM_LEGACY_SPECULATIVE_SLOWTASK_THRESHOLD = TEZ_AM_PREFIX + "legacy.speculative.slowtask.threshold"; + /** + * Long value. Specifies the timeout after which tasks on a single task vertex must be speculated. + * A negative value means not to use timeout for speculation of single task vertices. + */ + @Unstable + @ConfigurationScope(Scope.AM) + @ConfigurationProperty(type="long") + public static final String TEZ_AM_LEGACY_SPECULATIVE_SINGLE_TASK_VERTEX_TIMEOUT = + TEZ_AM_PREFIX + "legacy.speculative.single.task.vertex.timeout"; + public static final long TEZ_AM_LEGACY_SPECULATIVE_SINGLE_TASK_VERTEX_TIMEOUT_DEFAULT = -1; + /** * Int value. Upper limit on the number of threads user to launch containers in the app * master. Expert level setting. diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java index dd54d8662b..9fbea19f03 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java @@ -24,6 +24,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import org.apache.tez.dag.api.TezConfiguration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -67,6 +68,7 @@ public class LegacySpeculator { private static final double PROPORTION_RUNNING_TASKS_SPECULATABLE = 0.1; private static final double PROPORTION_TOTAL_TASKS_SPECULATABLE = 0.01; private static final int MINIMUM_ALLOWED_SPECULATIVE_TASKS = 10; + private static final int VERTEX_SIZE_THRESHOLD_FOR_TIMEOUT_SPECULATION = 1; private static final Logger LOG = LoggerFactory.getLogger(LegacySpeculator.class); @@ -88,7 +90,7 @@ public class LegacySpeculator { private Vertex vertex; private TaskRuntimeEstimator estimator; - + private final long taskTimeout; private final Clock clock; private long nextSpeculateTime = Long.MIN_VALUE; @@ -116,6 +118,9 @@ public LegacySpeculator(Configuration conf, Clock clock, Vertex vertex) { this.vertex = vertex; this.estimator = estimator; this.clock = clock; + taskTimeout = conf.getLong( + TezConfiguration.TEZ_AM_LEGACY_SPECULATIVE_SINGLE_TASK_VERTEX_TIMEOUT, + TezConfiguration.TEZ_AM_LEGACY_SPECULATIVE_SINGLE_TASK_VERTEX_TIMEOUT_DEFAULT); } /* ************************************************************* */ @@ -209,7 +214,12 @@ public void handle(SpeculatorEvent event) { // // All of these values are negative. Any value that should be allowed to // speculate is 0 or positive. - private long speculationValue(Task task, long now) { + // + // If shouldUseTimeout is true, we will use timeout to decide on + // speculation instead of the task statistics. This can be useful, for + // example for single task vertices for which there are no tasks to compare + // with + private long speculationValue(Task task, long now, boolean shouldUseTimeout) { Map attempts = task.getAttempts(); TezTaskID taskID = task.getTaskId(); long acceptableRuntime = Long.MIN_VALUE; @@ -220,7 +230,7 @@ private long speculationValue(Task task, long now) { return NOT_RUNNING; } - if (!mayHaveSpeculated.contains(taskID)) { + if (!mayHaveSpeculated.contains(taskID) && !shouldUseTimeout) { acceptableRuntime = estimator.thresholdRuntime(taskID); if (acceptableRuntime == Long.MAX_VALUE) { return ON_SCHEDULE; @@ -239,8 +249,6 @@ private long speculationValue(Task task, long now) { } runningTaskAttemptID = taskAttempt.getID(); - long estimatedRunTime = estimator.estimatedRuntime(runningTaskAttemptID); - long taskAttemptStartTime = estimator.attemptEnrolledTime(runningTaskAttemptID); if (taskAttemptStartTime > now) { @@ -249,43 +257,57 @@ private long speculationValue(Task task, long now) { return TOO_NEW; } - long estimatedEndTime = estimatedRunTime + taskAttemptStartTime; + if (shouldUseTimeout) { + if ((now - taskAttemptStartTime) > taskTimeout) { + // If the task has timed out, then we want to schedule a speculation + // immediately. However we cannot return immediately since we may + // already have a speculation running. + result = Long.MAX_VALUE; + } else { + // Task has not timed out so we are good + return ON_SCHEDULE; + } + } else { + long estimatedRunTime = estimator.estimatedRuntime(runningTaskAttemptID); - long estimatedReplacementEndTime - = now + estimator.newAttemptEstimatedRuntime(); + long estimatedEndTime = estimatedRunTime + taskAttemptStartTime; - float progress = taskAttempt.getProgress(); - TaskAttemptHistoryStatistics data = - runningTaskAttemptStatistics.get(runningTaskAttemptID); - if (data == null) { - runningTaskAttemptStatistics.put(runningTaskAttemptID, - new TaskAttemptHistoryStatistics(estimatedRunTime, progress, now)); - } else { - if (estimatedRunTime == data.getEstimatedRunTime() - && progress == data.getProgress()) { - // Previous stats are same as same stats - if (data.notHeartbeatedInAWhile(now)) { - // Stats have stagnated for a while, simulate heart-beat. - // Now simulate the heart-beat - statusUpdate(taskAttempt.getID(), taskAttempt.getState(), clock.getTime()); - } + long estimatedReplacementEndTime + = now + estimator.newAttemptEstimatedRuntime(); + + float progress = taskAttempt.getProgress(); + TaskAttemptHistoryStatistics data = + runningTaskAttemptStatistics.get(runningTaskAttemptID); + if (data == null) { + runningTaskAttemptStatistics.put(runningTaskAttemptID, + new TaskAttemptHistoryStatistics(estimatedRunTime, progress, now)); } else { - // Stats have changed - update our data structure - data.setEstimatedRunTime(estimatedRunTime); - data.setProgress(progress); - data.resetHeartBeatTime(now); + if (estimatedRunTime == data.getEstimatedRunTime() + && progress == data.getProgress()) { + // Previous stats are same as same stats + if (data.notHeartbeatedInAWhile(now)) { + // Stats have stagnated for a while, simulate heart-beat. + // Now simulate the heart-beat + statusUpdate(taskAttempt.getID(), taskAttempt.getState(), clock.getTime()); + } + } else { + // Stats have changed - update our data structure + data.setEstimatedRunTime(estimatedRunTime); + data.setProgress(progress); + data.resetHeartBeatTime(now); + } } - } - if (estimatedEndTime < now) { - return PROGRESS_IS_GOOD; - } + if (estimatedEndTime < now) { + return PROGRESS_IS_GOOD; + } - if (estimatedReplacementEndTime >= estimatedEndTime) { - return TOO_LATE_TO_SPECULATE; - } + if (estimatedReplacementEndTime >= estimatedEndTime) { + return TOO_LATE_TO_SPECULATE; + } - result = estimatedEndTime - estimatedReplacementEndTime; + result = estimatedEndTime - estimatedReplacementEndTime; + } } } @@ -296,7 +318,7 @@ private long speculationValue(Task task, long now) { - if (acceptableRuntime == Long.MIN_VALUE) { + if ((acceptableRuntime == Long.MIN_VALUE) && !shouldUseTimeout) { acceptableRuntime = estimator.thresholdRuntime(taskID); if (acceptableRuntime == Long.MAX_VALUE) { return ON_SCHEDULE; @@ -329,11 +351,15 @@ private int maybeScheduleASpeculation() { TezTaskID bestTaskID = null; long bestSpeculationValue = -1L; + boolean shouldUseTimeout = + (tasks.size() <= VERTEX_SIZE_THRESHOLD_FOR_TIMEOUT_SPECULATION) && + (taskTimeout >= 0); // this loop is potentially pricey. // TODO track the tasks that are potentially worth looking at for (Map.Entry taskEntry : tasks.entrySet()) { - long mySpeculationValue = speculationValue(taskEntry.getValue(), now); + long mySpeculationValue = speculationValue(taskEntry.getValue(), now, + shouldUseTimeout); if (mySpeculationValue == ALREADY_SPECULATING) { ++numberSpeculationsAlready; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java index 9a39facbe0..1df5af4fa3 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java @@ -19,6 +19,8 @@ package org.apache.tez.dag.app; import java.io.IOException; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.hadoop.conf.Configuration; @@ -97,7 +99,54 @@ void syncWithMockAppLauncher(boolean allowScheduling, AtomicBoolean mockAppLaunc mockAppLauncherGoFlag.notify(); } } - + + @Test (timeout = 10000) + public void testSingleTaskSpeculation() throws Exception { + // Map + Map confToExpected = new HashMap(); + confToExpected.put(Long.MAX_VALUE >> 1, 1); // Really long time to speculate + confToExpected.put(100L, 2); + confToExpected.put(-1L, 1); // Don't speculate + + for(Map.Entry entry : confToExpected.entrySet()) { + defaultConf.setLong( + TezConfiguration.TEZ_AM_LEGACY_SPECULATIVE_SINGLE_TASK_VERTEX_TIMEOUT, + entry.getKey()); + DAG dag = DAG.create("test"); + Vertex vA = Vertex.create("A", + ProcessorDescriptor.create("Proc.class"), + 1); + dag.addVertex(vA); + + MockTezClient tezClient = createTezSession(); + + DAGClient dagClient = tezClient.submitDAG(dag); + DAGImpl dagImpl = (DAGImpl) mockApp.getContext().getCurrentDAG(); + TezVertexID vertexId = TezVertexID.getInstance(dagImpl.getID(), 0); + // original attempt is killed and speculative one is successful + TezTaskAttemptID killedTaId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 0); + TezTaskAttemptID successTaId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 1); + + Thread.sleep(200); + // cause speculation trigger + mockLauncher.setStatusUpdatesForTask(killedTaId, 100); + + mockLauncher.startScheduling(true); + dagClient.waitForCompletion(); + Assert.assertEquals(DAGStatus.State.SUCCEEDED, dagClient.getDAGStatus(null).getState()); + Task task = dagImpl.getTask(killedTaId.getTaskID()); + Assert.assertEquals(entry.getValue().intValue(), task.getAttempts().size()); + if (entry.getValue() > 1) { + Assert.assertEquals(successTaId, task.getSuccessfulAttempt().getID()); + TaskAttempt killedAttempt = task.getAttempt(killedTaId); + Joiner.on(",").join(killedAttempt.getDiagnostics()).contains("Killed as speculative attempt"); + Assert.assertEquals(TaskAttemptTerminationCause.TERMINATED_EFFECTIVE_SPECULATION, + killedAttempt.getTerminationCause()); + } + tezClient.stop(); + } + } + public void testBasicSpeculation(boolean withProgress) throws Exception { DAG dag = DAG.create("test"); Vertex vA = Vertex.create("A", ProcessorDescriptor.create("Proc.class"), 5); From b04e7fce7ff61bb31b06919da7298aa3a04e1c5f Mon Sep 17 00:00:00 2001 From: Zhiyuan Yang Date: Fri, 25 Aug 2017 16:47:01 -0700 Subject: [PATCH 006/512] TEZ-3431. Add unit tests for container release (Taklon Stephen Wu via zhiyuany) --- .../tez/dag/app/rm/TestTaskScheduler.java | 123 ++++++++++-------- 1 file changed, 66 insertions(+), 57 deletions(-) diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java index 16c560e946..1a647b1f2b 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java @@ -89,6 +89,10 @@ public class TestTaskScheduler { static ContainerSignatureMatcher containerSignatureMatcher = new AlwaysMatchesContainerMatcher(); private ExecutorService contextCallbackExecutor; + private static final String DEFAULT_APP_HOST = "host"; + private static final String DEFAULT_APP_URL = "url"; + private static final String SUCCEED_APP_MESSAGE = "success"; + private static final int DEFAULT_APP_PORT = 0; @BeforeClass public static void beforeClass() { @@ -122,16 +126,12 @@ public void testTaskSchedulerNoReuse() throws Exception { AMRMClientAsyncForTest mockRMClient = spy( new AMRMClientAsyncForTest(new AMRMClientForTest(), 100)); - String appHost = "host"; - int appPort = 0; - String appUrl = "url"; - Configuration conf = new Configuration(); conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, false); int interval = 100; conf.setInt(TezConfiguration.TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX, interval); - TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf); + TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(DEFAULT_APP_HOST, DEFAULT_APP_PORT, DEFAULT_APP_URL, conf); TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp); TaskSchedulerWithDrainableContext scheduler = @@ -146,7 +146,7 @@ public void testTaskSchedulerNoReuse() throws Exception { scheduler.start(); drainableAppCallback.drain(); verify(mockRMClient).start(); - verify(mockRMClient).registerApplicationMaster(appHost, appPort, appUrl); + verify(mockRMClient).registerApplicationMaster(DEFAULT_APP_HOST, DEFAULT_APP_PORT, DEFAULT_APP_URL); RegisterApplicationMasterResponse regResponse = mockRMClient.getRegistrationResponse(); verify(mockApp).setApplicationRegistrationData(regResponse.getMaximumResourceCapability(), regResponse.getApplicationACLs(), @@ -372,23 +372,19 @@ public void testTaskSchedulerNoReuse() throws Exception { drainableAppCallback.drain(); verify(mockApp).appShutdownRequested(); - String appMsg = "success"; AppFinalStatus finalStatus = - new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl); + new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, SUCCEED_APP_MESSAGE, DEFAULT_APP_URL); when(mockApp.getFinalAppStatus()).thenReturn(finalStatus); scheduler.shutdown(); drainableAppCallback.drain(); verify(mockRMClient). unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, - appMsg, appUrl); + SUCCEED_APP_MESSAGE, DEFAULT_APP_URL); verify(mockRMClient).stop(); } @Test(timeout=10000) public void testTaskSchedulerInitiateStop() throws Exception { - String appHost = "host"; - int appPort = 0; - String appUrl = "url"; Configuration conf = new Configuration(); conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 0); @@ -396,7 +392,7 @@ public void testTaskSchedulerInitiateStop() throws Exception { conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, 10000); conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, 10000); - TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf); + TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(DEFAULT_APP_HOST, DEFAULT_APP_PORT, DEFAULT_APP_URL, conf); final TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp); TezAMRMClientAsync mockRMClient = spy( @@ -506,10 +502,6 @@ public void testTaskSchedulerWithReuse() throws Exception { TezAMRMClientAsync mockRMClient = spy( new AMRMClientAsyncForTest(new AMRMClientForTest(), 100)); - String appHost = "host"; - int appPort = 0; - String appUrl = "url"; - Configuration conf = new Configuration(); // to match all in the same pass conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 0); @@ -517,7 +509,7 @@ public void testTaskSchedulerWithReuse() throws Exception { conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, 0); conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, 0); - TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf); + TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(DEFAULT_APP_HOST, DEFAULT_APP_PORT, DEFAULT_APP_URL, conf); final TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp); TaskSchedulerWithDrainableContext scheduler = @@ -795,15 +787,14 @@ public void testTaskSchedulerWithReuse() throws Exception { drainableAppCallback.drain(); verify(mockApp).appShutdownRequested(); - String appMsg = "success"; AppFinalStatus finalStatus = - new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl); + new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, SUCCEED_APP_MESSAGE, DEFAULT_APP_URL); when(mockApp.getFinalAppStatus()).thenReturn(finalStatus); scheduler.shutdown(); drainableAppCallback.drain(); verify(mockRMClient). unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, - appMsg, appUrl); + SUCCEED_APP_MESSAGE, DEFAULT_APP_URL); verify(mockRMClient).stop(); } @@ -812,12 +803,8 @@ public void testTaskSchedulerDetermineMinHeldContainers() throws Exception { TezAMRMClientAsync mockRMClient = spy( new AMRMClientAsyncForTest(new AMRMClientForTest(), 100)); - String appHost = "host"; - int appPort = 0; - String appUrl = "url"; - - TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, true, - new Configuration()); + TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(DEFAULT_APP_HOST, DEFAULT_APP_PORT, DEFAULT_APP_URL, + true, new Configuration()); final TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp); TaskSchedulerWithDrainableContext scheduler = @@ -959,9 +946,48 @@ public void testTaskSchedulerDetermineMinHeldContainers() throws Exception { verify(mockRMClient, times(2)).releaseAssignedContainer((ContainerId)any()); Assert.assertEquals(5, scheduler.heldContainers.size()); - String appMsg = "success"; AppFinalStatus finalStatus = - new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl); + new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, SUCCEED_APP_MESSAGE, DEFAULT_APP_URL); + when(mockApp.getFinalAppStatus()).thenReturn(finalStatus); + scheduler.shutdown(); + } + + @Test (timeout=3000) + public void testTaskSchedulerHeldContainersReleaseAfterExpired() throws Exception { + final TezAMRMClientAsync mockRMClient = spy( + new AMRMClientAsyncForTest(new AMRMClientForTest(), 100)); + final TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(DEFAULT_APP_HOST, DEFAULT_APP_PORT, + DEFAULT_APP_URL, true, new Configuration()); + final TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp); + final TaskSchedulerWithDrainableContext scheduler = + new TaskSchedulerWithDrainableContext(drainableAppCallback, mockRMClient); + + scheduler.initialize(); + scheduler.start(); + + ApplicationAttemptId appId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 0), 0); + ContainerId containerId = ContainerId.newInstance(appId, 0); + Container c1 = mock(Container.class, RETURNS_DEEP_STUBS); + when(c1.getNodeId().getHost()).thenReturn(""); // we are mocking directly + + HeldContainer hc1 = mock(HeldContainer.class); + when(c1.getId()).thenReturn(containerId); + when(hc1.getContainer()).thenReturn(c1); + when(hc1.isNew()).thenReturn(false); + + // containerExpiryTime = 0 + scheduler.heldContainers.put(containerId, hc1); + + long currTime = System.currentTimeMillis(); + scheduler.delayedContainerManager.addDelayedContainer(hc1.getContainer(), currTime); + // sleep and wait for mainLoop() check-in to release this expired held container. + Thread.sleep(1000); + + verify(mockRMClient, times(1)).releaseAssignedContainer((ContainerId)any()); + Assert.assertEquals(0, scheduler.heldContainers.size()); + + AppFinalStatus finalStatus = + new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, SUCCEED_APP_MESSAGE, DEFAULT_APP_URL); when(mockApp.getFinalAppStatus()).thenReturn(finalStatus); scheduler.shutdown(); } @@ -971,10 +997,6 @@ public void testTaskSchedulerRandomReuseExpireTime() throws Exception { TezAMRMClientAsync mockRMClient = spy( new AMRMClientAsyncForTest(new AMRMClientForTest(), 100)); - String appHost = "host"; - int appPort = 0; - String appUrl = "url"; - long minTime = 1000l; long maxTime = 100000l; Configuration conf1 = new Configuration(); @@ -985,8 +1007,8 @@ public void testTaskSchedulerRandomReuseExpireTime() throws Exception { conf2.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, minTime); conf2.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, maxTime); - TaskSchedulerContext mockApp1 = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf1); - TaskSchedulerContext mockApp2 = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf2); + TaskSchedulerContext mockApp1 = setupMockTaskSchedulerContext(DEFAULT_APP_HOST, DEFAULT_APP_PORT, DEFAULT_APP_URL, conf1); + TaskSchedulerContext mockApp2 = setupMockTaskSchedulerContext(DEFAULT_APP_HOST, DEFAULT_APP_PORT, DEFAULT_APP_URL, conf2); final TaskSchedulerContextDrainable drainableAppCallback1 = createDrainableContext(mockApp1); final TaskSchedulerContextDrainable drainableAppCallback2 = createDrainableContext(mockApp2); @@ -1018,9 +1040,8 @@ public void testTaskSchedulerRandomReuseExpireTime() throws Exception { lastExpireTime = currExpireTime; } - String appMsg = "success"; AppFinalStatus finalStatus = - new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl); + new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, SUCCEED_APP_MESSAGE, DEFAULT_APP_URL); when(mockApp1.getFinalAppStatus()).thenReturn(finalStatus); when(mockApp2.getFinalAppStatus()).thenReturn(finalStatus); scheduler1.shutdown(); @@ -1033,16 +1054,12 @@ public void testTaskSchedulerPreemption() throws Exception { TezAMRMClientAsync mockRMClient = mock(TezAMRMClientAsync.class); - String appHost = "host"; - int appPort = 0; - String appUrl = "url"; - Configuration conf = new Configuration(); conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, false); conf.setInt(TezConfiguration.TEZ_AM_PREEMPTION_HEARTBEATS_BETWEEN_PREEMPTIONS, 3); - TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, false, - null, null, new PreemptionMatcher(), conf); + TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(DEFAULT_APP_HOST, DEFAULT_APP_PORT, DEFAULT_APP_URL, + false, null, null, new PreemptionMatcher(), conf); final TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp); final TaskSchedulerWithDrainableContext scheduler = @@ -1331,7 +1348,7 @@ public Object answer(InvocationOnMock invocation) { verify(mockRMClient, times(1)).releaseAssignedContainer(mockCId3A); AppFinalStatus finalStatus = - new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, "", appUrl); + new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, "", DEFAULT_APP_URL); when(mockApp.getFinalAppStatus()).thenReturn(finalStatus); scheduler.shutdown(); drainableAppCallback.drain(); @@ -1342,10 +1359,6 @@ public void testTaskSchedulerPreemption2() throws Exception { TezAMRMClientAsync mockRMClient = spy( new AMRMClientAsyncForTest(new AMRMClientForTest(), 100)); - String appHost = "host"; - int appPort = 0; - String appUrl = "url"; - int waitTime = 1000; Configuration conf = new Configuration(); @@ -1353,8 +1366,8 @@ public void testTaskSchedulerPreemption2() throws Exception { conf.setInt(TezConfiguration.TEZ_AM_PREEMPTION_HEARTBEATS_BETWEEN_PREEMPTIONS, 2); conf.setInt(TezConfiguration.TEZ_AM_PREEMPTION_MAX_WAIT_TIME_MS, waitTime); - TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, false, - null, null, new PreemptionMatcher(), conf); + TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(DEFAULT_APP_HOST, DEFAULT_APP_PORT, DEFAULT_APP_URL, + false, null, null, new PreemptionMatcher(), conf); final TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp); final TaskSchedulerWithDrainableContext scheduler = @@ -1491,7 +1504,7 @@ public Object answer(InvocationOnMock invocation) { Assert.assertTrue(oldStartWaitTime < scheduler.highestWaitingRequestWaitStartTime); AppFinalStatus finalStatus = - new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, "", appUrl); + new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, "", DEFAULT_APP_URL); when(mockApp.getFinalAppStatus()).thenReturn(finalStatus); scheduler.shutdown(); drainableAppCallback.drain(); @@ -1505,7 +1518,7 @@ public void testLocalityMatching() throws Exception { Configuration conf = new Configuration(); conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, false); - TaskSchedulerContext appClient = setupMockTaskSchedulerContext("host", 0, "", conf); + TaskSchedulerContext appClient = setupMockTaskSchedulerContext(DEFAULT_APP_HOST, DEFAULT_APP_PORT, "", conf); final TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(appClient); TaskSchedulerWithDrainableContext taskScheduler = @@ -1611,10 +1624,6 @@ public void testContainerExpired() throws Exception { TezAMRMClientAsync mockRMClient = spy( new AMRMClientAsyncForTest(new AMRMClientForTest(), 100)); - String appHost = "host"; - int appPort = 0; - String appUrl = "url"; - Configuration conf = new Configuration(); // to match all in the same pass conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 0); @@ -1622,7 +1631,7 @@ public void testContainerExpired() throws Exception { conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, 0); conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, 0); - TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf); + TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(DEFAULT_APP_HOST, DEFAULT_APP_PORT, DEFAULT_APP_URL, conf); final TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp); TaskSchedulerWithDrainableContext scheduler = new TaskSchedulerWithDrainableContext(drainableAppCallback, mockRMClient); From 287ef0e35c908d0ab95415e4bee7e863e3a85af8 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Fri, 1 Sep 2017 10:29:48 -0500 Subject: [PATCH 007/512] TEZ-3825. Tez UI DAGs page can't query RUNNING or SUBMITTED apps (jeagles) --- tez-ui/src/main/webapp/app/adapters/timeline.js | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tez-ui/src/main/webapp/app/adapters/timeline.js b/tez-ui/src/main/webapp/app/adapters/timeline.js index 605b0cb152..2de6eacafd 100644 --- a/tez-ui/src/main/webapp/app/adapters/timeline.js +++ b/tez-ui/src/main/webapp/app/adapters/timeline.js @@ -72,7 +72,7 @@ export default AbstractAdapter.extend({ var filter = this.get(`filters.${key}`); if(filter) { - if(!primaryFilter) { + if(!primaryFilter && !(filter === 'status' && value === 'RUNNING')) { primaryFilter = {}; primaryFilter[filter] = value; } From 9fd0578ccb315cc5b591c57635e889279a376f60 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Wed, 6 Sep 2017 16:10:04 -0500 Subject: [PATCH 008/512] TEZ-3827. TEZ Vertex status on DAG index page shows SUCCEEDED for SUCCEEDED_WITH_FAILURES final status (jeagles) --- tez-ui/src/main/webapp/app/controllers/dag/index/index.js | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tez-ui/src/main/webapp/app/controllers/dag/index/index.js b/tez-ui/src/main/webapp/app/controllers/dag/index/index.js index eb4aa040f7..c9adde4c6b 100644 --- a/tez-ui/src/main/webapp/app/controllers/dag/index/index.js +++ b/tez-ui/src/main/webapp/app/controllers/dag/index/index.js @@ -37,7 +37,7 @@ export default MultiTableController.extend({ },{ id: 'status', headerTitle: 'Status', - contentPath: 'status', + contentPath: 'finalStatus', cellComponentName: 'em-table-status-cell', observePath: true },{ From 7e895f54bb3f4e16d7d76970b602ef3c59271bd8 Mon Sep 17 00:00:00 2001 From: Zhiyuan Yang Date: Tue, 12 Sep 2017 13:39:55 -0700 Subject: [PATCH 009/512] TEZ-3828. Allow relaxing locality when retried task's priority is kept same (zhiyuany) --- .../org/apache/tez/dag/api/TezConfiguration.java | 12 +++++++++++- .../main/java/org/apache/tez/dag/app/dag/Vertex.java | 1 + .../apache/tez/dag/app/dag/impl/TaskAttemptImpl.java | 2 +- .../org/apache/tez/dag/app/dag/impl/VertexImpl.java | 9 +++++++++ .../apache/tez/dag/app/dag/impl/TestTaskAttempt.java | 9 +++++++++ 5 files changed, 31 insertions(+), 2 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 5df5259910..efe6d6c47b 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -593,7 +593,7 @@ public TezConfiguration(boolean loadDefaults) { /** * Boolean value. Specifies whether a re-scheduled attempt of a task, caused by previous - * failures gets special treatment - higher priority, dropped location hints. + * failures gets higher priority */ @ConfigurationScope(Scope.VERTEX) @ConfigurationProperty(type="boolean") @@ -601,6 +601,16 @@ public TezConfiguration(boolean loadDefaults) { TEZ_AM_PREFIX + "task.reschedule.higher.priority"; public static final boolean TEZ_AM_TASK_RESCHEDULE_HIGHER_PRIORITY_DEFAULT=true; + /** + * Boolean value. Specifies whether a re-scheduled attempt of a task, caused by previous + * failure get relaxed locality + */ + @ConfigurationScope(Scope.VERTEX) + @ConfigurationProperty(type="boolean") + public static final String TEZ_AM_TASK_RESCHEDULE_RELAXED_LOCALITY = + TEZ_AM_PREFIX + "task.reschedule.relaxed.locality"; + public static final boolean TEZ_AM_TASK_RESCHEDULE_RELAXED_LOCALITY_DEFAULT=true; + /** * Boolean value. Enabled blacklisting of nodes of nodes that are considered faulty. These nodes * will not be used to execute tasks. diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java index 0a6e9c5bc9..4d0a4bfd14 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java @@ -206,5 +206,6 @@ public TaskAttemptEventInfo getTaskAttemptTezEvents(TezTaskAttemptID attemptID, interface VertexConfig { int getMaxFailedTaskAttempts(); boolean getTaskRescheduleHigherPriority(); + boolean getTaskRescheduleRelaxedLocality(); } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java index 3c8a9b55ff..1fe65a91a5 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java @@ -1314,7 +1314,7 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl ta, TaskAttemptEvent ta.taskRacks = racks; // Ask for hosts / racks only if not a re-scheduled task. - if (ta.isRescheduled && ta.getVertex().getVertexConfig().getTaskRescheduleHigherPriority()) { + if (ta.isRescheduled && ta.getVertex().getVertexConfig().getTaskRescheduleRelaxedLocality()) { locationHint = null; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index 59552f2798..209db5a0a0 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -4648,6 +4648,7 @@ static class VertexConfigImpl implements VertexConfig { private final int maxFailedTaskAttempts; private final boolean taskRescheduleHigherPriority; + private final boolean taskRescheduleRelaxedLocality; public VertexConfigImpl(Configuration conf) { this.maxFailedTaskAttempts = conf.getInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS, @@ -4655,6 +4656,9 @@ public VertexConfigImpl(Configuration conf) { this.taskRescheduleHigherPriority = conf.getBoolean(TezConfiguration.TEZ_AM_TASK_RESCHEDULE_HIGHER_PRIORITY, TezConfiguration.TEZ_AM_TASK_RESCHEDULE_HIGHER_PRIORITY_DEFAULT); + this.taskRescheduleRelaxedLocality = + conf.getBoolean(TezConfiguration.TEZ_AM_TASK_RESCHEDULE_RELAXED_LOCALITY, + TezConfiguration.TEZ_AM_TASK_RESCHEDULE_RELAXED_LOCALITY_DEFAULT); } @Override @@ -4666,5 +4670,10 @@ public int getMaxFailedTaskAttempts() { public boolean getTaskRescheduleHigherPriority() { return taskRescheduleHigherPriority; } + + @Override + public boolean getTaskRescheduleRelaxedLocality() { + return taskRescheduleRelaxedLocality; + } } } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java index a9d0c8d3df..7709bc0c4f 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java @@ -212,8 +212,14 @@ public void testRetriesAtSamePriorityConfig() { // Override the test defaults to setup the config change TezConfiguration vertexConf = new TezConfiguration(); vertexConf.setBoolean(TezConfiguration.TEZ_AM_TASK_RESCHEDULE_HIGHER_PRIORITY, false); + vertexConf.setBoolean(TezConfiguration.TEZ_AM_TASK_RESCHEDULE_RELAXED_LOCALITY, true); when(mockVertex.getVertexConfig()).thenReturn(new VertexImpl.VertexConfigImpl(vertexConf)); + // set locality + Set hosts = new TreeSet(); + hosts.add("host1"); + locationHint = TaskLocationHint.createTaskLocationHint(hosts, null); + TaskAttemptImpl.ScheduleTaskattemptTransition sta = new TaskAttemptImpl.ScheduleTaskattemptTransition(); @@ -241,12 +247,15 @@ public void testRetriesAtSamePriorityConfig() { verify(eventHandler, times(1)).handle(arg.capture()); AMSchedulerEventTALaunchRequest launchEvent = (AMSchedulerEventTALaunchRequest) arg.getValue(); Assert.assertEquals(2, launchEvent.getPriority()); + Assert.assertEquals(1, launchEvent.getLocationHint().getHosts().size()); + Assert.assertTrue(launchEvent.getLocationHint().getHosts().contains("host1")); // Verify priority for a retried attempt is the same sta.transition(taImplReScheduled, sEvent); verify(eventHandler, times(2)).handle(arg.capture()); launchEvent = (AMSchedulerEventTALaunchRequest) arg.getValue(); Assert.assertEquals(2, launchEvent.getPriority()); + Assert.assertNull(launchEvent.getLocationHint()); } @Test(timeout = 5000) From 09679bdfd8e78a623c15026479b6fce71f48a4df Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Wed, 13 Sep 2017 17:24:59 -0500 Subject: [PATCH 010/512] TEZ-3831. Reduce Unordered memory needed for storing empty completed events (Jonathan Eagles via kshukla) --- .../common/shuffle/impl/ShuffleManager.java | 41 ++++++------------- 1 file changed, 12 insertions(+), 29 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java index e142228546..8e9be124a1 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java @@ -702,7 +702,9 @@ private void registerCompletedInput(FetchedInput fetchedInput) { private void maybeInformInputReady(FetchedInput fetchedInput) { lock.lock(); try { - completedInputs.add(fetchedInput); + if (!(fetchedInput instanceof NullFetchedInput)) { + completedInputs.add(fetchedInput); + } if (!inputReadyNotificationSent.getAndSet(true)) { // TODO Should eventually be controlled by Inputs which are processing the data. inputContext.inputIsReady(); @@ -846,20 +848,6 @@ public void shutdown() throws InterruptedException { } } - /////////////////// Methods for walking the available inputs - - /** - * @return true if there is another input ready for consumption. - */ - public boolean newInputAvailable() { - FetchedInput head = completedInputs.peek(); - if (head == null || head instanceof NullFetchedInput) { - return false; - } else { - return true; - } - } - /** * @return true if all of the required inputs have been fetched. */ @@ -878,21 +866,16 @@ public boolean allInputsFetched() { * but more may become available. */ public FetchedInput getNextInput() throws InterruptedException { - FetchedInput input = null; - do { - // Check for no additional inputs - lock.lock(); - try { - input = completedInputs.peek(); - if (input == null && allInputsFetched()) { - break; - } - } finally { - lock.unlock(); + // Check for no additional inputs + lock.lock(); + try { + if (completedInputs.peek() == null && allInputsFetched()) { + return null; } - input = completedInputs.take(); // block - } while (input instanceof NullFetchedInput); - return input; + } finally { + lock.unlock(); + } + return completedInputs.take(); // block } public int getNumInputs() { From 0033da85d4ccc856c699935d13dd617cf817d955 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Thu, 14 Sep 2017 16:07:51 -0500 Subject: [PATCH 011/512] TEZ-3832. TEZ DAG status shows SUCCEEDED for SUCCEEDED_WITH_FAILURES final status (Jonathan Eagles via kshukla) --- .../main/webapp/app/controllers/app/dags.js | 2 +- .../main/webapp/app/controllers/home/index.js | 2 +- tez-ui/src/main/webapp/app/models/dag.js | 9 +++++++++ tez-ui/src/main/webapp/app/serializers/dag.js | 1 + .../main/webapp/app/templates/dag/index.hbs | 2 +- .../main/webapp/tests/unit/models/dag-test.js | 18 +++++++++++++++++- .../webapp/tests/unit/serializers/dag-test.js | 2 +- 7 files changed, 31 insertions(+), 5 deletions(-) diff --git a/tez-ui/src/main/webapp/app/controllers/app/dags.js b/tez-ui/src/main/webapp/app/controllers/app/dags.js index e0c5e68408..bb4502a9b1 100644 --- a/tez-ui/src/main/webapp/app/controllers/app/dags.js +++ b/tez-ui/src/main/webapp/app/controllers/app/dags.js @@ -48,7 +48,7 @@ export default MultiTableController.extend({ },{ id: 'status', headerTitle: 'Status', - contentPath: 'status', + contentPath: 'finalStatus', cellComponentName: 'em-table-status-cell', observePath: true },{ diff --git a/tez-ui/src/main/webapp/app/controllers/home/index.js b/tez-ui/src/main/webapp/app/controllers/home/index.js index 5d8b290eef..754e5e6b99 100644 --- a/tez-ui/src/main/webapp/app/controllers/home/index.js +++ b/tez-ui/src/main/webapp/app/controllers/home/index.js @@ -120,7 +120,7 @@ export default TableController.extend({ },{ id: 'status', headerTitle: 'Status', - contentPath: 'status', + contentPath: 'finalStatus', cellComponentName: 'em-table-status-cell', observePath: true },{ diff --git a/tez-ui/src/main/webapp/app/models/dag.js b/tez-ui/src/main/webapp/app/models/dag.js index 4ddec74551..d03b14c3b9 100644 --- a/tez-ui/src/main/webapp/app/models/dag.js +++ b/tez-ui/src/main/webapp/app/models/dag.js @@ -83,6 +83,15 @@ export default DAGInfoModel.extend({ callerType: Ember.computed.or("callerData.callerType", "info.callerData.callerType"), amWsVersion: DS.attr("string"), + failedTaskAttempts: DS.attr("number"), + + finalStatus: Ember.computed("status", "failedTaskAttempts", function () { + var status = this.get("status"); + if(status === "SUCCEEDED" && this.get("failedTaskAttempts")) { + status = "SUCCEEDED_WITH_FAILURES"; + } + return status; + }), info: DS.attr("object"), diff --git a/tez-ui/src/main/webapp/app/serializers/dag.js b/tez-ui/src/main/webapp/app/serializers/dag.js index 9b3edfc458..de60985f88 100644 --- a/tez-ui/src/main/webapp/app/serializers/dag.js +++ b/tez-ui/src/main/webapp/app/serializers/dag.js @@ -128,5 +128,6 @@ export default DAGInfoSerializer.extend({ vertexNameIdMap: 'otherinfo.vertexNameIdMapping', amWsVersion: 'otherinfo.amWebServiceVersion', + failedTaskAttempts: 'otherinfo.numFailedTaskAttempts', } }); diff --git a/tez-ui/src/main/webapp/app/templates/dag/index.hbs b/tez-ui/src/main/webapp/app/templates/dag/index.hbs index ad1e093a0a..ae148025ca 100644 --- a/tez-ui/src/main/webapp/app/templates/dag/index.hbs +++ b/tez-ui/src/main/webapp/app/templates/dag/index.hbs @@ -51,7 +51,7 @@ Status - {{em-table-status-cell content=model.status}} + {{em-table-status-cell content=model.finalStatus}} Progress diff --git a/tez-ui/src/main/webapp/tests/unit/models/dag-test.js b/tez-ui/src/main/webapp/tests/unit/models/dag-test.js index 468c87f0e5..7be7def930 100644 --- a/tez-ui/src/main/webapp/tests/unit/models/dag-test.js +++ b/tez-ui/src/main/webapp/tests/unit/models/dag-test.js @@ -62,6 +62,8 @@ test('Basic creation test', function(assert) { assert.ok(model.info); assert.ok(model.amWsVersion); + assert.ok(model.failedTaskAttempts); + assert.ok(model.finalStatus); }); test('app loadType test', function(assert) { @@ -83,6 +85,20 @@ test('app loadType test', function(assert) { assert.equal(loadType(record), undefined); }); +test('status test', function(assert) { + let model = this.subject(); + + Ember.run(function () { + model.set("status", "SUCCEEDED"); + assert.equal(model.get("status"), "SUCCEEDED"); + assert.equal(model.get("finalStatus"), "SUCCEEDED"); + + model.set("failedTaskAttempts", 1); + assert.equal(model.get("status"), "SUCCEEDED"); + assert.equal(model.get("finalStatus"), "SUCCEEDED_WITH_FAILURES"); + }); +}); + test('queue test', function(assert) { let model = this.subject(), queueName = "queueName", @@ -138,4 +154,4 @@ test('vertices, edges & vertexGroups test', function(assert) { assert.equal(model.get("edges"), testEdges); assert.equal(model.get("vertexGroups"), testVertexGroups); }); -}); \ No newline at end of file +}); diff --git a/tez-ui/src/main/webapp/tests/unit/serializers/dag-test.js b/tez-ui/src/main/webapp/tests/unit/serializers/dag-test.js index f57d1af378..6334569491 100644 --- a/tez-ui/src/main/webapp/tests/unit/serializers/dag-test.js +++ b/tez-ui/src/main/webapp/tests/unit/serializers/dag-test.js @@ -36,7 +36,7 @@ test('Basic creation test', function(assert) { assert.ok(serializer.maps.containerLogs); assert.ok(serializer.maps.vertexIdNameMap); - assert.equal(Object.keys(serializer.get("maps")).length, 12 + 7); //12 own & 9 inherited (2 overwritten) + assert.equal(Object.keys(serializer.get("maps")).length, 13 + 7); //13 own & 9 inherited (2 overwritten) }); test('atsStatus test', function(assert) { From 4c29635880446c7b13ca6318f5429d2887538d45 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Fri, 15 Sep 2017 14:41:41 -0500 Subject: [PATCH 012/512] TEZ-3831 addendum. Reduce Unordered memory needed for storing empty completed events (Jonathan Eagles via kshukla) --- .../library/common/shuffle/impl/ShuffleManager.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java index 8e9be124a1..0a0286e1e7 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java @@ -721,6 +721,10 @@ private void adjustCompletedInputs(FetchedInput fetchedInput) { int numComplete = numCompletedInputs.incrementAndGet(); if (numComplete == numInputs) { + // Poison pill End of Input message to awake blocking take call + if (fetchedInput instanceof NullFetchedInput) { + completedInputs.add(fetchedInput); + } LOG.info("All inputs fetched for input vertex : " + inputContext.getSourceVertexName()); } } finally { @@ -875,7 +879,12 @@ public FetchedInput getNextInput() throws InterruptedException { } finally { lock.unlock(); } - return completedInputs.take(); // block + // Block until next input or End of Input message + FetchedInput fetchedInput = completedInputs.take(); + if (fetchedInput instanceof NullFetchedInput) { + fetchedInput = null; + } + return fetchedInput; } public int getNumInputs() { From 7236d1563ef71a4f551da2bca5501713cbf1d037 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Mon, 18 Sep 2017 13:22:47 -0500 Subject: [PATCH 013/512] TEZ-3724. Tez UI on HTTP corrects HTTPS REST calls to HTTP (Jonathan Eagles via kshukla) --- tez-ui/src/main/webapp/app/services/hosts.js | 28 +++++++------------ .../webapp/tests/unit/services/hosts-test.js | 8 +++--- 2 files changed, 14 insertions(+), 22 deletions(-) diff --git a/tez-ui/src/main/webapp/app/services/hosts.js b/tez-ui/src/main/webapp/app/services/hosts.js index c9e7d25b5a..ddab5c58a9 100644 --- a/tez-ui/src/main/webapp/app/services/hosts.js +++ b/tez-ui/src/main/webapp/app/services/hosts.js @@ -23,26 +23,18 @@ export default Ember.Service.extend({ env: Ember.inject.service("env"), correctProtocol: function (url, localProto) { - var urlProto; - - localProto = localProto || window.location.protocol; - - if(url.match("://")) { - urlProto = url.substr(0, url.indexOf("//")); - } - - if(localProto === "file:") { - urlProto = urlProto || "http:"; + var index = url.indexOf("://"); + if(index === -1) { + localProto = localProto || window.location.protocol; + return localProto + "//" + url; } - else { - urlProto = localProto; + var urlProto = url.substr(0, index + 1); + if(urlProto === "file:") { + urlProto = localProto || "http:"; + url = url.substr(index + 3); + return urlProto + "//" + url; } - - if(url.match("://")) { - url = url.substr(url.indexOf("://") + 3); - } - - return urlProto + "//" + url; + return url; }, normalizeURL: function (url) { diff --git a/tez-ui/src/main/webapp/tests/unit/services/hosts-test.js b/tez-ui/src/main/webapp/tests/unit/services/hosts-test.js index 026f21b304..afa527cb03 100644 --- a/tez-ui/src/main/webapp/tests/unit/services/hosts-test.js +++ b/tez-ui/src/main/webapp/tests/unit/services/hosts-test.js @@ -36,11 +36,11 @@ test('Test correctProtocol', function(assert) { // Correction assert.equal(service.correctProtocol("localhost:8088"), "http://localhost:8088"); - assert.equal(service.correctProtocol("https://localhost:8088"), "http://localhost:8088"); + assert.equal(service.correctProtocol("https://localhost:8088"), "https://localhost:8088"); assert.equal(service.correctProtocol("file://localhost:8088"), "http://localhost:8088"); assert.equal(service.correctProtocol("localhost:8088", "http:"), "http://localhost:8088"); - assert.equal(service.correctProtocol("https://localhost:8088", "http:"), "http://localhost:8088"); + assert.equal(service.correctProtocol("https://localhost:8088", "http:"), "https://localhost:8088"); assert.equal(service.correctProtocol("file://localhost:8088", "http:"), "http://localhost:8088"); assert.equal(service.correctProtocol("localhost:8088", "https:"), "https://localhost:8088"); @@ -72,6 +72,6 @@ test('Test host URLs with ENV set', function(assert) { rm: "https://localhost:4444" } }; - assert.equal(service.get("timeline"), "http://localhost:3333"); - assert.equal(service.get("rm"), "http://localhost:4444"); + assert.equal(service.get("timeline"), "https://localhost:3333"); + assert.equal(service.get("rm"), "https://localhost:4444"); }); From f86128a8dd8a7bbda6b2b61bcaee336b35c6100a Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Tue, 19 Sep 2017 13:17:11 -0500 Subject: [PATCH 014/512] TEZ-3834. TaskSchedulerManager NullPointerException during shutdown when failed to start. Contributed by Jonathan Eagles --- .../tez/dag/app/rm/TaskSchedulerManager.java | 24 +++++++++++++------ .../dag/app/rm/TestTaskSchedulerManager.java | 17 +++++++++++++ 2 files changed, 34 insertions(+), 7 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java index 640e8f62a0..7c1b926e95 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java @@ -684,12 +684,14 @@ protected void notifyForTest() { public void initiateStop() { for (int i = 0 ; i < taskSchedulers.length ; i++) { - try { - taskSchedulers[i].getTaskScheduler().initiateStop(); - } catch (Exception e) { - // Ignore for now as scheduler stop invoked on shutdown - LOG.error("Failed to do a clean initiateStop for Scheduler: " - + Utils.getTaskSchedulerIdentifierString(i, appContext), e); + if (taskSchedulers[i] != null) { + try { + taskSchedulers[i].getTaskScheduler().initiateStop(); + } catch (Exception e) { + // Ignore for now as scheduler stop invoked on shutdown + LOG.error("Failed to do a clean initiateStop for Scheduler: " + + Utils.getTaskSchedulerIdentifierString(i, appContext), e); + } } } } @@ -978,13 +980,21 @@ public ContainerSignatureMatcher getContainerSignatureMatcher() { } public boolean hasUnregistered() { + // Only return true if all task schedulers that were registered successfully unregister + if (taskSchedulers.length == 0) { + return false; + } boolean result = true; - for (int i = 0 ; i < taskSchedulers.length ; i++) { + for (int i = 0; i < taskSchedulers.length; i++) { // Explicitly not catching any exceptions around this API // No clear route to recover. Better to crash. + if (taskSchedulers[i] == null) { + return false; + } try { result = result & this.taskSchedulers[i].hasUnregistered(); } catch (Exception e) { + result = false; String msg = "Error in TaskScheduler when checking if a scheduler has unregistered" + ", scheduler=" + Utils.getTaskSchedulerIdentifierString(i, appContext); LOG.error(msg, e); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java index 43805f3de8..5df25de963 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java @@ -568,6 +568,23 @@ mockTaskAttempt2, mock(TaskLocationHint.class), 1, mock(ContainerContext.class), eq(launchRequest2)); } + @SuppressWarnings("unchecked") + @Test(timeout = 5000) + public void testShutdownBeforeStartTaskScheduler() { + Configuration conf = new TezConfiguration(); + AppContext appContext = mock(AppContext.class, RETURNS_DEEP_STUBS); + doReturn(conf).when(appContext).getAMConf(); + + List list = new LinkedList<>(); + list.add(null); + + TaskSchedulerManager taskSchedulerManager = + new TaskSchedulerManager(appContext, null, null, + null, null, list, false,null); + assertFalse("Should not return true unless actually unregistered successfully", + taskSchedulerManager.hasUnregistered()); + } + @SuppressWarnings("unchecked") @Test(timeout = 5000) public void testReportFailureFromTaskScheduler() { From 83a8b5097ef3bebc81d85f551ca2d7ba0fbb2e05 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Tue, 19 Sep 2017 13:25:21 -0500 Subject: [PATCH 015/512] TEZ-3839. Tez Shuffle Handler prints disk error stack traces for every read failure. (Kuhu Shukla via jeagles) --- .../java/org/apache/tez/auxservices/ShuffleHandler.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java index a4d1495ba9..d48cc013fa 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java @@ -18,6 +18,7 @@ package org.apache.tez.auxservices; +import org.apache.hadoop.util.DiskChecker; import static org.fusesource.leveldbjni.JniDBFactory.asString; import static org.fusesource.leveldbjni.JniDBFactory.bytes; import static org.jboss.netty.buffer.ChannelBuffers.wrappedBuffer; @@ -1160,7 +1161,11 @@ public ChannelFuture sendMap(ReduceContext reduceContext) } nextMap.addListener(new ReduceMapFileCount(reduceContext)); } catch (IOException e) { - LOG.error("Shuffle error :", e); + if (e instanceof DiskChecker.DiskErrorException) { + LOG.error("Shuffle error :" + e); + } else { + LOG.error("Shuffle error :", e); + } String errorMessage = getErrorMessage(e); sendError(reduceContext.getCtx(), errorMessage, INTERNAL_SERVER_ERROR); From 0109af29314ee97b8703e0be85591e6e610f4cea Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Tue, 19 Sep 2017 14:04:40 -0500 Subject: [PATCH 016/512] TEZ-3840. Tez should write TEZ_DAG_ID before TEZ_EXTRA_INFO (Jonathan Eagles via kshukla) --- .../history/logging/ats/HistoryEventTimelineConversion.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java index 235a29299c..3b5dbfd8e3 100644 --- a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java +++ b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java @@ -87,8 +87,8 @@ public static List convertToTimelineEntities(HistoryEvent histor convertContainerStoppedEvent((ContainerStoppedEvent) historyEvent)); case DAG_SUBMITTED: return Lists.newArrayList( - convertDAGSubmittedToDAGExtraInfoEntity((DAGSubmittedEvent)historyEvent), - convertDAGSubmittedEvent((DAGSubmittedEvent)historyEvent)); + convertDAGSubmittedEvent((DAGSubmittedEvent)historyEvent), + convertDAGSubmittedToDAGExtraInfoEntity((DAGSubmittedEvent)historyEvent)); case DAG_INITIALIZED: return Collections.singletonList( convertDAGInitializedEvent((DAGInitializedEvent) historyEvent)); From 3d3d04c6fa9d0d740cc00b1a0f526863e397c1ca Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Wed, 20 Sep 2017 10:40:05 -0500 Subject: [PATCH 017/512] TEZ-3840 addendum. Tez should write TEZ_DAG_ID before TEZ_EXTRA_INFO (Jonathan Eagles via kshukla) --- .../history/logging/ats/HistoryEventTimelineConversion.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java index 3b5dbfd8e3..874be1d4a4 100644 --- a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java +++ b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java @@ -96,8 +96,8 @@ public static List convertToTimelineEntities(HistoryEvent histor return Collections.singletonList(convertDAGStartedEvent((DAGStartedEvent) historyEvent)); case DAG_FINISHED: return Lists.newArrayList( - convertDAGFinishedToDAGExtraInfoEntity((DAGFinishedEvent) historyEvent), - convertDAGFinishedEvent((DAGFinishedEvent) historyEvent)); + convertDAGFinishedEvent((DAGFinishedEvent) historyEvent), + convertDAGFinishedToDAGExtraInfoEntity((DAGFinishedEvent) historyEvent)); case VERTEX_INITIALIZED: return Collections.singletonList( convertVertexInitializedEvent((VertexInitializedEvent) historyEvent)); From 8f61c515d6882a99efebf5473b1d23985d3a1d49 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Mon, 25 Sep 2017 08:51:05 -0500 Subject: [PATCH 018/512] TEZ-3836. Tez UI task page sort does not work on RHEL7/Fedora. Contributed by Sreenath Somarajapuram --- tez-ui/src/main/webapp/app/styles/shared.less | 43 +++++++++++++++++++ 1 file changed, 43 insertions(+) diff --git a/tez-ui/src/main/webapp/app/styles/shared.less b/tez-ui/src/main/webapp/app/styles/shared.less index 24da150f46..b93875b74a 100644 --- a/tez-ui/src/main/webapp/app/styles/shared.less +++ b/tez-ui/src/main/webapp/app/styles/shared.less @@ -60,4 +60,47 @@ b { margin-top: 10px; } } + + .table-body .table-scroll-body .table-column .header-body { + .sort-icon { + right: 7px; + top: 50%; + margin-top: -10px; + width: 5px; + height: 20px; + + &:before { + top: .3em; + } + &:after { + top: 1.2em; + } + + &.asc{ + &:after { + top: .3em; + } + } + &.desc { + &:before { + top: .3em; + } + } + } + + .resize-column { + position: absolute; + right: 0px; + top: 50%; + margin-top: -10px; + width: 5px; + + &:after { + content: "\F142"; + font: normal normal normal 13px/1 FontAwesome; + position: absolute; + top: .3em; + } + } + } } From bc08b19b2e8df3f5d202ea9dfda0bc1b7e7e5e2f Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Thu, 28 Sep 2017 10:52:21 -0500 Subject: [PATCH 019/512] TEZ-3843. Tez UI Vertex/Tasks log links for running tasks are missing (jeagles) --- tez-ui/src/main/webapp/app/serializers/task.js | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tez-ui/src/main/webapp/app/serializers/task.js b/tez-ui/src/main/webapp/app/serializers/task.js index 991f67e335..65b8c9f0cd 100644 --- a/tez-ui/src/main/webapp/app/serializers/task.js +++ b/tez-ui/src/main/webapp/app/serializers/task.js @@ -26,6 +26,6 @@ export default TimelineSerializer.extend({ failedTaskAttempts: 'otherinfo.numFailedTaskAttempts', successfulAttemptID: 'otherinfo.successfulAttemptId', - attemptIDs: 'otherinfo.relatedentities.TEZ_TASK_ATTEMPT_ID', + attemptIDs: 'relatedentities.TEZ_TASK_ATTEMPT_ID', } }); From a4a3c6dce3eaaa74b64591b181dc6fdcd5b8aa5e Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Thu, 28 Sep 2017 16:38:11 -0500 Subject: [PATCH 020/512] TEZ-3845. Tez UI Cleanup Stats Table (Jonathan Eagles via kshukla) --- tez-ui/src/main/webapp/app/components/stats-link.js | 6 ------ .../webapp/app/templates/components/stats-link.hbs | 12 +++++++++--- .../main/webapp/app/templates/dag/index/index.hbs | 12 ++++++------ .../src/main/webapp/app/templates/vertex/index.hbs | 10 +++++++++- 4 files changed, 24 insertions(+), 16 deletions(-) diff --git a/tez-ui/src/main/webapp/app/components/stats-link.js b/tez-ui/src/main/webapp/app/components/stats-link.js index 0fb56df9e1..4131dc80d6 100644 --- a/tez-ui/src/main/webapp/app/components/stats-link.js +++ b/tez-ui/src/main/webapp/app/components/stats-link.js @@ -24,10 +24,4 @@ export default Ember.Component.extend({ statsType: null, searchText: Ember.computed.oneWay("statsType"), - _statsType: Ember.computed("statsType", function () { - var type = this.get("statsType"); - if(type) { - return Ember.String.capitalize(type.toLowerCase()); - } - }) }); diff --git a/tez-ui/src/main/webapp/app/templates/components/stats-link.hbs b/tez-ui/src/main/webapp/app/templates/components/stats-link.hbs index 0d91ae9b26..d8155eec61 100644 --- a/tez-ui/src/main/webapp/app/templates/components/stats-link.hbs +++ b/tez-ui/src/main/webapp/app/templates/components/stats-link.hbs @@ -17,9 +17,15 @@ }} {{#if value}} - {{#link-to routeName (query-params searchText=searchText)}} - {{txt value type="number"}} {{_statsType}} - {{/link-to}} + {{#if searchText}} + {{#link-to routeName (query-params searchText=searchText)}} + {{txt value type="number"}} + {{/link-to}} + {{else}} + {{#link-to routeName}} + {{txt value type="number"}} + {{/link-to}} + {{/if}} {{else}} {{txt value type="number"}} {{/if}} diff --git a/tez-ui/src/main/webapp/app/templates/dag/index/index.hbs b/tez-ui/src/main/webapp/app/templates/dag/index/index.hbs index e39b514e97..de425af2d7 100644 --- a/tez-ui/src/main/webapp/app/templates/dag/index/index.hbs +++ b/tez-ui/src/main/webapp/app/templates/dag/index/index.hbs @@ -25,22 +25,22 @@ + + Total Vertices + {{stats-link value=stats.totalVertices routeName="dag.vertices"}} + Succeeded Vertices {{stats-link value=stats.succeededVertices routeName="dag.vertices" statsType="SUCCEEDED"}} - Total Vertices - {{stats.totalVertices}} + Total Tasks + {{stats-link value=stats.totalTasks routeName="dag.tasks"}} Succeeded Tasks {{stats-link value=stats.succeededTasks routeName="dag.tasks" statsType="SUCCEEDED"}} - - Total Tasks - {{stats.totalTasks}} - Failed Tasks {{stats-link value=stats.failedTasks routeName="dag.tasks" statsType="FAILED"}} diff --git a/tez-ui/src/main/webapp/app/templates/vertex/index.hbs b/tez-ui/src/main/webapp/app/templates/vertex/index.hbs index ff4f4fb7a7..ce083436df 100644 --- a/tez-ui/src/main/webapp/app/templates/vertex/index.hbs +++ b/tez-ui/src/main/webapp/app/templates/vertex/index.hbs @@ -68,7 +68,7 @@ Total Tasks - {{txt model.totalTasks type="number"}} + {{stats-link value=model.totalTasks routeName="vertex.tasks"}} Succeeded Tasks @@ -82,6 +82,14 @@ Killed Tasks {{stats-link value=model.killedTasks routeName="vertex.tasks" statsType="KILLED"}} + + Failed Task Attempts + {{stats-link value=model.failedTaskAttempts routeName="vertex.attempts" statsType="FAILED"}} + + + Killed Task Attempts + {{stats-link value=model.killedTaskAttempts routeName="vertex.attempts" statsType="KILLED"}} + First Task Start Time From 14cc28275f35bfa8c3316d12508cfd54363062c0 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Fri, 29 Sep 2017 12:57:58 -0500 Subject: [PATCH 021/512] TEZ-3833. Tasks should report codec errors during shuffle as fetch failures. Contributed by Kuhu Shukla --- .../apache/tez/runtime/library/common/shuffle/Fetcher.java | 6 +++--- .../shuffle/orderedgrouped/FetcherOrderedGrouped.java | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java index 9f657e5e9f..a3bbeca365 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java @@ -683,7 +683,7 @@ public void freeResources(FetchedInput fetchedInput) { long endTime = System.currentTimeMillis(); fetcherCallback.fetchSucceeded(host, srcAttemptId, fetchedInput, idxRecord.getPartLength(), idxRecord.getRawLength(), (endTime - startTime)); - } catch (IOException e) { + } catch (IOException | InternalError e) { hasFailures = true; cleanupFetchedInput(fetchedInput); if (isShutDown.get()) { @@ -980,7 +980,7 @@ private InputAttemptIdentifier[] fetchInputs(DataInputStream input, // metrics.successFetch(); } srcAttemptsRemaining.remove(inputAttemptIdentifier.toString()); - } catch (IOException ioe) { + } catch (IOException | InternalError ioe) { if (isShutDown.get()) { cleanupFetchedInput(fetchedInput); if (isDebugEnabled) { @@ -1037,7 +1037,7 @@ private void cleanupFetchedInput(FetchedInput fetchedInput) { * @return true to indicate connection retry. false otherwise. * @throws IOException */ - private boolean shouldRetry(InputAttemptIdentifier srcAttemptId, IOException ioe) { + private boolean shouldRetry(InputAttemptIdentifier srcAttemptId, Throwable ioe) { if (!(ioe instanceof SocketTimeoutException)) { return false; } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java index 68a54e911f..af9b929643 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java @@ -565,7 +565,7 @@ protected InputAttemptIdentifier[] copyMapOutput(MapHost host, endTime - startTime, mapOutput, false); } remaining.remove(inputAttemptIdentifier.toString()); - } catch(IOException ioe) { + } catch(IOException | InternalError ioe) { if (stopped) { if (LOG.isDebugEnabled()) { LOG.debug("Not reporting fetch failure for exception during data copy: [" @@ -614,7 +614,7 @@ protected InputAttemptIdentifier[] copyMapOutput(MapHost host, * @return true to indicate connection retry. false otherwise. * @throws IOException */ - private boolean shouldRetry(MapHost host, IOException ioe) { + private boolean shouldRetry(MapHost host, Throwable ioe) { if (!(ioe instanceof SocketTimeoutException)) { return false; } @@ -723,7 +723,7 @@ protected void setupLocalDiskFetch(MapHost host) throws InterruptedException { long endTime = System.currentTimeMillis(); scheduler.copySucceeded(srcAttemptId, host, indexRecord.getPartLength(), indexRecord.getRawLength(), (endTime - startTime), mapOutput, true); - } catch (IOException e) { + } catch (IOException | InternalError e) { if (mapOutput != null) { mapOutput.abort(); } From 4d6690ae17a07b6f87efda2851524f2e9cc18cec Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Mon, 2 Oct 2017 11:36:06 -0500 Subject: [PATCH 022/512] TEZ-3830. HistoryEventTimelineConversion should not hard code the Task state. (Kuhu Shukla via jeagles) --- .../tez/dag/history/events/TaskStartedEvent.java | 11 +++++++++++ .../logging/ats/HistoryEventTimelineConversion.java | 2 +- 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java index 7516090a7c..07dc2f9d99 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java @@ -22,6 +22,7 @@ import java.io.InputStream; import java.io.OutputStream; +import org.apache.tez.dag.api.oldrecords.TaskState; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.records.TezTaskID; @@ -33,13 +34,20 @@ public class TaskStartedEvent implements HistoryEvent { private String vertexName; private long scheduledTime; private long startTime; + private TaskState state; public TaskStartedEvent(TezTaskID taskId, String vertexName, long scheduledTime, long startTime) { + this(taskId, vertexName, scheduledTime, startTime, TaskState.SCHEDULED); + } + + public TaskStartedEvent(TezTaskID taskId, + String vertexName, long scheduledTime, long startTime, TaskState state) { this.vertexName = vertexName; this.taskID = taskId; this.scheduledTime = scheduledTime; this.startTime = startTime; + this.state = state; } public TaskStartedEvent() { @@ -108,4 +116,7 @@ public long getStartTime() { return startTime; } + public TaskState getState() { + return state; + } } diff --git a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java index 874be1d4a4..c40d3a8f78 100644 --- a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java +++ b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java @@ -632,7 +632,7 @@ private static TimelineEntity convertTaskStartedEvent(TaskStartedEvent event) { atsEntity.addOtherInfo(ATSConstants.START_TIME, event.getStartTime()); atsEntity.addOtherInfo(ATSConstants.SCHEDULED_TIME, event.getScheduledTime()); - atsEntity.addOtherInfo(ATSConstants.STATUS, TaskState.SCHEDULED.name()); + atsEntity.addOtherInfo(ATSConstants.STATUS, event.getState().name()); return atsEntity; } From 3b2933f01d9cf3a431f8230d4a3a7f83e7099788 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Mon, 2 Oct 2017 17:12:52 -0500 Subject: [PATCH 023/512] TEZ-3847. AM web controller task counters are empty sometimes. Contributed by Jonathan Eagles --- .../tez/dag/app/dag/impl/TaskAttemptImpl.java | 8 ++- .../tez/dag/app/dag/impl/TestTaskAttempt.java | 66 +++++++++++++++++++ 2 files changed, 72 insertions(+), 2 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java index 1fe65a91a5..1218543ca4 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java @@ -1575,8 +1575,12 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) { TaskStatusUpdateEvent statusEvent = sEvent.getStatusEvent(); ta.reportedStatus.state = ta.getState(); ta.reportedStatus.progress = statusEvent.getProgress(); - ta.reportedStatus.counters = statusEvent.getCounters(); - ta.statistics = statusEvent.getStatistics(); + if (statusEvent.getCounters() != null) { + ta.reportedStatus.counters = statusEvent.getCounters(); + } + if (statusEvent.getStatistics() != null) { + ta.statistics = statusEvent.getStatistics(); + } if (statusEvent.getProgressNotified()) { ta.lastNotifyProgressTimestamp = ta.clock.getTime(); } else { diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java index 7709bc0c4f..2bad2ef8d9 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java @@ -62,6 +62,7 @@ import org.apache.log4j.Level; import org.apache.log4j.LogManager; import org.apache.tez.common.MockDNSToSwitchMapping; +import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.TezConstants; import org.apache.tez.dag.app.dag.event.TaskAttemptEventSubmitted; import org.apache.tez.dag.app.dag.event.TaskEventTAFailed; @@ -1072,6 +1073,71 @@ public void testProgressTimeStampUpdate() throws Exception { Assert.assertTrue("This should have been an attempt failed event!", arg.getValue() instanceof TaskAttemptEventAttemptFailed); } + @Test + public void testStatusUpdateWithNullCounters() throws Exception { + ApplicationId appId = ApplicationId.newInstance(1, 2); + ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance( + appId, 0); + TezDAGID dagID = TezDAGID.getInstance(appId, 1); + TezVertexID vertexID = TezVertexID.getInstance(dagID, 1); + TezTaskID taskID = TezTaskID.getInstance(vertexID, 1); + + MockEventHandler eventHandler = spy(new MockEventHandler()); + TaskCommunicatorManagerInterface taListener = createMockTaskAttemptListener(); + + Configuration taskConf = new Configuration(); + taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class); + taskConf.setBoolean("fs.file.impl.disable.cache", true); + + locationHint = TaskLocationHint.createTaskLocationHint( + new HashSet(Arrays.asList(new String[]{"127.0.0.1"})), null); + Resource resource = Resource.newInstance(1024, 1); + + NodeId nid = NodeId.newInstance("127.0.0.1", 0); + ContainerId contId = ContainerId.newContainerId(appAttemptId, 3); + Container container = mock(Container.class); + when(container.getId()).thenReturn(contId); + when(container.getNodeId()).thenReturn(nid); + when(container.getNodeHttpAddress()).thenReturn("localhost:0"); + + AMContainerMap containers = new AMContainerMap( + mock(ContainerHeartbeatHandler.class), mock(TaskCommunicatorManagerInterface.class), + new ContainerContextMatcher(), appCtx); + containers.addContainerIfNew(container, 0, 0, 0); + + doReturn(new ClusterInfo()).when(appCtx).getClusterInfo(); + doReturn(containers).when(appCtx).getAllContainers(); + + TaskHeartbeatHandler mockHeartbeatHandler = mock(TaskHeartbeatHandler.class); + TaskAttemptImpl taImpl = new MockTaskAttemptImpl(taskID, 1, eventHandler, + taListener, taskConf, new SystemClock(), + mockHeartbeatHandler, appCtx, false, + resource, createFakeContainerContext(), false); + TezTaskAttemptID taskAttemptID = taImpl.getID(); + + taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); + taImpl.handle(new TaskAttemptEventSubmitted(taskAttemptID, contId)); + taImpl.handle(new TaskAttemptEventStartedRemotely(taskAttemptID)); + assertEquals("Task attempt is not in the RUNNING state", taImpl.getState(), TaskAttemptState.RUNNING); + verify(mockHeartbeatHandler).register(taskAttemptID); + + TezCounters counters = new TezCounters(); + counters.findCounter("group", "counter").increment(1); + taImpl.handle(new TaskAttemptEventStatusUpdate( + taskAttemptID, new TaskStatusUpdateEvent(counters, 0.1f, null, false))); + assertEquals(1, taImpl.getCounters().findCounter("group", "counter").getValue()); + taImpl.handle(new TaskAttemptEventStatusUpdate( + taskAttemptID, new TaskStatusUpdateEvent(null, 0.1f, null, false))); + assertEquals(1, taImpl.getCounters().findCounter("group", "counter").getValue()); + counters.findCounter("group", "counter").increment(1); + taImpl.handle(new TaskAttemptEventStatusUpdate( + taskAttemptID, new TaskStatusUpdateEvent(counters, 0.1f, null, false))); + assertEquals(2, taImpl.getCounters().findCounter("group", "counter").getValue()); + taImpl.handle(new TaskAttemptEventStatusUpdate( + taskAttemptID, new TaskStatusUpdateEvent(null, 0.1f, null, false))); + assertEquals(2, taImpl.getCounters().findCounter("group", "counter").getValue()); + } + @Test (timeout = 5000) public void testNoProgressFail() throws Exception { ApplicationId appId = ApplicationId.newInstance(1, 2); From b875eb1f2e0a56d4b9c01127b40d81983e9be896 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Wed, 4 Oct 2017 16:43:32 -0500 Subject: [PATCH 024/512] TEZ-3848. Tez Local mode doesn't localize distributed cache files (Jacob Tolar via jeagles) --- .../app/launcher/LocalContainerLauncher.java | 42 +++- .../app/launcher/TezLocalCacheManager.java | 184 ++++++++++++++++++ 2 files changed, 223 insertions(+), 3 deletions(-) create mode 100644 tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java index d50b49eb52..9764daaaef 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java @@ -18,7 +18,6 @@ package org.apache.tez.dag.app.launcher; - import java.io.IOException; import java.net.InetAddress; import java.net.UnknownHostException; @@ -48,7 +47,6 @@ import org.apache.tez.common.ReflectionUtils; import org.apache.tez.common.TezUtils; import org.apache.tez.dag.records.TezDAGID; -import org.apache.tez.hadoop.shim.DefaultHadoopShim; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; @@ -100,6 +98,9 @@ public class LocalContainerLauncher extends DagContainerLauncher { runningContainers = new ConcurrentHashMap(); + private final ConcurrentHashMap + cacheManagers = new ConcurrentHashMap<>(); + private final ExecutorService callbackExecutor = Executors.newFixedThreadPool(1, new ThreadFactoryBuilder().setDaemon(true).setNameFormat("CallbackExecutor").build()); @@ -229,6 +230,10 @@ void sendContainerLaunchFailedMsg(ContainerId containerId, String message) { private void handleLaunchFailed(Throwable t, ContainerId containerId) { String message; + + // clean up distributed cache files + cleanupCacheFiles(containerId); + if (t instanceof RejectedExecutionException) { message = "Failed to queue container launch for container Id: " + containerId; } else { @@ -244,10 +249,22 @@ private void launch(ContainerLaunchRequest event) { String tokenIdentifier = context.getApplicationID().toString(); try { TezChild tezChild; + try { int taskCommId = context.getTaskCommunicatorIdentifier(event.getTaskCommunicatorName()); + + Configuration conf = context.getAMConf(); + if (isLocalMode) { + TezLocalCacheManager cacheManager = new TezLocalCacheManager( + event.getContainerLaunchContext().getLocalResources(), + conf + ); + cacheManagers.put(event.getContainerId(), cacheManager); + cacheManager.localize(); + } + tezChild = - createTezChild(context.getAMConf(), event.getContainerId(), tokenIdentifier, + createTezChild(conf, event.getContainerId(), tokenIdentifier, context.getApplicationAttemptId().getAttemptId(), context.getLocalDirs(), ((TezTaskCommunicatorImpl)tal.getTaskCommunicator(taskCommId).getTaskCommunicator()).getUmbilical(), TezCommonUtils.parseCredentialsBytes(event.getContainerLaunchContext().getTokens().array())); @@ -322,6 +339,9 @@ public void onSuccess(TezChild.ContainerExecutionResult result) { (result.getThrowable() == null ? null : result.getThrowable().getMessage()) : result.getErrorMessage(), TaskAttemptEndReason.APPLICATION_ERROR); } + + // clean up distributed cache files + cleanupCacheFiles(containerId); } @Override @@ -341,6 +361,22 @@ public void onFailure(Throwable t) { TezChild.ContainerExecutionResult.ExitStatus.SUCCESS.getExitCode(), "CancellationException", TaskAttemptEndReason.COMMUNICATION_ERROR.CONTAINER_EXITED); } + + // clean up distributed cache files + cleanupCacheFiles(containerId); + } + } + + private void cleanupCacheFiles(ContainerId container) { + if (isLocalMode) { + TezLocalCacheManager manager = cacheManagers.remove(container); + try { + if (manager != null) { + manager.cleanup(); + } + } catch (IOException e) { + LOG.info("Unable to clean up local cache files: ", e); + } } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java new file mode 100644 index 0000000000..80f73aa9aa --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java @@ -0,0 +1,184 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.app.launcher; + +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileContext; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.LocalResource; +import org.apache.hadoop.yarn.api.records.LocalResourceType; +import org.apache.hadoop.yarn.util.FSDownload; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.ThreadFactory; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class is responsible for localizing files from the distributed cache for Tez local mode. + */ +public class TezLocalCacheManager { + + private static final Logger LOG = LoggerFactory.getLogger(TezLocalCacheManager.class); + + final private Map resources; + final private Configuration conf; + final private UserGroupInformation ugi; + final private FileContext fileContext; + final private java.nio.file.Path tempDir; + + final private Map resourceInfo = new HashMap<>(); + + public TezLocalCacheManager(Map resources, Configuration conf) throws IOException { + this.ugi = UserGroupInformation.getCurrentUser(); + this.fileContext = FileContext.getLocalFSFileContext(); + this.resources = resources; + this.conf = conf; + this.tempDir = Files.createTempDirectory(Paths.get("."), "tez-local-cache"); + } + + /** + * Localize this instance's resources by downloading and symlinking them. + * + * @throws IOException when an error occurs in download or link + */ + public void localize() throws IOException { + String absPath = Paths.get(".").toAbsolutePath().normalize().toString(); + Path cwd = fileContext.makeQualified(new Path(absPath)); + ExecutorService threadPool = null; + + try { + // construct new threads with helpful names + ThreadFactory threadFactory = new ThreadFactoryBuilder() + .setNameFormat("TezLocalCacheManager Downloader #%d") + .build(); + threadPool = Executors.newCachedThreadPool(threadFactory); + + // start all fetches + for (Map.Entry entry : resources.entrySet()) { + String resourceName = entry.getKey(); + LocalResource resource = entry.getValue(); + + if (resource.getType() == LocalResourceType.PATTERN) { + throw new IllegalArgumentException("Resource type PATTERN not supported."); + } + + // submit task to download the object + java.nio.file.Path downloadDir = Files.createTempDirectory(tempDir, resourceName); + Path dest = new Path(downloadDir.toAbsolutePath().toString()); + FSDownload downloader = new FSDownload(fileContext, ugi, conf, dest, resource); + Future downloadedPath = threadPool.submit(downloader); + + // linkPath is the path we want to symlink the file/directory into + Path linkPath = new Path(cwd, entry.getKey()); + resourceInfo.put(resource, new ResourceInfo(downloadedPath, linkPath)); + } + + // Link each file + for (Map.Entry entry : resourceInfo.entrySet()) { + LocalResource resource = entry.getKey(); + ResourceInfo resourceMeta = entry.getValue(); + + Path linkPath = resourceMeta.linkPath; + Path targetPath; + + try { + // this blocks on the download completing + targetPath = resourceMeta.downloadPath.get(); + } catch (InterruptedException | ExecutionException e) { + throw new IOException(e); + } + + if (createSymlink(targetPath, linkPath)) { + LOG.info("Localized file: {} as {}", resource, linkPath); + } else { + LOG.warn("Failed to create symlink: {} <- {}", targetPath, linkPath); + } + } + } finally { + if (threadPool != null) { + threadPool.shutdownNow(); + } + } + } + + /** + * Clean up any symlinks and temp files that were created. + * + * @throws IOException when an error occurs in cleanup + */ + public void cleanup() throws IOException { + for (ResourceInfo info : resourceInfo.values()) { + if (fileContext.util().exists(info.linkPath)) { + fileContext.delete(info.linkPath, true); + } + } + + Path temp = new Path(tempDir.toString()); + if (fileContext.util().exists(temp)) { + fileContext.delete(temp, true); + } + } + + /** + * Create a symlink. + */ + private boolean createSymlink(Path target, Path link) throws IOException { + LOG.info("Creating symlink: {} <- {}", target, link); + String targetPath = target.toUri().getPath(); + String linkPath = link.toUri().getPath(); + + if (fileContext.util().exists(link)) { + LOG.warn("File already exists at symlink path: {}", link); + return false; + } else { + try { + Files.createSymbolicLink(Paths.get(linkPath), Paths.get(targetPath)); + return true; + } catch (UnsupportedOperationException e) { + LOG.warn("Unable to create symlink {} <- {}: UnsupportedOperationException", target, link); + return false; + } + } + } + + /** + * Wrapper to keep track of download path and link path + */ + private static class ResourceInfo { + final Future downloadPath; + final Path linkPath; + + public ResourceInfo(Future downloadPath, Path linkPath) { + this.downloadPath = downloadPath; + this.linkPath = linkPath; + } + } +} From c82b2eadea9d75db4dd47a455c0b4d6f7dda2cc8 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Tue, 10 Oct 2017 11:48:24 -0500 Subject: [PATCH 025/512] TEZ-3844. Tez UI Dag Counters show no records for a RUNNING DAG. (jeagles) --- tez-ui/src/main/webapp/app/routes/single-am-pollster.js | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/tez-ui/src/main/webapp/app/routes/single-am-pollster.js b/tez-ui/src/main/webapp/app/routes/single-am-pollster.js index 27bcced4a9..87fb941bf0 100644 --- a/tez-ui/src/main/webapp/app/routes/single-am-pollster.js +++ b/tez-ui/src/main/webapp/app/routes/single-am-pollster.js @@ -21,10 +21,13 @@ import AmPollsterRoute from './am-pollster'; export default AmPollsterRoute.extend({ - canPoll: Ember.computed("polledRecords", "loadedValue.dag.isComplete", "loadedValue.app.isComplete", function () { + canPoll: Ember.computed("polledRecords", "loadedValue.dag.isComplete", "loadedValue.isComplete", "loadedValue.app.isComplete", function () { var isComplete = this.get("loadedValue.dag.isComplete"); if(isComplete === undefined) { - isComplete = this.get("loadedValue.app.isComplete"); + isComplete = this.get("loadedValue.isComplete"); + if(isComplete === undefined) { + isComplete = this.get("loadedValue.app.isComplete"); + } } return isComplete === false && this._super(); }), From 0207281b3cea4f78b5a57fb81fef0cf14d8c24e1 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Thu, 12 Oct 2017 09:33:14 -0500 Subject: [PATCH 026/512] TEZ-3852. Optimize ContainerContext.isSuperSet to speed container reuse decisions. Contributed by Jonathan Eagles --- .../apache/tez/dag/app/ContainerContext.java | 52 +++++++------------ .../tez/dag/app/rm/TestContainerReuse.java | 4 ++ 2 files changed, 24 insertions(+), 32 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerContext.java b/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerContext.java index f00b27b243..d88daff8e4 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerContext.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerContext.java @@ -129,26 +129,24 @@ public boolean isExactMatch(ContainerContext otherContext) { // classpath modification private static boolean localResourcesCompatible(Map srcLRs, Map reqLRs) { - Map reqLRsCopy = new HashMap(reqLRs); - for (Entry srcLREntry : srcLRs.entrySet()) { - LocalResource requestedLocalResource = reqLRsCopy.remove(srcLREntry.getKey()); - if (requestedLocalResource != null && !srcLREntry.getValue().equals(requestedLocalResource)) { + for (Entry reqLREntry : reqLRs.entrySet()) { + LocalResource requestedLocalResource = srcLRs.get(reqLREntry.getKey()); + if (requestedLocalResource == null) { + LocalResource lr = reqLREntry.getValue(); + if (!LocalResourceType.FILE.equals(lr.getType())) { + if (LOG.isDebugEnabled()) { + LOG.debug("Cannot match container: Additional local resource needed is not of type FILE" + + ", resourceName: " + reqLREntry.getKey() + + ", resourceDetails: " + reqLREntry); + } + return false; + } + } else if(!reqLREntry.getValue().equals(requestedLocalResource)) { if (LOG.isDebugEnabled()) { LOG.debug("Cannot match container: Attempting to use same target resource name: " - + srcLREntry.getKey() + + reqLREntry.getKey() + ", but with different source resources. Already localized: " - + srcLREntry.getValue() + ", requested: " + requestedLocalResource); - } - return false; - } - } - for (Entry additionalLREntry : reqLRsCopy.entrySet()) { - LocalResource lr = additionalLREntry.getValue(); - if (EnumSet.of(LocalResourceType.ARCHIVE, LocalResourceType.PATTERN).contains(lr.getType())) { - if (LOG.isDebugEnabled()) { - LOG.debug("Cannot match container: Additional local resource needed is not of type FILE" - + ", resourceName: " + additionalLREntry.getKey() - + ", resourceDetails: " + additionalLREntry); + + requestedLocalResource + ", requested: " + reqLREntry.getValue()); } return false; } @@ -161,24 +159,14 @@ private static boolean isSuperSet(Map srcMap, Map matchMap, for (Entry oEntry : matchMap.entrySet()) { K oKey = oEntry.getKey(); V oVal = oEntry.getValue(); - if (srcMap.containsKey(oKey)) { - if (!oVal.equals(srcMap.get(oKey))) { - if (LOG.isDebugEnabled()) { - LOG.debug("Incompatible container context" + V srcVal = srcMap.get(oKey); + if (!oVal.equals(srcVal)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Incompatible container context" + ", matchInfo=" + matchInfo + ", thisKey=" + oKey - + ", thisVal=" + srcMap.get(oKey) + + ", thisVal=" + srcVal + ", otherVal=" + oVal); - } - return false; - } - } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Incompatible container context" - + ", matchInfo=" + matchInfo - + ", thisKey=" + oKey - + ", thisVal=null" - + ", otherVal=" + oVal); } return false; } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java index 7e9e9abbb6..4e29dd5917 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java @@ -38,6 +38,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.hadoop.yarn.api.records.LocalResourceType; import org.apache.tez.common.TezUtils; import org.apache.tez.serviceplugins.api.TaskScheduler; import org.mockito.ArgumentCaptor; @@ -1127,8 +1128,11 @@ public void testReuseConflictLocalResources() throws IOException, InterruptedExc String rsrc1 = "rsrc1"; String rsrc2 = "rsrc2"; LocalResource lr1 = mock(LocalResource.class); + doReturn(LocalResourceType.FILE).when(lr1).getType(); LocalResource lr2 = mock(LocalResource.class); + doReturn(LocalResourceType.FILE).when(lr2).getType(); LocalResource lr3 = mock(LocalResource.class); + doReturn(LocalResourceType.FILE).when(lr3).getType(); AMContainerEventAssignTA assignEvent = null; From d8767248e426eeea628ddb2f3831c45880606e7d Mon Sep 17 00:00:00 2001 From: Sreenath Somarajapuram Date: Mon, 16 Oct 2017 13:54:27 +0530 Subject: [PATCH 027/512] TEZ-3850. Enable header as sort button on Tez UI (Jonathan Eagles) --- tez-ui/src/main/webapp/app/controllers/dag/tasks.js | 1 + tez-ui/src/main/webapp/app/controllers/table.js | 3 ++- tez-ui/src/main/webapp/app/controllers/vertex/tasks.js | 1 + tez-ui/src/main/webapp/package.json | 2 +- tez-ui/src/main/webapp/yarn.lock | 6 +++--- 5 files changed, 8 insertions(+), 5 deletions(-) diff --git a/tez-ui/src/main/webapp/app/controllers/dag/tasks.js b/tez-ui/src/main/webapp/app/controllers/dag/tasks.js index a742644b34..92f674ad8e 100644 --- a/tez-ui/src/main/webapp/app/controllers/dag/tasks.js +++ b/tez-ui/src/main/webapp/app/controllers/dag/tasks.js @@ -83,6 +83,7 @@ export default MultiTableController.extend({ headerTitle: 'Successful/Last Attempt Log', cellComponentName: 'em-table-tasks-log-link-cell', enableSearch: false, + contentPath: 'logs', getCellContent: function (row) { var attemptID = row.get("successfulAttemptID"); if(!attemptID) { diff --git a/tez-ui/src/main/webapp/app/controllers/table.js b/tez-ui/src/main/webapp/app/controllers/table.js index fd01a5076f..57adf00469 100644 --- a/tez-ui/src/main/webapp/app/controllers/table.js +++ b/tez-ui/src/main/webapp/app/controllers/table.js @@ -51,7 +51,8 @@ export default AbstractController.extend({ searchText: this.get("searchText"), sortColumnId: this.get("sortColumnId"), sortOrder: this.get("sortOrder"), - pageNo: this.get("pageNo") + pageNo: this.get("pageNo"), + headerAsSortButton: true, }); }), diff --git a/tez-ui/src/main/webapp/app/controllers/vertex/tasks.js b/tez-ui/src/main/webapp/app/controllers/vertex/tasks.js index 86261fd9ab..560c8ba0f9 100644 --- a/tez-ui/src/main/webapp/app/controllers/vertex/tasks.js +++ b/tez-ui/src/main/webapp/app/controllers/vertex/tasks.js @@ -73,6 +73,7 @@ export default MultiTableController.extend(AutoCounterColumn, { headerTitle: 'Successful/Last Attempt Log', cellComponentName: 'em-table-tasks-log-link-cell', enableSearch: false, + contentPath: 'logs', getCellContent: function (row) { var attemptID = row.get("successfulAttemptID"); if(!attemptID) { diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json index 513de5271a..ccb85eaef1 100644 --- a/tez-ui/src/main/webapp/package.json +++ b/tez-ui/src/main/webapp/package.json @@ -62,7 +62,7 @@ }, "dependencies": { "em-helpers": "0.8.0", - "em-table": "0.7.2", + "em-table": "0.11.2", "em-tgraph": "0.0.10" } } diff --git a/tez-ui/src/main/webapp/yarn.lock b/tez-ui/src/main/webapp/yarn.lock index bf694fb2c3..abbb702bae 100644 --- a/tez-ui/src/main/webapp/yarn.lock +++ b/tez-ui/src/main/webapp/yarn.lock @@ -1401,9 +1401,9 @@ em-helpers@0.8.0: optionalDependencies: phantomjs-prebuilt "2.1.13" -em-table@0.7.2: - version "0.7.2" - resolved "https://registry.yarnpkg.com/em-table/-/em-table-0.7.2.tgz#867ff734701df9765f2505e02acd74768edb0f71" +em-table@0.11.2: + version "0.11.2" + resolved "https://registry.yarnpkg.com/em-table/-/em-table-0.11.2.tgz#1248f5f7a59dfc5d6592164933eebb02f3a5d410" dependencies: ember-cli-htmlbars "^1.0.1" ember-cli-less "^1.4.0" From f760ad3573d13dc981182765c42b8a7d8ea46696 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Tue, 17 Oct 2017 10:57:44 -0500 Subject: [PATCH 028/512] TEZ-3854. Make use of new improved em-table sort-icon (Jonathan Eagles via kshukla) --- tez-ui/src/main/webapp/app/styles/shared.less | 18 ------------------ tez-ui/src/main/webapp/package.json | 2 +- tez-ui/src/main/webapp/yarn.lock | 6 +++--- 3 files changed, 4 insertions(+), 22 deletions(-) diff --git a/tez-ui/src/main/webapp/app/styles/shared.less b/tez-ui/src/main/webapp/app/styles/shared.less index b93875b74a..dedac8c91b 100644 --- a/tez-ui/src/main/webapp/app/styles/shared.less +++ b/tez-ui/src/main/webapp/app/styles/shared.less @@ -68,24 +68,6 @@ b { margin-top: -10px; width: 5px; height: 20px; - - &:before { - top: .3em; - } - &:after { - top: 1.2em; - } - - &.asc{ - &:after { - top: .3em; - } - } - &.desc { - &:before { - top: .3em; - } - } } .resize-column { diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json index ccb85eaef1..3d6846834c 100644 --- a/tez-ui/src/main/webapp/package.json +++ b/tez-ui/src/main/webapp/package.json @@ -62,7 +62,7 @@ }, "dependencies": { "em-helpers": "0.8.0", - "em-table": "0.11.2", + "em-table": "0.11.3", "em-tgraph": "0.0.10" } } diff --git a/tez-ui/src/main/webapp/yarn.lock b/tez-ui/src/main/webapp/yarn.lock index abbb702bae..93210b7910 100644 --- a/tez-ui/src/main/webapp/yarn.lock +++ b/tez-ui/src/main/webapp/yarn.lock @@ -1401,9 +1401,9 @@ em-helpers@0.8.0: optionalDependencies: phantomjs-prebuilt "2.1.13" -em-table@0.11.2: - version "0.11.2" - resolved "https://registry.yarnpkg.com/em-table/-/em-table-0.11.2.tgz#1248f5f7a59dfc5d6592164933eebb02f3a5d410" +em-table@0.11.3: + version "0.11.3" + resolved "https://registry.yarnpkg.com/em-table/-/em-table-0.11.3.tgz#20e605cc3814214e644199399a2383cee8d23eeb" dependencies: ember-cli-htmlbars "^1.0.1" ember-cli-less "^1.4.0" From bbe0f96a7b048f887e5a5e46667393407381e6e2 Mon Sep 17 00:00:00 2001 From: Zhiyuan Yang Date: Thu, 19 Oct 2017 12:45:37 -0700 Subject: [PATCH 029/512] TEZ-3853. Binary incompatibility caused by DEFAULT_LOG_LEVEL (zhiyuany) --- .../main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java index efd6463530..1d06b7aeaf 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java @@ -204,12 +204,12 @@ private static String getChildLogLevel(Configuration conf, boolean isMap) { if (isMap) { return conf.get( MRJobConfig.MAP_LOG_LEVEL, - JobConf.DEFAULT_LOG_LEVEL.toString() + MRJobConfig.DEFAULT_LOG_LEVEL ); } else { return conf.get( MRJobConfig.REDUCE_LOG_LEVEL, - JobConf.DEFAULT_LOG_LEVEL.toString() + MRJobConfig.DEFAULT_LOG_LEVEL ); } } From d5ac3b75f73b908b292f4f7e1a3d619696d957b8 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Wed, 25 Oct 2017 11:01:28 -0500 Subject: [PATCH 030/512] TEZ-3849. Combiner+PipelinedSorter silently drops records (Jacob Tolar via kshukla) --- .../processor/reduce/ReduceProcessor.java | 6 +++ .../tez/mapreduce/combine/TestMRCombiner.java | 11 +++- .../common/sort/impl/PipelinedSorter.java | 50 +++++++++++++++++-- .../library/common/sort/impl/TezMerger.java | 36 +++++++++---- .../sort/impl/TezRawKeyValueIterator.java | 11 +++- .../common/sort/impl/dflt/DefaultSorter.java | 6 +++ .../input/OrderedGroupedInputLegacy.java | 5 ++ .../common/sort/impl/TestPipelinedSorter.java | 39 +++++++++++++++ 8 files changed, 148 insertions(+), 16 deletions(-) diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java index 4b79c78c45..63b168f267 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java @@ -309,6 +309,12 @@ public boolean isSameKey() throws IOException { public DataInputBuffer getValue() throws IOException { return rawIter.getValue(); } + + @Override + public boolean hasNext() throws IOException { + return rawIter.hasNext(); + } + public boolean next() throws IOException { boolean ret = rawIter.next(); reporter.setProgress(rawIter.getProgress().getProgress()); diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/combine/TestMRCombiner.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/combine/TestMRCombiner.java index a796e59326..7668d96af1 100644 --- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/combine/TestMRCombiner.java +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/combine/TestMRCombiner.java @@ -152,7 +152,16 @@ private static class TezRawKeyValueIteratorTest implements @Override public boolean next() throws IOException { - if (i++ < keys.length - 1) { + boolean hasNext = hasNext(); + if (hasNext) { + i += 1; + } + + return hasNext; + } + + public boolean hasNext() throws IOException { + if (i < (keys.length - 1)) { return true; } return false; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java index 88d10d0f11..07c2fe2e4d 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java @@ -574,15 +574,14 @@ public boolean spill(boolean ignoreEmptySpills) throws IOException { //write merged output to disk long segmentStart = out.getPos(); Writer writer = null; - boolean hasNext = kvIter.next(); + boolean hasNext = kvIter.hasNext(); if (hasNext || !sendEmptyPartitionDetails) { writer = new Writer(conf, out, keyClass, valClass, codec, spilledRecordsCounter, null, merger.needsRLE()); } if (combiner == null) { - while (hasNext) { + while (kvIter.next()) { writer.append(kvIter.getKey(), kvIter.getValue()); - hasNext = kvIter.next(); } } else { if (hasNext) { @@ -842,6 +841,7 @@ public void flush() throws IOException { private interface PartitionedRawKeyValueIterator extends TezRawKeyValueIterator { int getPartition(); + Integer peekPartition(); } private static class BufferStreamWrapper extends OutputStream @@ -1129,6 +1129,11 @@ public boolean next() { return true; } + @Override + public boolean hasNext() { + return (kvindex == maxindex); + } + public void close() { } @@ -1146,6 +1151,14 @@ public int getPartition() { return partition; } + public Integer peekPartition() { + if (!hasNext()) { + return null; + } else { + return kvmeta.get(span.offsetFor(kvindex + 1) + PARTITION); + } + } + @SuppressWarnings("unused") public int size() { return (maxindex - kvindex); @@ -1264,6 +1277,23 @@ public boolean next() throws IOException { return false; } + @Override + public boolean hasNext() throws IOException { + if (dirty || iter.hasNext()) { + Integer part; + if (dirty) { + part = iter.getPartition(); + } else { + part = iter.peekPartition(); + } + + if (part != null) { + return (part >>> (32 - partitionBits)) == partition; + } + } + return false; + } + public void reset(int partition) { this.partition = partition; } @@ -1403,6 +1433,20 @@ public final boolean next() { return false; } + @Override + public boolean hasNext() { + return peek() != null; + } + + public Integer peekPartition() { + if (!hasNext()) { + return null; + } else { + SpanIterator peek = peek(); + return peek.getPartition(); + } + } + public DataInputBuffer getKey() { return key; } public DataInputBuffer getValue() { return value; } public int getPartition() { return partition; } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java index 6eb9a40c93..0e18eadae6 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java @@ -638,19 +638,10 @@ void compareKeyWithNextTopKey(Segment current) throws IOException { } public boolean next() throws IOException { - if (size() == 0) + if (!hasNext()) { return false; - - if (minSegment != null) { - //minSegment is non-null for all invocations of next except the first - //one. For the first invocation, the priority queue is ready for use - //but for the subsequent invocations, first adjust the queue - adjustPriorityQueue(minSegment); - if (size() == 0) { - minSegment = null; - return false; - } } + minSegment = top(); long startPos = minSegment.getPosition(); KeyValueBuffer nextKey = minSegment.getKey(); @@ -1036,6 +1027,24 @@ public boolean isSameKey() throws IOException { return (hasNext != null) && (hasNext == KeyState.SAME_KEY); } + public boolean hasNext() throws IOException { + if (size() == 0) + return false; + + if (minSegment != null) { + //minSegment is non-null for all invocations of next except the first + //one. For the first invocation, the priority queue is ready for use + //but for the subsequent invocations, first adjust the queue + adjustPriorityQueue(minSegment); + if (size() == 0) { + minSegment = null; + return false; + } + } + + return true; + } + } private static class EmptyIterator implements TezRawKeyValueIterator { @@ -1060,6 +1069,11 @@ public boolean next() throws IOException { return false; } + @Override + public boolean hasNext() throws IOException { + return false; + } + @Override public void close() throws IOException { } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezRawKeyValueIterator.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezRawKeyValueIterator.java index 4e2ce3a7f0..683c9b9ce9 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezRawKeyValueIterator.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezRawKeyValueIterator.java @@ -55,7 +55,16 @@ public interface TezRawKeyValueIterator { * @throws IOException */ boolean next() throws IOException; - + + /** + * Returns true if any items are left in the iterator. + * + * @return true if a call to next will succeed + * false otherwise. + * @throws IOException + */ + boolean hasNext() throws IOException; + /** * Closes the iterator so that the underlying streams can be closed. * diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java index 268e237afe..85e0003991 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java @@ -1122,6 +1122,12 @@ public MRResultIterator(int start, int end) { this.end = end; current = start - 1; } + + @Override + public boolean hasNext() throws IOException { + return (current + 1) < end; + } + public boolean next() throws IOException { return ++current < end; } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedInputLegacy.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedInputLegacy.java index 6ae156a382..b697be5a1d 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedInputLegacy.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedInputLegacy.java @@ -57,6 +57,11 @@ public boolean next() throws IOException { return false; } + @Override + public boolean hasNext() throws IOException { + return false; + } + @Override public void close() throws IOException { } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java index f85272bcaa..d6f62739ce 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java @@ -39,9 +39,11 @@ import org.apache.tez.runtime.api.ExecutionContext; import org.apache.tez.runtime.api.OutputContext; import org.apache.tez.runtime.api.OutputStatisticsReporter; +import org.apache.tez.runtime.api.TaskContext; import org.apache.tez.runtime.api.impl.ExecutionContextImpl; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration.ReportPartitionStats; +import org.apache.tez.runtime.library.common.combine.Combiner; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.runtime.library.conf.OrderedPartitionedKVOutputConfig.SorterImpl; import org.apache.tez.runtime.library.partitioner.HashPartitioner; @@ -438,6 +440,43 @@ public void testMultipleSpills() throws IOException { verifyCounters(sorter, outputContext); } + @Test + public void testWithCombiner() throws IOException { + Configuration conf = getConf(); + conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT, true); + conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_COMBINER_CLASS, DummyCombiner.class.getName()); + this.numOutputs = 5; + this.initialAvailableMem = 5 * 1024 * 1024; + conf.setInt(TezRuntimeConfiguration + .TEZ_RUNTIME_PIPELINED_SORTER_MIN_BLOCK_SIZE_IN_MB, 3); + PipelinedSorter sorter = new PipelinedSorter(this.outputContext, conf, numOutputs, + initialAvailableMem); + + writeData(sorter, 1, 20); + + Path outputFile = sorter.finalOutputFile; + FileSystem fs = outputFile.getFileSystem(conf); + IFile.Reader reader = new IFile.Reader(fs, outputFile, null, null, null, false, -1, 4096); + verifyData(reader); + reader.close(); + + verifyCounters(sorter, outputContext); + } + + // for testWithCombiner + public static class DummyCombiner implements Combiner { + public DummyCombiner(TaskContext ctx) { + // do nothing + } + + @Override + public void combine(TezRawKeyValueIterator rawIter, IFile.Writer writer) throws InterruptedException, IOException { + while (rawIter.next()) { + writer.append(rawIter.getKey(), rawIter.getValue()); + } + } + } + @Test public void testMultipleSpills_WithRLE() throws IOException { Configuration conf = getConf(); From ec9135145fda48917b319b1accc273254c707ae5 Mon Sep 17 00:00:00 2001 From: Zhiyuan Yang Date: Wed, 25 Oct 2017 23:43:46 -0700 Subject: [PATCH 031/512] TEZ-3856. API to access counters in InputInitializerContext (Prasanth Jayachandran via zhiyuany) --- .../tez/runtime/api/InputInitializerContext.java | 7 +++++++ .../main/java/org/apache/tez/dag/app/dag/Vertex.java | 6 ++++++ .../dag/impl/TezRootInputInitializerContextImpl.java | 5 +++++ .../org/apache/tez/dag/app/dag/impl/VertexImpl.java | 11 ++++++++++- .../java/org/apache/tez/mapreduce/TezTestUtils.java | 6 ++++++ 5 files changed, 34 insertions(+), 1 deletion(-) diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializerContext.java b/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializerContext.java index 6a123cf1a1..ccfac46e21 100644 --- a/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializerContext.java +++ b/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializerContext.java @@ -26,6 +26,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.UserPayload; import org.apache.tez.dag.api.event.VertexState; import org.apache.tez.dag.api.event.VertexStateUpdate; @@ -117,4 +118,10 @@ public interface InputInitializerContext { */ void registerForVertexStateUpdates(String vertexName, @Nullable Set stateSet); + /** + * Add custom counters + * + * @param tezCounters counters to add + */ + void addCounters(TezCounters tezCounters); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java index 4d0a4bfd14..ba7624c8f5 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java @@ -86,6 +86,12 @@ public interface Vertex extends Comparable { */ TezCounters getCachedCounters(); + /** + * Add custom counters to the vertex + * @param tezCounters counters to add + */ + void addCounters(TezCounters tezCounters); + int getMaxTaskConcurrency(); Map getTasks(); Task getTask(TezTaskID taskID); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java index 4ca4024d27..f7130545bb 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java @@ -24,6 +24,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.InputDescriptor; import org.apache.tez.dag.api.InputInitializerDescriptor; import org.apache.tez.dag.api.RootInputLeafOutput; @@ -119,4 +120,8 @@ public void registerForVertexStateUpdates(String vertexName, Set st manager.registerForVertexUpdates(vertexName, input.getName(), stateSet); } + @Override + public void addCounters(final TezCounters tezCounters) { + vertex.addCounters(tezCounters); + } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index 209db5a0a0..0bd73ee405 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -231,6 +231,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl // must be a linked map for ordering volatile LinkedHashMap tasks = new LinkedHashMap(); private Object fullCountersLock = new Object(); + private TezCounters counters = new TezCounters(); private TezCounters fullCounters = null; private TezCounters cachedCounters = null; private long cachedCountersTimestamp = 0; @@ -1189,6 +1190,7 @@ public TezCounters getAllCounters() { } TezCounters counters = new TezCounters(); + counters.incrAllCounters(this.counters); return incrTaskCounters(counters, tasks.values()); } finally { @@ -1217,13 +1219,19 @@ public TezCounters getCachedCounters() { } TezCounters counters = new TezCounters(); + counters.incrAllCounters(this.counters); cachedCounters = incrTaskCounters(counters, tasks.values()); return cachedCounters; } finally { readLock.unlock(); } } - + + @Override + public void addCounters(final TezCounters tezCounters) { + counters.incrAllCounters(tezCounters); + } + @Override public int getMaxTaskConcurrency() { return vertexConf.getInt(TezConfiguration.TEZ_AM_VERTEX_MAX_TASK_CONCURRENCY, @@ -3308,6 +3316,7 @@ private VertexStatisticsImpl constructStatistics() { @Private public void constructFinalFullcounters() { this.fullCounters = new TezCounters(); + this.fullCounters.incrAllCounters(counters); this.vertexStats = new VertexStats(); for (Task t : this.tasks.values()) { diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TezTestUtils.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TezTestUtils.java index 8912ad29e5..369afbe6b3 100644 --- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TezTestUtils.java +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TezTestUtils.java @@ -19,6 +19,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.UserPayload; import org.apache.tez.dag.api.event.VertexState; import org.apache.tez.dag.records.TezDAGID; @@ -119,6 +120,11 @@ public void registerForVertexStateUpdates(String vertexName, Set st throw new UnsupportedOperationException("getVertexNumTasks not implemented in this mock"); } + @Override + public void addCounters(TezCounters tezCounters) { + throw new UnsupportedOperationException("addCounters not implemented in this mock"); + } + @Override public UserPayload getUserPayload() { throw new UnsupportedOperationException("getUserPayload not implemented in this mock"); From 8247a643f9fd62b270395ae255036706f5153d7c Mon Sep 17 00:00:00 2001 From: Zhiyuan Yang Date: Thu, 26 Oct 2017 11:40:47 -0700 Subject: [PATCH 032/512] TEZ-3666. Integer overflow in ShuffleVertexManagerBase (Ming Ma via zhiyuany) --- .../FairShuffleVertexManager.java | 9 +- .../ShuffleVertexManagerBase.java | 22 +++-- .../TestFairShuffleVertexManager.java | 99 +++++++++++++------ .../TestShuffleVertexManagerBase.java | 4 +- .../TestShuffleVertexManagerUtils.java | 37 +++++-- 5 files changed, 121 insertions(+), 50 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/FairShuffleVertexManager.java b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/FairShuffleVertexManager.java index a8b336c56a..f3971eb998 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/FairShuffleVertexManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/FairShuffleVertexManager.java @@ -234,7 +234,9 @@ public long[] estimatePartitionSize() { } else { for (int i = 0; i < numOfPartitions; i++) { estimatedPartitionOutputSize[i] = - MB * getExpectedStatsAtIndex(i); + getExpectedStatsAtIndex(i); + LOG.info("Partition index {} with size {}", i, + estimatedPartitionOutputSize[i]); } } return estimatedPartitionOutputSize; @@ -419,9 +421,12 @@ public void compute() { } Iterator it = iterator(); while(it.hasNext()) { + DestinationTaskInputsProperty property = it.next(); sourceVertexInfo.getDestinationInputsProperties().put( - destinationIndex,it.next()); + destinationIndex, property); destinationIndex++; + LOG.info("Destination Index {}: Input Property {}", + destinationIndex, property); } startNextPartitionsGroup(); } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/ShuffleVertexManagerBase.java b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/ShuffleVertexManagerBase.java index 967d0ea7a1..bb63bd5ce2 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/ShuffleVertexManagerBase.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/ShuffleVertexManagerBase.java @@ -148,9 +148,14 @@ int getNumTasks() { int getNumCompletedTasks() { return finishedTaskSet.cardinality(); } - int getExpectedStatsInMBAtIndex(int index) { + + BigInteger getExpectedStatsAtIndex(int index) { return (numVMEventsReceived == 0) ? - 0: statsInMB[index] * numTasks / numVMEventsReceived; + BigInteger.ZERO : + BigInteger.valueOf(statsInMB[index]). + multiply(BigInteger.valueOf(numTasks)). + divide(BigInteger.valueOf(numVMEventsReceived)). + multiply(BigInteger.valueOf(MB)); } } @@ -464,12 +469,17 @@ int getCurrentlyKnownStatsAtIndex(int index) { return stats; } - int getExpectedStatsAtIndex(int index) { - int stats = 0; + long getExpectedStatsAtIndex(int index) { + BigInteger stats = BigInteger.ZERO; for(SourceVertexInfo entry : getAllSourceVertexInfo()) { - stats += entry.getExpectedStatsInMBAtIndex(index); + stats = stats.add(entry.getExpectedStatsAtIndex(index)); + } + if (stats.compareTo(BigInteger.valueOf(Long.MAX_VALUE)) > 0) { + LOG.warn("Partition {}'s size {} exceeded Long.MAX_VALUE", index, stats); + return Long.MAX_VALUE; + } else { + return stats.longValue(); } - return stats; } /** diff --git a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestFairShuffleVertexManager.java b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestFairShuffleVertexManager.java index 61ca785ab6..de857bc06f 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestFairShuffleVertexManager.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestFairShuffleVertexManager.java @@ -112,17 +112,22 @@ public void testInvalidSetup() { @Test(timeout = 5000) public void testReduceSchedulingWithPartitionStats() throws Exception { + final int numScatherAndGatherSourceTasks = 300; final Map newEdgeManagers = new HashMap(); - testSchedulingWithPartitionStats(FairRoutingType.REDUCE_PARALLELISM, - 2, 2, newEdgeManagers); + long[] partitionStats = new long[]{(MB), (2 * MB), (5 * MB)}; + testSchedulingWithPartitionStats( + FairRoutingType.REDUCE_PARALLELISM, numScatherAndGatherSourceTasks, + partitionStats, 2,2, 2, newEdgeManagers); EdgeManagerPluginOnDemand edgeManager = (EdgeManagerPluginOnDemand)newEdgeManagers.values().iterator().next(); // The first destination task fetches two partitions from all source tasks. - // 6 == 3 source tasks * 2 merged partitions - Assert.assertEquals(6, edgeManager.getNumDestinationTaskPhysicalInputs(0)); - for (int sourceTaskIndex = 0; sourceTaskIndex < 3; sourceTaskIndex++) { + // Thus the # of inputs == # of source tasks * 2 merged partitions + Assert.assertEquals(numScatherAndGatherSourceTasks * 2, + edgeManager.getNumDestinationTaskPhysicalInputs(0)); + for (int sourceTaskIndex = 0; + sourceTaskIndex < numScatherAndGatherSourceTasks; sourceTaskIndex++) { for (int j = 0; j < 2; j++) { if (j == 0) { EdgeManagerPluginOnDemand.CompositeEventRouteMetadata routeMetadata = @@ -144,19 +149,26 @@ public void testReduceSchedulingWithPartitionStats() throws Exception { @Test(timeout = 5000) public void testFairSchedulingWithPartitionStats() throws Exception { + final int numScatherAndGatherSourceTasks = 300; final Map newEdgeManagers = new HashMap(); - testSchedulingWithPartitionStats(FairRoutingType.FAIR_PARALLELISM, - 3, 2, newEdgeManagers); + long[] partitionStats = new long[]{(MB), (2 * MB), (5 * MB)}; + + testSchedulingWithPartitionStats( + FairRoutingType.FAIR_PARALLELISM, + numScatherAndGatherSourceTasks, partitionStats, + 2, 3, 2, newEdgeManagers); // Get the first edgeManager which is SCATTER_GATHER. EdgeManagerPluginOnDemand edgeManager = (EdgeManagerPluginOnDemand)newEdgeManagers.values().iterator().next(); // The first destination task fetches two partitions from all source tasks. - // 6 == 3 source tasks * 2 merged partitions - Assert.assertEquals(6, edgeManager.getNumDestinationTaskPhysicalInputs(0)); - for (int sourceTaskIndex = 0; sourceTaskIndex < 3; sourceTaskIndex++) { + // Thus the # of inputs == # of source tasks * 2 merged partitions + Assert.assertEquals(numScatherAndGatherSourceTasks * 2, + edgeManager.getNumDestinationTaskPhysicalInputs(0)); + for (int sourceTaskIndex = 0; sourceTaskIndex < numScatherAndGatherSourceTasks; + sourceTaskIndex++) { for (int j = 0; j < 2; j++) { if (j == 0) { EdgeManagerPluginOnDemand.CompositeEventRouteMetadata routeMetadata = @@ -175,9 +187,10 @@ public void testFairSchedulingWithPartitionStats() throws Exception { } } - // The 2nd destination task fetches one partition from the first source - // task. - Assert.assertEquals(1, edgeManager.getNumDestinationTaskPhysicalInputs(1)); + // The 2nd destination task fetches one partition from the first half of + // source tasks. + Assert.assertEquals(numScatherAndGatherSourceTasks / 2, + edgeManager.getNumDestinationTaskPhysicalInputs(1)); for (int j = 0; j < 2; j++) { if (j == 0) { EdgeManagerPluginOnDemand.CompositeEventRouteMetadata routeMetadata = @@ -193,33 +206,59 @@ public void testFairSchedulingWithPartitionStats() throws Exception { } } - // The 3rd destination task fetches one partition from the 2nd and 3rd - // source task. - Assert.assertEquals(2, edgeManager.getNumDestinationTaskPhysicalInputs(2)); - for (int sourceTaskIndex = 1; sourceTaskIndex < 3; sourceTaskIndex++) { + // The 3rd destination task fetches one partition from 2nd half of + // source tasks. + Assert.assertEquals(numScatherAndGatherSourceTasks / 2, + edgeManager.getNumDestinationTaskPhysicalInputs(2)); + for (int sourceTaskIndex = numScatherAndGatherSourceTasks / 2; + sourceTaskIndex < numScatherAndGatherSourceTasks; sourceTaskIndex++) { for (int j = 0; j < 2; j++) { if (j == 0) { EdgeManagerPluginOnDemand.CompositeEventRouteMetadata routeMetadata = edgeManager.routeCompositeDataMovementEventToDestination(sourceTaskIndex, 2); Assert.assertEquals(1, routeMetadata.getCount()); Assert.assertEquals(2, routeMetadata.getSource()); - Assert.assertEquals(sourceTaskIndex - 1, routeMetadata.getTarget()); + Assert.assertEquals( + sourceTaskIndex - numScatherAndGatherSourceTasks / 2, + routeMetadata.getTarget()); } else { EdgeManagerPluginOnDemand.EventRouteMetadata routeMetadata = edgeManager.routeInputSourceTaskFailedEventToDestination(sourceTaskIndex, 2); Assert.assertEquals(1, routeMetadata.getNumEvents()); - Assert.assertEquals(sourceTaskIndex - 1, routeMetadata.getTargetIndices()[0]); + Assert.assertEquals(sourceTaskIndex - numScatherAndGatherSourceTasks / 2, + routeMetadata.getTargetIndices()[0]); } } } } + @Test(timeout = 500000) + public void testOverflow() throws Exception { + final int numScatherAndGatherSourceTasks = 30000; + final Map newEdgeManagers = + new HashMap(); + final int firstPartitionSize = 1; + final int secondPartitionSize = 2; + final int thirdPartitionSize = 500; + long[] partitionStats = new long[]{(firstPartitionSize * MB), + (secondPartitionSize * MB), (thirdPartitionSize * MB)}; + final int expectedDestinationTasks = + (firstPartitionSize + secondPartitionSize + thirdPartitionSize) + * numScatherAndGatherSourceTasks / 1000; + + testSchedulingWithPartitionStats( + FairRoutingType.FAIR_PARALLELISM, + numScatherAndGatherSourceTasks, partitionStats, 1000, + expectedDestinationTasks, 3, newEdgeManagers); + } + // Create a DAG with one destination vertexes connected to 3 source vertexes. // There are 3 tasks for each vertex. One edge is of type SCATTER_GATHER. // The other edges are BROADCAST. private void testSchedulingWithPartitionStats( - FairRoutingType fairRoutingType, int expectedScheduledTasks, - int expectedNumDestinationConsumerTasks, + FairRoutingType fairRoutingType, int numTasks, long[] partitionStats, + int numCompletedEvents, + int expectedScheduledTasks, int expectedNumDestinationConsumerTasks, Map newEdgeManagers) throws Exception { Configuration conf = new Configuration(); @@ -227,7 +266,7 @@ private void testSchedulingWithPartitionStats( HashMap mockInputVertices = new HashMap(); String r1 = "R1"; - final int numOfTasksInr1 = 3; + final int numOfTasksInr1 = numTasks; EdgeProperty eProp1 = EdgeProperty.create( EdgeProperty.DataMovementType.SCATTER_GATHER, EdgeProperty.DataSourceType.PERSISTED, @@ -291,20 +330,16 @@ private void testSchedulingWithPartitionStats( manager.totalNumBipartiteSourceTasks); Assert.assertEquals(0, manager.numBipartiteSourceTasksCompleted); - //Send an event for r1. - manager.onSourceTaskCompleted(createTaskAttemptIdentifier(r1, 0)); Assert.assertTrue(manager.pendingTasks.size() == numOfTasksInDestination); // no tasks scheduled Assert.assertTrue(manager.totalNumBipartiteSourceTasks == numOfTasksInr1); - long[] sizes = new long[]{(50 * MB), (200 * MB), (500 * MB)}; - VertexManagerEvent vmEvent = getVertexManagerEvent(sizes, 800 * MB, - r1, true); - manager.onVertexManagerEventReceived(vmEvent); //send VM event - //stats from another task - sizes = new long[]{(60 * MB), (300 * MB), (600 * MB)}; - vmEvent = getVertexManagerEvent(sizes, 1200 * MB, r1, true); - manager.onVertexManagerEventReceived(vmEvent); //send VM event + for (int i = 0; i < numCompletedEvents; i++) { + VertexManagerEvent vmEvent = getVertexManagerEvent(partitionStats, 0, + r1, true); + manager.onSourceTaskCompleted(vmEvent.getProducerAttemptIdentifier()); + manager.onVertexManagerEventReceived(vmEvent); //send VM event + } //Send an event for m2. manager.onSourceTaskCompleted(createTaskAttemptIdentifier(m2, 0)); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerBase.java b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerBase.java index 96f46d60f1..9c3a5b34d8 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerBase.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerBase.java @@ -210,7 +210,7 @@ public void testPartitionStats() throws IOException { //{5,9,12,18} in bitmap final long MB = 1024l * 1024l; long[] sizes = new long[]{(0l), (1 * MB), (964 * MB), (48 * MB)}; - VertexManagerEvent vmEvent = getVertexManagerEvent(sizes, 1L, "Vertex", false); + VertexManagerEvent vmEvent = getVertexManagerEvent(sizes, 0, "Vertex", false); manager = createManager(conf, mockContext, 0.01f, 0.75f); manager.onVertexStarted(emptyCompletions); @@ -239,7 +239,7 @@ public void testPartitionStats() throws IOException { Assert.assertEquals(10, manager.getCurrentlyKnownStatsAtIndex(3)); //10 MB bucket // Testing for detailed partition stats - vmEvent = getVertexManagerEvent(sizes, 1L, "Vertex", true); + vmEvent = getVertexManagerEvent(sizes, 0, "Vertex", true); manager = createManager(conf, mockContext, 0.01f, 0.75f); manager.onVertexStarted(emptyCompletions); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerUtils.java b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerUtils.java index 439d65011b..92812224cd 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerUtils.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerUtils.java @@ -118,24 +118,35 @@ VertexManagerPluginContext createVertexManagerContext( } VertexManagerEvent getVertexManagerEvent(long[] sizes, - long totalSize, String vertexName) throws IOException { - return getVertexManagerEvent(sizes, totalSize, vertexName, false); + long inputSize, String vertexName) throws IOException { + return getVertexManagerEvent(sizes, inputSize, vertexName, false); } - VertexManagerEvent getVertexManagerEvent(long[] sizes, - long totalSize, String vertexName, boolean reportDetailedStats) + VertexManagerEvent getVertexManagerEvent(long[] partitionSizes, + long uncompressedTotalSize, String vertexName, boolean reportDetailedStats) throws IOException { ByteBuffer payload; - if (sizes != null) { - RoaringBitmap partitionStats = ShuffleUtils.getPartitionStatsForPhysicalOutput(sizes); + long totalSize = 0; + // Use partition sizes to compute the total size. + if (partitionSizes != null) { + totalSize = estimatedUncompressedSum(partitionSizes); + } else { + totalSize = uncompressedTotalSize; + } + if (partitionSizes != null) { + RoaringBitmap partitionStats = + ShuffleUtils.getPartitionStatsForPhysicalOutput(partitionSizes); DataOutputBuffer dout = new DataOutputBuffer(); partitionStats.serialize(dout); ByteString - partitionStatsBytes = TezCommonUtils.compressByteArrayToByteString(dout.getData()); + partitionStatsBytes = TezCommonUtils.compressByteArrayToByteString( + dout.getData()); if (reportDetailedStats) { payload = VertexManagerEventPayloadProto.newBuilder() .setOutputSize(totalSize) - .setDetailedPartitionStats(ShuffleUtils.getDetailedPartitionStatsForPhysicalOutput(sizes)) + .setDetailedPartitionStats( + ShuffleUtils.getDetailedPartitionStatsForPhysicalOutput( + partitionSizes)) .build().toByteString() .asReadOnlyByteBuffer(); } else { @@ -159,6 +170,16 @@ VertexManagerEvent getVertexManagerEvent(long[] sizes, return vmEvent; } + // Assume 3 : 1 compression ratio to estimate the total size + // of all partitions. + long estimatedUncompressedSum(long[] partitionStats) { + long sum = 0; + for (long partition : partitionStats) { + sum += partition; + } + return sum * 3; + } + public static TaskAttemptIdentifier createTaskAttemptIdentifier(String vName, int tId) { VertexIdentifier mockVertex = mock(VertexIdentifier.class); when(mockVertex.getName()).thenReturn(vName); From 87d7c145ffc71707d1d393fddf94efa2a77d8822 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Fri, 27 Oct 2017 16:48:03 -0500 Subject: [PATCH 033/512] TEZ-3858. Misleading dag level diagnostics in case of invalid vertex event (Zhiyuan Yang via kshukla) --- .../java/org/apache/tez/dag/app/dag/impl/VertexImpl.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index 0bd73ee405..c329ec71ae 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -4061,11 +4061,9 @@ private static class InternalErrorTransition implements SingleArcTransition { @Override public void transition(VertexImpl vertex, VertexEvent event) { - LOG.error("Invalid event " + event.getType() + " on Vertex " - + vertex.getLogIdentifier()); - vertex.eventHandler.handle(new DAGEventDiagnosticsUpdate( - vertex.getDAGId(), "Invalid event " + event.getType() - + " on Vertex " + vertex.getLogIdentifier())); + String msg = "Invalid event on Vertex " + vertex.getLogIdentifier(); + LOG.error(msg); + vertex.eventHandler.handle(new DAGEventDiagnosticsUpdate(vertex.getDAGId(), msg)); vertex.setFinishTime(); vertex.trySetTerminationCause(VertexTerminationCause.INTERNAL_ERROR); vertex.cancelCommits(); From 3fb57c869601ed69436707fd6eeed3331622718b Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Wed, 1 Nov 2017 11:16:19 -0500 Subject: [PATCH 034/512] TEZ-3862. Tez UI: Upgrade em-tgraph to version 0.0.14 (Jonathan Eagles via kshukla) --- tez-ui/src/main/webapp/package.json | 2 +- tez-ui/src/main/webapp/yarn.lock | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json index 3d6846834c..0d2a46ffed 100644 --- a/tez-ui/src/main/webapp/package.json +++ b/tez-ui/src/main/webapp/package.json @@ -63,6 +63,6 @@ "dependencies": { "em-helpers": "0.8.0", "em-table": "0.11.3", - "em-tgraph": "0.0.10" + "em-tgraph": "0.0.14" } } diff --git a/tez-ui/src/main/webapp/yarn.lock b/tez-ui/src/main/webapp/yarn.lock index 93210b7910..866677b01f 100644 --- a/tez-ui/src/main/webapp/yarn.lock +++ b/tez-ui/src/main/webapp/yarn.lock @@ -1411,9 +1411,9 @@ em-table@0.11.3: optionalDependencies: phantomjs-prebuilt "2.1.13" -em-tgraph@0.0.10: - version "0.0.10" - resolved "https://registry.yarnpkg.com/em-tgraph/-/em-tgraph-0.0.10.tgz#8b82a9a1853e4bfd615ff1a9d810435302b0db29" +em-tgraph@0.0.14: + version "0.0.14" + resolved "https://registry.yarnpkg.com/em-tgraph/-/em-tgraph-0.0.14.tgz#4d48b911760f85dec41904e4056ec52542391cc1" dependencies: ember-cli-htmlbars "^1.0.1" ember-cli-less "^1.4.0" From a51af593f3cf9cf42120cb741a2c69eb83a08768 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Wed, 1 Nov 2017 11:26:54 -0500 Subject: [PATCH 035/512] TEZ-3857. Tez TaskImpl can throw Invalid state transition for leaf tasks that do Retro Active Transition (kshukla) --- .../apache/tez/dag/app/dag/impl/TaskImpl.java | 13 ++--- .../tez/dag/app/dag/impl/TestTaskImpl.java | 56 +++++++++++++++++++ 2 files changed, 62 insertions(+), 7 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java index bed41415fb..99cb2e06e7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java @@ -1248,12 +1248,6 @@ private static class TaskRetroactiveFailureTransition @Override public TaskStateInternal transition(TaskImpl task, TaskEvent event) { - if (task.leafVertex) { - LOG.error("Unexpected event for task of leaf vertex " + event.getType() + ", taskId: " - + task.getTaskId()); - task.internalError(event.getType()); - } - TaskEventTAFailed castEvent = (TaskEventTAFailed) event; TezTaskAttemptID failedAttemptId = castEvent.getTaskAttemptID(); TaskAttempt failedAttempt = task.getAttempt(failedAttemptId); @@ -1277,7 +1271,12 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { task.taskAttemptStatus.put(failedAttemptId.getId(), true); return TaskStateInternal.SUCCEEDED; } - + + if (task.leafVertex) { + LOG.error("Unexpected event for task of leaf vertex " + event.getType() + ", taskId: " + + task.getTaskId()); + task.internalError(event.getType()); + } Preconditions.checkState(castEvent.getCausalEvent() != null); TaskAttemptEventOutputFailed destinationEvent = (TaskAttemptEventOutputFailed) castEvent.getCausalEvent(); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java index e03e2829d9..d13e6546e9 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java @@ -1067,6 +1067,62 @@ public void testFailedAttemptStatus() throws InterruptedException { assertTrue("Task should have failed!", mockTask.getState() == TaskState.FAILED); } + @SuppressWarnings("rawtypes") + @Test (timeout = 10000L) + public void testSucceededLeafTaskWithRetroFailures() throws InterruptedException { + Configuration newConf = new Configuration(conf); + newConf.setInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS, 1); + Vertex vertex = mock(Vertex.class); + doReturn(new VertexImpl.VertexConfigImpl(newConf)).when(vertex).getVertexConfig(); + mockTask = new MockTaskImpl(vertexId, partition, + eventHandler, conf, taskCommunicatorManagerInterface, clock, + taskHeartbeatHandler, appContext, true, + taskResource, containerContext, vertex); + TezTaskID taskId = getNewTaskID(); + scheduleTaskAttempt(taskId); + MockTaskAttemptImpl firstMockTaskAttempt = mockTask.getAttemptList().get(0); + launchTaskAttempt(firstMockTaskAttempt.getID()); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + MockTaskAttemptImpl secondMockTaskAttempt = mockTask.getAttemptList().get(1); + launchTaskAttempt(secondMockTaskAttempt.getID()); + + firstMockTaskAttempt.handle(new TaskAttemptEventSchedule( + TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()), 10, 10)); + secondMockTaskAttempt.handle(new TaskAttemptEventSchedule( + TezTaskAttemptID.fromString(secondMockTaskAttempt.toString()), 10, 10)); + firstMockTaskAttempt.handle(new TaskAttemptEventSubmitted( + TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()), mockContainer.getId())); + secondMockTaskAttempt.handle(new TaskAttemptEventSubmitted( + TezTaskAttemptID.fromString(secondMockTaskAttempt.toString()), mockContainer.getId())); + + secondMockTaskAttempt.handle( + new TaskAttemptEventStartedRemotely(TezTaskAttemptID.fromString(secondMockTaskAttempt.toString()))); + firstMockTaskAttempt.handle( + new TaskAttemptEventStartedRemotely(TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()))); + secondMockTaskAttempt.handle( + new TaskAttemptEvent(TezTaskAttemptID.fromString(secondMockTaskAttempt.toString()), + TaskAttemptEventType.TA_DONE)); + firstMockTaskAttempt.handle( + new TaskAttemptEventAttemptFailed(TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()), + TaskAttemptEventType.TA_FAILED, TaskFailureType.NON_FATAL, "test", + TaskAttemptTerminationCause.CONTAINER_EXITED)); + + mockTask.handle(new TaskEventTASucceeded(secondMockTaskAttempt.getID())); + firstMockTaskAttempt.handle(new TaskAttemptEventContainerTerminated(mockContainerId, + firstMockTaskAttempt.getID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); + + InputReadErrorEvent mockReEvent = InputReadErrorEvent.create("", 0, 0); + TezTaskAttemptID mockDestId = firstMockTaskAttempt.getID(); + EventMetaData meta = new EventMetaData(EventProducerConsumerType.INPUT, "Vertex", "Edge", mockDestId); + TezEvent tzEvent = new TezEvent(mockReEvent, meta); + TaskAttemptEventOutputFailed outputFailedEvent = + new TaskAttemptEventOutputFailed(mockDestId, tzEvent, 1); + firstMockTaskAttempt.handle(outputFailedEvent); + mockTask.handle(new TaskEventTAFailed(firstMockTaskAttempt.getID(), TaskFailureType.NON_FATAL, + mock(TaskAttemptEvent.class))); + Assert.assertEquals(mockTask.getInternalState(), TaskStateInternal.SUCCEEDED); + } + private void failAttempt(MockTaskAttemptImpl taskAttempt, int index, int expectedIncompleteAttempts) { InputReadErrorEvent mockReEvent = InputReadErrorEvent.create("", 0, index); TezTaskAttemptID mockDestId = mock(TezTaskAttemptID.class); From a24652735929af0b6ccd42c3ae097ae9847998f1 Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Thu, 2 Nov 2017 03:36:15 +0530 Subject: [PATCH 036/512] TEZ-3861. PipelineSorter setting negative progess (rbalamohan) --- .../tez/runtime/library/common/sort/impl/PipelinedSorter.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java index 07c2fe2e4d..c4782f6a6a 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java @@ -1122,10 +1122,10 @@ public DataInputBuffer getValue() { public boolean next() { // caveat: since we use this as a comparable in the merger if(kvindex == maxindex) return false; + kvindex += 1; if(kvindex % 100 == 0) { - progress.set((kvindex-maxindex) / (float)maxindex); + progress.set(1 - ((maxindex - kvindex) / (float) maxindex)); } - kvindex += 1; return true; } From b96f79fa75dc6cf47e4d648b028ccb12f02308a6 Mon Sep 17 00:00:00 2001 From: Zhiyuan Yang Date: Fri, 10 Nov 2017 16:44:29 -0800 Subject: [PATCH 037/512] TEZ-3855. Allow vertex manager to send event to processor (zhiyuany) --- .../dag/api/VertexManagerPluginContext.java | 13 ++++ .../api/events/CustomProcessorEvent.java | 65 +++++++++++++++++++ tez-api/src/main/proto/Events.proto | 5 ++ .../tez/dag/app/dag/impl/VertexImpl.java | 12 ++++ .../tez/dag/app/dag/impl/VertexManager.java | 26 ++++++++ .../dag/app/dag/impl/TestVertexManager.java | 57 ++++++++++++++-- .../apache/tez/common/ProtoConverters.java | 18 +++++ .../tez/runtime/api/impl/EventType.java | 1 + .../apache/tez/runtime/api/impl/TezEvent.java | 15 +++++ 9 files changed, 208 insertions(+), 4 deletions(-) create mode 100644 tez-api/src/main/java/org/apache/tez/runtime/api/events/CustomProcessorEvent.java diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/VertexManagerPluginContext.java b/tez-api/src/main/java/org/apache/tez/dag/api/VertexManagerPluginContext.java index b858a6531a..b89b279fad 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/VertexManagerPluginContext.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/VertexManagerPluginContext.java @@ -29,8 +29,10 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.tez.dag.api.event.VertexState; +import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.InputSpecUpdate; import org.apache.tez.runtime.api.VertexStatistics; +import org.apache.tez.runtime.api.events.CustomProcessorEvent; import org.apache.tez.runtime.api.events.InputDataInformationEvent; import com.google.common.base.Preconditions; @@ -268,6 +270,17 @@ public void reconfigureVertex(@Nullable Map rootInputSp * task to which events need to be sent. */ public void addRootInputEvents(String inputName, Collection events); + + /** + * Allows a VertexManagerPlugin to send events of custom payload to processor + * of a specific task of managed vertex + * + * It's up to user to make sure taskId is valid + * + * @param events events to be sent + * @param taskId id of a task of managed vertex + */ + public void sendEventToProcessor(Collection events, int taskId); @Deprecated /** diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/events/CustomProcessorEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/CustomProcessorEvent.java new file mode 100644 index 0000000000..9cfb02ea1c --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/runtime/api/events/CustomProcessorEvent.java @@ -0,0 +1,65 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.runtime.api.events; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.tez.runtime.api.Event; + +import java.nio.ByteBuffer; + +public class CustomProcessorEvent extends Event { + private ByteBuffer payload; + + /** + * Version number to indicate what app attempt generated this Event + */ + private int version; + + private CustomProcessorEvent(ByteBuffer payload) { + this(payload, -1); + } + + private CustomProcessorEvent(ByteBuffer payload, int version) { + this.payload = payload; + this.version = version; + } + + public static CustomProcessorEvent create(ByteBuffer payload) { + return new CustomProcessorEvent(payload); + } + + @Private + public static CustomProcessorEvent create(ByteBuffer payload, int version) { + return new CustomProcessorEvent(payload, version); + } + + public ByteBuffer getPayload() { + return payload; + } + + @Private + public void setVersion(int version) { + this.version = version; + } + + public int getVersion() { + return version; + } +} diff --git a/tez-api/src/main/proto/Events.proto b/tez-api/src/main/proto/Events.proto index e018864939..71235004da 100644 --- a/tez-api/src/main/proto/Events.proto +++ b/tez-api/src/main/proto/Events.proto @@ -69,3 +69,8 @@ message RootInputInitializerEventProto { optional string target_input_name = 2; optional bytes user_payload = 3; } + +message CustomProcessorEventProto { + optional bytes user_payload = 1; + required int32 version = 2; +} diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index c329ec71ae..13cfb8fa1f 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -167,6 +167,7 @@ import org.apache.tez.runtime.api.TaskAttemptIdentifier; import org.apache.tez.runtime.api.VertexStatistics; import org.apache.tez.runtime.api.events.CompositeDataMovementEvent; +import org.apache.tez.runtime.api.events.CustomProcessorEvent; import org.apache.tez.runtime.api.events.DataMovementEvent; import org.apache.tez.runtime.api.events.InputDataInformationEvent; import org.apache.tez.runtime.api.events.InputFailedEvent; @@ -3884,6 +3885,17 @@ private void handleRoutedTezEvents(List tezEvents, boolean isPendingEv } EventMetaData sourceMeta = tezEvent.getSourceInfo(); switch(tezEvent.getEventType()) { + case CUSTOM_PROCESSOR_EVENT: + { + // set version as app attempt id + ((CustomProcessorEvent) tezEvent.getEvent()).setVersion( + appContext.getApplicationAttemptId().getAttemptId()); + // route event to task + EventMetaData destinationMeta = tezEvent.getDestinationInfo(); + Task targetTask = getTask(destinationMeta.getTaskAttemptID().getTaskID()); + targetTask.registerTezEvent(tezEvent); + } + break; case INPUT_FAILED_EVENT: case DATA_MOVEMENT_EVENT: case COMPOSITE_DATA_MOVEMENT_EVENT: diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java index b7d3428c6b..7a1547f0e3 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java @@ -36,6 +36,9 @@ import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.tez.dag.app.dag.event.DAGEventInternalError; +import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent; +import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.runtime.api.events.CustomProcessorEvent; import org.apache.tez.runtime.api.impl.GroupInputSpec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -272,6 +275,29 @@ public TezEvent apply(InputDataInformationEvent riEvent) { // Recovery handling is taken care of by the Vertex. } + @Override + public void sendEventToProcessor(Collection events, int taskId) { + checkAndThrowIfDone(); + Preconditions.checkArgument(taskId >= 0 && taskId < managedVertex.getTotalTasks(), + "Invalid taskId " + taskId + "; " + "There are " + managedVertex.getTotalTasks() + + " tasks in total."); + + if (events != null && events.size() > 0) { + List tezEvents = new ArrayList<>(); + for (CustomProcessorEvent event : events) { + TezEvent tezEvent = new TezEvent(event, null); + // use dummy task attempt id since this is not an task attempt specific event and task + // attempt id won't be used anyway + EventMetaData destinationMeta = new EventMetaData(EventProducerConsumerType.PROCESSOR, + managedVertex.getName(), managedVertex.getName(), + TezTaskAttemptID.getInstance(managedVertex.getTask(taskId).getTaskId(), -1)); + tezEvent.setDestinationInfo(destinationMeta); + tezEvents.add(tezEvent); + } + appContext.getEventHandler().handle( + new VertexEventRouteEvent(managedVertex.getVertexId(), tezEvents)); + } + } @Override public synchronized void setVertexLocationHint(VertexLocationHint locationHint) { diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java index 3d9f2714af..c850d6880a 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java @@ -18,14 +18,17 @@ package org.apache.tez.dag.app.dag.impl; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Mockito.RETURNS_DEEP_STUBS; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -56,8 +59,10 @@ import org.apache.tez.dag.app.dag.Vertex; import org.apache.tez.dag.app.dag.event.CallableEvent; import org.apache.tez.dag.app.dag.event.VertexEventInputDataInformation; +import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent; import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.TaskAttemptIdentifier; +import org.apache.tez.runtime.api.events.CustomProcessorEvent; import org.apache.tez.runtime.api.events.InputDataInformationEvent; import org.apache.tez.runtime.api.events.VertexManagerEvent; import org.apache.tez.runtime.api.impl.GroupInputSpec; @@ -214,10 +219,9 @@ public void testOnRootVertexInitialized2() throws Exception { @Test(timeout = 5000) public void testVMPluginCtxGetInputVertexGroup() throws Exception { VertexManager vm = - new VertexManager( - VertexManagerPluginDescriptor.create(CustomVertexManager.class - .getName()), UserGroupInformation.getCurrentUser(), - mockVertex, mockAppContext, mock(StateChangeNotifier.class)); + new VertexManager(VertexManagerPluginDescriptor.create(CustomVertexManager.class.getName()), + UserGroupInformation.getCurrentUser(), mockVertex, mockAppContext, + mock(StateChangeNotifier.class)); assertTrue(vm.pluginContext.getInputVertexGroups().isEmpty()); @@ -232,6 +236,51 @@ public void testVMPluginCtxGetInputVertexGroup() throws Exception { assertTrue(groups.get(group).contains(v2)); } + @Test(timeout = 5000) + public void testSendCustomProcessorEvent() throws Exception { + VertexManager vm = + new VertexManager(VertexManagerPluginDescriptor.create(CustomVertexManager.class.getName()), + UserGroupInformation.getCurrentUser(), mockVertex, mockAppContext, + mock(StateChangeNotifier.class)); + ArgumentCaptor requestCaptor = + ArgumentCaptor.forClass(VertexEventRouteEvent.class); + + when(mockVertex.getTotalTasks()).thenReturn(2); + + List events = new ArrayList<>(); + // task id too small, should fail + try { + vm.pluginContext.sendEventToProcessor(events, -1); + fail("Should fail for invalid task id"); + } catch (IllegalArgumentException exception) { + assertTrue(exception.getMessage().contains("Invalid taskId")); + } + // task id too large, should fail + try { + vm.pluginContext.sendEventToProcessor(events, 10); + fail("Should fail for invalid task id"); + } catch (IllegalArgumentException exception) { + assertTrue(exception.getMessage().contains("Invalid taskId")); + } + + // null event, do nothing + vm.pluginContext.sendEventToProcessor(null, 0); + verify(mockHandler, never()).handle(requestCaptor.capture()); + + // empty event + vm.pluginContext.sendEventToProcessor(events, 1); + verify(mockHandler, never()).handle(requestCaptor.capture()); + + //events.add(); + byte[] payload = new byte[] {1,2,3}; + events.add(CustomProcessorEvent.create(ByteBuffer.wrap(payload))); + vm.pluginContext.sendEventToProcessor(events, 1); + verify(mockHandler, times(1)).handle(requestCaptor.capture()); + CustomProcessorEvent cpe = + (CustomProcessorEvent)(requestCaptor.getValue().getEvents().get(0).getEvent()); + assertArrayEquals(payload, cpe.getPayload().array()); + } + public static class CustomVertexManager extends VertexManagerPlugin { private Map> cachedEventMap = new HashMap>(); diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/common/ProtoConverters.java b/tez-runtime-internals/src/main/java/org/apache/tez/common/ProtoConverters.java index ea90158d52..e4285708a5 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/common/ProtoConverters.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/common/ProtoConverters.java @@ -21,6 +21,7 @@ import com.google.protobuf.ByteString; import org.apache.tez.runtime.api.events.CompositeDataMovementEvent; +import org.apache.tez.runtime.api.events.CustomProcessorEvent; import org.apache.tez.runtime.api.events.DataMovementEvent; import org.apache.tez.runtime.api.events.CompositeRoutedDataMovementEvent; import org.apache.tez.runtime.api.events.EventProtos; @@ -31,6 +32,23 @@ public class ProtoConverters { + public static EventProtos.CustomProcessorEventProto convertCustomProcessorEventToProto( + CustomProcessorEvent event) { + EventProtos.CustomProcessorEventProto.Builder builder = + EventProtos.CustomProcessorEventProto.newBuilder(); + if (event.getPayload() != null) { + builder.setUserPayload(ByteString.copyFrom(event.getPayload())); + } + builder.setVersion(event.getVersion()); + return builder.build(); + } + + public static CustomProcessorEvent convertCustomProcessorEventFromProto( + EventProtos.CustomProcessorEventProto proto) { + return CustomProcessorEvent.create(proto.getUserPayload() != null ? + proto.getUserPayload().asReadOnlyByteBuffer() : null, proto.getVersion()); + } + public static EventProtos.DataMovementEventProto convertDataMovementEventToProto( DataMovementEvent event) { EventProtos.DataMovementEventProto.Builder builder = diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventType.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventType.java index e573526671..7e365b1870 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventType.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventType.java @@ -31,4 +31,5 @@ public enum EventType { COMPOSITE_DATA_MOVEMENT_EVENT, ROOT_INPUT_INITIALIZER_EVENT, COMPOSITE_ROUTED_DATA_MOVEMENT_EVENT, + CUSTOM_PROCESSOR_EVENT, } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java index 1a90ada78b..e7af4a1ebe 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java @@ -30,6 +30,7 @@ import org.apache.tez.dag.api.TezUncheckedException; import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.events.CompositeDataMovementEvent; +import org.apache.tez.runtime.api.events.CustomProcessorEvent; import org.apache.tez.runtime.api.events.DataMovementEvent; import org.apache.tez.runtime.api.events.CompositeRoutedDataMovementEvent; import org.apache.tez.runtime.api.events.EventProtos; @@ -57,6 +58,8 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import static org.apache.tez.runtime.api.events.EventProtos.*; + public class TezEvent implements Writable { private EventType eventType; @@ -82,6 +85,8 @@ public TezEvent(Event event, EventMetaData sourceInfo, long time) { this.setSourceInfo(sourceInfo); if (event instanceof DataMovementEvent) { eventType = EventType.DATA_MOVEMENT_EVENT; + } else if (event instanceof CustomProcessorEvent) { + eventType = EventType.CUSTOM_PROCESSOR_EVENT; } else if (event instanceof CompositeDataMovementEvent) { eventType = EventType.COMPOSITE_DATA_MOVEMENT_EVENT; } else if (event instanceof CompositeRoutedDataMovementEvent) { @@ -157,6 +162,11 @@ private void serializeEvent(DataOutput out) throws IOException { } else { AbstractMessage message; switch (eventType) { + case CUSTOM_PROCESSOR_EVENT: + message = + ProtoConverters.convertCustomProcessorEventToProto( + (CustomProcessorEvent) event); + break; case DATA_MOVEMENT_EVENT: message = ProtoConverters.convertDataMovementEventToProto( @@ -260,6 +270,11 @@ private void deserializeEvent(DataInput in) throws IOException { } input = CodedInputStream.newInstance(eventBytes, startOffset, eventBytesLen); switch (eventType) { + case CUSTOM_PROCESSOR_EVENT: + CustomProcessorEventProto cpProto = + CustomProcessorEventProto.parseFrom(input); + event = ProtoConverters.convertCustomProcessorEventFromProto(cpProto); + break; case DATA_MOVEMENT_EVENT: DataMovementEventProto dmProto = DataMovementEventProto.parseFrom(input); From b61e55ce9362e3a11761d57c876c83f09b03b159 Mon Sep 17 00:00:00 2001 From: Zhiyuan Yang Date: Wed, 15 Nov 2017 13:48:43 -0800 Subject: [PATCH 038/512] TEZ-3855 addendum. Allow vertex manager to send event to processor (zhiyuany) --- .../org/apache/tez/runtime/api/events/CustomProcessorEvent.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/events/CustomProcessorEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/CustomProcessorEvent.java index 9cfb02ea1c..7d84a1d79d 100644 --- a/tez-api/src/main/java/org/apache/tez/runtime/api/events/CustomProcessorEvent.java +++ b/tez-api/src/main/java/org/apache/tez/runtime/api/events/CustomProcessorEvent.java @@ -51,7 +51,7 @@ public static CustomProcessorEvent create(ByteBuffer payload, int version) { } public ByteBuffer getPayload() { - return payload; + return payload.asReadOnlyBuffer(); } @Private From c85f9277429c65570749de5769e5c74b9849a112 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Wed, 29 Nov 2017 06:57:46 -0600 Subject: [PATCH 039/512] TEZ-3867. testSendCustomProcessorEvent try to get array out of read only ByteBuffer (Zhiyuan Yang via kshukla) --- .../apache/tez/dag/app/dag/impl/TestVertexManager.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java index c850d6880a..f7a60377c4 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java @@ -278,7 +278,15 @@ public void testSendCustomProcessorEvent() throws Exception { verify(mockHandler, times(1)).handle(requestCaptor.capture()); CustomProcessorEvent cpe = (CustomProcessorEvent)(requestCaptor.getValue().getEvents().get(0).getEvent()); - assertArrayEquals(payload, cpe.getPayload().array()); + + // should be able to get payload any times + for (int i = 0; i < 2; i++) { + ByteBuffer payloadBuffer = cpe.getPayload(); + assertEquals(payload.length, payloadBuffer.remaining()); + for (byte aPayload : payload) { + assertEquals(aPayload, payloadBuffer.get()); + } + } } public static class CustomVertexManager extends VertexManagerPlugin { From 07c4c3a49bf6b252186ea40a777d930f63cffc5f Mon Sep 17 00:00:00 2001 From: Siddharth Seth Date: Fri, 1 Dec 2017 13:27:56 -0800 Subject: [PATCH 040/512] TEZ-3868. Update website to factor in the TEZ trademark registration. --- docs/src/site/markdown/index.md | 4 +-- .../markdown/releases/apache-tez-0-5-0.md | 4 +-- .../markdown/releases/apache-tez-0-5-1.md | 4 +-- .../markdown/releases/apache-tez-0-5-2.md | 4 +-- .../markdown/releases/apache-tez-0-5-3.md | 4 +-- .../markdown/releases/apache-tez-0-5-4.md | 4 +-- .../markdown/releases/apache-tez-0-6-0.md | 4 +-- .../markdown/releases/apache-tez-0-6-1.md | 4 +-- .../markdown/releases/apache-tez-0-6-2.md | 4 +-- .../markdown/releases/apache-tez-0-7-0.md | 4 +-- .../markdown/releases/apache-tez-0-7-1.md | 4 +-- .../releases/apache-tez-0-8-0-alpha.md | 4 +-- .../releases/apache-tez-0-8-1-alpha.md | 4 +-- .../markdown/releases/apache-tez-0-8-2.md | 4 +-- .../markdown/releases/apache-tez-0-8-3.md | 4 +-- .../markdown/releases/apache-tez-0-8-4.md | 4 +-- .../markdown/releases/apache-tez-0-8-5.md | 4 +-- .../markdown/releases/apache-tez-0-9-0.md | 4 +-- docs/src/site/markdown/releases/index.md | 36 +++++++++---------- docs/src/site/site.xml | 4 +-- 20 files changed, 56 insertions(+), 56 deletions(-) diff --git a/docs/src/site/markdown/index.md b/docs/src/site/markdown/index.md index d0f307a4b4..d41a98b12b 100644 --- a/docs/src/site/markdown/index.md +++ b/docs/src/site/markdown/index.md @@ -15,12 +15,12 @@ limitations under the License. --> -Welcome to Apache Tez™ +Welcome to Apache TEZ® Introduction ------------ -The Apache Tez™ project is aimed at building an application framework +The Apache TEZ® project is aimed at building an application framework which allows for a complex directed-acyclic-graph of tasks for processing data. It is currently built atop [Apache Hadoop YARN](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html). diff --git a/docs/src/site/markdown/releases/apache-tez-0-5-0.md b/docs/src/site/markdown/releases/apache-tez-0-5-0.md index f206ea6f7f..c8cd2e696f 100644 --- a/docs/src/site/markdown/releases/apache-tez-0-5-0.md +++ b/docs/src/site/markdown/releases/apache-tez-0-5-0.md @@ -15,9 +15,9 @@ limitations under the License. --> -Apache Tez™ 0.5.0 +Apache TEZ® 0.5.0 -Apache Tez™ 0.5.0 +Apache TEZ® 0.5.0 ---------------- - [Download Release Artifacts](http://archive.apache.org/dist/tez/0.5.0/) diff --git a/docs/src/site/markdown/releases/apache-tez-0-5-1.md b/docs/src/site/markdown/releases/apache-tez-0-5-1.md index 01417ba347..12c3c001ab 100644 --- a/docs/src/site/markdown/releases/apache-tez-0-5-1.md +++ b/docs/src/site/markdown/releases/apache-tez-0-5-1.md @@ -15,9 +15,9 @@ limitations under the License. --> -Apache Tez™ 0.5.1 +Apache TEZ® 0.5.1 -Apache Tez™ 0.5.1 +Apache TEZ® 0.5.1 ---------------- - [Download Release Artifacts](http://archive.apache.org/dist/tez/0.5.1/) diff --git a/docs/src/site/markdown/releases/apache-tez-0-5-2.md b/docs/src/site/markdown/releases/apache-tez-0-5-2.md index 99ca44b759..6acb130e47 100644 --- a/docs/src/site/markdown/releases/apache-tez-0-5-2.md +++ b/docs/src/site/markdown/releases/apache-tez-0-5-2.md @@ -15,9 +15,9 @@ limitations under the License. --> -Apache Tez™ 0.5.2 +Apache TEZ® 0.5.2 -Apache Tez™ 0.5.2 +Apache TEZ® 0.5.2 ---------------- - [Download Release Artifacts](http://archive.apache.org/dist/tez/0.5.2/) diff --git a/docs/src/site/markdown/releases/apache-tez-0-5-3.md b/docs/src/site/markdown/releases/apache-tez-0-5-3.md index 7e00fe9092..750fe30536 100644 --- a/docs/src/site/markdown/releases/apache-tez-0-5-3.md +++ b/docs/src/site/markdown/releases/apache-tez-0-5-3.md @@ -15,9 +15,9 @@ limitations under the License. --> -Apache Tez™ 0.5.3 +Apache TEZ® 0.5.3 -Apache Tez™ 0.5.3 +Apache TEZ® 0.5.3 ---------------- - [Download Release Artifacts](http://archive.apache.org/dist/tez/0.5.3/) diff --git a/docs/src/site/markdown/releases/apache-tez-0-5-4.md b/docs/src/site/markdown/releases/apache-tez-0-5-4.md index 9d3f96c03b..54c5d0501d 100644 --- a/docs/src/site/markdown/releases/apache-tez-0-5-4.md +++ b/docs/src/site/markdown/releases/apache-tez-0-5-4.md @@ -15,9 +15,9 @@ limitations under the License. --> -Apache Tez™ 0.5.4 +Apache TEZ® 0.5.4 -Apache Tez™ 0.5.4 +Apache TEZ® 0.5.4 ---------------- - [Download Release Artifacts](http://www.apache.org/dyn/closer.lua/tez/0.5.4/) diff --git a/docs/src/site/markdown/releases/apache-tez-0-6-0.md b/docs/src/site/markdown/releases/apache-tez-0-6-0.md index 473d03b807..96048bcb28 100644 --- a/docs/src/site/markdown/releases/apache-tez-0-6-0.md +++ b/docs/src/site/markdown/releases/apache-tez-0-6-0.md @@ -15,9 +15,9 @@ limitations under the License. --> -Apache Tez™ 0.6.0 +Apache TEZ® 0.6.0 -Apache Tez™ 0.6.0 +Apache TEZ® 0.6.0 ---------------- - [Download Release Artifacts](http://archive.apache.org/dist/tez/0.6.0/) diff --git a/docs/src/site/markdown/releases/apache-tez-0-6-1.md b/docs/src/site/markdown/releases/apache-tez-0-6-1.md index 12854778b2..17b7d647e5 100644 --- a/docs/src/site/markdown/releases/apache-tez-0-6-1.md +++ b/docs/src/site/markdown/releases/apache-tez-0-6-1.md @@ -15,9 +15,9 @@ limitations under the License. --> -Apache Tez™ 0.6.1 +Apache TEZ® 0.6.1 -Apache Tez™ 0.6.1 +Apache TEZ® 0.6.1 ---------------- - [Download Release Artifacts](http://archive.apache.org/dist/tez/tez/0.6.1/) diff --git a/docs/src/site/markdown/releases/apache-tez-0-6-2.md b/docs/src/site/markdown/releases/apache-tez-0-6-2.md index 5a898e6c96..b1f97572b2 100644 --- a/docs/src/site/markdown/releases/apache-tez-0-6-2.md +++ b/docs/src/site/markdown/releases/apache-tez-0-6-2.md @@ -15,9 +15,9 @@ limitations under the License. --> -Apache Tez™ 0.6.2 +Apache TEZ® 0.6.2 -Apache Tez™ 0.6.2 +Apache TEZ® 0.6.2 ---------------- - [Download Release Artifacts](http://www.apache.org/dyn/closer.lua/tez/0.6.2/) diff --git a/docs/src/site/markdown/releases/apache-tez-0-7-0.md b/docs/src/site/markdown/releases/apache-tez-0-7-0.md index 8c0f956c60..fd8b48d646 100644 --- a/docs/src/site/markdown/releases/apache-tez-0-7-0.md +++ b/docs/src/site/markdown/releases/apache-tez-0-7-0.md @@ -15,9 +15,9 @@ limitations under the License. --> -Apache Tez™ 0.7.0 +Apache TEZ® 0.7.0 -Apache Tez™ 0.7.0 +Apache TEZ® 0.7.0 ---------------- - [Download Release Artifacts](http://archive.apache.org/dist/tez/0.7.0/) diff --git a/docs/src/site/markdown/releases/apache-tez-0-7-1.md b/docs/src/site/markdown/releases/apache-tez-0-7-1.md index 1c1766f241..152bd47053 100644 --- a/docs/src/site/markdown/releases/apache-tez-0-7-1.md +++ b/docs/src/site/markdown/releases/apache-tez-0-7-1.md @@ -15,9 +15,9 @@ limitations under the License. --> -Apache Tez™ 0.7.1 +Apache TEZ® 0.7.1 -Apache Tez™ 0.7.1 +Apache TEZ® 0.7.1 ---------------- - [Download Release Artifacts](http://www.apache.org/dyn/closer.lua/tez/0.7.1/) diff --git a/docs/src/site/markdown/releases/apache-tez-0-8-0-alpha.md b/docs/src/site/markdown/releases/apache-tez-0-8-0-alpha.md index 5854ecd74f..2a1d4d18ec 100644 --- a/docs/src/site/markdown/releases/apache-tez-0-8-0-alpha.md +++ b/docs/src/site/markdown/releases/apache-tez-0-8-0-alpha.md @@ -15,9 +15,9 @@ limitations under the License. --> -Apache Tez™ 0.8.0-alpha +Apache TEZ® 0.8.0-alpha -Apache Tez™ 0.8.0-alpha +Apache TEZ® 0.8.0-alpha ---------------------- - [Download Release Artifacts](http://archive.apache.org/dist/tez/0.8.0-alpha/) diff --git a/docs/src/site/markdown/releases/apache-tez-0-8-1-alpha.md b/docs/src/site/markdown/releases/apache-tez-0-8-1-alpha.md index 806292beb9..c8db8ed56b 100644 --- a/docs/src/site/markdown/releases/apache-tez-0-8-1-alpha.md +++ b/docs/src/site/markdown/releases/apache-tez-0-8-1-alpha.md @@ -15,9 +15,9 @@ limitations under the License. --> -Apache Tez™ 0.8.1-alpha +Apache TEZ® 0.8.1-alpha -Apache Tez™ 0.8.1-alpha +Apache TEZ® 0.8.1-alpha ---------------------- - [Download Release Artifacts](http://archive.apache.org/dist/tez/0.8.1-alpha/) diff --git a/docs/src/site/markdown/releases/apache-tez-0-8-2.md b/docs/src/site/markdown/releases/apache-tez-0-8-2.md index 8c25155323..f7282ff1b6 100644 --- a/docs/src/site/markdown/releases/apache-tez-0-8-2.md +++ b/docs/src/site/markdown/releases/apache-tez-0-8-2.md @@ -15,9 +15,9 @@ limitations under the License. --> -Apache Tez™ 0.8.2 +Apache TEZ® 0.8.2 -Apache Tez™ 0.8.2 +Apache TEZ® 0.8.2 ---------------------- - [Download Release Artifacts](http://archive.apache.org/dist/tez/0.8.2/) diff --git a/docs/src/site/markdown/releases/apache-tez-0-8-3.md b/docs/src/site/markdown/releases/apache-tez-0-8-3.md index 5318652b6f..ca871779ba 100644 --- a/docs/src/site/markdown/releases/apache-tez-0-8-3.md +++ b/docs/src/site/markdown/releases/apache-tez-0-8-3.md @@ -15,9 +15,9 @@ limitations under the License. --> -Apache Tez™ 0.8.3 +Apache TEZ® 0.8.3 -Apache Tez™ 0.8.3 +Apache TEZ® 0.8.3 ---------------------- - [Download Release Artifacts](http://www.apache.org/dyn/closer.lua/tez/0.8.3/) diff --git a/docs/src/site/markdown/releases/apache-tez-0-8-4.md b/docs/src/site/markdown/releases/apache-tez-0-8-4.md index 2250dbee4c..802db939ab 100644 --- a/docs/src/site/markdown/releases/apache-tez-0-8-4.md +++ b/docs/src/site/markdown/releases/apache-tez-0-8-4.md @@ -15,9 +15,9 @@ limitations under the License. --> -Apache Tez™ 0.8.4 +Apache TEZ® 0.8.4 -Apache Tez™ 0.8.4 +Apache TEZ® 0.8.4 ---------------------- - [Download Release Artifacts](http://www.apache.org/dyn/closer.lua/tez/0.8.4/) diff --git a/docs/src/site/markdown/releases/apache-tez-0-8-5.md b/docs/src/site/markdown/releases/apache-tez-0-8-5.md index 9b73cec00a..cebc17a405 100644 --- a/docs/src/site/markdown/releases/apache-tez-0-8-5.md +++ b/docs/src/site/markdown/releases/apache-tez-0-8-5.md @@ -15,9 +15,9 @@ limitations under the License. --> -Apache Tez™ 0.8.5 +Apache TEZ® 0.8.5 -Apache Tez™ 0.8.5 +Apache TEZ® 0.8.5 ---------------------- - [Download Release Artifacts](http://www.apache.org/dyn/closer.lua/tez/0.8.5/) diff --git a/docs/src/site/markdown/releases/apache-tez-0-9-0.md b/docs/src/site/markdown/releases/apache-tez-0-9-0.md index 5059790dd8..12095ac2b6 100644 --- a/docs/src/site/markdown/releases/apache-tez-0-9-0.md +++ b/docs/src/site/markdown/releases/apache-tez-0-9-0.md @@ -15,9 +15,9 @@ limitations under the License. --> -Apache Tez™ 0.9.0 +Apache TEZ® 0.9.0 -Apache Tez™ 0.9.0 +Apache TEZ® 0.9.0 ---------------------- - [Download Release Artifacts](http://www.apache.org/dyn/closer.lua/tez/0.9.0/) diff --git a/docs/src/site/markdown/releases/index.md b/docs/src/site/markdown/releases/index.md index 9e9df225ff..08f4819113 100644 --- a/docs/src/site/markdown/releases/index.md +++ b/docs/src/site/markdown/releases/index.md @@ -15,24 +15,24 @@ limitations under the License. --> -Apache Tez™ Releases +Apache TEZ® Releases Releases ------------ -- [Apache Tez™ 0.9.0](./apache-tez-0-9-0.html) (Jul 27, 2017) -- [Apache Tez™ 0.8.5](./apache-tez-0-8-5.html) (Mar 13, 2017) -- [Apache Tez™ 0.8.4](./apache-tez-0-8-4.html) (Jul 08, 2016) -- [Apache Tez™ 0.8.3](./apache-tez-0-8-3.html) (Apr 15, 2016) -- [Apache Tez™ 0.8.2](./apache-tez-0-8-2.html) (Jan 19, 2016) -- [Apache Tez™ 0.8.1-alpha](./apache-tez-0-8-1-alpha.html) (Oct 12, 2015) -- [Apache Tez™ 0.8.0-alpha](./apache-tez-0-8-0-alpha.html) (Sep 01, 2015) -- [Apache Tez™ 0.7.1](./apache-tez-0-7-1.html) (May 10, 2016) -- [Apache Tez™ 0.7.0](./apache-tez-0-7-0.html) (May 18, 2015) -- [Apache Tez™ 0.6.2](./apache-tez-0-6-2.html) (Aug 07, 2015) -- [Apache Tez™ 0.6.1](./apache-tez-0-6-1.html) (May 18, 2015) -- [Apache Tez™ 0.6.0](./apache-tez-0-6-0.html) (Jan 23, 2015) -- [Apache Tez™ 0.5.4](./apache-tez-0-5-4.html) (Jun 26, 2015) -- [Apache Tez™ 0.5.3](./apache-tez-0-5-3.html) (Dec 10, 2014) -- [Apache Tez™ 0.5.2](./apache-tez-0-5-2.html) (Nov 07, 2014) -- [Apache Tez™ 0.5.1](./apache-tez-0-5-1.html) (Oct 08, 2014) -- [Apache Tez™ 0.5.0](./apache-tez-0-5-0.html) (Sep 04, 2014) +- [Apache TEZ® 0.9.0](./apache-tez-0-9-0.html) (Jul 27, 2017) +- [Apache TEZ® 0.8.5](./apache-tez-0-8-5.html) (Mar 13, 2017) +- [Apache TEZ® 0.8.4](./apache-tez-0-8-4.html) (Jul 08, 2016) +- [Apache TEZ® 0.8.3](./apache-tez-0-8-3.html) (Apr 15, 2016) +- [Apache TEZ® 0.8.2](./apache-tez-0-8-2.html) (Jan 19, 2016) +- [Apache TEZ® 0.8.1-alpha](./apache-tez-0-8-1-alpha.html) (Oct 12, 2015) +- [Apache TEZ® 0.8.0-alpha](./apache-tez-0-8-0-alpha.html) (Sep 01, 2015) +- [Apache TEZ® 0.7.1](./apache-tez-0-7-1.html) (May 10, 2016) +- [Apache TEZ® 0.7.0](./apache-tez-0-7-0.html) (May 18, 2015) +- [Apache TEZ® 0.6.2](./apache-tez-0-6-2.html) (Aug 07, 2015) +- [Apache TEZ® 0.6.1](./apache-tez-0-6-1.html) (May 18, 2015) +- [Apache TEZ® 0.6.0](./apache-tez-0-6-0.html) (Jan 23, 2015) +- [Apache TEZ® 0.5.4](./apache-tez-0-5-4.html) (Jun 26, 2015) +- [Apache TEZ® 0.5.3](./apache-tez-0-5-3.html) (Dec 10, 2014) +- [Apache TEZ® 0.5.2](./apache-tez-0-5-2.html) (Nov 07, 2014) +- [Apache TEZ® 0.5.1](./apache-tez-0-5-1.html) (Oct 08, 2014) +- [Apache TEZ® 0.5.0](./apache-tez-0-5-0.html) (Sep 04, 2014) diff --git a/docs/src/site/site.xml b/docs/src/site/site.xml index 91d15a911f..226293eac6 100644 --- a/docs/src/site/site.xml +++ b/docs/src/site/site.xml @@ -94,7 +94,7 @@ - +

@@ -120,7 +120,7 @@ - + From cfede26b054a8a0243e969ee0e62b25a6f542f10 Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Mon, 4 Dec 2017 03:34:32 +0530 Subject: [PATCH 041/512] TEZ-3869. Analyzer: Fix VertexInfo::getLastTaskToFinish comparison (rbalamohan) --- .../history/parser/datamodel/VertexInfo.java | 17 +++++------------ 1 file changed, 5 insertions(+), 12 deletions(-) diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java index 0f6831b873..038a874de3 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java @@ -521,9 +521,7 @@ public final TaskInfo getFirstTaskToStart() { } Collections.sort(taskInfoList, new Comparator() { @Override public int compare(TaskInfo o1, TaskInfo o2) { - return (o1.getStartTimeInterval() < o2.getStartTimeInterval()) ? -1 : - ((o1.getStartTimeInterval() == o2.getStartTimeInterval()) ? - 0 : 1); + return Long.compare(o1.getStartTimeInterval(), o2.getStartTimeInterval()); } }); return taskInfoList.get(0); @@ -541,9 +539,7 @@ public final TaskInfo getLastTaskToFinish() { } Collections.sort(taskInfoList, new Comparator() { @Override public int compare(TaskInfo o1, TaskInfo o2) { - return (o1.getFinishTimeInterval() > o2.getFinishTimeInterval()) ? -1 : - ((o1.getStartTimeInterval() == o2.getStartTimeInterval()) ? - 0 : 1); + return -1 * Long.compare(o1.getFinishTimeInterval(), o2.getFinishTimeInterval()); } }); return taskInfoList.get(0); @@ -589,8 +585,7 @@ public final long getMaxTaskDuration() { private Ordering orderingOnTimeTaken() { return Ordering.from(new Comparator() { @Override public int compare(TaskInfo o1, TaskInfo o2) { - return (o1.getTimeTaken() < o2.getTimeTaken()) ? -1 : - ((o1.getTimeTaken() == o2.getTimeTaken()) ? 0 : 1); + return Long.compare(o1.getTimeTaken(), o2.getTimeTaken()); } }); } @@ -598,8 +593,7 @@ private Ordering orderingOnTimeTaken() { private Ordering orderingOnStartTime() { return Ordering.from(new Comparator() { @Override public int compare(TaskInfo o1, TaskInfo o2) { - return (o1.getStartTimeInterval() < o2.getStartTimeInterval()) ? -1 : - ((o1.getStartTimeInterval() == o2.getStartTimeInterval()) ? 0 : 1); + return Long.compare(o1.getStartTimeInterval(), o2.getStartTimeInterval()); } }); } @@ -607,8 +601,7 @@ private Ordering orderingOnStartTime() { private Ordering orderingOnAttemptStartTime() { return Ordering.from(new Comparator() { @Override public int compare(TaskAttemptInfo o1, TaskAttemptInfo o2) { - return (o1.getStartTimeInterval() < o2.getStartTimeInterval()) ? -1 : - ((o1.getStartTimeInterval() == o2.getStartTimeInterval()) ? 0 : 1); + return Long.compare(o1.getStartTimeInterval(), o2.getStartTimeInterval()); } }); } From 4c378b443b20e1f643e894e81ec41271d0356b3f Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Mon, 11 Dec 2017 14:40:04 -0600 Subject: [PATCH 042/512] TEZ-3876. Bug in local mode distributed cache files (Jacob Tolar via jeagles) --- .../app/launcher/TezLocalCacheManager.java | 60 ++++++---- .../launcher/TestTezLocalCacheManager.java | 107 ++++++++++++++++++ 2 files changed, 142 insertions(+), 25 deletions(-) create mode 100644 tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestTezLocalCacheManager.java diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java index 80f73aa9aa..45e55403a1 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java @@ -31,7 +31,9 @@ import java.nio.file.Files; import java.nio.file.Paths; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; +import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -90,15 +92,20 @@ public void localize() throws IOException { throw new IllegalArgumentException("Resource type PATTERN not supported."); } - // submit task to download the object - java.nio.file.Path downloadDir = Files.createTempDirectory(tempDir, resourceName); - Path dest = new Path(downloadDir.toAbsolutePath().toString()); - FSDownload downloader = new FSDownload(fileContext, ugi, conf, dest, resource); - Future downloadedPath = threadPool.submit(downloader); - // linkPath is the path we want to symlink the file/directory into Path linkPath = new Path(cwd, entry.getKey()); - resourceInfo.put(resource, new ResourceInfo(downloadedPath, linkPath)); + + if (resourceInfo.containsKey(resource)) { + // We've already downloaded this resource and just need to add another link. + resourceInfo.get(resource).linkPaths.add(linkPath); + } else { + // submit task to download the object + java.nio.file.Path downloadDir = Files.createTempDirectory(tempDir, resourceName); + Path dest = new Path(downloadDir.toAbsolutePath().toString()); + FSDownload downloader = new FSDownload(fileContext, ugi, conf, dest, resource); + Future downloadedPath = threadPool.submit(downloader); + resourceInfo.put(resource, new ResourceInfo(downloadedPath, linkPath)); + } } // Link each file @@ -106,20 +113,21 @@ public void localize() throws IOException { LocalResource resource = entry.getKey(); ResourceInfo resourceMeta = entry.getValue(); - Path linkPath = resourceMeta.linkPath; - Path targetPath; - - try { - // this blocks on the download completing - targetPath = resourceMeta.downloadPath.get(); - } catch (InterruptedException | ExecutionException e) { - throw new IOException(e); - } - - if (createSymlink(targetPath, linkPath)) { - LOG.info("Localized file: {} as {}", resource, linkPath); - } else { - LOG.warn("Failed to create symlink: {} <- {}", targetPath, linkPath); + for (Path linkPath : resourceMeta.linkPaths) { + Path targetPath; + + try { + // this blocks on the download completing + targetPath = resourceMeta.downloadPath.get(); + } catch (InterruptedException | ExecutionException e) { + throw new IOException(e); + } + + if (createSymlink(targetPath, linkPath)) { + LOG.info("Localized file: {} as {}", resource, linkPath); + } else { + LOG.warn("Failed to create symlink: {} <- {}", targetPath, linkPath); + } } } } finally { @@ -136,8 +144,10 @@ public void localize() throws IOException { */ public void cleanup() throws IOException { for (ResourceInfo info : resourceInfo.values()) { - if (fileContext.util().exists(info.linkPath)) { - fileContext.delete(info.linkPath, true); + for (Path linkPath : info.linkPaths) { + if (fileContext.util().exists(linkPath)) { + fileContext.delete(linkPath, true); + } } } @@ -174,11 +184,11 @@ private boolean createSymlink(Path target, Path link) throws IOException { */ private static class ResourceInfo { final Future downloadPath; - final Path linkPath; + final Set linkPaths = new HashSet<>(); public ResourceInfo(Future downloadPath, Path linkPath) { this.downloadPath = downloadPath; - this.linkPath = linkPath; + this.linkPaths.add(linkPath); } } } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestTezLocalCacheManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestTezLocalCacheManager.java new file mode 100644 index 0000000000..fb23a1d403 --- /dev/null +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestTezLocalCacheManager.java @@ -0,0 +1,107 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.app.launcher; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileContext; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.api.records.LocalResource; +import org.apache.hadoop.yarn.api.records.LocalResourceType; +import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; +import org.apache.hadoop.yarn.api.records.URL; +import org.apache.hadoop.yarn.factories.RecordFactory; +import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; +import org.apache.hadoop.yarn.util.ConverterUtils; +import org.junit.Assert; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.net.URISyntaxException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.HashMap; +import java.util.Map; + +public class TestTezLocalCacheManager { + + @Test + public void testManager() throws URISyntaxException, IOException { + Map resources = new HashMap<>(); + + // Test that localization works for regular files and verify that if multiple symlinks are created, + // they all work + LocalResource resourceOne = createFile("content-one"); + LocalResource resourceTwo = createFile("content-two"); + + resources.put("file-one", resourceOne); + resources.put("file-two", resourceTwo); + resources.put("file-three", resourceTwo); + + TezLocalCacheManager manager = new TezLocalCacheManager(resources, new Configuration()); + + try { + manager.localize(); + + Assert.assertEquals( + "content-one", + new String(Files.readAllBytes(Paths.get("./file-one"))) + ); + + Assert.assertEquals( + "content-two", + new String(Files.readAllBytes(Paths.get("./file-two"))) + ); + + Assert.assertEquals( + "content-two", + new String(Files.readAllBytes(Paths.get("./file-three"))) + ); + } finally { + manager.cleanup(); + } + + // verify that symlinks were removed + Assert.assertFalse(Files.exists(Paths.get("./file-one"))); + Assert.assertFalse(Files.exists(Paths.get("./file-two"))); + Assert.assertFalse(Files.exists(Paths.get("./file-three"))); + } + + // create a temporary file with the given content and return a LocalResource + private static LocalResource createFile(String content) throws IOException { + FileContext fs = FileContext.getLocalFSFileContext(); + + java.nio.file.Path tempFile = Files.createTempFile("test-cache-manager", ".txt"); + File temp = tempFile.toFile(); + temp.deleteOnExit(); + Path p = new Path("file:///" + tempFile.toAbsolutePath().toString()); + + Files.write(tempFile, content.getBytes()); + + RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); + LocalResource ret = recordFactory.newRecordInstance(LocalResource.class); + URL yarnUrlFromPath = ConverterUtils.getYarnUrlFromPath(p); + ret.setResource(yarnUrlFromPath); + ret.setSize(content.getBytes().length); + ret.setType(LocalResourceType.FILE); + ret.setVisibility(LocalResourceVisibility.PRIVATE); + ret.setTimestamp(fs.getFileStatus(p).getModificationTime()); + return ret; + } +} From 4c4bf99eb749e2db0f211710d3a625dc91588d14 Mon Sep 17 00:00:00 2001 From: Zhiyuan Yang Date: Thu, 4 Jan 2018 14:46:56 -0800 Subject: [PATCH 043/512] TEZ-3882. Changes for 0.9.1 release (zhiyuany) --- Tez_DOAP.rdf | 7 +++++ .../markdown/releases/apache-tez-0-9-1.md | 30 +++++++++++++++++++ docs/src/site/markdown/releases/index.md | 1 + docs/src/site/site.xml | 1 + 4 files changed, 39 insertions(+) create mode 100644 docs/src/site/markdown/releases/apache-tez-0-9-1.md diff --git a/Tez_DOAP.rdf b/Tez_DOAP.rdf index d50e5ab602..7e32221f2f 100644 --- a/Tez_DOAP.rdf +++ b/Tez_DOAP.rdf @@ -34,6 +34,13 @@ Java + + + Version 0.9.1 + 2018-01-04 + 0.9.1 + + Version 0.9.0 diff --git a/docs/src/site/markdown/releases/apache-tez-0-9-1.md b/docs/src/site/markdown/releases/apache-tez-0-9-1.md new file mode 100644 index 0000000000..452bce5f5d --- /dev/null +++ b/docs/src/site/markdown/releases/apache-tez-0-9-1.md @@ -0,0 +1,30 @@ + + +Apache TEZ® 0.9.1 + +Apache TEZ® 0.9.1 +---------------------- + +- [Download Release Artifacts](http://www.apache.org/dyn/closer.lua/tez/0.9.1/) +- [Release Notes](0.9.1/release-notes.txt) +- Documentation + - [API Javadocs](0.9.1/tez-api-javadocs/index.html) : Documentation for the Tez APIs + - [Runtime Library Javadocs](0.9.1/tez-runtime-library-javadocs/index.html) : Documentation for built-in implementations of useful Inputs, Outputs, Processors etc. written based on the Tez APIs + - [Tez Mapreduce Javadocs](0.9.1/tez-mapreduce-javadocs/index.html) : Documentation for built-in implementations of Mapreduce compatible Inputs, Outputs, Processors etc. written based on the Tez APIs + - [Tez Configuration](0.9.1/tez-api-javadocs/configs/TezConfiguration.html) : Documentation for configurations of Tez. These configurations are typically specified in tez-site.xml. + - [Tez Runtime Configuration](0.9.1/tez-runtime-library-javadocs/configs/TezRuntimeConfiguration.html) : Documentation for runtime configurations of Tez. These configurations are typically specified by job submitters. diff --git a/docs/src/site/markdown/releases/index.md b/docs/src/site/markdown/releases/index.md index 08f4819113..3c2b393b1a 100644 --- a/docs/src/site/markdown/releases/index.md +++ b/docs/src/site/markdown/releases/index.md @@ -19,6 +19,7 @@ Releases ------------ +- [Apache TEZ® 0.9.1](./apache-tez-0-9-1.html) (Jan 04, 2018) - [Apache TEZ® 0.9.0](./apache-tez-0-9-0.html) (Jul 27, 2017) - [Apache TEZ® 0.8.5](./apache-tez-0-8-5.html) (Mar 13, 2017) - [Apache TEZ® 0.8.4](./apache-tez-0-8-4.html) (Jul 08, 2016) diff --git a/docs/src/site/site.xml b/docs/src/site/site.xml index 226293eac6..79a613beb1 100644 --- a/docs/src/site/site.xml +++ b/docs/src/site/site.xml @@ -127,6 +127,7 @@ +

From d777f455bba95ae65f286729918b09780a0df231 Mon Sep 17 00:00:00 2001 From: Zhiyuan Yang Date: Thu, 4 Jan 2018 15:23:34 -0800 Subject: [PATCH 044/512] TEZ-3883. Update version in master to 0.9.2 (zhiyuany) --- docs/pom.xml | 2 +- hadoop-shim-impls/hadoop-shim-2.7/pom.xml | 2 +- hadoop-shim-impls/hadoop-shim-2.8/pom.xml | 2 +- hadoop-shim-impls/pom.xml | 2 +- hadoop-shim/pom.xml | 2 +- pom.xml | 2 +- tez-api/pom.xml | 2 +- tez-common/pom.xml | 2 +- tez-dag/pom.xml | 2 +- tez-dist/pom.xml | 2 +- tez-examples/pom.xml | 2 +- tez-ext-service-tests/pom.xml | 2 +- tez-mapreduce/pom.xml | 2 +- tez-plugins/pom.xml | 2 +- tez-plugins/tez-aux-services/pom.xml | 2 +- tez-plugins/tez-history-parser/pom.xml | 2 +- tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml | 2 +- tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml | 2 +- tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml | 2 +- tez-plugins/tez-yarn-timeline-history/pom.xml | 2 +- tez-runtime-internals/pom.xml | 2 +- tez-runtime-library/pom.xml | 2 +- tez-tests/pom.xml | 2 +- tez-tools/analyzers/job-analyzer/pom.xml | 2 +- tez-tools/analyzers/pom.xml | 2 +- tez-tools/pom.xml | 2 +- tez-tools/tez-javadoc-tools/pom.xml | 2 +- tez-tools/tez-tfile-parser/pom.xml | 2 +- tez-ui/pom.xml | 2 +- 29 files changed, 29 insertions(+), 29 deletions(-) diff --git a/docs/pom.xml b/docs/pom.xml index 472975d04d..b5fc0f62db 100644 --- a/docs/pom.xml +++ b/docs/pom.xml @@ -27,7 +27,7 @@ org.apache.tez tez - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez-docs pom diff --git a/hadoop-shim-impls/hadoop-shim-2.7/pom.xml b/hadoop-shim-impls/hadoop-shim-2.7/pom.xml index 027ab4b0ef..a4b0a5bdae 100644 --- a/hadoop-shim-impls/hadoop-shim-2.7/pom.xml +++ b/hadoop-shim-impls/hadoop-shim-2.7/pom.xml @@ -19,7 +19,7 @@ hadoop-shim-impls org.apache.tez - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT 4.0.0 hadoop-shim-2.7 diff --git a/hadoop-shim-impls/hadoop-shim-2.8/pom.xml b/hadoop-shim-impls/hadoop-shim-2.8/pom.xml index cb9d63f063..64ce4a959b 100644 --- a/hadoop-shim-impls/hadoop-shim-2.8/pom.xml +++ b/hadoop-shim-impls/hadoop-shim-2.8/pom.xml @@ -19,7 +19,7 @@ hadoop-shim-impls org.apache.tez - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT 4.0.0 hadoop-shim-2.8 diff --git a/hadoop-shim-impls/pom.xml b/hadoop-shim-impls/pom.xml index c645eedc80..5eb304c9de 100644 --- a/hadoop-shim-impls/pom.xml +++ b/hadoop-shim-impls/pom.xml @@ -20,7 +20,7 @@ tez org.apache.tez - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT hadoop-shim-impls pom diff --git a/hadoop-shim/pom.xml b/hadoop-shim/pom.xml index 63c68ddcb9..ab612b2929 100644 --- a/hadoop-shim/pom.xml +++ b/hadoop-shim/pom.xml @@ -20,7 +20,7 @@ tez org.apache.tez - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT hadoop-shim diff --git a/pom.xml b/pom.xml index 64acfceef4..c703e2ec50 100644 --- a/pom.xml +++ b/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez pom - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez diff --git a/tez-api/pom.xml b/tez-api/pom.xml index 78dd415572..763534f636 100644 --- a/tez-api/pom.xml +++ b/tez-api/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez-api diff --git a/tez-common/pom.xml b/tez-common/pom.xml index 8a23335bea..76fdfb894e 100644 --- a/tez-common/pom.xml +++ b/tez-common/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez-common diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml index 76ecfe7311..210ed2bcf7 100644 --- a/tez-dag/pom.xml +++ b/tez-dag/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez-dag diff --git a/tez-dist/pom.xml b/tez-dist/pom.xml index 2187bfc3a0..854a548a31 100644 --- a/tez-dist/pom.xml +++ b/tez-dist/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez-dist diff --git a/tez-examples/pom.xml b/tez-examples/pom.xml index b20e973785..6490ef4b7c 100644 --- a/tez-examples/pom.xml +++ b/tez-examples/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez-examples diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml index d9d47c9726..9cc73362fe 100644 --- a/tez-ext-service-tests/pom.xml +++ b/tez-ext-service-tests/pom.xml @@ -20,7 +20,7 @@ tez org.apache.tez - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez-ext-service-tests diff --git a/tez-mapreduce/pom.xml b/tez-mapreduce/pom.xml index 24b443223e..e72cc66f9b 100644 --- a/tez-mapreduce/pom.xml +++ b/tez-mapreduce/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez-mapreduce diff --git a/tez-plugins/pom.xml b/tez-plugins/pom.xml index e524077b9f..fb0691a0a6 100644 --- a/tez-plugins/pom.xml +++ b/tez-plugins/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez-plugins pom diff --git a/tez-plugins/tez-aux-services/pom.xml b/tez-plugins/tez-aux-services/pom.xml index c61fefe11f..ac874ae0d0 100644 --- a/tez-plugins/tez-aux-services/pom.xml +++ b/tez-plugins/tez-aux-services/pom.xml @@ -20,7 +20,7 @@ tez-plugins org.apache.tez - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez-aux-services diff --git a/tez-plugins/tez-history-parser/pom.xml b/tez-plugins/tez-history-parser/pom.xml index 3eeb911c6b..9ccf2fe9eb 100644 --- a/tez-plugins/tez-history-parser/pom.xml +++ b/tez-plugins/tez-history-parser/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez-plugins - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez-history-parser diff --git a/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml b/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml index 99af09989f..2b40ac974c 100644 --- a/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml +++ b/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez-yarn-timeline-cache-plugin diff --git a/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml b/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml index 791f4e410c..692b9158f3 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez-yarn-timeline-history-with-acls diff --git a/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml b/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml index 627e72d147..0866e3d980 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez-yarn-timeline-history-with-fs diff --git a/tez-plugins/tez-yarn-timeline-history/pom.xml b/tez-plugins/tez-yarn-timeline-history/pom.xml index 2d4192c52b..b43d49b7cb 100644 --- a/tez-plugins/tez-yarn-timeline-history/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez-yarn-timeline-history diff --git a/tez-runtime-internals/pom.xml b/tez-runtime-internals/pom.xml index 0cef94bb1c..4f4cd6e2dc 100644 --- a/tez-runtime-internals/pom.xml +++ b/tez-runtime-internals/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez-runtime-internals diff --git a/tez-runtime-library/pom.xml b/tez-runtime-library/pom.xml index b3a19a3668..cd70f6662f 100644 --- a/tez-runtime-library/pom.xml +++ b/tez-runtime-library/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez-runtime-library diff --git a/tez-tests/pom.xml b/tez-tests/pom.xml index cc6da2954a..9bacb438ba 100644 --- a/tez-tests/pom.xml +++ b/tez-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez-tests diff --git a/tez-tools/analyzers/job-analyzer/pom.xml b/tez-tools/analyzers/job-analyzer/pom.xml index bca2a19065..3acf2bfdfa 100644 --- a/tez-tools/analyzers/job-analyzer/pom.xml +++ b/tez-tools/analyzers/job-analyzer/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-perf-analyzer - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez-job-analyzer diff --git a/tez-tools/analyzers/pom.xml b/tez-tools/analyzers/pom.xml index 0eb2f67ca7..7f6920b84f 100644 --- a/tez-tools/analyzers/pom.xml +++ b/tez-tools/analyzers/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-tools - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez-perf-analyzer pom diff --git a/tez-tools/pom.xml b/tez-tools/pom.xml index d23606f5aa..8d1a5b6dbd 100644 --- a/tez-tools/pom.xml +++ b/tez-tools/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez-tools pom diff --git a/tez-tools/tez-javadoc-tools/pom.xml b/tez-tools/tez-javadoc-tools/pom.xml index 4ef229e0b1..56f390ba5e 100644 --- a/tez-tools/tez-javadoc-tools/pom.xml +++ b/tez-tools/tez-javadoc-tools/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-tools - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez-javadoc-tools diff --git a/tez-tools/tez-tfile-parser/pom.xml b/tez-tools/tez-tfile-parser/pom.xml index d7ba4cade3..4ada794af5 100644 --- a/tez-tools/tez-tfile-parser/pom.xml +++ b/tez-tools/tez-tfile-parser/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-tools - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez-tfile-parser diff --git a/tez-ui/pom.xml b/tez-ui/pom.xml index 4309bc45f8..1dcffe96d1 100644 --- a/tez-ui/pom.xml +++ b/tez-ui/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT tez-ui war From f7feaa72b4fc42676b54e9581165439e9c6d3df7 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Tue, 16 Jan 2018 11:22:10 -0600 Subject: [PATCH 045/512] TEZ-3877. Delete unordered spill files once merge is done (Jason Lowe via jeagles) --- .../writers/UnorderedPartitionedKVWriter.java | 21 ++++++++++++++++--- .../TestUnorderedPartitionedKVWriter.java | 17 +++++++++++---- 2 files changed, 31 insertions(+), 7 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java index 6ea0385000..f4ebc97547 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java @@ -111,8 +111,8 @@ public class UnorderedPartitionedKVWriter extends BaseUnorderedPartitionedKVWrit WrappedBuffer currentBuffer; private final FileSystem rfs; - private final List spillInfoList = Collections - .synchronizedList(new ArrayList()); + @VisibleForTesting + final List spillInfoList = Collections.synchronizedList(new ArrayList()); private final ListeningExecutorService spillExecutor; @@ -1039,12 +1039,26 @@ private void mergeAll() throws IOException { if (out != null) { out.close(); } + deleteIntermediateSpills(); } finalSpillRecord.writeToFile(finalIndexPath, conf); fileOutputBytesCounter.increment(indexFileSizeEstimate); LOG.info(destNameTrimmed + ": " + "Finished final spill after merging : " + numSpills.get() + " spills"); } + private void deleteIntermediateSpills() { + // Delete the intermediate spill files + synchronized (spillInfoList) { + for (SpillInfo spill : spillInfoList) { + try { + rfs.delete(spill.outPath, false); + } catch (IOException e) { + LOG.warn("Unable to delete intermediate spill " + spill.outPath, e); + } + } + } + } + private void writeLargeRecord(final Object key, final Object value, final int partition) throws IOException { numAdditionalSpillsCounter.increment(1); @@ -1359,7 +1373,8 @@ private static class SpillResult { } } - private static class SpillInfo { + @VisibleForTesting + static class SpillInfo { final TezSpillRecord spillRecord; final Path outPath; diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java index f1cea7ecc9..ae396cbf03 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java @@ -54,6 +54,7 @@ import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.runtime.api.TaskFailureType; import org.apache.tez.runtime.api.events.VertexManagerEvent; +import org.apache.tez.runtime.library.common.writers.UnorderedPartitionedKVWriter.SpillInfo; import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.VertexManagerEventPayloadProto; import org.apache.tez.runtime.library.utils.DATA_RANGE_IN_MB; import org.roaringbitmap.RoaringBitmap; @@ -1238,13 +1239,21 @@ private void baseTest(int numRecords, int numPartitions, Set skippedPar Path outputFilePath = kvWriter.finalOutPath; Path spillFilePath = kvWriter.finalIndexPath; - if (numRecordsWritten > 0) { - assertTrue(localFs.exists(outputFilePath)); - assertTrue(localFs.exists(spillFilePath)); - } else { + if (numRecordsWritten <= 0) { return; } + assertTrue(localFs.exists(outputFilePath)); + assertTrue(localFs.exists(spillFilePath)); + + // verify no intermediate spill files have been left around + synchronized (kvWriter.spillInfoList) { + for (SpillInfo spill : kvWriter.spillInfoList) { + assertFalse("lingering intermediate spill file " + spill.outPath, + localFs.exists(spill.outPath)); + } + } + // Special case for 0 records. TezSpillRecord spillRecord = new TezSpillRecord(spillFilePath, conf); DataInputBuffer keyBuffer = new DataInputBuffer(); From 3c7640d710710740a96a9c524f27a8dde4cfc09f Mon Sep 17 00:00:00 2001 From: Sergey Shelukhin Date: Tue, 16 Jan 2018 13:56:46 -0800 Subject: [PATCH 046/512] TEZ-3880: Do not count rejected tasks as killed in vertex progress (Sergey Shelukhin, reviewed by Gunther Hagleitner) Signed-off-by: Gopal V --- .../apache/tez/dag/api/client/Progress.java | 13 +++++++- tez-api/src/main/proto/DAGApiRecords.proto | 1 + .../tez/dag/api/client/ProgressBuilder.java | 4 +++ .../org/apache/tez/dag/app/dag/Vertex.java | 3 ++ .../apache/tez/dag/app/dag/impl/DAGImpl.java | 6 ++++ .../tez/dag/app/dag/impl/TaskAttemptImpl.java | 2 +- .../apache/tez/dag/app/dag/impl/TaskImpl.java | 18 +++++++++- .../tez/dag/app/dag/impl/VertexImpl.java | 12 +++++++ .../tez/dag/app/dag/impl/TestTaskImpl.java | 33 +++++++++++++++++-- .../tez/tests/TestExternalTezServices.java | 2 +- 10 files changed, 88 insertions(+), 6 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/Progress.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/Progress.java index 110ac90d8d..656838dc5a 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/Progress.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/Progress.java @@ -63,6 +63,10 @@ public int getKilledTaskAttemptCount() { return proxy.getKilledTaskAttemptCount(); } + public int getRejectedTaskAttemptCount() { + return proxy.getRejectedTaskAttemptCount(); + } + @Override public boolean equals(Object obj) { if (obj instanceof Progress){ @@ -73,7 +77,8 @@ && getRunningTaskCount() == other.getRunningTaskCount() && getFailedTaskCount() == other.getFailedTaskCount() && getKilledTaskCount() == other.getKilledTaskCount() && getFailedTaskAttemptCount() == other.getFailedTaskAttemptCount() - && getKilledTaskAttemptCount() == other.getKilledTaskAttemptCount(); + && getKilledTaskAttemptCount() == other.getKilledTaskAttemptCount() + && getRejectedTaskAttemptCount() == other.getRejectedTaskAttemptCount(); } return false; } @@ -94,6 +99,8 @@ public int hashCode() { getFailedTaskAttemptCount(); result = prime * result + getKilledTaskAttemptCount(); + result = prime * result + + getRejectedTaskAttemptCount(); return result; } @@ -119,6 +126,10 @@ public String toString() { sb.append(" KilledTaskAttempts: "); sb.append(getKilledTaskAttemptCount()); } + if (getRejectedTaskAttemptCount() > 0) { + sb.append(" RejectedTaskAttempts: "); + sb.append(getRejectedTaskAttemptCount()); + } return sb.toString(); } diff --git a/tez-api/src/main/proto/DAGApiRecords.proto b/tez-api/src/main/proto/DAGApiRecords.proto index c84094b51e..34c369d430 100644 --- a/tez-api/src/main/proto/DAGApiRecords.proto +++ b/tez-api/src/main/proto/DAGApiRecords.proto @@ -227,6 +227,7 @@ message ProgressProto { optional int32 killedTaskCount = 5; optional int32 failedTaskAttemptCount = 6; optional int32 killedTaskAttemptCount = 7; + optional int32 rejectedTaskAttemptCount = 8; } enum VertexStatusStateProto { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/ProgressBuilder.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/ProgressBuilder.java index 538151899b..9dc13549cb 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/ProgressBuilder.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/ProgressBuilder.java @@ -59,6 +59,10 @@ public void setKilledTaskAttemptCount(int count) { getBuilder().setKilledTaskAttemptCount(count); } + public void setRejectedTaskAttemptCount(int count) { + getBuilder().setRejectedTaskAttemptCount(count); + } + private ProgressProto.Builder getBuilder() { return (Builder) this.proxy; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java index ba7624c8f5..0e54e9fc0b 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java @@ -214,4 +214,7 @@ interface VertexConfig { boolean getTaskRescheduleHigherPriority(); boolean getTaskRescheduleRelaxedLocality(); } + + void incrementRejectedTaskAttemptCount(); + int getRejectedTaskAttemptCount(); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java index 481353b6b6..6c67e68f66 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java @@ -867,6 +867,7 @@ public DAGStatusBuilder getDAGStatus(Set statusOptions) { int totalKilledTaskCount = 0; int totalFailedTaskAttemptCount = 0; int totalKilledTaskAttemptCount = 0; + int totalRejectedTaskAttemptCount = 0; readLock.lock(); try { for(Map.Entry entry : vertexMap.entrySet()) { @@ -879,6 +880,7 @@ public DAGStatusBuilder getDAGStatus(Set statusOptions) { totalKilledTaskCount += progress.getKilledTaskCount(); totalFailedTaskAttemptCount += progress.getFailedTaskAttemptCount(); totalKilledTaskAttemptCount += progress.getKilledTaskAttemptCount(); + totalRejectedTaskAttemptCount += progress.getRejectedTaskAttemptCount(); } ProgressBuilder dagProgress = new ProgressBuilder(); dagProgress.setTotalTaskCount(totalTaskCount); @@ -888,6 +890,7 @@ public DAGStatusBuilder getDAGStatus(Set statusOptions) { dagProgress.setKilledTaskCount(totalKilledTaskCount); dagProgress.setFailedTaskAttemptCount(totalFailedTaskAttemptCount); dagProgress.setKilledTaskAttemptCount(totalKilledTaskAttemptCount); + dagProgress.setRejectedTaskAttemptCount(totalRejectedTaskAttemptCount); status.setState(getState()); status.setDiagnostics(diagnostics); status.setDAGProgress(dagProgress); @@ -942,6 +945,7 @@ private ProgressBuilder getDAGProgress() { int totalKilledTaskCount = 0; int totalFailedTaskAttemptCount = 0; int totalKilledTaskAttemptCount = 0; + int totalRejectedTaskAttemptCount = 0; readLock.lock(); try { for(Map.Entry entry : vertexMap.entrySet()) { @@ -953,6 +957,7 @@ private ProgressBuilder getDAGProgress() { totalKilledTaskCount += progress.getKilledTaskCount(); totalFailedTaskAttemptCount += progress.getFailedTaskAttemptCount(); totalKilledTaskAttemptCount += progress.getKilledTaskAttemptCount(); + totalRejectedTaskAttemptCount += progress.getRejectedTaskAttemptCount(); } ProgressBuilder dagProgress = new ProgressBuilder(); dagProgress.setTotalTaskCount(totalTaskCount); @@ -962,6 +967,7 @@ private ProgressBuilder getDAGProgress() { dagProgress.setKilledTaskCount(totalKilledTaskCount); dagProgress.setFailedTaskAttemptCount(totalFailedTaskAttemptCount); dagProgress.setKilledTaskAttemptCount(totalKilledTaskAttemptCount); + dagProgress.setRejectedTaskAttemptCount(totalRejectedTaskAttemptCount); return dagProgress; } finally { readLock.unlock(); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java index 1218543ca4..c43bd984ad 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java @@ -1413,7 +1413,7 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) { ta.sendEvent(createDAGCounterUpdateEventTAFinished(ta, helper.getTaskAttemptState())); // Send out events to the Task - indicating TaskAttemptTermination(F/K) - ta.sendEvent(helper.getTaskEvent(ta.attemptId, event)); + ta.sendEvent(helper.getTaskEvent(ta.attemptId, event)); } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java index 99cb2e06e7..9e1d85f9e9 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java @@ -75,8 +75,10 @@ import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptKilled; import org.apache.tez.dag.app.dag.event.TaskAttemptEventKillRequest; import org.apache.tez.dag.app.dag.event.TaskAttemptEventOutputFailed; +import org.apache.tez.dag.app.dag.event.TaskAttemptEventTerminationCauseEvent; import org.apache.tez.dag.app.dag.event.TaskEvent; import org.apache.tez.dag.app.dag.event.TaskEventScheduleTask; +import org.apache.tez.dag.app.dag.event.TaskEventTAKilled; import org.apache.tez.dag.app.dag.event.TaskEventTAUpdate; import org.apache.tez.dag.app.dag.event.TaskEventTermination; import org.apache.tez.dag.app.dag.event.TaskEventType; @@ -1145,7 +1147,21 @@ public void transition(TaskImpl task, TaskEvent event) { TaskAttemptStateInternal.KILLED); // we KillWaitAttemptCompletedTransitionready have a spare task.taskAttemptStatus.put(castEvent.getTaskAttemptID().getId(), true); - task.getVertex().incrementKilledTaskAttemptCount(); + + boolean isRejection = false; + if (event instanceof TaskEventTAKilled) { + TaskEventTAKilled killEvent = (TaskEventTAKilled) event; + if (killEvent.getCausalEvent() instanceof TaskAttemptEventTerminationCauseEvent) { + TaskAttemptEventTerminationCauseEvent cause = + (TaskAttemptEventTerminationCauseEvent)killEvent.getCausalEvent(); + isRejection = cause.getTerminationCause() == TaskAttemptTerminationCause.SERVICE_BUSY; + } + } + if (isRejection) { // TODO: remove as part of TEZ-3881. + task.getVertex().incrementRejectedTaskAttemptCount(); + } else { + task.getVertex().incrementKilledTaskAttemptCount(); + } if (task.shouldScheduleNewAttempt()) { task.addAndScheduleAttempt(castEvent.getTaskAttemptID()); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index 13cfb8fa1f..d727e39a57 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -676,6 +676,7 @@ private void augmentStateMachine() { AtomicInteger failedTaskAttemptCount = new AtomicInteger(0); @VisibleForTesting AtomicInteger killedTaskAttemptCount = new AtomicInteger(0); + AtomicInteger rejectedTaskAttemptCount = new AtomicInteger(0); @VisibleForTesting long initTimeRequested; // Time at which INIT request was received. @@ -1429,6 +1430,7 @@ public ProgressBuilder getVertexProgress() { progress.setKilledTaskCount(killedTaskCount); progress.setFailedTaskAttemptCount(failedTaskAttemptCount.get()); progress.setKilledTaskAttemptCount(killedTaskAttemptCount.get()); + progress.setRejectedTaskAttemptCount(rejectedTaskAttemptCount.get()); return progress; } finally { this.readLock.unlock(); @@ -1550,6 +1552,11 @@ public void incrementKilledTaskAttemptCount() { this.killedTaskAttemptCount.incrementAndGet(); } + @Override + public void incrementRejectedTaskAttemptCount() { + this.rejectedTaskAttemptCount.incrementAndGet(); + } + @Override public int getFailedTaskAttemptCount() { return this.failedTaskAttemptCount.get(); @@ -1560,6 +1567,11 @@ public int getKilledTaskAttemptCount() { return this.killedTaskAttemptCount.get(); } + @Override + public int getRejectedTaskAttemptCount() { + return this.rejectedTaskAttemptCount.get(); + } + private void setTaskLocationHints(VertexLocationHint vertexLocationHint) { if (vertexLocationHint != null && vertexLocationHint.getTaskLocationHints() != null && diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java index d13e6546e9..b142bb9b08 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java @@ -40,6 +40,7 @@ import org.apache.tez.dag.app.dag.event.TaskAttemptEventStartedRemotely; import org.apache.tez.dag.app.dag.event.TaskAttemptEventSubmitted; import org.apache.tez.dag.app.dag.event.DAGEventType; +import org.apache.tez.dag.app.dag.event.TaskAttemptEventTerminationCauseEvent; import org.apache.tez.dag.app.dag.event.TaskEvent; import org.apache.tez.dag.app.dag.event.TaskEventTAFailed; import org.apache.tez.dag.app.dag.event.TaskEventTAKilled; @@ -104,7 +105,6 @@ import org.junit.Test; public class TestTaskImpl { - private static final Logger LOG = LoggerFactory.getLogger(TestTaskImpl.class); private int taskCounter = 0; @@ -185,7 +185,7 @@ public void setup() { Vertex vertex = mock(Vertex.class); doReturn(new VertexImpl.VertexConfigImpl(conf)).when(vertex).getVertexConfig(); eventHandler = new TestEventHandler(); - + mockTask = new MockTaskImpl(vertexId, partition, eventHandler, conf, taskCommunicatorManagerInterface, clock, taskHeartbeatHandler, appContext, leafVertex, @@ -508,6 +508,23 @@ public void testKillRunningTaskAttempt() { Assert.assertEquals(lastTAId, mockTask.getLastAttempt().getSchedulingCausalTA()); } + @Test(timeout = 5000) + /** + * Kill running attempt + * {@link TaskState#RUNNING}->{@link TaskState#RUNNING} + */ + public void testKillTaskAttemptServiceBusy() { + LOG.info("--- START: testKillTaskAttemptServiceBusy ---"); + TezTaskID taskId = getNewTaskID(); + scheduleTaskAttempt(taskId); + launchTaskAttempt(mockTask.getLastAttempt().getID()); + mockTask.handle(createTaskTAKilledEvent( + mockTask.getLastAttempt().getID(), new ServiceBusyEvent())); + assertTaskRunningState(); + verify(mockTask.getVertex(), times(0)).incrementKilledTaskAttemptCount(); + verify(mockTask.getVertex(), times(1)).incrementRejectedTaskAttemptCount(); + } + /** * {@link TaskState#KILLED}->{@link TaskState#KILLED} */ @@ -1386,4 +1403,16 @@ public ContainerId getAssignedContainerID() { } } + public class ServiceBusyEvent extends TezAbstractEvent + implements TaskAttemptEventTerminationCauseEvent { + public ServiceBusyEvent() { + super(TaskAttemptEventType.TA_KILLED); + } + + @Override + public TaskAttemptTerminationCause getTerminationCause() { + return TaskAttemptTerminationCause.SERVICE_BUSY; + } + } } + diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java index 920534a598..c135d7a433 100644 --- a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java +++ b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java @@ -200,7 +200,7 @@ private void runExceptionSimulation() throws IOException, TezException, Interrup DAGStatus dagStatus = dagClient.waitForCompletion(); assertEquals(DAGStatus.State.SUCCEEDED, dagStatus.getState()); assertEquals(1, dagStatus.getDAGProgress().getFailedTaskAttemptCount()); - assertEquals(1, dagStatus.getDAGProgress().getKilledTaskAttemptCount()); + assertEquals(1, dagStatus.getDAGProgress().getRejectedTaskAttemptCount()); } From a9b8bb5a6840a89844cd75ec86aab2b7457c8028 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Thu, 25 Jan 2018 09:35:33 -0600 Subject: [PATCH 047/512] TEZ-3770. DAG-aware YARN task scheduler (jlowe) --- .../apache/tez/dag/api/TezConfiguration.java | 9 + .../tez/serviceplugins/api/DagInfo.java | 6 + .../api/TaskSchedulerContext.java | 1 + .../tez/common/MockDNSToSwitchMapping.java | 14 +- tez-dag/findbugs-exclude.xml | 5 + .../java/org/apache/tez/dag/app/dag/DAG.java | 1 - .../apache/tez/dag/app/dag/impl/DAGImpl.java | 39 + .../dag/app/rm/DagAwareYarnTaskScheduler.java | 2064 +++++++++++++++++ .../dag/app/rm/TaskSchedulerContextImpl.java | 5 + .../rm/TaskSchedulerContextImplWrapper.java | 5 + .../tez/dag/app/rm/TaskSchedulerManager.java | 24 +- .../org/apache/tez/dag/app/MockClock.java | 24 +- .../app/rm/TestDagAwareYarnTaskScheduler.java | 1510 ++++++++++++ .../dag/app/rm/TestTaskSchedulerHelpers.java | 5 + .../tez/dag/helpers/DagInfoImplForTest.java | 12 + .../ControlledScheduledExecutorService.java | 239 ++ 16 files changed, 3954 insertions(+), 9 deletions(-) create mode 100644 tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java create mode 100644 tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java create mode 100644 tez-dag/src/test/java/org/apache/tez/test/ControlledScheduledExecutorService.java diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index efe6d6c47b..6d3050d745 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -680,6 +680,15 @@ public TezConfiguration(boolean loadDefaults) { public static final String TEZ_AM_DAG_SCHEDULER_CLASS_DEFAULT = "org.apache.tez.dag.app.dag.impl.DAGSchedulerNaturalOrder"; + /** + * String value. The class to be used for the YARN task scheduler. Expert level setting. + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty + public static final String TEZ_AM_YARN_SCHEDULER_CLASS = TEZ_AM_PREFIX + "yarn.scheduler.class"; + public static final String TEZ_AM_YARN_SCHEDULER_CLASS_DEFAULT = + "org.apache.tez.dag.app.rm.YarnTaskSchedulerService"; + /** Int value. The amount of memory in MB to be used by the AppMaster */ @ConfigurationScope(Scope.AM) @ConfigurationProperty(type="integer") diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/DagInfo.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/DagInfo.java index 4a8b9fa8ce..328cb62c71 100644 --- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/DagInfo.java +++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/DagInfo.java @@ -16,6 +16,8 @@ import org.apache.hadoop.security.Credentials; +import java.util.BitSet; + public interface DagInfo { /** @@ -35,4 +37,8 @@ public interface DagInfo { * @return the credentials for the dag. */ Credentials getCredentials(); + + int getTotalVertices(); + + BitSet getVertexDescendants(int vertexIndex); } diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java index 97fe7ae834..e188231e14 100644 --- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java +++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java @@ -220,4 +220,5 @@ void setApplicationRegistrationData( */ AMState getAMState(); + int getVertexIndexForTask(Object task); } diff --git a/tez-common/src/test/java/org/apache/tez/common/MockDNSToSwitchMapping.java b/tez-common/src/test/java/org/apache/tez/common/MockDNSToSwitchMapping.java index ec6881f0ae..afcd687356 100644 --- a/tez-common/src/test/java/org/apache/tez/common/MockDNSToSwitchMapping.java +++ b/tez-common/src/test/java/org/apache/tez/common/MockDNSToSwitchMapping.java @@ -19,7 +19,10 @@ package org.apache.tez.common; import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; @@ -32,6 +35,8 @@ * Overrides CachedDNSToSwitchMapping to ensure that it does not try to resolve hostnames */ public class MockDNSToSwitchMapping extends CachedDNSToSwitchMapping implements DNSToSwitchMapping { + private static final Map rackMap = + Collections.synchronizedMap(new HashMap()); private final String defaultRack = "/default-rack"; @@ -43,7 +48,11 @@ public MockDNSToSwitchMapping() { public List resolve(List strings) { List resolvedHosts = new ArrayList(); for (String h : strings) { - resolvedHosts.add(defaultRack); + String rack = rackMap.get(h); + if (rack == null) { + rack = defaultRack; + } + resolvedHosts.add(rack); } return resolvedHosts; } @@ -62,4 +71,7 @@ public static void initializeMockRackResolver() { RackResolver.init(rackResolverConf); } + public static void addRackMapping(String host, String rack) { + rackMap.put(host, rack); + } } diff --git a/tez-dag/findbugs-exclude.xml b/tez-dag/findbugs-exclude.xml index c3e099e5b3..1150ccb3c7 100644 --- a/tez-dag/findbugs-exclude.xml +++ b/tez-dag/findbugs-exclude.xml @@ -92,6 +92,11 @@ + + + + + diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java index 6c8e8f9b57..10c4257c5b 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java @@ -68,7 +68,6 @@ public interface DAG extends DagInfo { Map getVertices(); Vertex getVertex(TezVertexID vertexId); List getDiagnostics(); - int getTotalVertices(); int getSuccessfulVertices(); float getProgress(); float getCompletedTaskProgress(); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java index 6c67e68f66..0a775a6db8 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java @@ -22,6 +22,7 @@ import java.net.URL; import java.security.PrivilegedExceptionAction; import java.util.ArrayList; +import java.util.BitSet; import java.util.Collection; import java.util.Collections; import java.util.EnumSet; @@ -190,6 +191,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG, volatile Map vertices = new HashMap(); @VisibleForTesting Map edges = new HashMap(); + ArrayList vertexDescendants; private TezCounters dagCounters = new TezCounters(); private Object fullCountersLock = new Object(); @VisibleForTesting @@ -1457,6 +1459,16 @@ public int getTotalVertices() { } + @Override + public BitSet getVertexDescendants(int vertexIndex) { + readLock.lock(); + try { + return vertexDescendants.get(vertexIndex); + } finally { + readLock.unlock(); + } + } + @Override public int getSuccessfulVertices() { readLock.lock(); @@ -1560,6 +1572,8 @@ DAGState initializeDAG() { parseVertexEdges(this, edgePlans, v); } + computeVertexDescendants(); + // Initialize the edges, now that the payload and vertices have been set. for (Edge e : edges.values()) { try { @@ -1616,6 +1630,31 @@ private void createDAGEdges(DAGImpl dag) throws TezException { } } + private void computeVertexDescendants() { + vertexDescendants = new ArrayList<>(numVertices); + for (int i = 0; i < numVertices; ++i) { + vertexDescendants.add(new BitSet(numVertices)); + } + BitSet verticesVisited = new BitSet(numVertices); + for (Vertex v : vertices.values()) { + computeVertexDescendants(verticesVisited, v); + } + } + + private BitSet computeVertexDescendants(BitSet verticesVisited, Vertex v) { + int vertexIndex = v.getVertexId().getId(); + BitSet descendants = vertexDescendants.get(vertexIndex); + if (!verticesVisited.get(vertexIndex)) { + for (Vertex child : v.getOutputVertices().keySet()) { + descendants.set(child.getVertexId().getId()); + BitSet childDescendants = computeVertexDescendants(verticesVisited, child); + descendants.or(childDescendants); + } + verticesVisited.set(vertexIndex); + } + return descendants; + } + private static void assignDAGScheduler(DAGImpl dag) throws TezException { String dagSchedulerClassName = dag.dagConf.get(TezConfiguration.TEZ_AM_DAG_SCHEDULER_CLASS, TezConfiguration.TEZ_AM_DAG_SCHEDULER_CLASS_DEFAULT); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java new file mode 100644 index 0000000000..dab1cad8ba --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java @@ -0,0 +1,2064 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.app.rm; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.commons.lang.mutable.MutableInt; +import org.apache.commons.math3.random.RandomDataGenerator; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.util.Time; +import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.NodeReport; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.client.api.AMRMClient; +import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; +import org.apache.hadoop.yarn.client.api.async.impl.AMRMClientAsyncImpl; +import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes; +import org.apache.hadoop.yarn.util.RackResolver; +import org.apache.hadoop.yarn.util.resource.Resources; +import org.apache.tez.common.ContainerSignatureMatcher; +import org.apache.tez.common.TezUtils; +import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.serviceplugins.api.DagInfo; +import org.apache.tez.serviceplugins.api.TaskAttemptEndReason; +import org.apache.tez.serviceplugins.api.TaskScheduler; +import org.apache.tez.serviceplugins.api.TaskSchedulerContext; +import org.apache.tez.serviceplugins.api.TaskSchedulerContext.AMState; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.GuardedBy; +import java.util.ArrayList; +import java.util.BitSet; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.NavigableMap; +import java.util.PriorityQueue; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +/** + * A YARN task scheduler that is aware of the dependencies between vertices + * in the DAG and takes them into account when deciding how to schedule + * and preempt tasks. + * + * This scheduler makes the assumption that vertex IDs start at 0 and are + * densely allocated (i.e.: there are no "gaps" in the vertex ID space). + */ +public class DagAwareYarnTaskScheduler extends TaskScheduler + implements AMRMClientAsync.CallbackHandler { + private static final Logger LOG = LoggerFactory.getLogger(DagAwareYarnTaskScheduler.class); + private static final Comparator PREEMPT_ORDER_COMPARATOR = new PreemptOrderComparator(); + + private final RandomDataGenerator random = new RandomDataGenerator(); + private AMRMClientAsyncWrapper client; + private ScheduledExecutorService reuseExecutor; + private ResourceCalculator resourceCalculator; + private int numHeartbeats = 0; + private Resource totalResources = Resource.newInstance(0, 0); + @GuardedBy("this") + private Resource allocatedResources = Resource.newInstance(0, 0); + private final Set blacklistedNodes = Collections.newSetFromMap(new ConcurrentHashMap()); + private final ContainerSignatureMatcher signatureMatcher; + @GuardedBy("this") + private final RequestTracker requestTracker = new RequestTracker(); + @GuardedBy("this") + private final Map heldContainers = new HashMap<>(); + @GuardedBy("this") + private final IdleContainerTracker idleTracker = new IdleContainerTracker(); + @GuardedBy("this") + private final Map taskAssignments = new HashMap<>(); + + /** A mapping from the vertex ID to the set of containers assigned to tasks for that vertex */ + @GuardedBy("this") + private final Map> vertexAssignments = new HashMap<>(); + + /** If vertex N has at least one task assigned to a container then the corresponding bit at index N is set */ + @GuardedBy("this") + private final BitSet assignedVertices = new BitSet(); + + /** + * Tracks assigned tasks for released containers so the app can be notified properly when the + * container completion event finally arrives. + */ + @GuardedBy("this") + private final Map releasedContainers = new HashMap<>(); + + @GuardedBy("this") + private final Set sessionContainers = new HashSet<>(); + + /** + * Tracks the set of descendant vertices in the DAG for each vertex. The BitSet for descendants of vertex N + * are at array index N. If a bit is set at index X in the descendants BitSet then vertex X is a descendant + * of vertex N in the DAG. + */ + @GuardedBy("this") + private ArrayList vertexDescendants = null; + + private volatile boolean stopRequested = false; + private volatile boolean shouldUnregister = false; + private volatile boolean hasUnregistered = false; + + // cached configuration parameters + private boolean shouldReuseContainers; + private boolean reuseRackLocal; + private boolean reuseNonLocal; + private long localitySchedulingDelay; + private long idleContainerTimeoutMin; + private long idleContainerTimeoutMax; + private int sessionNumMinHeldContainers; + private int preemptionPercentage; + private int numHeartbeatsBetweenPreemptions; + private int lastPreemptionHeartbeat = 0; + private long preemptionMaxWaitTime; + + public DagAwareYarnTaskScheduler(TaskSchedulerContext taskSchedulerContext) { + super(taskSchedulerContext); + signatureMatcher = taskSchedulerContext.getContainerSignatureMatcher(); + } + + @Override + public void initialize() throws Exception { + initialize(new AMRMClientAsyncWrapper(new AMRMClientImpl(), 1000, this)); + } + + void initialize(AMRMClientAsyncWrapper client) throws Exception { + super.initialize(); + this.client = client; + Configuration conf = TezUtils.createConfFromUserPayload(getContext().getInitialUserPayload()); + client.init(conf); + + int heartbeatIntervalMax = conf.getInt( + TezConfiguration.TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX, + TezConfiguration.TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX_DEFAULT); + client.setHeartbeatInterval(heartbeatIntervalMax); + + shouldReuseContainers = conf.getBoolean( + TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, + TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED_DEFAULT); + reuseRackLocal = conf.getBoolean( + TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, + TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED_DEFAULT); + reuseNonLocal = conf + .getBoolean( + TezConfiguration.TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED, + TezConfiguration.TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED_DEFAULT); + Preconditions.checkArgument( + ((!reuseRackLocal && !reuseNonLocal) || (reuseRackLocal)), + "Re-use Rack-Local cannot be disabled if Re-use Non-Local has been" + + " enabled"); + + localitySchedulingDelay = conf.getLong( + TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, + TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS_DEFAULT); + Preconditions.checkArgument(localitySchedulingDelay >= 0, + "Locality Scheduling delay should be >=0"); + + idleContainerTimeoutMin = conf.getLong( + TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, + TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS_DEFAULT); + Preconditions.checkArgument(idleContainerTimeoutMin >= 0 || idleContainerTimeoutMin == -1, + "Idle container release min timeout should be either -1 or >=0"); + + idleContainerTimeoutMax = conf.getLong( + TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, + TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS_DEFAULT); + Preconditions.checkArgument( + idleContainerTimeoutMax >= 0 && idleContainerTimeoutMax >= idleContainerTimeoutMin, + "Idle container release max timeout should be >=0 and >= " + + TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS); + + sessionNumMinHeldContainers = conf.getInt(TezConfiguration.TEZ_AM_SESSION_MIN_HELD_CONTAINERS, + TezConfiguration.TEZ_AM_SESSION_MIN_HELD_CONTAINERS_DEFAULT); + Preconditions.checkArgument(sessionNumMinHeldContainers >= 0, + "Session minimum held containers should be >=0"); + + preemptionPercentage = conf.getInt(TezConfiguration.TEZ_AM_PREEMPTION_PERCENTAGE, + TezConfiguration.TEZ_AM_PREEMPTION_PERCENTAGE_DEFAULT); + Preconditions.checkArgument(preemptionPercentage >= 0 && preemptionPercentage <= 100, + "Preemption percentage should be between 0-100"); + + numHeartbeatsBetweenPreemptions = conf.getInt( + TezConfiguration.TEZ_AM_PREEMPTION_HEARTBEATS_BETWEEN_PREEMPTIONS, + TezConfiguration.TEZ_AM_PREEMPTION_HEARTBEATS_BETWEEN_PREEMPTIONS_DEFAULT); + Preconditions.checkArgument(numHeartbeatsBetweenPreemptions >= 1, + "Heartbeats between preemptions should be >=1"); + + preemptionMaxWaitTime = conf.getInt(TezConfiguration.TEZ_AM_PREEMPTION_MAX_WAIT_TIME_MS, + TezConfiguration.TEZ_AM_PREEMPTION_MAX_WAIT_TIME_MS_DEFAULT); + Preconditions.checkArgument(preemptionMaxWaitTime >=0, "Preemption max wait time must be >=0"); + + LOG.info("scheduler initialized with maxRMHeartbeatInterval:" + heartbeatIntervalMax + + " reuseEnabled:" + shouldReuseContainers + + " reuseRack:" + reuseRackLocal + + " reuseAny:" + reuseNonLocal + + " localityDelay:" + localitySchedulingDelay + + " preemptPercentage:" + preemptionPercentage + + " preemptMaxWaitTime:" + preemptionMaxWaitTime + + " numHeartbeatsBetweenPreemptions:" + numHeartbeatsBetweenPreemptions + + " idleContainerMinTimeout:" + idleContainerTimeoutMin + + " idleContainerMaxTimeout:" + idleContainerTimeoutMax + + " sessionMinHeldContainers:" + sessionNumMinHeldContainers); + } + + @Override + public void start() throws Exception { + super.start(); + client.start(); + if (shouldReuseContainers) { + reuseExecutor = createExecutor(); + } + TaskSchedulerContext ctx = getContext(); + RegisterApplicationMasterResponse response = client.registerApplicationMaster( + ctx.getAppHostName(), ctx.getAppClientPort(), ctx.getAppTrackingUrl()); + ctx.setApplicationRegistrationData(response.getMaximumResourceCapability(), + response.getApplicationACLs(), response.getClientToAMTokenMasterKey(), + response.getQueue()); + if (response.getSchedulerResourceTypes().contains(SchedulerResourceTypes.CPU)) { + resourceCalculator = new MemCpuResourceCalculator(); + } else { + resourceCalculator = new MemResourceCalculator(); + } + } + + protected ScheduledExecutorService createExecutor() { + return new ReuseContainerExecutor(); + } + + protected long now() { + return Time.monotonicNow(); + } + + @Override + public void initiateStop() { + super.initiateStop(); + LOG.debug("Initiating stop of task scheduler"); + stopRequested = true; + List releasedLaunchedContainers; + synchronized (this) { + releasedLaunchedContainers = new ArrayList<>(heldContainers.size()); + List heldList = new ArrayList<>(heldContainers.values()); + for (HeldContainer hc : heldList) { + if (releaseContainer(hc)) { + releasedLaunchedContainers.add(hc.getId()); + } + } + + List tasks = requestTracker.getTasks(); + for (Object task : tasks) { + removeTaskRequest(task); + } + } + + // perform app callback outside of locks + for (ContainerId id : releasedLaunchedContainers) { + getContext().containerBeingReleased(id); + } + } + + @Override + public void shutdown() throws Exception { + super.shutdown(); + if (reuseExecutor != null) { + reuseExecutor.shutdown(); + reuseExecutor.awaitTermination(2, TimeUnit.SECONDS); + } + synchronized (this) { + if (shouldUnregister && !hasUnregistered) { + TaskSchedulerContext.AppFinalStatus status = getContext().getFinalAppStatus(); + LOG.info("Unregistering from RM, exitStatus={} exitMessage={} trackingURL={}", + status.exitStatus, status.exitMessage, status.postCompletionTrackingUrl); + client.unregisterApplicationMaster(status.exitStatus, + status.exitMessage, + status.postCompletionTrackingUrl); + hasUnregistered = true; + } + } + client.stop(); + } + + @Override + public void onContainersAllocated(List containers) { + AMState appState = getContext().getAMState(); + if (stopRequested || appState == AMState.COMPLETED) { + LOG.info("Ignoring {} allocations since app is terminating", containers.size()); + for (Container c : containers) { + client.releaseAssignedContainer(c.getId()); + } + return; + } + List assignments = assignNewContainers(containers, getContext().getAMState(), getContext().isSession()); + informAppAboutAssignments(assignments); + } + + private synchronized List assignNewContainers(List newContainers, + AMState appState, boolean isSession) { + // try to assign the containers as node-local + List assignments = new ArrayList<>(newContainers.size()); + List unassigned = new ArrayList<>(newContainers.size()); + for (Container c : newContainers) { + HeldContainer hc = new HeldContainer(c); + heldContainers.put(hc.getId(), hc); + Resources.addTo(allocatedResources, c.getResource()); + tryAssignNewContainer(hc, hc.getHost(), assignments, unassigned); + } + + // try to assign the remaining containers as rack-local + List containers = unassigned; + unassigned = new ArrayList<>(containers.size()); + for (HeldContainer hc : containers) { + tryAssignNewContainer(hc, hc.getRack(), assignments, unassigned); + } + + // try to assign the remaining containers without locality + containers = unassigned; + unassigned = new ArrayList<>(containers.size()); + for (HeldContainer hc : containers) { + tryAssignNewContainer(hc, ResourceRequest.ANY, assignments, unassigned); + } + + for (HeldContainer hc : unassigned) { + if (shouldReuseContainers) { + idleTracker.add(hc); + TaskRequest assigned = tryAssignReuseContainer(hc, appState, isSession); + if (assigned != null) { + assignments.add(new Assignment(assigned, hc.getContainer())); + } + } else { + releaseContainer(hc); + } + } + + return assignments; + } + + /** + * Try to assign a newly acquired container to a task of the same priority. + * + * @param hc the container to assign + * @param location the locality to consider for assignment + * @param assignments list to update if container is assigned + * @param unassigned list to update if container is not assigned + */ + @GuardedBy("this") + private void tryAssignNewContainer(HeldContainer hc, String location, + List assignments, List unassigned) { + List> results = client.getMatchingRequests(hc.getPriority(), + location, hc.getCapability()); + if (!results.isEmpty()) { + for (Collection requests : results) { + if (!requests.isEmpty()) { + TaskRequest request = requests.iterator().next(); + assignContainer(request, hc, location); + assignments.add(new Assignment(request, hc.getContainer())); + return; + } + } + } + + unassigned.add(hc); + } + + @GuardedBy("this") + @Nullable + private TaskRequest tryAssignReuseContainer(HeldContainer hc, + AMState appState, boolean isSession) { + if (stopRequested) { + return null; + } + + TaskRequest assignedRequest = null; + switch (appState) { + case IDLE: + handleReuseContainerWhenIdle(hc, isSession); + break; + case RUNNING_APP: + if (requestTracker.isEmpty()) { + // treat no requests as if app is idle + handleReuseContainerWhenIdle(hc, isSession); + } else { + assignedRequest = tryAssignReuseContainerAppRunning(hc); + if (assignedRequest == null) { + if (hc.atMaxMatchLevel()) { + LOG.info("Releasing idle container {} due to pending requests", hc.getId()); + releaseContainer(hc); + } else { + hc.scheduleForReuse(localitySchedulingDelay); + } + } + } + break; + case COMPLETED: + LOG.info("Releasing container {} because app has completed", hc.getId()); + releaseContainer(hc); + break; + default: + throw new IllegalStateException("Unexpected app state " + appState); + } + + return assignedRequest; + } + + @GuardedBy("this") + private void handleReuseContainerWhenIdle(HeldContainer hc, boolean isSession) { + if (isSession && sessionContainers.isEmpty() && sessionNumMinHeldContainers > 0) { + computeSessionContainers(); + } + + if (sessionContainers.contains(hc)) { + LOG.info("Retaining container {} since it is a session container"); + hc.resetMatchingLevel(); + } else { + long now = now(); + long expiration = hc.getIdleExpirationTimestamp(now); + if (now >= expiration) { + LOG.info("Releasing expired idle container {}", hc.getId()); + releaseContainer(hc); + } else { + hc.scheduleForReuse(expiration - now); + } + } + } + + @GuardedBy("this") + @Nullable + private TaskRequest tryAssignReuseContainerAppRunning(HeldContainer hc) { + if (!hc.isAssignable()) { + LOG.debug("Skipping scheduling of container {} because it state is {}", hc.getId(), hc.getState()); + return null; + } + + TaskRequest assignedRequest = tryAssignReuseContainerForAffinity(hc); + if (assignedRequest != null) { + return assignedRequest; + } + + for (Entry entry : requestTracker.getStatsEntries()) { + Priority priority = entry.getKey(); + RequestPriorityStats stats = entry.getValue(); + if (!stats.allowedVertices.intersects(stats.vertices)) { + LOG.debug("Skipping requests at priority {} because all requesting vertices are blocked by higher priority requests", + priority); + continue; + } + + String matchLocation = hc.getMatchingLocation(); + if (stats.localityCount <= 0) { + LOG.debug("Overriding locality match of container {} to ANY since there are no locality requests at priority {}", + hc.getId(), priority); + matchLocation = ResourceRequest.ANY; + } + assignedRequest = tryAssignReuseContainerForPriority(hc, matchLocation, + priority, stats.allowedVertices); + if (assignedRequest != null) { + break; + } + } + return assignedRequest; + } + + @GuardedBy("this") + @Nullable + private TaskRequest tryAssignReuseContainerForAffinity(HeldContainer hc) { + Collection affinities = hc.getAffinities(); + if (affinities != null) { + for (TaskRequest request : affinities) { + if (requestTracker.isRequestBlocked(request)) { + LOG.debug("Cannot assign task {} to container {} since vertex {} is a descendant of pending tasks", + request.getTask(), hc.getId(), request.getVertexIndex()); + } else { + assignContainer(request, hc, hc.getId()); + return request; + } + } + } + return null; + } + + @GuardedBy("this") + @Nullable + private TaskRequest tryAssignReuseContainerForPriority(HeldContainer hc, String matchLocation, + Priority priority, BitSet allowedVertices) { + List> results = client.getMatchingRequests(priority, matchLocation, hc.getCapability()); + if (results.isEmpty()) { + return null; + } + + for (Collection requests : results) { + for (TaskRequest request : requests) { + final int vertexIndex = request.getVertexIndex(); + if (!allowedVertices.get(vertexIndex)) { + LOG.debug("Not assigning task {} since it is a descendant of a pending vertex", request.getTask()); + continue; + } + + Object signature = hc.getSignature(); + if (signature == null || signatureMatcher.isSuperSet(signature, request.getContainerSignature())) { + assignContainer(request, hc, matchLocation); + return request; + } + } + } + return null; + } + + private void informAppAboutAssignments(List assignments) { + if (!assignments.isEmpty()) { + for (Assignment a : assignments) { + informAppAboutAssignment(a.request, a.container); + } + } + } + + /** + * Inform the app about a task assignment. This should not be called with + * any locks held. + * + * @param request the corresponding task request + * @param container the container assigned to the task + */ + private void informAppAboutAssignment(TaskRequest request, Container container) { + if (blacklistedNodes.contains(container.getNodeId())) { + Object task = request.getTask(); + LOG.info("Container {} allocated for task {} on blacklisted node {}", + container.getId(), container.getNodeId(), task); + deallocateContainer(container.getId()); + // its ok to submit the same request again because the RM will not give us + // the bad/unhealthy nodes again. The nodes may become healthy/unblacklisted + // and so its better to give the RM the full information. + allocateTask(task, request.getCapability(), + (request.getNodes() == null ? null : + request.getNodes().toArray(new String[request.getNodes().size()])), + (request.getRacks() == null ? null : + request.getRacks().toArray(new String[request.getRacks().size()])), + request.getPriority(), + request.getContainerSignature(), + request.getCookie()); + } else { + getContext().taskAllocated(request.getTask(), request.getCookie(), container); + } + } + + @GuardedBy("this") + private void computeSessionContainers() { + Map rackHeldNumber = new HashMap<>(); + Map> nodeHeldContainers = new HashMap<>(); + for(HeldContainer heldContainer : heldContainers.values()) { + if (heldContainer.getSignature() == null) { + // skip containers that have not been launched as there is no process to reuse + continue; + } + MutableInt count = rackHeldNumber.get(heldContainer.getRack()); + if (count == null) { + count = new MutableInt(0); + rackHeldNumber.put(heldContainer.getRack(), count); + } + count.increment(); + String host = heldContainer.getHost(); + List nodeContainers = nodeHeldContainers.get(host); + if (nodeContainers == null) { + nodeContainers = new LinkedList<>(); + nodeHeldContainers.put(host, nodeContainers); + } + nodeContainers.add(heldContainer); + } + + Map rackToHoldNumber = new HashMap<>(); + for (String rack : rackHeldNumber.keySet()) { + rackToHoldNumber.put(rack, new MutableInt(0)); + } + + // distribute evenly across nodes + // the loop assigns 1 container per rack over all racks + int containerCount = 0; + while (containerCount < sessionNumMinHeldContainers && !rackHeldNumber.isEmpty()) { + Iterator> iter = rackHeldNumber.entrySet().iterator(); + while (containerCount < sessionNumMinHeldContainers && iter.hasNext()) { + Entry entry = iter.next(); + MutableInt rackCount = entry.getValue(); + rackCount.decrement(); + if (rackCount.intValue() >=0) { + containerCount++; + rackToHoldNumber.get(entry.getKey()).increment(); + } else { + iter.remove(); + } + } + } + + // distribute containers evenly across nodes while not exceeding rack limit + // the loop assigns 1 container per node over all nodes + containerCount = 0; + while (containerCount < sessionNumMinHeldContainers && !nodeHeldContainers.isEmpty()) { + Iterator>> iter = nodeHeldContainers.entrySet().iterator(); + while (containerCount < sessionNumMinHeldContainers && iter.hasNext()) { + List nodeContainers = iter.next().getValue(); + if (nodeContainers.isEmpty()) { + // node is empty. remove it. + iter.remove(); + continue; + } + HeldContainer heldContainer = nodeContainers.remove(nodeContainers.size() - 1); + MutableInt holdCount = rackToHoldNumber.get(heldContainer.getRack()); + holdCount.decrement(); + if (holdCount.intValue() >= 0) { + // rack can hold a container + containerCount++; + sessionContainers.add(heldContainer); + } else { + // rack limit reached. remove node. + iter.remove(); + } + } + } + + LOG.info("Identified {} session containers out of {} total containers", + sessionContainers.size(), heldContainers.size()); + } + + @GuardedBy("this") + private void activateSessionContainers() { + if (!sessionContainers.isEmpty()) { + for (HeldContainer hc : sessionContainers) { + if (hc.isAssignable()) { + hc.scheduleForReuse(localitySchedulingDelay); + } + } + sessionContainers.clear(); + } + } + + @Override + public void onContainersCompleted(List statuses) { + if (stopRequested) { + return; + } + + List taskStatusList = new ArrayList<>(statuses.size()); + synchronized (this) { + for (ContainerStatus status : statuses) { + ContainerId cid = status.getContainerId(); + LOG.info("Container {} completed with status {}", cid, status); + Object task = releasedContainers.remove(cid); + if (task == null) { + HeldContainer hc = heldContainers.get(cid); + if (hc != null) { + task = containerCompleted(hc); + } + } + if (task != null) { + taskStatusList.add(new TaskStatus(task, status)); + } + } + } + + // perform app callback outside of locks + for (TaskStatus taskStatus : taskStatusList) { + getContext().containerCompleted(taskStatus.task, taskStatus.status); + } + } + + @Override + public void onNodesUpdated(List updatedNodes) { + if (!stopRequested) { + getContext().nodesUpdated(updatedNodes); + } + } + + @Override + public float getProgress() { + if (stopRequested) { + return 1; + } + + Collection preemptedContainers; + synchronized (this) { + Resource freeResources = getAvailableResources(); + if (totalResources.getMemory() == 0) { + // assume this is the first allocate callback. nothing is allocated. + // available resource = totalResource + // TODO this will not handle dynamic changes in resources + totalResources = Resources.clone(freeResources); + LOG.info("App total resource memory: {} cpu: {} activeAssignments: {}", + totalResources.getMemory(), totalResources.getVirtualCores(), taskAssignments.size()); + } + + ++numHeartbeats; + if (LOG.isDebugEnabled() || numHeartbeats % 50 == 1) { + LOG.info(constructPeriodicLog(freeResources)); + } + + preemptedContainers = maybePreempt(freeResources); + if (preemptedContainers != null && !preemptedContainers.isEmpty()) { + lastPreemptionHeartbeat = numHeartbeats; + } + } + + // perform app callback outside of locks + if (preemptedContainers != null && !preemptedContainers.isEmpty()) { + for (ContainerId cid : preemptedContainers) { + LOG.info("Preempting container {} currently allocated to a task", cid); + getContext().preemptContainer(cid); + } + } + + return getContext().getProgress(); + } + + @Override + public void onShutdownRequest() { + if (!stopRequested) { + getContext().appShutdownRequested(); + } + } + + @Override + public void onError(Throwable e) { + LOG.error("Error from ARMRMClient", e); + if (!stopRequested) { + getContext().reportError(YarnTaskSchedulerServiceError.RESOURCEMANAGER_ERROR, + StringUtils.stringifyException(e), null); + } + } + + @Override + public Resource getAvailableResources() { + return client.getAvailableResources(); + } + + @Override + public Resource getTotalResources() { + return totalResources; + } + + @Override + public int getClusterNodeCount() { + return client.getClusterNodeCount(); + } + + @Override + public synchronized void blacklistNode(NodeId nodeId) { + LOG.info("Blacklisting node: {}", nodeId); + blacklistedNodes.add(nodeId); + client.updateBlacklist(Collections.singletonList(nodeId.getHost()), null); + } + + @Override + public synchronized void unblacklistNode(NodeId nodeId) { + if (blacklistedNodes.remove(nodeId)) { + LOG.info("Removing blacklist for node: {}", nodeId); + client.updateBlacklist(null, Collections.singletonList(nodeId.getHost())); + } + } + + @Override + public void allocateTask(Object task, Resource capability, String[] hosts, String[] racks, + Priority priority, Object containerSignature, Object clientCookie) { + int vertexIndex = getContext().getVertexIndexForTask(task); + TaskRequest request = new TaskRequest(task, vertexIndex, capability, hosts, racks, + priority, containerSignature, clientCookie); + addTaskRequest(request); + } + + @Override + public void allocateTask(Object task, Resource capability, ContainerId containerId, + Priority priority, Object containerSignature, Object clientCookie) { + String[] hosts = null; + synchronized (this) { + HeldContainer held = heldContainers.get(containerId); + if (held != null) { + if (held.canFit(capability)) { + hosts = new String[]{held.getHost()}; + } else { + LOG.warn("Match request to container {} but {} does not fit in {}", + containerId, capability, held.getCapability()); + containerId = null; + } + } else { + LOG.info("Ignoring match request to unknown container {}", containerId); + containerId = null; + } + } + int vertexIndex = getContext().getVertexIndexForTask(task); + TaskRequest request = new TaskRequest(task, vertexIndex, capability, hosts, null, + priority, containerSignature, clientCookie, containerId); + addTaskRequest(request); + } + + @Override + public boolean deallocateTask(Object task, boolean taskSucceeded, + TaskAttemptEndReason endReason, String diagnostics) { + ContainerId releasedLaunchedContainer = null; + AMState appState = getContext().getAMState(); + boolean isSession = getContext().isSession(); + TaskRequest newAssignment = null; + HeldContainer hc; + synchronized (this) { + TaskRequest request = removeTaskRequest(task); + if (request != null) { + LOG.debug("Deallocating task {} before it was allocated", task); + return false; + } + + hc = removeTaskAssignment(task); + if (hc != null) { + if (taskSucceeded && shouldReuseContainers) { + idleTracker.add(hc); + newAssignment = tryAssignReuseContainer(hc, appState, isSession); + if (newAssignment == null && hc.isReleasedAndUsed()) { + releasedLaunchedContainer = hc.getId(); + } + } else { + if (releaseContainer(hc)) { + releasedLaunchedContainer = hc.getId(); + } + } + } + } + + // perform app callback outside of locks + if (newAssignment != null) { + informAppAboutAssignment(newAssignment, hc.getContainer()); + return true; + } + if (releasedLaunchedContainer != null) { + getContext().containerBeingReleased(releasedLaunchedContainer); + return true; + } + return hc != null; + } + + @Override + public Object deallocateContainer(ContainerId containerId) { + Object task = null; + ContainerId releasedLaunchedContainer = null; + synchronized (this) { + HeldContainer hc = heldContainers.remove(containerId); + if (hc != null) { + task = hc.getAssignedTask(); + if (task != null) { + LOG.info("Deallocated container {} from task {}", containerId, task); + } + if (releaseContainer(hc)) { + releasedLaunchedContainer = hc.getId(); + } + } else { + LOG.info("Ignoring deallocation of unknown container {}", containerId); + } + } + + // perform app callback outside of locks + if (releasedLaunchedContainer != null) { + getContext().containerBeingReleased(releasedLaunchedContainer); + } + return task; + } + + @GuardedBy("this") + private void assignContainer(TaskRequest request, HeldContainer hc, Object match) { + LOG.info("Assigning container {} to task {} host={} priority={} capability={} match={} lastTask={}", + hc.getId(), request.getTask(), hc.getHost(), hc.getPriority(), hc.getCapability(), match, hc.getLastTask()); + removeTaskRequest(request.getTask()); + addTaskAssignment(request, hc); + idleTracker.remove(hc); + } + + private synchronized boolean releaseContainer(HeldContainer hc) { + Object task = containerCompleted(hc); + client.releaseAssignedContainer(hc.getId()); + if (task != null) { + releasedContainers.put(hc.getId(), task); + return true; + } + return false; + } + + @GuardedBy("this") + private void addTaskAssignment(TaskRequest request, HeldContainer hc) { + HeldContainer oldContainer = taskAssignments.put(request.getTask(), hc); + if (oldContainer != null) { + LOG.error("Task {} being assigned to container {} but was already assigned to container {}", + request.getTask(), hc.getId(), oldContainer.getId()); + } + Integer vertexIndex = request.vertexIndex; + Set cset = vertexAssignments.get(vertexIndex); + if (cset == null) { + cset = new HashSet<>(); + vertexAssignments.put(vertexIndex, cset); + assignedVertices.set(vertexIndex); + } + cset.add(hc); + hc.assignTask(request); + } + + @GuardedBy("this") + private HeldContainer removeTaskAssignment(Object task) { + HeldContainer hc = taskAssignments.remove(task); + if (hc != null) { + TaskRequest request = hc.removeAssignment(); + if (request != null) { + Integer vertexIndex = request.vertexIndex; + Set cset = vertexAssignments.get(vertexIndex); + if (cset != null && cset.remove(hc) && cset.isEmpty()) { + vertexAssignments.remove(vertexIndex); + assignedVertices.clear(vertexIndex); + } + } else { + LOG.error("Container {} had assigned task {} but no request?!?", hc.getId(), task); + } + } + return hc; + } + + @GuardedBy("this") + @Nullable + private Object containerCompleted(HeldContainer hc) { + idleTracker.remove(hc); + heldContainers.remove(hc.getId()); + Resources.subtractFrom(allocatedResources, hc.getCapability()); + removeTaskAssignment(hc.getAssignedTask()); + hc.released(); + return hc.getLastTask(); + } + + @GuardedBy("this") + private void ensureVertexDescendants() { + if (vertexDescendants == null) { + DagInfo info = getContext().getCurrentDagInfo(); + if (info == null) { + throw new IllegalStateException("Scheduling tasks but no current DAG info?"); + } + int numVertices = info.getTotalVertices(); + ArrayList descendants = new ArrayList<>(numVertices); + for (int i = 0; i < numVertices; ++i) { + descendants.add(info.getVertexDescendants(i)); + } + vertexDescendants = descendants; + } + } + + private void addTaskRequest(TaskRequest request) { + Container assignedContainer = null; + synchronized (this) { + if (shouldReuseContainers && !stopRequested && getContext().getAMState() != AMState.COMPLETED) { + ensureVertexDescendants(); + activateSessionContainers(); + HeldContainer hc = tryAssignTaskToIdleContainer(request); + if (hc != null) { + assignedContainer = hc.getContainer(); + } + } + + if (assignedContainer == null) { + ensureVertexDescendants(); + TaskRequest old = requestTracker.add(request); + if (old != null) { + removeTaskRequestByRequest(request); + } + client.addContainerRequest(request); + + HeldContainer hc = heldContainers.get(request.getAffinity()); + if (hc != null) { + hc.addAffinity(request); + } + } + } + + // perform app callback outside of locks + if (assignedContainer != null) { + informAppAboutAssignment(request, assignedContainer); + } + } + + @Nullable + private synchronized TaskRequest removeTaskRequest(Object task) { + TaskRequest request = requestTracker.remove(task); + if (request != null) { + removeTaskRequestByRequest(request); + } + return request; + } + + @GuardedBy("this") + private void removeTaskRequestByRequest(TaskRequest request) { + client.removeContainerRequest(request); + HeldContainer hc = heldContainers.get(request.getAffinity()); + if (hc != null) { + hc.removeAffinity(request); + } + } + + @GuardedBy("this") + @Nullable + private HeldContainer tryAssignTaskToIdleContainer(TaskRequest request) { + if (requestTracker.isRequestBlocked(request)) { + LOG.debug("Cannot assign task {} to an idle container since vertex {} is a descendant of pending tasks", + request.getTask(), request.getVertexIndex()); + return null; + } + + // check if container affinity can be satisfied immediately + ContainerId affinity = request.getAffinity(); + if (affinity != null) { + HeldContainer hc = heldContainers.get(affinity); + if (hc != null && hc.isAssignable()) { + assignContainer(request, hc, affinity); + return hc; + } + } + + // try to match the task against idle containers in order from best locality to worst + HeldContainer hc; + if (request.hasLocality()) { + hc = tryAssignTaskToIdleContainer(request, request.getNodes(), HeldContainerState.MATCHES_LOCAL_STATES); + if (hc == null) { + hc = tryAssignTaskToIdleContainer(request, request.getRacks(), HeldContainerState.MATCHES_RACK_STATES); + if (hc == null) { + hc = tryAssignTaskToIdleContainer(request, ResourceRequest.ANY, HeldContainerState.MATCHES_ANY_STATES); + } + } + } else { + hc = tryAssignTaskToIdleContainer(request, ResourceRequest.ANY, HeldContainerState.MATCHES_LOCAL_STATES); + } + + return hc; + } + + @GuardedBy("this") + @Nullable + private HeldContainer tryAssignTaskToIdleContainer(TaskRequest request, + List locations, EnumSet eligibleStates) { + if (locations != null && !locations.isEmpty()) { + for (String location : locations) { + HeldContainer hc = tryAssignTaskToIdleContainer(request, location, eligibleStates); + if (hc != null) { + return hc; + } + } + } + return null; + } + + @GuardedBy("this") + @Nullable + private HeldContainer tryAssignTaskToIdleContainer(TaskRequest request, + String location, EnumSet eligibleStates) { + Set containers = idleTracker.getByLocation(location); + HeldContainer bestMatch = null; + if (containers != null && !containers.isEmpty()) { + for (HeldContainer hc : containers) { + if (eligibleStates.contains(hc.getState())) { + Object csig = hc.getSignature(); + if (csig == null || signatureMatcher.isSuperSet(csig, request.getContainerSignature())) { + int numAffinities = hc.getNumAffinities(); + if (numAffinities == 0) { + bestMatch = hc; + break; + } + if (bestMatch == null || numAffinities < bestMatch.getNumAffinities()) { + bestMatch = hc; + } + } else { + LOG.debug("Unable to assign task {} to container {} due to signature mismatch", request.getTask(), hc.getId()); + } + } + } + } + if (bestMatch != null) { + assignContainer(request, bestMatch, location); + } + return bestMatch; + } + + @Override + public void setShouldUnregister() { + shouldUnregister = true; + } + + @Override + public boolean hasUnregistered() { + return hasUnregistered; + } + + @Override + public synchronized void dagComplete() { + for (HeldContainer hc : sessionContainers) { + hc.resetMatchingLevel(); + } + vertexDescendants = null; + } + + @GuardedBy("this") + @Nullable + private Collection maybePreempt(Resource freeResources) { + if (preemptionPercentage == 0 || numHeartbeats - lastPreemptionHeartbeat < numHeartbeatsBetweenPreemptions) { + return null; + } + if (!requestTracker.isPreemptionDeadlineExpired() && requestTracker.fitsHighestPriorityRequest(freeResources)) { + if (numHeartbeats % 50 == 1) { + LOG.info("Highest priority request fits in free resources {}", freeResources); + } + return null; + } + + int numIdleContainers = idleTracker.getNumContainers(); + if (numIdleContainers > 0) { + if (numHeartbeats % 50 == 1) { + LOG.info("Avoiding preemption since there are {} idle containers", numIdleContainers); + } + return null; + } + + BitSet blocked = requestTracker.createVertexBlockedSet(); + if (!blocked.intersects(assignedVertices)) { + if (numHeartbeats % 50 == 1) { + LOG.info("Avoiding preemption since there are no descendants of the highest priority requests running"); + } + return null; + } + + Resource preemptLeft = requestTracker.getAmountToPreempt(preemptionPercentage); + if (!resourceCalculator.anyAvailable(preemptLeft)) { + if (numHeartbeats % 50 == 1) { + LOG.info("Avoiding preemption since amount to preempt is {}", preemptLeft); + } + return null; + } + + PriorityQueue candidates = new PriorityQueue<>(11, PREEMPT_ORDER_COMPARATOR); + blocked.and(assignedVertices); + for (int i = blocked.nextSetBit(0); i >= 0; i = blocked.nextSetBit(i + 1)) { + Collection containers = vertexAssignments.get(i); + if (containers != null) { + candidates.addAll(containers); + } else { + LOG.error("Vertex {} in assignedVertices but no assignments?", i); + } + } + + ArrayList preemptedContainers = new ArrayList<>(); + HeldContainer hc; + while ((hc = candidates.poll()) != null) { + LOG.info("Preempting container {} currently allocated to task {}", hc.getId(), hc.getAssignedTask()); + preemptedContainers.add(hc.getId()); + resourceCalculator.deductFrom(preemptLeft, hc.getCapability()); + if (!resourceCalculator.anyAvailable(preemptLeft)) { + break; + } + } + + return preemptedContainers; + } + + @GuardedBy("this") + private String constructPeriodicLog(Resource freeResource) { + Priority highestPriority = requestTracker.getHighestPriority(); + return "Allocated: " + allocatedResources + + " Free: " + freeResource + + " pendingRequests: " + requestTracker.getNumRequests() + + " heldContainers: " + heldContainers.size() + + " heartbeats: " + numHeartbeats + + " lastPreemptionHeartbeat: " + lastPreemptionHeartbeat + + ((highestPriority != null) ? + (" highestWaitingRequestWaitStartTime: " + requestTracker.getHighestPriorityWaitTimestamp() + + " highestWaitingRequestPriority: " + highestPriority) : ""); + } + + @VisibleForTesting + int getNumBlacklistedNodes() { + return blacklistedNodes.size(); + } + + @VisibleForTesting + Collection getSessionContainers() { + return sessionContainers; + } + + // Wrapper class to work around lack of blacklisting APIs in async client. + // This can be removed once Tez requires YARN >= 2.7.0 + static class AMRMClientAsyncWrapper extends AMRMClientAsyncImpl { + AMRMClientAsyncWrapper(AMRMClient syncClient, int intervalMs, CallbackHandler handler) { + super(syncClient, intervalMs, handler); + } + + public void updateBlacklist(List additions, List removals) { + client.updateBlacklist(additions, removals); + } + } + + /** + * A utility class to track a task allocation. + */ + static class TaskRequest extends AMRMClient.ContainerRequest { + final Object task; + final int vertexIndex; + final Object signature; + final Object cookie; + final ContainerId affinityContainerId; + + TaskRequest(Object task, int vertexIndex, Resource capability, String[] hosts, String[] racks, + Priority priority, Object signature, Object cookie) { + this(task, vertexIndex, capability, hosts, racks, priority, signature, cookie, null); + } + + TaskRequest(Object task, int vertexIndex, Resource capability, String[] hosts, String[] racks, + Priority priority, Object signature, Object cookie, ContainerId affinityContainerId) { + super(capability, hosts, racks, priority); + this.task = task; + this.vertexIndex = vertexIndex; + this.signature = signature; + this.cookie = cookie; + this.affinityContainerId = affinityContainerId; + } + + Object getTask() { + return task; + } + + int getVertexIndex() { + return vertexIndex; + } + + Object getContainerSignature() { + return signature; + } + + Object getCookie() { + return cookie; + } + + @Nullable + ContainerId getAffinity() { + return affinityContainerId; + } + + boolean hasLocality() { + List nodes = getNodes(); + List racks = getRacks(); + return (nodes != null && !nodes.isEmpty()) || (racks != null && !racks.isEmpty()); + } + } + + private enum HeldContainerState { + MATCHING_LOCAL(true), + MATCHING_RACK(true), + MATCHING_ANY(true), + ASSIGNED(false), + RELEASED(false); + + private static final EnumSet MATCHES_LOCAL_STATES = EnumSet.of( + HeldContainerState.MATCHING_LOCAL, HeldContainerState.MATCHING_RACK, HeldContainerState.MATCHING_ANY); + private static final EnumSet MATCHES_RACK_STATES = EnumSet.of( + HeldContainerState.MATCHING_RACK, HeldContainerState.MATCHING_ANY); + private static final EnumSet MATCHES_ANY_STATES = EnumSet.of(HeldContainerState.MATCHING_ANY); + + private final boolean assignable; + + HeldContainerState(boolean assignable) { + this.assignable = assignable; + } + + boolean isAssignable() { + return assignable; + } + } + + /** + * Tracking for an allocated container. + */ + @VisibleForTesting + class HeldContainer implements Callable { + final Container container; + final String rack; + @GuardedBy("DagAwareYarnTaskScheduler.this") + HeldContainerState state = HeldContainerState.MATCHING_LOCAL; + + /** The Future received when scheduling an idle container for re-allocation at a later time. */ + @GuardedBy("DagAwareYarnTaskScheduler.this") + Future future = null; + + /** The collection of task requests that have specified this container as a scheduling affinity. */ + @GuardedBy("DagAwareYarnTaskScheduler.this") + Collection affinities = null; + + /** + * The task request corresponding to the currently assigned task to this container. + * This field is null when the container is not currently assigned. + */ + @GuardedBy("DagAwareYarnTaskScheduler.this") + TaskRequest assignedRequest = null; + + /** The task request corresponding to the last task that was assigned to this container. */ + @GuardedBy("DagAwareYarnTaskScheduler.this") + TaskRequest lastRequest = null; + + /** The timestamp when the idle container will expire. 0 if the container is not idle. */ + @GuardedBy("DagAwareYarnTaskScheduler.this") + long idleExpirationTimestamp = 0; + + /** The timestamp when this container was assigned. 0 if the container is not assigned. */ + @GuardedBy("DagAwareYarnTaskScheduler.this") + long assignmentTimestamp = 0; + + HeldContainer(Container container) { + this.container = container; + this.rack = RackResolver.resolve(container.getNodeId().getHost()).getNetworkLocation(); + } + + HeldContainerState getState() { + return state; + } + + boolean isAssignable() { + return state.isAssignable(); + } + + boolean isReleasedAndUsed() { + return state == HeldContainerState.RELEASED && getLastTask() != null; + } + + Container getContainer() { + return container; + } + + ContainerId getId() { + return container.getId(); + } + + String getHost() { + return container.getNodeId().getHost(); + } + + String getRack() { + return rack; + } + + Priority getPriority() { + return container.getPriority(); + } + + Resource getCapability() { + return container.getResource(); + } + + @Nullable + Object getAssignedTask() { + return assignedRequest != null ? assignedRequest.getTask() : null; + } + + void assignTask(TaskRequest request) { + assert state != HeldContainerState.ASSIGNED && state != HeldContainerState.RELEASED; + if (assignedRequest != null) { + LOG.error("Container {} assigned task {} but already running task {}", + getId(), request.getTask(), assignedRequest.getTask()); + } + assignedRequest = request; + lastRequest = request; + state = HeldContainerState.ASSIGNED; + idleExpirationTimestamp = 0; + assignmentTimestamp = now(); + if (future != null) { + future.cancel(false); + future = null; + } + } + + TaskRequest removeAssignment() { + assert state == HeldContainerState.ASSIGNED; + TaskRequest result = assignedRequest; + assignedRequest = null; + assignmentTimestamp = 0; + state = HeldContainerState.MATCHING_LOCAL; + return result; + } + + void addAffinity(TaskRequest request) { + if (affinities == null) { + affinities = new HashSet<>(); + } + affinities.add(request); + } + + void removeAffinity(TaskRequest request) { + if (affinities != null && affinities.remove(request) && affinities.isEmpty()) { + affinities = null; + } + } + + int getNumAffinities() { + return affinities != null ? affinities.size() : 0; + } + + @Nullable + Collection getAffinities() { + return affinities; + } + + void scheduleForReuse(long delayMillis) { + assert state != HeldContainerState.ASSIGNED && state != HeldContainerState.RELEASED; + try { + if (future != null) { + future.cancel(false); + } + future = reuseExecutor.schedule(this, delayMillis, TimeUnit.MILLISECONDS); + } catch (RejectedExecutionException e) { + if (!stopRequested) { + LOG.error("Container {} could not be scheduled for reuse!", getId(), e); + } + } + } + + @Nullable + Object getSignature() { + return lastRequest != null ? lastRequest.getContainerSignature() : null; + } + + @Nullable + Object getLastTask() { + return lastRequest != null ? lastRequest.getTask() : null; + } + + String getMatchingLocation() { + switch (state) { + case MATCHING_LOCAL: + return getHost(); + case MATCHING_RACK: + return getRack(); + case MATCHING_ANY: + return ResourceRequest.ANY; + default: + throw new IllegalStateException("Container " + getId() + " trying to match in state " + state); + } + } + + void moveToNextMatchingLevel() { + switch (state) { + case MATCHING_LOCAL: + if (reuseRackLocal) { + state = HeldContainerState.MATCHING_RACK; + } + break; + case MATCHING_RACK: + if (reuseNonLocal) { + state = HeldContainerState.MATCHING_ANY; + } + break; + case MATCHING_ANY: + break; + default: + throw new IllegalStateException("Container " + getId() + " trying to match in state " + state); + } + } + + boolean atMaxMatchLevel() { + switch (state) { + case MATCHING_LOCAL: + return !reuseRackLocal; + case MATCHING_RACK: + return !reuseNonLocal; + case MATCHING_ANY: + return true; + default: + throw new IllegalStateException("Container " + getId() + " trying to match in state " + state); + } + } + + void resetMatchingLevel() { + if (isAssignable()) { + state = HeldContainerState.MATCHING_LOCAL; + } + } + + long getIdleExpirationTimestamp(long now) { + if (idleExpirationTimestamp == 0) { + if (idleContainerTimeoutMin > 0) { + idleExpirationTimestamp = now + random.nextLong(idleContainerTimeoutMin, idleContainerTimeoutMax); + } else { + idleExpirationTimestamp = Long.MAX_VALUE; + } + } + return idleExpirationTimestamp; + } + + long getAssignmentTimestamp() { + return assignmentTimestamp; + } + + boolean canFit(Resource capability) { + Resource cr = container.getResource(); + return cr.getMemory() >= capability.getMemory() && cr.getVirtualCores() >= capability.getVirtualCores(); + } + + @Override + public Void call() throws Exception { + AMState appState = getContext().getAMState(); + boolean isSession = getContext().isSession(); + TaskRequest assigned = null; + ContainerId released = null; + synchronized (DagAwareYarnTaskScheduler.this) { + future = null; + if (isAssignable()) { + moveToNextMatchingLevel(); + assigned = tryAssignReuseContainer(this, appState, isSession); + if (assigned == null && isReleasedAndUsed()) { + released = getId(); + } + } + } + if (assigned != null) { + informAppAboutAssignment(assigned, container); + } + if (released != null) { + getContext().containerBeingReleased(released); + } + return null; + } + + void released() { + assert state != HeldContainerState.RELEASED; + state = HeldContainerState.RELEASED; + if (future != null) { + future.cancel(false); + } + future = null; + } + } + + /** + * Utility comparator to order containers by assignment timestamp from + * most recent to least recent. + */ + private static class PreemptOrderComparator implements Comparator { + @Override + public int compare(HeldContainer o1, HeldContainer o2) { + long timestamp1 = o1.getAssignmentTimestamp(); + if (timestamp1 == 0) { + timestamp1 = Long.MAX_VALUE; + } + long timestamp2 = o2.getAssignmentTimestamp(); + if (timestamp2 == 0) { + timestamp2 = Long.MAX_VALUE; + } + return Long.compare(timestamp2, timestamp1); + } + } + + /** + * Utility class for a request, container pair + */ + private static class Assignment { + final TaskRequest request; + final Container container; + + Assignment(TaskRequest request, Container container) { + this.request = request; + this.container = container; + } + } + + /** + * Utility class for a task, container exit status pair + */ + private static class TaskStatus { + final Object task; + final ContainerStatus status; + + TaskStatus(Object task, ContainerStatus status) { + this.task = task; + this.status = status; + } + } + + /** + * The task allocation request tracker tracks task allocations + * and keeps statistics on which priorities have requests and which vertices + * should be blocked from container reuse due to DAG topology. + */ + private class RequestTracker { + private final Map requests = new HashMap<>(); + /** request map ordered by priority with highest priority first */ + private final NavigableMap priorityStats = + new TreeMap<>(Collections.reverseOrder()); + private Priority highestPriority = null; + private long highestPriorityWaitTimestamp = 0; + + @GuardedBy("DagAwareYarnTaskScheduler.this") + @Nullable + TaskRequest add(TaskRequest request) { + TaskRequest oldRequest = requests.put(request.getTask(), request); + Priority priority = request.getPriority(); + RequestPriorityStats stats = priorityStats.get(priority); + if (stats == null) { + stats = addStatsForPriority(priority); + } + ++stats.requestCount; + if (request.hasLocality()) { + ++stats.localityCount; + } + incrVertexTaskCount(priority, stats, request.getVertexIndex()); + + if (oldRequest != null) { + updateStatsForRemoval(oldRequest); + } + return oldRequest; + } + + @GuardedBy("DagAwareYarnTaskScheduler.this") + @Nullable + TaskRequest remove(Object task) { + TaskRequest request = requests.remove(task); + if (request != null) { + updateStatsForRemoval(request); + return request; + } + return null; + } + + private RequestPriorityStats addStatsForPriority(Priority priority) { + BitSet allowedVerts = new BitSet(vertexDescendants.size()); + Entry lowerEntry = priorityStats.lowerEntry(priority); + if (lowerEntry != null) { + // initialize the allowed vertices BitSet using the information derived + // from the next higher priority entry + RequestPriorityStats priorStats = lowerEntry.getValue(); + allowedVerts.or(priorStats.allowedVertices); + allowedVerts.andNot(priorStats.descendants); + } else { + // no higher priority entry so this priority is currently the highest + highestPriority = priority; + highestPriorityWaitTimestamp = now(); + allowedVerts.set(0, vertexDescendants.size()); + } + RequestPriorityStats stats = new RequestPriorityStats(vertexDescendants.size(), allowedVerts); + priorityStats.put(priority, stats); + return stats; + } + + private void updateStatsForRemoval(TaskRequest request) { + Priority priority = request.getPriority(); + RequestPriorityStats stats = priorityStats.get(priority); + decrVertexTaskCount(priority, stats, request.getVertexIndex()); + --stats.requestCount; + if (request.hasLocality()) { + --stats.localityCount; + } + if (stats.requestCount == 0) { + priorityStats.remove(priority); + if (highestPriority.equals(priority)) { + if (priorityStats.isEmpty()) { + highestPriority = null; + highestPriorityWaitTimestamp = 0; + } else { + highestPriority = priorityStats.firstKey(); + highestPriorityWaitTimestamp = now(); + } + } + } + } + + @GuardedBy("DagAwareYarnTaskScheduler.this") + boolean isEmpty() { + return requests.isEmpty(); + } + + @GuardedBy("DagAwareYarnTaskScheduler.this") + int getNumRequests() { + return requests.size(); + } + + @GuardedBy("DagAwareYarnTaskScheduler.this") + List getTasks() { + return new ArrayList<>(requests.keySet()); + } + + @GuardedBy("DagAwareYarnTaskScheduler.this") + Collection> getStatsEntries() { + return priorityStats.entrySet(); + } + + @GuardedBy("DagAwareYarnTaskScheduler.this") + @Nullable + Priority getHighestPriority() { + if (priorityStats.isEmpty()) { + return null; + } + return priorityStats.firstKey(); + } + + @GuardedBy("DagAwareYarnTaskScheduler.this") + long getHighestPriorityWaitTimestamp() { + return highestPriorityWaitTimestamp; + } + + @GuardedBy("DagAwareYarnTaskScheduler.this") + boolean isRequestBlocked(TaskRequest request) { + Entry entry = priorityStats.floorEntry(request.getPriority()); + if (entry != null) { + RequestPriorityStats stats = entry.getValue(); + int vertexIndex = request.getVertexIndex(); + return !stats.allowedVertices.get(vertexIndex) || stats.descendants.get(vertexIndex); + } + return false; + } + + private void incrVertexTaskCount(Priority priority, RequestPriorityStats stats, int vertexIndex) { + Integer vertexIndexInt = vertexIndex; + MutableInt taskCount = stats.vertexTaskCount.get(vertexIndexInt); + if (taskCount != null) { + taskCount.increment(); + } else { + addVertexToRequestStats(priority, stats, vertexIndexInt); + } + } + + private void decrVertexTaskCount(Priority priority, RequestPriorityStats stats, int vertexIndex) { + Integer vertexIndexInt = vertexIndex; + MutableInt taskCount = stats.vertexTaskCount.get(vertexIndexInt); + taskCount.decrement(); + if (taskCount.intValue() <= 0) { + removeVertexFromRequestStats(priority, stats, vertexIndexInt); + } + } + + /** + * Add a new vertex to a RequestPriorityStats. + * + * Adding a vertex to the request stats requires updating the stats descendants bitmask to include the descendants + * of the new vertex and also updating the allowedVertices bitmask for all lower priority requests to prevent any + * task request from a descendant vertex in the DAG from being allocated. This avoids assigning allocations to + * lower priority requests when a higher priority request of an ancestor is still pending, but it allows lower + * priority requests to be satisfied if higher priority requests are not ancestors. This is particularly useful + * for DAGs that have independent trees of vertices or significant, parallel branches within a tree. + * + * Requests are blocked by taking the specified vertex's full descendant vertex bitmask in vertexDescendants and + * clearing those bits for all lower priority requests. For the following example DAG where each vertex index + * corresponds to its letter position (i.e.: A=0, B=1, C=2, etc.) + * + * A + * | + * C---B----E + * | | + * D F + * | + * G---H + * + * Vertices F, G, and H are descendants of E but all other vertices are not. The vertexDescendants bitmask for + * vertex E is therefore 11100000b or 0xE0. When the first vertex E task request arrives we need to disallow + * requests for all descendants of E. That is accomplished by iterating through the request stats for all lower + * priority requests and clearing the allowedVertex bits corresponding to the descendants, + * i.e: allowedVertices = allowedVertices & ~descendants + */ + private void addVertexToRequestStats(Priority priority, RequestPriorityStats stats, Integer vertexIndexInt) { + // Creating a new vertex entry for this priority, so the allowed vertices for all + // lower priorities need to be updated based on the descendants of the new vertex. + stats.vertexTaskCount.put(vertexIndexInt, new MutableInt(1)); + int vertexIndex = vertexIndexInt; + stats.vertices.set(vertexIndex); + BitSet d = vertexDescendants.get(vertexIndex); + stats.descendants.or(d); + for (RequestPriorityStats lowerStat : priorityStats.tailMap(priority, false).values()) { + lowerStat.allowedVertices.andNot(d); + } + } + + /** + * Removes a vertex from a RequestPriorityStats. + * + * Removing a vertex is more expensive than adding a vertex. The stats contain bitmasks which only store on/off + * values rather than reference counts. Therefore we must rebuild the descendants bitmasks from the remaining + * vertices in the request stats. Once the new descendants mask is computed we then need to rebuild the + * allowedVertices BitSet for all lower priority request stats in case the removal of this vertex unblocks lower + * priority requests of a descendant vertex. + * + * Rebuilding allowedVertices for the lower priorities involves starting with the allowedVertices mask at the + * current priority then masking off the descendants at each priority level encountered, accumulating the results. + * Any descendants of a level will be blocked at all lower levels. See the addVertexToRequestStats documentation + * for details on how vertices map to the descendants and allowedVertices bit masks. + */ + private void removeVertexFromRequestStats(Priority priority, RequestPriorityStats stats, Integer vertexIndexInt) { + stats.vertexTaskCount.remove(vertexIndexInt); + int vertexIndex = vertexIndexInt; + stats.vertices.clear(vertexIndex); + + // Rebuild the descendants BitSet for the remaining vertices at this priority. + stats.descendants.clear(); + for (Integer vIndex : stats.vertexTaskCount.keySet()) { + stats.descendants.or(vertexDescendants.get(vIndex)); + } + + // The allowedVertices for all lower priorities need to be recalculated where the vertex descendants at each + // level are removed from the list of allowed vertices at all subsequent levels. + Collection tailStats = priorityStats.tailMap(priority, false).values(); + if (!tailStats.isEmpty()) { + BitSet cumulativeAllowed = new BitSet(vertexDescendants.size()); + cumulativeAllowed.or(stats.allowedVertices); + cumulativeAllowed.andNot(stats.descendants); + for (RequestPriorityStats s : tailStats) { + s.allowedVertices.clear(); + s.allowedVertices.or(cumulativeAllowed); + cumulativeAllowed.andNot(s.descendants); + } + } + } + + @GuardedBy("DagAwareYarnTaskScheduler.this") + boolean isPreemptionDeadlineExpired() { + return highestPriorityWaitTimestamp != 0 + && now() - highestPriorityWaitTimestamp > preemptionMaxWaitTime; + } + + @GuardedBy("DagAwareYarnTaskScheduler.this") + boolean fitsHighestPriorityRequest(Resource freeResources) { + if (priorityStats.isEmpty()) { + return true; + } + Priority priority = priorityStats.firstKey(); + List requestsList = client.getMatchingRequests( + priority, ResourceRequest.ANY, freeResources); + return !requestsList.isEmpty(); + } + + @GuardedBy("DagAwareYarnTaskScheduler.this") + Resource getAmountToPreempt(int preemptionPercentage) { + if (priorityStats.isEmpty()) { + return Resources.none(); + } + Priority priority = priorityStats.firstKey(); + List> requestsList = client.getMatchingRequests( + priority, ResourceRequest.ANY, Resources.unbounded()); + int numRequests = 0; + for (Collection requests : requestsList) { + numRequests += requests.size(); + } + numRequests = (int) Math.ceil(numRequests * (preemptionPercentage / 100.f)); + Resource toPreempt = Resource.newInstance(0, 0); + if (numRequests != 0) { + outer_loop: + for (Collection requests : requestsList) { + for (TaskRequest request : requests) { + Resources.addTo(toPreempt, request.getCapability()); + if (--numRequests == 0) { + break outer_loop; + } + } + } + } + return toPreempt; + } + + // Create a new BitSet that represents all of the vertices that should not be + // scheduled due to outstanding requests from higher priority predecessor vertices. + @GuardedBy("DagAwareYarnTaskScheduler.this") + BitSet createVertexBlockedSet() { + BitSet blocked = new BitSet(); + Entry entry = priorityStats.lastEntry(); + if (entry != null) { + RequestPriorityStats stats = entry.getValue(); + blocked.or(stats.allowedVertices); + blocked.flip(0, blocked.length()); + blocked.or(stats.descendants); + } + return blocked; + } + } + + /** + * Tracks statistics on vertices that are requesting tasks at a particular priority + */ + private static class RequestPriorityStats { + /** Map from vertex ID to number of task requests for that vertex */ + final Map vertexTaskCount = new HashMap<>(); + /** BitSet of vertices that have oustanding requests at this priority */ + final BitSet vertices; + /** BitSet of vertices that are descendants of this vertex */ + final BitSet descendants; + /** + * BitSet of vertices that are allowed to be scheduled at this priority + * (i.e.: no oustanding predecessors requesting at higher priorities) + */ + final BitSet allowedVertices; + int requestCount = 0; + int localityCount = 0; + + RequestPriorityStats(int numTotalVertices, BitSet allowedVertices) { + this.vertices = new BitSet(numTotalVertices); + this.descendants = new BitSet(numTotalVertices); + this.allowedVertices = allowedVertices; + } + } + + /** + * Tracks idle containers and facilitates faster matching of task requests + * against those containers given a desired location. + */ + private static class IdleContainerTracker { + /** + * Map of location ID (e.g.: a specific host, rack, or ANY) to set of + * idle containers matching that location + */ + final Map> containersByLocation = new HashMap<>(); + int numContainers = 0; + + @GuardedBy("DagAwareYarnTaskScheduler.this") + void add(HeldContainer hc) { + add(hc, hc.getHost()); + add(hc, hc.getRack()); + add(hc, ResourceRequest.ANY); + ++numContainers; + } + + @GuardedBy("DagAwareYarnTaskScheduler.this") + void remove(HeldContainer hc) { + remove(hc, hc.getHost()); + remove(hc, hc.getRack()); + remove(hc, ResourceRequest.ANY); + --numContainers; + } + + @GuardedBy("DagAwareYarnTaskScheduler.this") + int getNumContainers() { + return numContainers; + } + + private void add(HeldContainer hc, String location) { + Set containers = containersByLocation.get(location); + if (containers == null) { + containers = new HashSet<>(); + containersByLocation.put(location, containers); + } + containers.add(hc); + } + + private void remove(HeldContainer hc, String location) { + Set containers = containersByLocation.get(location); + if (containers != null) { + if (containers.remove(hc) && containers.isEmpty()) { + containersByLocation.remove(location); + } + } + } + + @GuardedBy("DagAwareYarnTaskScheduler.this") + @Nullable + Set getByLocation(String location) { + return containersByLocation.get(location); + } + } + + private interface ResourceCalculator { + boolean anyAvailable(Resource rsrc); + void deductFrom(Resource total, Resource toSubtract); + } + + /** + * ResourceCalculator for memory-only allocation + */ + private static class MemResourceCalculator implements ResourceCalculator { + + @Override + public boolean anyAvailable(Resource rsrc) { + return rsrc.getMemory() > 0; + } + + @Override + public void deductFrom(Resource total, Resource toSubtract) { + total.setMemory(total.getMemory() - toSubtract.getMemory()); + } + } + + /** + * ResourceCalculator for memory and vcore allocation + */ + private static class MemCpuResourceCalculator extends MemResourceCalculator { + + @Override + public boolean anyAvailable(Resource rsrc) { + return super.anyAvailable(rsrc) || rsrc.getVirtualCores() > 0; + } + + @Override + public void deductFrom(Resource total, Resource toSubtract) { + super.deductFrom(total, toSubtract); + total.setVirtualCores(total.getVirtualCores() - toSubtract.getVirtualCores()); + } + } + + /** + * Scheduled thread pool executor that logs any errors that escape the worker thread. + * This can be replaced with HadoopThreadPoolExecutor once Tez requires Hadoop 2.8 or later. + */ + static class ReuseContainerExecutor extends ScheduledThreadPoolExecutor { + ReuseContainerExecutor() { + super(1, new ThreadFactoryBuilder().setNameFormat("ReuseContainerExecutor #%d").build()); + setRemoveOnCancelPolicy(true); + setExecuteExistingDelayedTasksAfterShutdownPolicy(false); + } + + @Override + protected void afterExecute(Runnable r, Throwable t) { + super.afterExecute(r, t); + + if (t == null && r instanceof Future) { + try { + ((Future) r).get(); + } catch (ExecutionException ee) { + LOG.warn("Execution exception when running task in {}", Thread.currentThread().getName()); + t = ee.getCause(); + } catch (InterruptedException ie) { + LOG.warn("Thread ({}) interrupted: ", Thread.currentThread(), ie); + Thread.currentThread().interrupt(); + } catch (Throwable throwable) { + t = throwable; + } + } + + if (t != null) { + LOG.warn("Caught exception in thread {}", Thread.currentThread().getName(), t); + } + } + } +} diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java index 39000d6b0a..f273c8b04a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java @@ -182,6 +182,11 @@ public AMState getAMState() { } } + @Override + public int getVertexIndexForTask(Object task) { + return taskSchedulerManager.getVertexIndexForTask(task); + } + @Override public void reportError(ServicePluginError servicePluginError, String diagnostics, DagInfo dagInfo) { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java index 49ab77dcb8..5e45e70993 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java @@ -187,6 +187,11 @@ public AMState getAMState() { return real.getAMState(); } + @Override + public int getVertexIndexForTask(Object task) { + return real.getVertexIndexForTask(task); + } + // End of getters which do not need to go through a thread. Underlying implementation // does not use locks. diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java index 7c1b926e95..5777a2a4fe 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java @@ -139,6 +139,8 @@ public class TaskSchedulerManager extends AbstractService implements BlockingQueue eventQueue = new LinkedBlockingQueue(); + private final String yarnSchedulerClassName; + // Not tracking container / task to schedulerId. Instead relying on everything flowing through // the system and being propagated back via events. @@ -164,6 +166,8 @@ public TaskSchedulerManager(TaskScheduler taskScheduler, AppContext appContext, this.historyUrl = null; this.isLocalMode = false; this.hadoopShim = new HadoopShimsLoader(appContext.getAMConf()).getHadoopShim(); + this.yarnSchedulerClassName = appContext.getAMConf().get(TezConfiguration.TEZ_AM_YARN_SCHEDULER_CLASS, + TezConfiguration.TEZ_AM_YARN_SCHEDULER_CLASS_DEFAULT); } /** @@ -196,6 +200,8 @@ public TaskSchedulerManager(AppContext appContext, this.historyUrl = getHistoryUrl(); this.isLocalMode = isLocalMode; this.hadoopShim = hadoopShim; + this.yarnSchedulerClassName = appContext.getAMConf().get(TezConfiguration.TEZ_AM_YARN_SCHEDULER_CLASS, + TezConfiguration.TEZ_AM_YARN_SCHEDULER_CLASS_DEFAULT); this.appCallbackExecutor = createAppCallbackExecutorService(); if (this.webUI != null) { this.webUI.setHistoryUrl(this.historyUrl); @@ -574,9 +580,11 @@ TaskSchedulerContext wrapTaskSchedulerContext(TaskSchedulerContext rawContext) { @VisibleForTesting TaskScheduler createYarnTaskScheduler(TaskSchedulerContext taskSchedulerContext, - int schedulerId) { - LOG.info("Creating TaskScheduler: YarnTaskSchedulerService"); - return new YarnTaskSchedulerService(taskSchedulerContext); + int schedulerId) throws TezException { + LOG.info("Creating YARN TaskScheduler: {}", yarnSchedulerClassName); + return ReflectionUtils.createClazzInstance(yarnSchedulerClassName, + new Class[] { TaskSchedulerContext.class }, + new Object[] { taskSchedulerContext }); } @VisibleForTesting @@ -888,11 +896,10 @@ public void reportError(int taskSchedulerIndex, ServicePluginError servicePlugin LOG.info("Error reported by scheduler {} - {}", Utils.getTaskSchedulerIdentifierString(taskSchedulerIndex, appContext) + ": " + diagnostics); - if (taskSchedulerDescriptors[taskSchedulerIndex].getClassName() - .equals(YarnTaskSchedulerService.class.getName())) { + if (taskSchedulerDescriptors[taskSchedulerIndex].getClassName().equals(yarnSchedulerClassName)) { LOG.warn( "Reporting a SchedulerServiceError to the DAGAppMaster since the error" + - " was reported by the default YARN Task Scheduler"); + " was reported by the YARN task scheduler"); sendEvent(new DAGAppMasterEventSchedulingServiceError(diagnostics)); } } else if (servicePluginError.getErrorType() == ServicePluginError.ErrorType.PERMANENT) { @@ -934,6 +941,11 @@ public void dagSubmitted() { // the context has updated information. } + public int getVertexIndexForTask(Object task) { + TaskAttempt attempt = (TaskAttempt) task; + return attempt.getVertexID().getId(); + } + public void preemptContainer(int schedulerId, ContainerId containerId) { // TODO Why is this making a call back into the scheduler, when the call is originating from there. // An AMContainer instance should already exist if an attempt is being made to preempt it diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockClock.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockClock.java index d01571446d..c7cf74eb84 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockClock.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockClock.java @@ -20,9 +20,21 @@ import org.apache.hadoop.yarn.util.Clock; +import java.util.Collection; +import java.util.LinkedList; + public class MockClock implements Clock { - long time = 1000; + long time; + Collection listeners = new LinkedList<>(); + + public MockClock() { + this(1000); + } + + public MockClock(long initTime) { + time = initTime; + } @Override public long getTime() { @@ -31,6 +43,16 @@ public long getTime() { public void incrementTime(long inc) { time += inc; + for (MockClockListener listener : listeners) { + listener.onTimeUpdated(time); + } + } + + public void register(MockClockListener listener) { + listeners.add(listener); } + public interface MockClockListener { + void onTimeUpdated(long newTime); + } } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java new file mode 100644 index 0000000000..553306f534 --- /dev/null +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java @@ -0,0 +1,1510 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.app.rm; + +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; +import org.apache.hadoop.yarn.api.records.ApplicationAccessType; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.NodeReport; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; +import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes; +import org.apache.hadoop.yarn.util.resource.Resources; +import org.apache.tez.common.MockDNSToSwitchMapping; +import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.app.MockClock; +import org.apache.tez.dag.app.rm.DagAwareYarnTaskScheduler.AMRMClientAsyncWrapper; +import org.apache.tez.dag.app.rm.DagAwareYarnTaskScheduler.HeldContainer; +import org.apache.tez.dag.app.rm.DagAwareYarnTaskScheduler.TaskRequest; +import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.TaskSchedulerContextDrainable; +import org.apache.tez.serviceplugins.api.DagInfo; +import org.apache.tez.serviceplugins.api.TaskScheduler; +import org.apache.tez.serviceplugins.api.TaskSchedulerContext; +import org.apache.tez.serviceplugins.api.TaskSchedulerContext.AppFinalStatus; +import org.apache.tez.test.ControlledScheduledExecutorService; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.mockito.ArgumentCaptor; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.BitSet; +import java.util.Collections; +import java.util.EnumSet; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; + +import static org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.createCountingExecutingService; +import static org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.setupMockTaskSchedulerContext; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyInt; +import static org.mockito.Matchers.anyListOf; +import static org.mockito.Matchers.eq; +import static org.mockito.Matchers.isNull; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class TestDagAwareYarnTaskScheduler { + private ExecutorService contextCallbackExecutor; + + @BeforeClass + public static void beforeClass() { + + MockDNSToSwitchMapping.initializeMockRackResolver(); + } + + @Before + public void preTest() { + contextCallbackExecutor = Executors.newSingleThreadExecutor( + new ThreadFactoryBuilder().setNameFormat("TaskSchedulerAppCallbackExecutor #%d") + .setDaemon(true) + .build()); + } + + @After + public void postTest() { + contextCallbackExecutor.shutdownNow(); + } + + private TaskSchedulerContextDrainable createDrainableContext( + TaskSchedulerContext taskSchedulerContext) { + TaskSchedulerContextImplWrapper wrapper = + new TaskSchedulerContextImplWrapper(taskSchedulerContext, + createCountingExecutingService(contextCallbackExecutor)); + return new TaskSchedulerContextDrainable(wrapper); + } + + @SuppressWarnings({ "unchecked" }) + @Test(timeout=30000) + public void testNoReuse() throws Exception { + AMRMClientAsyncWrapperForTest mockRMClient = spy(new AMRMClientAsyncWrapperForTest()); + + String appHost = "host"; + int appPort = 0; + String appUrl = "url"; + + Configuration conf = new Configuration(); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, false); + conf.setInt(TezConfiguration.TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX, 100); + + DagInfo mockDagInfo = mock(DagInfo.class); + when(mockDagInfo.getTotalVertices()).thenReturn(10); + when(mockDagInfo.getVertexDescendants(anyInt())).thenReturn(new BitSet()); + TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf); + when(mockApp.getCurrentDagInfo()).thenReturn(mockDagInfo); + TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp); + + MockClock clock = new MockClock(1000); + NewTaskSchedulerForTest scheduler = new NewTaskSchedulerForTest(drainableAppCallback, + mockRMClient, clock); + + scheduler.initialize(); + drainableAppCallback.drain(); + + scheduler.start(); + drainableAppCallback.drain(); + verify(mockRMClient).start(); + verify(mockRMClient).registerApplicationMaster(appHost, appPort, appUrl); + RegisterApplicationMasterResponse regResponse = mockRMClient.getRegistrationResponse(); + verify(mockApp).setApplicationRegistrationData(regResponse.getMaximumResourceCapability(), + regResponse.getApplicationACLs(), regResponse.getClientToAMTokenMasterKey(), + regResponse.getQueue()); + + assertEquals(scheduler.getClusterNodeCount(), mockRMClient.getClusterNodeCount()); + + Object mockTask1 = new MockTask("task1"); + Object mockCookie1 = new Object(); + Resource mockCapability = Resources.createResource(1024, 1); + String[] hosts = {"host1", "host5"}; + String[] racks = {"/default-rack", "/default-rack"}; + Priority mockPriority = Priority.newInstance(1); + ArgumentCaptor requestCaptor = + ArgumentCaptor.forClass(TaskRequest.class); + // allocate task + scheduler.allocateTask(mockTask1, mockCapability, hosts, + racks, mockPriority, null, mockCookie1); + drainableAppCallback.drain(); + verify(mockRMClient, times(1)). + addContainerRequest(any(TaskRequest.class)); + + // returned from task requests before allocation happens + assertFalse(scheduler.deallocateTask(mockTask1, true, null, null)); + verify(mockApp, times(0)).containerBeingReleased(any(ContainerId.class)); + verify(mockRMClient, times(1)). + removeContainerRequest(any(TaskRequest.class)); + verify(mockRMClient, times(0)). + releaseAssignedContainer((ContainerId) any()); + + // deallocating unknown task + assertFalse(scheduler.deallocateTask(mockTask1, true, null, null)); + verify(mockApp, times(0)).containerBeingReleased(any(ContainerId.class)); + verify(mockRMClient, times(1)). + removeContainerRequest(any(TaskRequest.class)); + verify(mockRMClient, times(0)). + releaseAssignedContainer((ContainerId) any()); + + // allocate tasks + Object mockTask2 = new MockTask("task2"); + Object mockCookie2 = new Object(); + Object mockTask3 = new MockTask("task3"); + Object mockCookie3 = new Object(); + scheduler.allocateTask(mockTask1, mockCapability, hosts, + racks, mockPriority, null, mockCookie1); + drainableAppCallback.drain(); + verify(mockRMClient, times(2)). + addContainerRequest(requestCaptor.capture()); + TaskRequest request1 = requestCaptor.getValue(); + scheduler.allocateTask(mockTask2, mockCapability, hosts, + racks, mockPriority, null, mockCookie2); + drainableAppCallback.drain(); + verify(mockRMClient, times(3)). + addContainerRequest(requestCaptor.capture()); + TaskRequest request2 = requestCaptor.getValue(); + scheduler.allocateTask(mockTask3, mockCapability, hosts, + racks, mockPriority, null, mockCookie3); + drainableAppCallback.drain(); + verify(mockRMClient, times(4)). + addContainerRequest(requestCaptor.capture()); + TaskRequest request3 = requestCaptor.getValue(); + + NodeId host1 = NodeId.newInstance("host1", 1); + NodeId host2 = NodeId.newInstance("host2", 2); + NodeId host3 = NodeId.newInstance("host3", 3); + NodeId host4 = NodeId.newInstance("host4", 4); + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 1), 1); + ContainerId mockCId1 = ContainerId.newContainerId(attemptId, 1); + Container mockContainer1 = Container.newInstance(mockCId1, host1, null, mockCapability, mockPriority, null); + ContainerId mockCId2 = ContainerId.newContainerId(attemptId, 2); + Container mockContainer2 = Container.newInstance(mockCId2, host2, null, mockCapability, mockPriority, null); + ContainerId mockCId3 = ContainerId.newContainerId(attemptId, 3); + Container mockContainer3 = Container.newInstance(mockCId3, host3, null, mockCapability, mockPriority, null); + ContainerId mockCId4 = ContainerId.newContainerId(attemptId, 4); + Container mockContainer4 = Container.newInstance(mockCId4, host4, null, mockCapability, mockPriority, null); + List containers = new ArrayList<>(); + containers.add(mockContainer1); + containers.add(mockContainer2); + containers.add(mockContainer3); + containers.add(mockContainer4); + scheduler.onContainersAllocated(containers); + drainableAppCallback.drain(); + // first container allocated + verify(mockApp).taskAllocated(mockTask1, mockCookie1, mockContainer1); + verify(mockApp).taskAllocated(mockTask2, mockCookie2, mockContainer2); + verify(mockApp).taskAllocated(mockTask3, mockCookie3, mockContainer3); + // no other allocations returned + verify(mockApp, times(3)).taskAllocated(any(), any(), (Container) any()); + verify(mockRMClient).removeContainerRequest(request1); + verify(mockRMClient).removeContainerRequest(request2); + verify(mockRMClient).removeContainerRequest(request3); + // verify unwanted container released + verify(mockRMClient).releaseAssignedContainer(mockCId4); + + // deallocate allocated task + assertTrue(scheduler.deallocateTask(mockTask1, true, null, null)); + drainableAppCallback.drain(); + verify(mockApp).containerBeingReleased(mockCId1); + verify(mockRMClient).releaseAssignedContainer(mockCId1); + // deallocate allocated container + assertEquals(mockTask2, scheduler.deallocateContainer(mockCId2)); + drainableAppCallback.drain(); + verify(mockRMClient).releaseAssignedContainer(mockCId2); + verify(mockRMClient, times(3)).releaseAssignedContainer((ContainerId) any()); + + List statuses = new ArrayList<>(); + ContainerStatus mockStatus1 = mock(ContainerStatus.class); + when(mockStatus1.getContainerId()).thenReturn(mockCId1); + statuses.add(mockStatus1); + ContainerStatus mockStatus2 = mock(ContainerStatus.class); + when(mockStatus2.getContainerId()).thenReturn(mockCId2); + statuses.add(mockStatus2); + ContainerStatus mockStatus3 = mock(ContainerStatus.class); + when(mockStatus3.getContainerId()).thenReturn(mockCId3); + statuses.add(mockStatus3); + ContainerStatus mockStatus4 = mock(ContainerStatus.class); + when(mockStatus4.getContainerId()).thenReturn(mockCId4); + statuses.add(mockStatus4); + + scheduler.onContainersCompleted(statuses); + drainableAppCallback.drain(); + // released container status returned + verify(mockApp).containerCompleted(mockTask1, mockStatus1); + verify(mockApp).containerCompleted(mockTask2, mockStatus2); + // currently allocated container status returned and not released + verify(mockApp).containerCompleted(mockTask3, mockStatus3); + // no other statuses returned + verify(mockApp, times(3)).containerCompleted(any(), (ContainerStatus) any()); + verify(mockRMClient, times(3)).releaseAssignedContainer((ContainerId) any()); + + // verify blacklisting + verify(mockRMClient, times(0)).updateBlacklist(anyListOf(String.class), anyListOf(String.class)); + String badHost = "host6"; + NodeId badNodeId = NodeId.newInstance(badHost, 1); + scheduler.blacklistNode(badNodeId); + List badNodeList = Collections.singletonList(badHost); + verify(mockRMClient, times(1)).updateBlacklist(eq(badNodeList), isNull(List.class)); + Object mockTask4 = new MockTask("task4"); + Object mockCookie4 = new Object(); + scheduler.allocateTask(mockTask4, mockCapability, null, + null, mockPriority, null, mockCookie4); + drainableAppCallback.drain(); + verify(mockRMClient, times(5)).addContainerRequest(requestCaptor.capture()); + ContainerId mockCId5 = ContainerId.newContainerId(attemptId, 5); + Container mockContainer5 = Container.newInstance(mockCId5, badNodeId, null, mockCapability, mockPriority, null); + containers.clear(); + containers.add(mockContainer5); + scheduler.onContainersAllocated(containers); + drainableAppCallback.drain(); + // no new allocation + verify(mockApp, times(3)).taskAllocated(any(), any(), (Container) any()); + // verify blacklisted container released + verify(mockRMClient).releaseAssignedContainer(mockCId5); + verify(mockRMClient, times(4)).releaseAssignedContainer((ContainerId) any()); + // verify request added back + verify(mockRMClient, times(6)).addContainerRequest(requestCaptor.capture()); + NodeId host6 = NodeId.newInstance("host6", 6); + ContainerId mockCId6 = ContainerId.newContainerId(attemptId, 6); + Container mockContainer6 = Container.newInstance(mockCId6, host6, null, mockCapability, mockPriority, null); + containers.clear(); + containers.add(mockContainer6); + scheduler.onContainersAllocated(containers); + drainableAppCallback.drain(); + // new allocation + verify(mockApp, times(4)).taskAllocated(any(), any(), (Container) any()); + verify(mockApp).taskAllocated(mockTask4, mockCookie4, mockContainer6); + // deallocate allocated task + assertTrue(scheduler.deallocateTask(mockTask4, true, null, null)); + drainableAppCallback.drain(); + verify(mockApp).containerBeingReleased(mockCId6); + verify(mockRMClient).releaseAssignedContainer(mockCId6); + verify(mockRMClient, times(5)).releaseAssignedContainer((ContainerId) any()); + // test unblacklist + scheduler.unblacklistNode(badNodeId); + verify(mockRMClient, times(1)).updateBlacklist(isNull(List.class), eq(badNodeList)); + assertEquals(0, scheduler.getNumBlacklistedNodes()); + + float progress = 0.5f; + when(mockApp.getProgress()).thenReturn(progress); + assertEquals(progress, scheduler.getProgress(), 0); + + // check duplicate allocation request + scheduler.allocateTask(mockTask1, mockCapability, hosts, racks, + mockPriority, null, mockCookie1); + drainableAppCallback.drain(); + verify(mockRMClient, times(7)).addContainerRequest(any(TaskRequest.class)); + verify(mockRMClient, times(6)). + removeContainerRequest(any(TaskRequest.class)); + scheduler.allocateTask(mockTask1, mockCapability, hosts, racks, + mockPriority, null, mockCookie1); + drainableAppCallback.drain(); + // old request removed and new one added + verify(mockRMClient, times(7)). + removeContainerRequest(any(TaskRequest.class)); + verify(mockRMClient, times(8)).addContainerRequest(any(TaskRequest.class)); + assertFalse(scheduler.deallocateTask(mockTask1, true, null, null)); + + List mockUpdatedNodes = mock(List.class); + scheduler.onNodesUpdated(mockUpdatedNodes); + drainableAppCallback.drain(); + verify(mockApp).nodesUpdated(mockUpdatedNodes); + + ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(String.class); + Exception mockException = new IOException("mockexception"); + scheduler.onError(mockException); + drainableAppCallback.drain(); + verify(mockApp) + .reportError(eq(YarnTaskSchedulerServiceError.RESOURCEMANAGER_ERROR), argumentCaptor.capture(), + any(DagInfo.class)); + assertTrue(argumentCaptor.getValue().contains("mockexception")); + + scheduler.onShutdownRequest(); + drainableAppCallback.drain(); + verify(mockApp).appShutdownRequested(); + + String appMsg = "success"; + AppFinalStatus finalStatus = + new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl); + when(mockApp.getFinalAppStatus()).thenReturn(finalStatus); + scheduler.shutdown(); + drainableAppCallback.drain(); + verify(mockRMClient). + unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, + appMsg, appUrl); + verify(mockRMClient).stop(); + } + + @Test(timeout=30000) + public void testSimpleReuseLocalMatching() throws Exception { + AMRMClientAsyncWrapperForTest mockRMClient = spy(new AMRMClientAsyncWrapperForTest()); + + String appHost = "host"; + int appPort = 0; + String appUrl = "url"; + + Configuration conf = new Configuration(); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, true); + conf.setInt(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 100); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, false); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED, false); + conf.setInt(TezConfiguration.TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX, 100); + + DagInfo mockDagInfo = mock(DagInfo.class); + when(mockDagInfo.getTotalVertices()).thenReturn(10); + when(mockDagInfo.getVertexDescendants(anyInt())).thenReturn(new BitSet()); + TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf); + when(mockApp.getCurrentDagInfo()).thenReturn(mockDagInfo); + TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp); + + MockClock clock = new MockClock(1000); + NewTaskSchedulerForTest scheduler = new NewTaskSchedulerForTest(drainableAppCallback, + mockRMClient, clock); + + scheduler.initialize(); + drainableAppCallback.drain(); + + scheduler.start(); + drainableAppCallback.drain(); + verify(mockRMClient).start(); + verify(mockRMClient).registerApplicationMaster(appHost, appPort, appUrl); + RegisterApplicationMasterResponse regResponse = mockRMClient.getRegistrationResponse(); + verify(mockApp).setApplicationRegistrationData(regResponse.getMaximumResourceCapability(), + regResponse.getApplicationACLs(), regResponse.getClientToAMTokenMasterKey(), + regResponse.getQueue()); + + assertEquals(scheduler.getClusterNodeCount(), mockRMClient.getClusterNodeCount()); + + Priority priorityv0 = Priority.newInstance(1); + Priority priorityv1 = Priority.newInstance(2); + String[] hostsv0t0 = { "host1", "host2" }; + MockTaskInfo taskv0t0 = new MockTaskInfo("taskv0t0", priorityv0, hostsv0t0); + MockTaskInfo taskv0t1 = new MockTaskInfo("taskv0t1", priorityv0, "host3"); + MockTaskInfo taskv0t2 = new MockTaskInfo("taskv0t2", priorityv0, hostsv0t0); + MockTaskInfo taskv1t0 = new MockTaskInfo("taskv1t0", priorityv1, hostsv0t0); + MockTaskInfo taskv1t1 = new MockTaskInfo("taskv1t1", priorityv1, hostsv0t0); + + TaskRequestCaptor taskRequestCaptor = new TaskRequestCaptor(mockRMClient, + scheduler, drainableAppCallback); + TaskRequest reqv0t0 = taskRequestCaptor.scheduleTask(taskv0t0); + taskRequestCaptor.scheduleTask(taskv0t1); + TaskRequest reqv0t2 = taskRequestCaptor.scheduleTask(taskv0t2); + TaskRequest reqv1t0 = taskRequestCaptor.scheduleTask(taskv1t0); + taskRequestCaptor.scheduleTask(taskv1t1); + + NodeId host1 = NodeId.newInstance("host1", 1); + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 1), 1); + ContainerId cid1 = ContainerId.newContainerId(attemptId, 1); + Container container1 = Container.newInstance(cid1, host1, null, taskv0t0.capability, priorityv0, null); + + // allocate one container at v0 priority + scheduler.onContainersAllocated(Collections.singletonList(container1)); + drainableAppCallback.drain(); + verify(mockApp).taskAllocated(taskv0t0.task, taskv0t0.cookie, container1); + verify(mockRMClient).removeContainerRequest(reqv0t0); + + // finish v0t0 successfully, verify v0t1 is skipped and v0t2 instead is assigned to the container + assertTrue(scheduler.deallocateTask(taskv0t0.task, true, null, null)); + clock.incrementTime(10000); + drainableAppCallback.drain(); + verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); + verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp).taskAllocated(taskv0t2.task, taskv0t2.cookie, container1); + verify(mockRMClient).removeContainerRequest(reqv0t2); + + // finish v0t2 successfully, verify v1t0 is assigned to the same container + assertTrue(scheduler.deallocateTask(taskv0t2.task, true, null, null)); + clock.incrementTime(10000); + drainableAppCallback.drain(); + verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); + verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp).taskAllocated(taskv1t0.task, taskv1t0.cookie, container1); + verify(mockRMClient).removeContainerRequest(reqv1t0); + + // fail v1t0 and verify container is released instead of reused for v1t1 + assertTrue(scheduler.deallocateTask(taskv1t0.task, false, null, null)); + clock.incrementTime(10000); + drainableAppCallback.drain(); + verify(mockApp).containerBeingReleased(cid1); + verify(mockRMClient).releaseAssignedContainer(cid1); + + String appMsg = "success"; + AppFinalStatus finalStatus = + new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl); + when(mockApp.getFinalAppStatus()).thenReturn(finalStatus); + scheduler.shutdown(); + drainableAppCallback.drain(); + verify(mockRMClient). + unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, + appMsg, appUrl); + verify(mockRMClient).stop(); + } + + @Test(timeout=30000) + public void testSimpleReuseRackMatching() throws Exception { + AMRMClientAsyncWrapperForTest mockRMClient = spy(new AMRMClientAsyncWrapperForTest()); + + String appHost = "host"; + int appPort = 0; + String appUrl = "url"; + + Configuration conf = new Configuration(); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, true); + conf.setInt(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 100); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, true); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED, false); + conf.setInt(TezConfiguration.TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX, 100); + + DagInfo mockDagInfo = mock(DagInfo.class); + when(mockDagInfo.getTotalVertices()).thenReturn(10); + when(mockDagInfo.getVertexDescendants(anyInt())).thenReturn(new BitSet()); + TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf); + when(mockApp.getCurrentDagInfo()).thenReturn(mockDagInfo); + TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp); + + MockClock clock = new MockClock(1000); + NewTaskSchedulerForTest scheduler = new NewTaskSchedulerForTest(drainableAppCallback, + mockRMClient, clock); + + scheduler.initialize(); + drainableAppCallback.drain(); + + scheduler.start(); + drainableAppCallback.drain(); + verify(mockRMClient).start(); + verify(mockRMClient).registerApplicationMaster(appHost, appPort, appUrl); + RegisterApplicationMasterResponse regResponse = mockRMClient.getRegistrationResponse(); + verify(mockApp).setApplicationRegistrationData(regResponse.getMaximumResourceCapability(), + regResponse.getApplicationACLs(), regResponse.getClientToAMTokenMasterKey(), + regResponse.getQueue()); + + assertEquals(scheduler.getClusterNodeCount(), mockRMClient.getClusterNodeCount()); + + Priority priorityv0 = Priority.newInstance(1); + Priority priorityv1 = Priority.newInstance(2); + String[] hostsv0t0 = { "host1", "host2" }; + MockTaskInfo taskv0t0 = new MockTaskInfo("taskv0t0", priorityv0, hostsv0t0); + MockTaskInfo taskv0t1 = new MockTaskInfo("taskv0t1", priorityv0, "host2"); + MockTaskInfo taskv0t2 = new MockTaskInfo("taskv0t2", priorityv0, "host4", "/somerack"); + MockTaskInfo taskv1t0 = new MockTaskInfo("taskv1t0", priorityv1, "host1"); + MockTaskInfo taskv1t1 = new MockTaskInfo("taskv1t1", priorityv1, "host5"); + + TaskRequestCaptor taskRequestCaptor = new TaskRequestCaptor(mockRMClient, + scheduler, drainableAppCallback); + TaskRequest reqv0t0 = taskRequestCaptor.scheduleTask(taskv0t0); + TaskRequest reqv0t1 = taskRequestCaptor.scheduleTask(taskv0t1); + taskRequestCaptor.scheduleTask(taskv0t2); + TaskRequest reqv1t0 = taskRequestCaptor.scheduleTask(taskv1t0); + taskRequestCaptor.scheduleTask(taskv1t1); + + NodeId host1 = NodeId.newInstance("host1", 1); + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 1), 1); + ContainerId cid1 = ContainerId.newContainerId(attemptId, 1); + Container container1 = Container.newInstance(cid1, host1, null, taskv0t0.capability, priorityv0, null); + + // allocate one container at v0 priority + scheduler.onContainersAllocated(Collections.singletonList(container1)); + drainableAppCallback.drain(); + verify(mockApp).taskAllocated(taskv0t0.task, taskv0t0.cookie, container1); + verify(mockRMClient).removeContainerRequest(reqv0t0); + + // finish v0t0 successfully, verify v0t1 is skipped and v1t0 assigned instead + // since host locality is preferred to rack locality and lower priority vertex + // is not blocked by higher priority vertex + assertTrue(scheduler.deallocateTask(taskv0t0.task, true, null, null)); + clock.incrementTime(10000); + drainableAppCallback.drain(); + verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); + verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp).taskAllocated(taskv1t0.task, taskv1t0.cookie, container1); + verify(mockRMClient).removeContainerRequest(reqv1t0); + + // finish v1t0 successfully, verify v0t1 is assigned + assertTrue(scheduler.deallocateTask(taskv1t0.task, true, null, null)); + clock.incrementTime(10000); + drainableAppCallback.drain(); + verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); + verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp).taskAllocated(taskv0t1.task, taskv0t1.cookie, container1); + verify(mockRMClient).removeContainerRequest(reqv0t1); + + // fail v0t1 and verify container is released instead of reused for v1t1 + assertTrue(scheduler.deallocateTask(taskv0t1.task, false, null, null)); + clock.incrementTime(10000); + drainableAppCallback.drain(); + verify(mockApp).containerBeingReleased(cid1); + verify(mockRMClient).releaseAssignedContainer(cid1); + + String appMsg = "success"; + AppFinalStatus finalStatus = + new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl); + when(mockApp.getFinalAppStatus()).thenReturn(finalStatus); + scheduler.shutdown(); + drainableAppCallback.drain(); + verify(mockRMClient). + unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, + appMsg, appUrl); + verify(mockRMClient).stop(); + } + + @Test(timeout=30000) + public void testSimpleReuseAnyMatching() throws Exception { + AMRMClientAsyncWrapperForTest mockRMClient = spy(new AMRMClientAsyncWrapperForTest()); + + String appHost = "host"; + int appPort = 0; + String appUrl = "url"; + + Configuration conf = new Configuration(); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, true); + conf.setInt(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 100); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, true); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED, true); + conf.setInt(TezConfiguration.TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX, 100); + + DagInfo mockDagInfo = mock(DagInfo.class); + when(mockDagInfo.getTotalVertices()).thenReturn(10); + when(mockDagInfo.getVertexDescendants(anyInt())).thenReturn(new BitSet()); + TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf); + when(mockApp.getCurrentDagInfo()).thenReturn(mockDagInfo); + TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp); + + MockClock clock = new MockClock(1000); + NewTaskSchedulerForTest scheduler = new NewTaskSchedulerForTest(drainableAppCallback, + mockRMClient, clock); + + scheduler.initialize(); + drainableAppCallback.drain(); + + scheduler.start(); + drainableAppCallback.drain(); + verify(mockRMClient).start(); + verify(mockRMClient).registerApplicationMaster(appHost, appPort, appUrl); + RegisterApplicationMasterResponse regResponse = mockRMClient.getRegistrationResponse(); + verify(mockApp).setApplicationRegistrationData(regResponse.getMaximumResourceCapability(), + regResponse.getApplicationACLs(), regResponse.getClientToAMTokenMasterKey(), + regResponse.getQueue()); + + assertEquals(scheduler.getClusterNodeCount(), mockRMClient.getClusterNodeCount()); + + Priority priorityv0 = Priority.newInstance(1); + Priority priorityv1 = Priority.newInstance(2); + String[] hostsv0t0 = { "host1", "host2" }; + MockTaskInfo taskv0t0 = new MockTaskInfo("taskv0t0", priorityv0, hostsv0t0); + MockTaskInfo taskv0t1 = new MockTaskInfo("taskv0t1", priorityv0, "host2"); + MockTaskInfo taskv0t2 = new MockTaskInfo("taskv0t2", priorityv0, "host4", "/rack4"); + MockTaskInfo taskv1t0 = new MockTaskInfo("taskv1t0", priorityv1, "host1"); + MockTaskInfo taskv1t1 = new MockTaskInfo("taskv1t1", priorityv1, "host6", "/rack6"); + + TaskRequestCaptor taskRequestCaptor = new TaskRequestCaptor(mockRMClient, + scheduler, drainableAppCallback); + TaskRequest reqv0t0 = taskRequestCaptor.scheduleTask(taskv0t0); + TaskRequest reqv0t1 = taskRequestCaptor.scheduleTask(taskv0t1); + TaskRequest reqv0t2 = taskRequestCaptor.scheduleTask(taskv0t2); + TaskRequest reqv1t0 = taskRequestCaptor.scheduleTask(taskv1t0); + taskRequestCaptor.scheduleTask(taskv1t1); + + NodeId host1 = NodeId.newInstance("host1", 1); + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 1), 1); + ContainerId cid1 = ContainerId.newContainerId(attemptId, 1); + Container container1 = Container.newInstance(cid1, host1, null, taskv0t0.capability, priorityv0, null); + + // allocate one container at v0 priority + scheduler.onContainersAllocated(Collections.singletonList(container1)); + drainableAppCallback.drain(); + verify(mockApp).taskAllocated(taskv0t0.task, taskv0t0.cookie, container1); + verify(mockRMClient).removeContainerRequest(reqv0t0); + + // finish v0t0 successfully, verify v0t1 is skipped and v1t0 assigned instead + // since host locality is preferred to rack locality and lower priority vertex + // is not blocked by higher priority vertex + assertTrue(scheduler.deallocateTask(taskv0t0.task, true, null, null)); + clock.incrementTime(10000); + drainableAppCallback.drain(); + verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); + verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp).taskAllocated(taskv1t0.task, taskv1t0.cookie, container1); + verify(mockRMClient).removeContainerRequest(reqv1t0); + + // finish v1t0 successfully, verify v0t1 is assigned + assertTrue(scheduler.deallocateTask(taskv1t0.task, true, null, null)); + clock.incrementTime(10000); + drainableAppCallback.drain(); + verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); + verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp).taskAllocated(taskv0t1.task, taskv0t1.cookie, container1); + verify(mockRMClient).removeContainerRequest(reqv0t1); + + // finish v0t1 successfully, verify v0t2 is assigned + assertTrue(scheduler.deallocateTask(taskv0t1.task, true, null, null)); + clock.incrementTime(10000); + drainableAppCallback.drain(); + verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); + verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp).taskAllocated(taskv0t2.task, taskv0t2.cookie, container1); + verify(mockRMClient).removeContainerRequest(reqv0t2); + + // fail v0t2 and verify container is released instead of reused for v1t1 + assertTrue(scheduler.deallocateTask(taskv0t2.task, false, null, null)); + clock.incrementTime(10000); + drainableAppCallback.drain(); + verify(mockApp).containerBeingReleased(cid1); + verify(mockRMClient).releaseAssignedContainer(cid1); + + String appMsg = "success"; + AppFinalStatus finalStatus = + new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl); + when(mockApp.getFinalAppStatus()).thenReturn(finalStatus); + scheduler.shutdown(); + drainableAppCallback.drain(); + verify(mockRMClient). + unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, + appMsg, appUrl); + verify(mockRMClient).stop(); + } + + @Test(timeout=30000) + public void testReuseWithAffinity() throws Exception { + AMRMClientAsyncWrapperForTest mockRMClient = spy(new AMRMClientAsyncWrapperForTest()); + + String appHost = "host"; + int appPort = 0; + String appUrl = "url"; + + Configuration conf = new Configuration(); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, true); + conf.setInt(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 100); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, true); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED, false); + conf.setInt(TezConfiguration.TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX, 100); + + DagInfo mockDagInfo = mock(DagInfo.class); + when(mockDagInfo.getTotalVertices()).thenReturn(10); + when(mockDagInfo.getVertexDescendants(anyInt())).thenReturn(new BitSet()); + TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf); + when(mockApp.getCurrentDagInfo()).thenReturn(mockDagInfo); + TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp); + + MockClock clock = new MockClock(1000); + NewTaskSchedulerForTest scheduler = new NewTaskSchedulerForTest(drainableAppCallback, + mockRMClient, clock); + + scheduler.initialize(); + drainableAppCallback.drain(); + + scheduler.start(); + drainableAppCallback.drain(); + verify(mockRMClient).start(); + verify(mockRMClient).registerApplicationMaster(appHost, appPort, appUrl); + RegisterApplicationMasterResponse regResponse = mockRMClient.getRegistrationResponse(); + verify(mockApp).setApplicationRegistrationData(regResponse.getMaximumResourceCapability(), + regResponse.getApplicationACLs(), regResponse.getClientToAMTokenMasterKey(), + regResponse.getQueue()); + + assertEquals(scheduler.getClusterNodeCount(), mockRMClient.getClusterNodeCount()); + + Priority priorityv0 = Priority.newInstance(1); + Priority priorityv1 = Priority.newInstance(2); + String[] hostsv0t0 = { "host1", "host2" }; + MockTaskInfo taskv0t0 = new MockTaskInfo("taskv0t0", priorityv0, hostsv0t0); + MockTaskInfo taskv0t1 = new MockTaskInfo("taskv0t1", priorityv0, hostsv0t0); + + TaskRequestCaptor taskRequestCaptor = new TaskRequestCaptor(mockRMClient, + scheduler, drainableAppCallback); + TaskRequest reqv0t0 = taskRequestCaptor.scheduleTask(taskv0t0); + taskRequestCaptor.scheduleTask(taskv0t1); + + NodeId host1 = NodeId.newInstance("host1", 1); + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 1), 1); + ContainerId cid1 = ContainerId.newContainerId(attemptId, 1); + Container container1 = Container.newInstance(cid1, host1, null, taskv0t0.capability, priorityv0, null); + + // allocate one container at v0 priority + scheduler.onContainersAllocated(Collections.singletonList(container1)); + drainableAppCallback.drain(); + verify(mockApp).taskAllocated(taskv0t0.task, taskv0t0.cookie, container1); + verify(mockRMClient).removeContainerRequest(reqv0t0); + + // add a new request for this container + MockTaskInfo taskv1t0 = new MockTaskInfo("taskv1t0", priorityv1, "host1"); + TaskRequest reqv1t0 = taskRequestCaptor.scheduleTask(taskv1t0, cid1); + + // finish v0t0 successfully, verify v0t1 is skipped even though it is node-local + // and v1t0 assigned instead for affinity + assertTrue(scheduler.deallocateTask(taskv0t0.task, true, null, null)); + clock.incrementTime(10000); + drainableAppCallback.drain(); + verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); + verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp).taskAllocated(taskv1t0.task, taskv1t0.cookie, container1); + verify(mockRMClient).removeContainerRequest(reqv1t0); + + String appMsg = "success"; + AppFinalStatus finalStatus = + new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl); + when(mockApp.getFinalAppStatus()).thenReturn(finalStatus); + scheduler.shutdown(); + drainableAppCallback.drain(); + verify(mockRMClient). + unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, + appMsg, appUrl); + verify(mockRMClient).stop(); + } + + @Test(timeout=30000) + public void testReuseVertexDescendants() throws Exception { + AMRMClientAsyncWrapperForTest mockRMClient = spy(new AMRMClientAsyncWrapperForTest()); + + String appHost = "host"; + int appPort = 0; + String appUrl = "url"; + + Configuration conf = new Configuration(); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, true); + conf.setInt(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 100); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, true); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED, false); + conf.setInt(TezConfiguration.TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX, 100); + + // vertex 0 and vertex 2 are root vertices and vertex 1 is a child of vertex 0 + DagInfo mockDagInfo = mock(DagInfo.class); + when(mockDagInfo.getTotalVertices()).thenReturn(3); + when(mockDagInfo.getVertexDescendants(0)).thenReturn(BitSet.valueOf(new long[] { 0x2 })); + when(mockDagInfo.getVertexDescendants(1)).thenReturn(new BitSet()); + when(mockDagInfo.getVertexDescendants(2)).thenReturn(new BitSet()); + TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf); + when(mockApp.getCurrentDagInfo()).thenReturn(mockDagInfo); + TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp); + + MockClock clock = new MockClock(1000); + NewTaskSchedulerForTest scheduler = new NewTaskSchedulerForTest(drainableAppCallback, + mockRMClient, clock); + + scheduler.initialize(); + drainableAppCallback.drain(); + + scheduler.start(); + drainableAppCallback.drain(); + verify(mockRMClient).start(); + verify(mockRMClient).registerApplicationMaster(appHost, appPort, appUrl); + RegisterApplicationMasterResponse regResponse = mockRMClient.getRegistrationResponse(); + verify(mockApp).setApplicationRegistrationData(regResponse.getMaximumResourceCapability(), + regResponse.getApplicationACLs(), regResponse.getClientToAMTokenMasterKey(), + regResponse.getQueue()); + + assertEquals(scheduler.getClusterNodeCount(), mockRMClient.getClusterNodeCount()); + + Priority priorityv0 = Priority.newInstance(1); + Priority priorityv1 = Priority.newInstance(2); + Priority priorityv2 = Priority.newInstance(3); + String[] hostsv0t0 = { "host1", "host2" }; + MockTaskInfo taskv0t0 = new MockTaskInfo("taskv0t0", priorityv0, hostsv0t0); + when(mockApp.getVertexIndexForTask(taskv0t0.task)).thenReturn(0); + MockTaskInfo taskv0t1 = new MockTaskInfo("taskv0t1", priorityv0, "host3"); + when(mockApp.getVertexIndexForTask(taskv0t1.task)).thenReturn(0); + MockTaskInfo taskv1t0 = new MockTaskInfo("taskv1t0", priorityv1, hostsv0t0); + when(mockApp.getVertexIndexForTask(taskv1t0.task)).thenReturn(1); + MockTaskInfo taskv2t0 = new MockTaskInfo("taskv2t0", priorityv2, hostsv0t0); + when(mockApp.getVertexIndexForTask(taskv2t0.task)).thenReturn(2); + MockTaskInfo taskv2t1 = new MockTaskInfo("taskv2t1", priorityv2, "host3"); + when(mockApp.getVertexIndexForTask(taskv2t1.task)).thenReturn(2); + + TaskRequestCaptor taskRequestCaptor = new TaskRequestCaptor(mockRMClient, + scheduler, drainableAppCallback); + TaskRequest reqv0t0 = taskRequestCaptor.scheduleTask(taskv0t0); + TaskRequest reqv0t1 = taskRequestCaptor.scheduleTask(taskv0t1); + TaskRequest reqv1t0 = taskRequestCaptor.scheduleTask(taskv1t0); + TaskRequest reqv2t0 = taskRequestCaptor.scheduleTask(taskv2t0); + taskRequestCaptor.scheduleTask(taskv2t1); + + NodeId host1 = NodeId.newInstance("host1", 1); + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 1), 1); + ContainerId cid1 = ContainerId.newContainerId(attemptId, 1); + Container container1 = Container.newInstance(cid1, host1, null, taskv0t0.capability, priorityv0, null); + + // allocate one container at v0 priority + scheduler.onContainersAllocated(Collections.singletonList(container1)); + drainableAppCallback.drain(); + verify(mockApp).taskAllocated(taskv0t0.task, taskv0t0.cookie, container1); + verify(mockRMClient).removeContainerRequest(reqv0t0); + + // finish v0t0 successfully, verify v1t0 is skipped and v2t0 assigned instead + // since host locality beats rack locality for unblocked vertex v2 and + // v1 is blocked by pending v0 request + assertTrue(scheduler.deallocateTask(taskv0t0.task, true, null, null)); + clock.incrementTime(10000); + drainableAppCallback.drain(); + verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); + verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp).taskAllocated(taskv2t0.task, taskv2t0.cookie, container1); + verify(mockRMClient).removeContainerRequest(reqv2t0); + + // finish v2t0 successfully, verify v0t1 is assigned since it is higher + // priority than v2 + assertTrue(scheduler.deallocateTask(taskv2t0.task, true, null, null)); + clock.incrementTime(10000); + drainableAppCallback.drain(); + verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); + verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp).taskAllocated(taskv0t1.task, taskv0t1.cookie, container1); + verify(mockRMClient).removeContainerRequest(reqv0t1); + + // finish v2t0 successfully, verify v1t0 is assigned since it is now unblocked + assertTrue(scheduler.deallocateTask(taskv0t1.task, true, null, null)); + clock.incrementTime(10000); + drainableAppCallback.drain(); + verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); + verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp).taskAllocated(taskv1t0.task, taskv1t0.cookie, container1); + verify(mockRMClient).removeContainerRequest(reqv1t0); + + // fail v1t0 and verify container is released instead of reused for v2t1 + assertTrue(scheduler.deallocateTask(taskv1t0.task, false, null, null)); + clock.incrementTime(10000); + drainableAppCallback.drain(); + verify(mockApp).containerBeingReleased(cid1); + verify(mockRMClient).releaseAssignedContainer(cid1); + + String appMsg = "success"; + AppFinalStatus finalStatus = + new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl); + when(mockApp.getFinalAppStatus()).thenReturn(finalStatus); + scheduler.shutdown(); + drainableAppCallback.drain(); + verify(mockRMClient). + unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, + appMsg, appUrl); + verify(mockRMClient).stop(); + } + + @Test(timeout=30000) + public void testSessionContainers() throws Exception { + AMRMClientAsyncWrapperForTest mockRMClient = spy(new AMRMClientAsyncWrapperForTest()); + + String appHost = "host"; + int appPort = 0; + String appUrl = "url"; + + Configuration conf = new Configuration(); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, true); + conf.setInt(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 100); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, true); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED, false); + conf.setInt(TezConfiguration.TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX, 100); + conf.setInt(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, 4000); + conf.setInt(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, 5000); + conf.setInt(TezConfiguration.TEZ_AM_SESSION_MIN_HELD_CONTAINERS, 5); + + DagInfo mockDagInfo = mock(DagInfo.class); + when(mockDagInfo.getTotalVertices()).thenReturn(10); + when(mockDagInfo.getVertexDescendants(anyInt())).thenReturn(new BitSet()); + TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf); + when(mockApp.getCurrentDagInfo()).thenReturn(mockDagInfo); + when(mockApp.isSession()).thenReturn(true); + TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp); + + MockClock clock = new MockClock(1000); + NewTaskSchedulerForTest scheduler = new NewTaskSchedulerForTest(drainableAppCallback, + mockRMClient, clock); + + scheduler.initialize(); + drainableAppCallback.drain(); + + scheduler.start(); + drainableAppCallback.drain(); + verify(mockRMClient).start(); + verify(mockRMClient).registerApplicationMaster(appHost, appPort, appUrl); + RegisterApplicationMasterResponse regResponse = mockRMClient.getRegistrationResponse(); + verify(mockApp).setApplicationRegistrationData(regResponse.getMaximumResourceCapability(), + regResponse.getApplicationACLs(), regResponse.getClientToAMTokenMasterKey(), + regResponse.getQueue()); + + assertEquals(scheduler.getClusterNodeCount(), mockRMClient.getClusterNodeCount()); + + final String rack1 = "/r1"; + final String rack2 = "/r2"; + final String rack3 = "/r3"; + final String node1Rack1 = "n1r1"; + final String node2Rack1 = "n2r1"; + final String node1Rack2 = "n1r2"; + final String node2Rack2 = "n2r2"; + final String node1Rack3 = "n1r3"; + MockDNSToSwitchMapping.addRackMapping(node1Rack1, rack1); + MockDNSToSwitchMapping.addRackMapping(node2Rack1, rack1); + MockDNSToSwitchMapping.addRackMapping(node1Rack2, rack2); + MockDNSToSwitchMapping.addRackMapping(node2Rack2, rack2); + MockDNSToSwitchMapping.addRackMapping(node1Rack3, rack3); + + Priority priorityv0 = Priority.newInstance(1); + MockTaskInfo taskv0t0 = new MockTaskInfo("taskv0t0", priorityv0, node1Rack1, rack1); + MockTaskInfo taskv0t1 = new MockTaskInfo("taskv0t1", priorityv0, node2Rack1, rack1); + MockTaskInfo taskv0t2 = new MockTaskInfo("taskv0t2", priorityv0, node1Rack1, rack1); + MockTaskInfo taskv0t3 = new MockTaskInfo("taskv0t3", priorityv0, node2Rack1, rack1); + MockTaskInfo taskv0t4 = new MockTaskInfo("taskv0t4", priorityv0, node1Rack2, rack2); + MockTaskInfo taskv0t5 = new MockTaskInfo("taskv0t5", priorityv0, node2Rack2, rack2); + MockTaskInfo taskv0t6 = new MockTaskInfo("taskv0t6", priorityv0, node1Rack3, rack3); + + TaskRequestCaptor taskRequestCaptor = new TaskRequestCaptor(mockRMClient, + scheduler, drainableAppCallback); + TaskRequest reqv0t0 = taskRequestCaptor.scheduleTask(taskv0t0); + TaskRequest reqv0t1 = taskRequestCaptor.scheduleTask(taskv0t1); + TaskRequest reqv0t2 = taskRequestCaptor.scheduleTask(taskv0t2); + TaskRequest reqv0t3 = taskRequestCaptor.scheduleTask(taskv0t3); + TaskRequest reqv0t4 = taskRequestCaptor.scheduleTask(taskv0t4); + TaskRequest reqv0t5 = taskRequestCaptor.scheduleTask(taskv0t5); + TaskRequest reqv0t6 = taskRequestCaptor.scheduleTask(taskv0t6); + + List containers = new ArrayList<>(); + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 1), 1); + ContainerId cid1 = ContainerId.newContainerId(attemptId, 1); + NodeId n1r1 = NodeId.newInstance(node1Rack1, 1); + Container container1 = Container.newInstance(cid1, n1r1, null, taskv0t0.capability, priorityv0, null); + containers.add(container1); + ContainerId cid2 = ContainerId.newContainerId(attemptId, 2); + NodeId n2r1 = NodeId.newInstance(node2Rack1, 1); + Container container2 = Container.newInstance(cid2, n2r1, null, taskv0t1.capability, priorityv0, null); + containers.add(container2); + ContainerId cid3 = ContainerId.newContainerId(attemptId, 3); + Container container3 = Container.newInstance(cid3, n1r1, null, taskv0t2.capability, priorityv0, null); + containers.add(container3); + ContainerId cid4 = ContainerId.newContainerId(attemptId, 4); + Container container4 = Container.newInstance(cid4, n2r1, null, taskv0t3.capability, priorityv0, null); + containers.add(container4); + ContainerId cid5 = ContainerId.newContainerId(attemptId, 5); + NodeId n1r2 = NodeId.newInstance(node1Rack2, 1); + Container container5 = Container.newInstance(cid5, n1r2, null, taskv0t4.capability, priorityv0, null); + containers.add(container5); + ContainerId cid6 = ContainerId.newContainerId(attemptId, 6); + NodeId n2r2 = NodeId.newInstance(node2Rack2, 1); + Container container6 = Container.newInstance(cid6, n2r2, null, taskv0t5.capability, priorityv0, null); + containers.add(container6); + ContainerId cid7 = ContainerId.newContainerId(attemptId, 7); + NodeId n1r3 = NodeId.newInstance(node1Rack3, 1); + Container container7 = Container.newInstance(cid7, n1r3, null, taskv0t6.capability, priorityv0, null); + containers.add(container7); + + scheduler.onContainersAllocated(containers); + drainableAppCallback.drain(); + verify(mockApp).taskAllocated(taskv0t0.task, taskv0t0.cookie, container1); + verify(mockRMClient).removeContainerRequest(reqv0t0); + verify(mockApp).taskAllocated(taskv0t1.task, taskv0t1.cookie, container2); + verify(mockRMClient).removeContainerRequest(reqv0t1); + verify(mockApp).taskAllocated(taskv0t2.task, taskv0t2.cookie, container3); + verify(mockRMClient).removeContainerRequest(reqv0t2); + verify(mockApp).taskAllocated(taskv0t3.task, taskv0t3.cookie, container4); + verify(mockRMClient).removeContainerRequest(reqv0t3); + verify(mockApp).taskAllocated(taskv0t4.task, taskv0t4.cookie, container5); + verify(mockRMClient).removeContainerRequest(reqv0t4); + verify(mockApp).taskAllocated(taskv0t5.task, taskv0t5.cookie, container6); + verify(mockRMClient).removeContainerRequest(reqv0t5); + verify(mockApp).taskAllocated(taskv0t6.task, taskv0t6.cookie, container7); + verify(mockRMClient).removeContainerRequest(reqv0t6); + + clock.incrementTime(10000); + drainableAppCallback.drain(); + assertTrue(scheduler.deallocateTask(taskv0t0.task, true, null, null)); + assertTrue(scheduler.deallocateTask(taskv0t1.task, true, null, null)); + assertTrue(scheduler.deallocateTask(taskv0t2.task, true, null, null)); + assertTrue(scheduler.deallocateTask(taskv0t3.task, true, null, null)); + assertTrue(scheduler.deallocateTask(taskv0t4.task, true, null, null)); + assertTrue(scheduler.deallocateTask(taskv0t5.task, true, null, null)); + assertTrue(scheduler.deallocateTask(taskv0t6.task, true, null, null)); + verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); + verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + + // verify only two of the containers were released after idle expiration + // and the rest were spread across the nodes and racks + clock.incrementTime(5000); + drainableAppCallback.drain(); + verify(mockApp, times(2)).containerBeingReleased(any(ContainerId.class)); + verify(mockRMClient, times(2)).releaseAssignedContainer(any(ContainerId.class)); + Set hosts = new HashSet<>(); + Set racks = new HashSet<>(); + for (HeldContainer hc : scheduler.getSessionContainers()) { + hosts.add(hc.getHost()); + racks.add(hc.getRack()); + } + assertEquals(5, hosts.size()); + assertEquals(3, racks.size()); + assertTrue(hosts.contains(node1Rack1)); + assertTrue(hosts.contains(node2Rack1)); + assertTrue(hosts.contains(node1Rack2)); + assertTrue(hosts.contains(node2Rack2)); + assertTrue(hosts.contains(node1Rack3)); + assertTrue(racks.contains(rack1)); + assertTrue(racks.contains(rack2)); + assertTrue(racks.contains(rack3)); + + String appMsg = "success"; + AppFinalStatus finalStatus = + new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl); + when(mockApp.getFinalAppStatus()).thenReturn(finalStatus); + scheduler.shutdown(); + drainableAppCallback.drain(); + verify(mockRMClient). + unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, + appMsg, appUrl); + verify(mockRMClient).stop(); + } + + @Test(timeout=50000) + public void testPreemptionNoHeadroom() throws Exception { + AMRMClientAsyncWrapperForTest mockRMClient = spy(new AMRMClientAsyncWrapperForTest()); + + String appHost = "host"; + int appPort = 0; + String appUrl = "url"; + + Configuration conf = new Configuration(); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, true); + conf.setInt(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 100); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, true); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED, false); + conf.setInt(TezConfiguration.TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX, 100); + conf.setInt(TezConfiguration.TEZ_AM_PREEMPTION_PERCENTAGE, 10); + conf.setInt(TezConfiguration.TEZ_AM_PREEMPTION_HEARTBEATS_BETWEEN_PREEMPTIONS, 3); + conf.setInt(TezConfiguration.TEZ_AM_PREEMPTION_MAX_WAIT_TIME_MS, 60 * 1000); + + // vertex 0 and vertex 2 are root vertices and vertex 1 is a child of vertex 0 + DagInfo mockDagInfo = mock(DagInfo.class); + when(mockDagInfo.getTotalVertices()).thenReturn(3); + when(mockDagInfo.getVertexDescendants(0)).thenReturn(BitSet.valueOf(new long[] { 0x2 })); + when(mockDagInfo.getVertexDescendants(1)).thenReturn(new BitSet()); + when(mockDagInfo.getVertexDescendants(2)).thenReturn(new BitSet()); + TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf); + when(mockApp.getCurrentDagInfo()).thenReturn(mockDagInfo); + TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp); + + MockClock clock = new MockClock(1000); + NewTaskSchedulerForTest scheduler = new NewTaskSchedulerForTest(drainableAppCallback, + mockRMClient, clock); + + scheduler.initialize(); + drainableAppCallback.drain(); + + scheduler.start(); + drainableAppCallback.drain(); + verify(mockRMClient).start(); + verify(mockRMClient).registerApplicationMaster(appHost, appPort, appUrl); + RegisterApplicationMasterResponse regResponse = mockRMClient.getRegistrationResponse(); + verify(mockApp).setApplicationRegistrationData(regResponse.getMaximumResourceCapability(), + regResponse.getApplicationACLs(), regResponse.getClientToAMTokenMasterKey(), + regResponse.getQueue()); + + assertEquals(scheduler.getClusterNodeCount(), mockRMClient.getClusterNodeCount()); + + Priority priorityv0 = Priority.newInstance(1); + Priority priorityv1 = Priority.newInstance(2); + Priority priorityv2 = Priority.newInstance(3); + String[] hostsv0t0 = { "host1", "host2" }; + MockTaskInfo taskv0t0 = new MockTaskInfo("taskv0t0", priorityv0, hostsv0t0); + when(mockApp.getVertexIndexForTask(taskv0t0.task)).thenReturn(0); + MockTaskInfo taskv0t1 = new MockTaskInfo("taskv0t1", priorityv0, hostsv0t0); + when(mockApp.getVertexIndexForTask(taskv0t1.task)).thenReturn(0); + MockTaskInfo taskv1t0 = new MockTaskInfo("taskv1t0", priorityv1, hostsv0t0); + when(mockApp.getVertexIndexForTask(taskv1t0.task)).thenReturn(1); + MockTaskInfo taskv1t1 = new MockTaskInfo("taskv1t1", priorityv1, hostsv0t0); + when(mockApp.getVertexIndexForTask(taskv1t1.task)).thenReturn(1); + MockTaskInfo taskv2t0 = new MockTaskInfo("taskv2t0", priorityv2, hostsv0t0); + when(mockApp.getVertexIndexForTask(taskv2t0.task)).thenReturn(2); + + // asks for two tasks for vertex 1 and start running one of them + TaskRequestCaptor taskRequestCaptor = new TaskRequestCaptor(mockRMClient, + scheduler, drainableAppCallback); + TaskRequest reqv1t0 = taskRequestCaptor.scheduleTask(taskv1t0); + TaskRequest reqv1t1 = taskRequestCaptor.scheduleTask(taskv1t1); + NodeId host1 = NodeId.newInstance("host1", 1); + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 1), 1); + ContainerId cid1 = ContainerId.newContainerId(attemptId, 1); + Container container1 = Container.newInstance(cid1, host1, null, taskv1t0.capability, priorityv1, null); + scheduler.onContainersAllocated(Collections.singletonList(container1)); + drainableAppCallback.drain(); + verify(mockApp).taskAllocated(taskv1t0.task, taskv1t0.cookie, container1); + verify(mockRMClient).removeContainerRequest(reqv1t0); + + // start running the other task for vertex 1 a bit later + clock.incrementTime(1000); + ContainerId cid2 = ContainerId.newContainerId(attemptId, 2); + Container container2 = Container.newInstance(cid2, host1, null, taskv1t0.capability, priorityv1, null); + scheduler.onContainersAllocated(Collections.singletonList(container2)); + drainableAppCallback.drain(); + verify(mockApp).taskAllocated(taskv1t1.task, taskv1t1.cookie, container2); + verify(mockRMClient).removeContainerRequest(reqv1t1); + + // add a request for vertex 0 but there is no headroom + when(mockRMClient.getAvailableResources()).thenReturn(Resources.none()); + TaskRequest reqv0t0 = taskRequestCaptor.scheduleTask(taskv0t0); + + // should preempt after enough heartbeats to get past preemption interval + // only the youngest container should be preempted to meet the demand + scheduler.getProgress(); + scheduler.getProgress(); + scheduler.getProgress(); + drainableAppCallback.drain(); + verify(mockApp, times(1)).preemptContainer(any(ContainerId.class)); + verify(mockApp).preemptContainer(cid2); + assertEquals(taskv1t1.task, scheduler.deallocateContainer(cid2)); + drainableAppCallback.drain(); + verify(mockApp).containerBeingReleased(cid2); + verify(mockRMClient).releaseAssignedContainer(cid2); + verify(mockApp, never()).containerBeingReleased(cid1); + verify(mockRMClient, never()).releaseAssignedContainer(cid1); + + // add a request for vertex 2 and allocate another container + clock.incrementTime(1000); + taskRequestCaptor.scheduleTask(taskv2t0); + ContainerId cid3 = ContainerId.newContainerId(attemptId, 3); + Container container3 = Container.newInstance(cid3, host1, null, taskv0t0.capability, priorityv0, null); + scheduler.onContainersAllocated(Collections.singletonList(container3)); + drainableAppCallback.drain(); + verify(mockApp).taskAllocated(taskv0t0.task, taskv0t0.cookie, container3); + verify(mockRMClient).removeContainerRequest(reqv0t0); + + // no more preemptions since v1 is not a descendant of v2 + scheduler.getProgress(); + scheduler.getProgress(); + scheduler.getProgress(); + drainableAppCallback.drain(); + verify(mockApp, times(1)).preemptContainer(any(ContainerId.class)); + + // adding request for v0 should trigger preemption on next heartbeat + taskRequestCaptor.scheduleTask(taskv0t1); + scheduler.getProgress(); + drainableAppCallback.drain(); + verify(mockApp, times(2)).preemptContainer(any(ContainerId.class)); + verify(mockApp).preemptContainer(cid1); + assertEquals(taskv1t0.task, scheduler.deallocateContainer(cid1)); + drainableAppCallback.drain(); + verify(mockApp).containerBeingReleased(cid1); + verify(mockRMClient).releaseAssignedContainer(cid1); + + String appMsg = "success"; + AppFinalStatus finalStatus = + new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl); + when(mockApp.getFinalAppStatus()).thenReturn(finalStatus); + scheduler.shutdown(); + drainableAppCallback.drain(); + verify(mockRMClient). + unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, + appMsg, appUrl); + verify(mockRMClient).stop(); + } + + @Test(timeout=50000) + public void testIdleContainerAssignment() throws Exception { + AMRMClientAsyncWrapperForTest mockRMClient = spy(new AMRMClientAsyncWrapperForTest()); + + String appHost = "host"; + int appPort = 0; + String appUrl = "url"; + + Configuration conf = new Configuration(); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, true); + conf.setInt(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 100); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, false); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED, false); + conf.setInt(TezConfiguration.TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX, 100); + conf.setInt(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, 4000); + conf.setInt(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, 5000); + conf.setInt(TezConfiguration.TEZ_AM_SESSION_MIN_HELD_CONTAINERS, 5); + + DagInfo mockDagInfo = mock(DagInfo.class); + when(mockDagInfo.getTotalVertices()).thenReturn(10); + when(mockDagInfo.getVertexDescendants(anyInt())).thenReturn(new BitSet()); + TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf); + when(mockApp.getCurrentDagInfo()).thenReturn(mockDagInfo); + when(mockApp.isSession()).thenReturn(true); + TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp); + + MockClock clock = new MockClock(1000); + NewTaskSchedulerForTest scheduler = new NewTaskSchedulerForTest(drainableAppCallback, + mockRMClient, clock); + + scheduler.initialize(); + drainableAppCallback.drain(); + + scheduler.start(); + drainableAppCallback.drain(); + verify(mockRMClient).start(); + verify(mockRMClient).registerApplicationMaster(appHost, appPort, appUrl); + RegisterApplicationMasterResponse regResponse = mockRMClient.getRegistrationResponse(); + verify(mockApp).setApplicationRegistrationData(regResponse.getMaximumResourceCapability(), + regResponse.getApplicationACLs(), regResponse.getClientToAMTokenMasterKey(), + regResponse.getQueue()); + + assertEquals(scheduler.getClusterNodeCount(), mockRMClient.getClusterNodeCount()); + + final String rack1 = "/r1"; + final String rack2 = "/r2"; + final String node1Rack1 = "n1r1"; + final String node2Rack1 = "n2r1"; + final String node1Rack2 = "n1r2"; + MockDNSToSwitchMapping.addRackMapping(node1Rack1, rack1); + MockDNSToSwitchMapping.addRackMapping(node2Rack1, rack1); + MockDNSToSwitchMapping.addRackMapping(node1Rack2, rack2); + + Priority priorityv0 = Priority.newInstance(1); + MockTaskInfo taskv0t0 = new MockTaskInfo("taskv0t0", priorityv0, node1Rack1, rack1); + + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 1), 1); + ContainerId cid1 = ContainerId.newContainerId(attemptId, 1); + NodeId n2r1 = NodeId.newInstance(node2Rack1, 1); + Container container1 = Container.newInstance(cid1, n2r1, null, taskv0t0.capability, priorityv0, null); + + // verify idle container is kept for now + scheduler.onContainersAllocated(Collections.singletonList(container1)); + clock.incrementTime(2000); + drainableAppCallback.drain(); + verify(mockApp, never()).containerBeingReleased(cid1); + verify(mockRMClient, never()).releaseAssignedContainer(cid1); + + // verify idle container is released without being assigned to a task because rack-local reuse is + // disabled + TaskRequestCaptor taskRequestCaptor = new TaskRequestCaptor(mockRMClient, + scheduler, drainableAppCallback); + TaskRequest reqv0t0 = taskRequestCaptor.scheduleTask(taskv0t0); + clock.incrementTime(10000); + drainableAppCallback.drain(); + verify(mockApp, never()).taskAllocated(taskv0t0.task, taskv0t0.cookie, container1); + verify(mockRMClient, never()).removeContainerRequest(reqv0t0); + verify(mockApp, never()).containerBeingReleased(cid1); + verify(mockRMClient).releaseAssignedContainer(cid1); + + // cancel the task request + assertFalse(scheduler.deallocateTask(taskv0t0.task, false, null, null)); + + // allocate another container that's node-local + ContainerId cid2 = ContainerId.newContainerId(attemptId, 2); + NodeId n1r1 = NodeId.newInstance(node1Rack1, 1); + Container container2 = Container.newInstance(cid2, n1r1, null, taskv0t0.capability, priorityv0, null); + scheduler.onContainersAllocated(Collections.singletonList(container2)); + clock.incrementTime(2000); + drainableAppCallback.drain(); + verify(mockApp, never()).containerBeingReleased(cid2); + verify(mockRMClient, never()).releaseAssignedContainer(cid2); + + // reschedule the task, verify it's now scheduled without a container request + // since node-local idle container is available + reqv0t0 = taskRequestCaptor.scheduleTask(taskv0t0, false); + verify(mockApp).taskAllocated(taskv0t0.task, taskv0t0.cookie, container2); + verify(mockRMClient).removeContainerRequest(reqv0t0); + } + + static class AMRMClientAsyncWrapperForTest extends AMRMClientAsyncWrapper { + AMRMClientAsyncWrapperForTest() { + super(new MockAMRMClient(), 10000, null); + } + + RegisterApplicationMasterResponse getRegistrationResponse() { + return ((MockAMRMClient) client).getRegistrationResponse(); + } + } + + static class MockAMRMClient extends AMRMClientImpl { + private RegisterApplicationMasterResponse mockRegResponse; + + MockAMRMClient() { + super(); + this.clusterAvailableResources = Resource.newInstance(4000, 4); + this.clusterNodeCount = 5; + } + + @Override + protected void serviceStart() { + } + + @Override + protected void serviceStop() { + } + + @SuppressWarnings("unchecked") + @Override + public RegisterApplicationMasterResponse registerApplicationMaster( + String appHostName, int appHostPort, String appTrackingUrl) { + mockRegResponse = mock(RegisterApplicationMasterResponse.class); + Resource mockMaxResource = Resources.createResource(1024*1024, 1024); + Map mockAcls = Collections.emptyMap(); + when(mockRegResponse.getMaximumResourceCapability()).thenReturn( + mockMaxResource); + when(mockRegResponse.getApplicationACLs()).thenReturn(mockAcls); + when(mockRegResponse.getSchedulerResourceTypes()).thenReturn( + EnumSet.of(SchedulerResourceTypes.MEMORY, SchedulerResourceTypes.CPU)); + return mockRegResponse; + } + + @Override + public void unregisterApplicationMaster(FinalApplicationStatus appStatus, + String appMessage, String appTrackingUrl) { + } + + RegisterApplicationMasterResponse getRegistrationResponse() { + return mockRegResponse; + } + } + + static class MockTask { + final String name; + + MockTask(String name) { + this.name = name; + } + + @Override + public String toString() { + return name; + } + } + + static class MockTaskInfo { + final static Object DEFAULT_SIGNATURE = new Object(); + + final MockTask task; + final Object cookie = new Object(); + final Object signature = DEFAULT_SIGNATURE; + final String[] hosts; + final String[] racks; + final Priority priority; + final Resource capability; + + MockTaskInfo(String name, Priority priority, String host) { + this(name, priority, host == null ? null : new String[] { host }); + } + + MockTaskInfo(String name, Priority priority, String[] hosts) { + this(name, priority, hosts, buildDefaultRacks(hosts)); + } + + MockTaskInfo(String name, Priority priority, String host, String rack) { + this(name, priority, host == null ? null : new String[] { host }, + rack == null ? null : new String[] { rack }); + } + + MockTaskInfo(String name, Priority priority, String[] hosts, String[] racks) { + this.task = new MockTask(name); + this.hosts = hosts; + this.racks = racks; + this.priority = priority; + this.capability = Resource.newInstance(1024, 1); + } + + static String[] buildDefaultRacks(String[] hosts) { + if (hosts == null) { + return null; + } + String[] racks = new String[hosts.length]; + Arrays.fill(racks, "/default-rack"); + return racks; + } + } + + static class TaskRequestCaptor { + final AMRMClientAsync client; + final TaskScheduler scheduler; + final TaskSchedulerContextDrainable drainableAppCallback; + final ArgumentCaptor captor = ArgumentCaptor.forClass(TaskRequest.class); + int invocationCount = 0; + + TaskRequestCaptor(AMRMClientAsync client, TaskScheduler scheduler, + TaskSchedulerContextDrainable drainableAppCallback) { + this.client = client; + this.scheduler = scheduler; + this.drainableAppCallback = drainableAppCallback; + } + + TaskRequest scheduleTask(MockTaskInfo taskInfo) throws Exception { + return scheduleTask(taskInfo, true); + } + + TaskRequest scheduleTask(MockTaskInfo taskInfo, boolean expectContainerRequest) throws Exception { + scheduler.allocateTask(taskInfo.task, taskInfo.capability, taskInfo.hosts, taskInfo.racks, + taskInfo.priority, taskInfo.signature, taskInfo.cookie); + drainableAppCallback.drain(); + if (expectContainerRequest) { + ++invocationCount; + } + verify(client, times(invocationCount)).addContainerRequest(captor.capture()); + TaskRequest request = captor.getValue(); + assertEquals(request.getTask(), taskInfo.task); + assertEquals(request.getCookie(), taskInfo.cookie); + return request; + } + + TaskRequest scheduleTask(MockTaskInfo taskInfo, ContainerId affinity) throws Exception { + scheduler.allocateTask(taskInfo.task, taskInfo.capability, affinity, taskInfo.priority, + taskInfo.signature, taskInfo.cookie); + drainableAppCallback.drain(); + verify(client, times(++invocationCount)).addContainerRequest(captor.capture()); + TaskRequest request = captor.getValue(); + assertEquals(request.getTask(), taskInfo.task); + assertEquals(request.getCookie(), taskInfo.cookie); + return request; + } + } + + static class NewTaskSchedulerForTest extends DagAwareYarnTaskScheduler { + final AMRMClientAsyncWrapper mockClient; + final MockClock clock; + + NewTaskSchedulerForTest( + TaskSchedulerContextDrainable appClient, + AMRMClientAsyncWrapper client, MockClock clock) { + super(appClient); + this.mockClient = client; + this.clock = clock; + setShouldUnregister(); + } + + @Override + public void initialize() throws Exception { + initialize(mockClient); + } + + @Override + protected ScheduledExecutorService createExecutor() { + return new ControlledScheduledExecutorService(clock); + } + + @Override + protected long now() { + return clock.getTime(); + } + } +} diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java index 2c2452b1f3..9cd1e231da 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java @@ -365,6 +365,11 @@ public AMState getAMState() { return real.getAMState(); } + @Override + public int getVertexIndexForTask(Object task) { + return real.getVertexIndexForTask(task); + } + @Override public void preemptContainer(ContainerId cId) { invocations++; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/helpers/DagInfoImplForTest.java b/tez-dag/src/test/java/org/apache/tez/dag/helpers/DagInfoImplForTest.java index b0eeaef933..ab446ac087 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/helpers/DagInfoImplForTest.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/helpers/DagInfoImplForTest.java @@ -17,6 +17,8 @@ import org.apache.hadoop.security.Credentials; import org.apache.tez.serviceplugins.api.DagInfo; +import java.util.BitSet; + public class DagInfoImplForTest implements DagInfo { private final int index; @@ -41,4 +43,14 @@ public String getName() { public Credentials getCredentials() { return null; } + + @Override + public int getTotalVertices() { + return 0; + } + + @Override + public BitSet getVertexDescendants(int vertexIndex) { + return null; + } } diff --git a/tez-dag/src/test/java/org/apache/tez/test/ControlledScheduledExecutorService.java b/tez-dag/src/test/java/org/apache/tez/test/ControlledScheduledExecutorService.java new file mode 100644 index 0000000000..f6da15b81a --- /dev/null +++ b/tez-dag/src/test/java/org/apache/tez/test/ControlledScheduledExecutorService.java @@ -0,0 +1,239 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.test; + +import org.apache.tez.dag.app.MockClock; +import org.apache.tez.dag.app.MockClock.MockClockListener; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.PriorityQueue; +import java.util.concurrent.Callable; +import java.util.concurrent.Delayed; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.FutureTask; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.RunnableScheduledFuture; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; + +/** A scheduled executor service with timing that can be controlled for unit tests. */ +public class ControlledScheduledExecutorService implements ScheduledExecutorService, MockClockListener { + private final MockClock clock; + private final PriorityQueue> queue = new PriorityQueue<>(); + private final AtomicLong nextSequenceNum = new AtomicLong(0); + private final AtomicBoolean stopped = new AtomicBoolean(false); + + public ControlledScheduledExecutorService(MockClock clock) { + this.clock = clock; + clock.register(this); + } + + @Override + public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) { + ScheduledFutureTask task = new ScheduledFutureTask<>(command, null, toTimestamp(delay, unit)); + schedule(task); + return task; + } + + @Override + public ScheduledFuture schedule(Callable callable, long delay, TimeUnit unit) { + ScheduledFutureTask task = new ScheduledFutureTask<>(callable, toTimestamp(delay, unit)); + schedule(task); + return task; + } + + @Override + public ScheduledFuture scheduleWithFixedDelay(Runnable command, long initialDelay, long delay, TimeUnit unit) { + ScheduledFutureTask task = new ScheduledFutureTask<>(command, null, + toTimestamp(initialDelay, unit), unit.toMillis(delay)); + schedule(task); + return task; + } + + @Override + public ScheduledFuture scheduleAtFixedRate(Runnable command, long initialDelay, long period, TimeUnit unit) { + return scheduleWithFixedDelay(command, initialDelay, period, unit); + } + + @Override + public Future submit(Callable callable) { + ScheduledFutureTask task = new ScheduledFutureTask<>(callable, 0); + schedule(task); + return task; + } + + @Override + public Future submit(Runnable runnable, T result) { + ScheduledFutureTask task = new ScheduledFutureTask<>(runnable, result, 0); + schedule(task); + return task; + } + + @Override + public Future submit(Runnable runnable) { + ScheduledFutureTask task = new ScheduledFutureTask<>(runnable, null, 0); + schedule(task); + return task; + } + + @Override + public List> invokeAll(Collection> tasks) { + throw new UnsupportedOperationException("invokeAll not yet implemented"); + } + + @Override + public List> invokeAll(Collection> tasks, long timeout, TimeUnit unit) throws InterruptedException { + throw new UnsupportedOperationException("invokeAll not yet implemented"); + } + + @Override + public T invokeAny(Collection> tasks) throws InterruptedException, ExecutionException { + throw new UnsupportedOperationException("invokeAny not yet implemented"); + } + + @Override + public T invokeAny(Collection> tasks, long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { + throw new UnsupportedOperationException("invokeAny not yet implemented"); + } + + @Override + public void execute(Runnable command) { + submit(command); + } + + @Override + public void shutdown() { + stopped.set(true); + } + + @Override + public List shutdownNow() { + stopped.set(true); + return new ArrayList(queue); + } + + @Override + public boolean isShutdown() { + return stopped.get(); + } + + @Override + public boolean isTerminated() { + return false; + } + + @Override + public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException { + return false; + } + + @Override + public void onTimeUpdated(long newTime) { + ScheduledFutureTask task = queue.peek(); + while (task != null && task.timestamp <= newTime) { + task = queue.poll(); + runTask(task); + task = queue.peek(); + } + } + + private long now() { + return clock.getTime(); + } + + private long toTimestamp(long delay, TimeUnit unit) { + return now() + unit.toMillis(delay); + } + + private void schedule(ScheduledFutureTask task) { + if (isShutdown()) { + throw new RejectedExecutionException("Executor has been shutdown"); + } + if (now() - task.timestamp >= 0) { + runTask(task); + } else { + queue.add(task); + } + } + + private void runTask(ScheduledFutureTask task) { + task.run(); + if (task.isPeriodic() && !isShutdown()) { + task.timestamp = toTimestamp(task.period, TimeUnit.MILLISECONDS); + queue.add(task); + } + } + + private class ScheduledFutureTask extends FutureTask implements RunnableScheduledFuture { + private final long sequenceNum; + private final long period; + private long timestamp; + + public ScheduledFutureTask(Callable callable, long timestamp) { + super(callable); + this.sequenceNum = nextSequenceNum.getAndIncrement(); + this.timestamp = timestamp; + this.period = 0; + } + + public ScheduledFutureTask(Runnable runnable, V result, long timestamp) { + super(runnable, result); + this.sequenceNum = nextSequenceNum.getAndIncrement(); + this.timestamp = timestamp; + this.period = 0; + } + + public ScheduledFutureTask(Runnable runnable, V result, long timestamp, long period) { + super(runnable, result); + this.sequenceNum = nextSequenceNum.getAndIncrement(); + this.timestamp = timestamp; + this.period = period; + } + + @Override + public boolean isPeriodic() { + return period != 0; + } + + @Override + public long getDelay(TimeUnit unit) { + return unit.convert(timestamp - now(), TimeUnit.MILLISECONDS); + } + + @Override + public int compareTo(Delayed o) { + if (o == this) { + return 0; + } + int result = Long.compare(getDelay(TimeUnit.MILLISECONDS), o.getDelay(TimeUnit.MILLISECONDS)); + if (result == 0 && o instanceof ScheduledFutureTask) { + ScheduledFutureTask otherTask = (ScheduledFutureTask) o; + result = Long.compare(sequenceNum, otherTask.sequenceNum); + } + return result; + } + } +} From 96c988cff7c3b6eb7cadf3c819aba4a10cbb0f12 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Tue, 6 Feb 2018 10:36:46 -0600 Subject: [PATCH 048/512] TEZ-3895. Missing name for local mode task scheduler service async request handler thread (Jonathan Eagles via kshukla) --- .../org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java | 1 + 1 file changed, 1 insertion(+) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java index 7dabb73aa0..3b034cdb9a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java @@ -144,6 +144,7 @@ public Object deallocateContainer(ContainerId containerId) { public void initialize() { taskRequestHandler = createRequestHandler(conf); asyncDelegateRequestThread = new Thread(taskRequestHandler); + asyncDelegateRequestThread.setName(LocalTaskSchedulerService.class.getSimpleName() + "RequestHandler"); asyncDelegateRequestThread.setDaemon(true); } From a1f2da8eb319218ff2a6dbe0f6de911336ac7e45 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Fri, 9 Feb 2018 13:54:11 -0600 Subject: [PATCH 049/512] TEZ-3894. Tez intermediate outputs implicitly rely on permissive umask for shuffle (Jason Lowe via kshukla) --- .../common/sort/impl/PipelinedSorter.java | 12 ++++ .../common/sort/impl/TezSpillRecord.java | 5 ++ .../common/sort/impl/dflt/DefaultSorter.java | 12 ++++ .../writers/UnorderedPartitionedKVWriter.java | 12 ++++ .../common/sort/impl/TestPipelinedSorter.java | 57 +++++++++++-------- .../sort/impl/dflt/TestDefaultSorter.java | 25 +++++++- .../TestUnorderedPartitionedKVWriter.java | 24 +++++++- 7 files changed, 119 insertions(+), 28 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java index c4782f6a6a..7915662bbe 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java @@ -38,6 +38,7 @@ import com.google.common.collect.Lists; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.permission.FsPermission; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.runtime.library.api.IOInterruptedException; import org.slf4j.Logger; @@ -68,6 +69,8 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; +import static org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord.SPILL_FILE_PERMS; + @SuppressWarnings({"unchecked", "rawtypes"}) public class PipelinedSorter extends ExternalSorter { @@ -479,6 +482,9 @@ private void spillSingleRecord(final Object key, final Object value, * MAP_OUTPUT_INDEX_RECORD_LENGTH); spillFilePaths.put(numSpills, filename); FSDataOutputStream out = rfs.create(filename, true, 4096); + if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { + rfs.setPermission(filename, SPILL_FILE_PERMS); + } try { LOG.info(outputContext.getDestinationVertexName() + ": Spilling to " + filename.toString() + @@ -564,6 +570,9 @@ public boolean spill(boolean ignoreEmptySpills) throws IOException { mapOutputFile.getSpillFileForWrite(numSpills, size); spillFilePaths.put(numSpills, filename); out = rfs.create(filename, true, 4096); + if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { + rfs.setPermission(filename, SPILL_FILE_PERMS); + } LOG.info(outputContext.getDestinationVertexName() + ": Spilling to " + filename.toString()); for (int i = 0; i < partitions; ++i) { if (isThreadInterrupted()) { @@ -749,6 +758,9 @@ public void flush() throws IOException { } //The output stream for the final single output file FSDataOutputStream finalOut = rfs.create(finalOutputFile, true, 4096); + if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { + rfs.setPermission(finalOutputFile, SPILL_FILE_PERMS); + } final TezSpillRecord spillRec = new TezSpillRecord(partitions); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.java index ab4142b51d..48bd211dfe 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.java @@ -30,11 +30,13 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.util.PureJavaCrc32; import org.apache.tez.runtime.library.common.Constants; public class TezSpillRecord { + public static final FsPermission SPILL_FILE_PERMS = new FsPermission((short) 0640); /** Backing store */ private final ByteBuffer buf; @@ -140,6 +142,9 @@ public void writeToFile(Path loc, Configuration job, Checksum crc) } else { out.close(); } + if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(job)))) { + rfs.setPermission(loc, SPILL_FILE_PERMS); + } } } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java index 85e0003991..cfcbd56103 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java @@ -32,6 +32,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import org.apache.commons.lang.exception.ExceptionUtils; +import org.apache.hadoop.fs.permission.FsPermission; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.runtime.library.api.IOInterruptedException; import org.slf4j.Logger; @@ -63,6 +64,8 @@ import com.google.common.base.Preconditions; +import static org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord.SPILL_FILE_PERMS; + @SuppressWarnings({"unchecked", "rawtypes"}) public final class DefaultSorter extends ExternalSorter implements IndexedSortable { @@ -893,6 +896,9 @@ protected void spill(int mstart, int mend, long sameKeyCount, long totalKeysCoun mapOutputFile.getSpillFileForWrite(numSpills, size); spillFilePaths.put(numSpills, filename); out = rfs.create(filename); + if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { + rfs.setPermission(filename, SPILL_FILE_PERMS); + } int spindex = mstart; final InMemValBytes value = createInMemValBytes(); @@ -1000,6 +1006,9 @@ private void spillSingleRecord(final Object key, final Object value, mapOutputFile.getSpillFileForWrite(numSpills, size); spillFilePaths.put(numSpills, filename); out = rfs.create(filename); + if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { + rfs.setPermission(filename, SPILL_FILE_PERMS); + } // we don't run the combiner for a single record for (int i = 0; i < partitions; ++i) { @@ -1273,6 +1282,9 @@ private void mergeParts() throws IOException, InterruptedException { //The output stream for the final single output file FSDataOutputStream finalOut = rfs.create(finalOutputFile, true, 4096); + if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { + rfs.setPermission(finalOutputFile, SPILL_FILE_PERMS); + } if (numSpills == 0) { // TODO Change event generation to say there is no data rather than generating a dummy file diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java index f4ebc97547..b9f0edf9fa 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java @@ -47,6 +47,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.io.DataInputBuffer; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.tez.common.CallableWithNdc; @@ -83,6 +84,8 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.protobuf.ByteString; +import static org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord.SPILL_FILE_PERMS; + public class UnorderedPartitionedKVWriter extends BaseUnorderedPartitionedKVWriter { private static final Logger LOG = LoggerFactory.getLogger(UnorderedPartitionedKVWriter.class); @@ -588,6 +591,9 @@ protected SpillResult callInternal() throws IOException { this.spillIndex = spillPathDetails.spillIndex; } FSDataOutputStream out = rfs.create(spillPathDetails.outputFilePath); + if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { + rfs.setPermission(spillPathDetails.outputFilePath, SPILL_FILE_PERMS); + } TezSpillRecord spillRecord = new TezSpillRecord(numPartitions); DataInputBuffer key = new DataInputBuffer(); DataInputBuffer val = new DataInputBuffer(); @@ -984,6 +990,9 @@ private void mergeAll() throws IOException { FSDataOutputStream out = null; try { out = rfs.create(finalOutPath); + if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { + rfs.setPermission(finalOutPath, SPILL_FILE_PERMS); + } Writer writer = null; for (int i = 0; i < numPartitions; i++) { @@ -1072,6 +1081,9 @@ private void writeLargeRecord(final Object key, final Object value, final int pa final TezSpillRecord spillRecord = new TezSpillRecord(numPartitions); final Path outPath = spillPathDetails.outputFilePath; out = rfs.create(outPath); + if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { + rfs.setPermission(outPath, SPILL_FILE_PERMS); + } BitSet emptyPartitions = null; if (pipelinedShuffle || !isFinalMergeEnabled) { emptyPartitions = new BitSet(numPartitions); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java index d6f62739ce..727f8acab1 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java @@ -21,7 +21,9 @@ import com.google.common.collect.Maps; import org.apache.commons.lang.RandomStringUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.DataInputBuffer; import org.apache.hadoop.io.DataOutputBuffer; @@ -43,6 +45,7 @@ import org.apache.tez.runtime.api.impl.ExecutionContextImpl; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration.ReportPartitionStats; +import org.apache.tez.runtime.library.common.Constants; import org.apache.tez.runtime.library.common.combine.Combiner; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.runtime.library.conf.OrderedPartitionedKVOutputConfig.SorterImpl; @@ -70,8 +73,10 @@ import static org.mockito.internal.verification.VerificationModeFactory.times; public class TestPipelinedSorter { + private static Configuration conf; private static FileSystem localFs = null; private static Path workDir = null; + private static LocalDirAllocator dirAllocator; private OutputContext outputContext; private int numOutputs; @@ -81,13 +86,14 @@ public class TestPipelinedSorter { private static TreeMap sortedDataMap = Maps.newTreeMap(); static { - Configuration conf = getConf(); + conf = getConf(); try { localFs = FileSystem.getLocal(conf); workDir = new Path( new Path(System.getProperty("test.build.data", "/tmp")), TestPipelinedSorter.class.getName()) .makeQualified(localFs.getUri(), localFs.getWorkingDirectory()); + dirAllocator = new LocalDirAllocator(TezRuntimeFrameworkConfigs.LOCAL_DIRS); } catch (IOException e) { throw new RuntimeException(e); } @@ -100,10 +106,11 @@ public static void cleanup() throws IOException { @Before public void setup() throws IOException { + conf = getConf(); ApplicationId appId = ApplicationId.newInstance(10000, 1); TezCounters counters = new TezCounters(); String uniqueId = UUID.randomUUID().toString(); - String auxiliaryService = getConf().get(TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID, + String auxiliaryService = conf.get(TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID, TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID_DEFAULT); this.outputContext = createMockOutputContext(counters, appId, uniqueId, auxiliaryService); } @@ -111,6 +118,7 @@ public void setup() throws IOException { public static Configuration getConf() { Configuration conf = new Configuration(); conf.set("fs.defaultFS", "file:///"); + conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "077"); //To enable PipelinedSorter conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_SORTER_CLASS, SorterImpl.PIPELINED.name()); @@ -139,15 +147,13 @@ public void basicTest() throws IOException { //TODO: need to support multiple partition testing later //# partition, # of keys, size per key, InitialMem, blockSize - Configuration conf = getConf(); conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_IO_SORT_MB, 5); basicTest(1, 100000, 100, (10 * 1024l * 1024l), 3 << 20); - + verifyOutputPermissions(outputContext.getUniqueIdentifier()); } @Test public void testWithoutPartitionStats() throws IOException { - Configuration conf = getConf(); conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_REPORT_PARTITION_STATS, false); //# partition, # of keys, size per key, InitialMem, blockSize basicTest(1, 0, 0, (10 * 1024l * 1024l), 3 << 20); @@ -156,7 +162,6 @@ public void testWithoutPartitionStats() throws IOException { @Test public void testWithEmptyData() throws IOException { - Configuration conf = getConf(); conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_IO_SORT_MB, 5); //# partition, # of keys, size per key, InitialMem, blockSize basicTest(1, 0, 0, (10 * 1024l * 1024l), 3 << 20); @@ -166,7 +171,6 @@ public void testWithEmptyData() throws IOException { public void testEmptyDataWithPipelinedShuffle() throws IOException { this.numOutputs = 1; this.initialAvailableMem = 1 *1024 * 1024; - Configuration conf = getConf(); conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT, false); conf.setInt(TezRuntimeConfiguration .TEZ_RUNTIME_PIPELINED_SORTER_MIN_BLOCK_SIZE_IN_MB, 1); @@ -207,7 +211,6 @@ public void testEmptyPartitionsHelper(int numKeys, boolean sendEmptyPartitionDet int partitions = 50; this.numOutputs = partitions; this.initialAvailableMem = 1 *1024 * 1024; - Configuration conf = getConf(); conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_EMPTY_PARTITION_INFO_VIA_EVENTS_ENABLED, sendEmptyPartitionDetails); conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT, true); conf.setInt(TezRuntimeConfiguration @@ -222,6 +225,7 @@ public void testEmptyPartitionsHelper(int numKeys, boolean sendEmptyPartitionDet assertTrue(sorter.getNumSpills() == numKeys + 1); } verifyCounters(sorter, outputContext); + verifyOutputPermissions(outputContext.getUniqueIdentifier()); Path indexFile = sorter.getFinalIndexFile(); TezSpillRecord spillRecord = new TezSpillRecord(indexFile, conf); for (int i = 0; i < partitions; i++) { @@ -264,7 +268,6 @@ public void testKVExceedsBuffer2() throws IOException { @Test public void testExceedsKVWithMultiplePartitions() throws IOException { - Configuration conf = getConf(); conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT, true); this.numOutputs = 5; this.initialAvailableMem = 1 * 1024 * 1024; @@ -275,13 +278,13 @@ public void testExceedsKVWithMultiplePartitions() throws IOException { writeData(sorter, 100, 1<<20); verifyCounters(sorter, outputContext); + verifyOutputPermissions(outputContext.getUniqueIdentifier()); } @Test public void testExceedsKVWithPipelinedShuffle() throws IOException { this.numOutputs = 1; this.initialAvailableMem = 1 *1024 * 1024; - Configuration conf = getConf(); conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT, false); conf.setInt(TezRuntimeConfiguration .TEZ_RUNTIME_PIPELINED_SORTER_MIN_BLOCK_SIZE_IN_MB, 1); @@ -301,7 +304,6 @@ public void testExceedsKVWithPipelinedShuffle() throws IOException { public void test_TEZ_2602_50mb() throws IOException { this.numOutputs = 1; this.initialAvailableMem = 1 *1024 * 1024; - Configuration conf = getConf(); conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT, true); conf.setInt(TezRuntimeConfiguration .TEZ_RUNTIME_PIPELINED_SORTER_MIN_BLOCK_SIZE_IN_MB, 1); @@ -318,13 +320,13 @@ public void test_TEZ_2602_50mb() throws IOException { sorter.flush(); sorter.close(); + verifyOutputPermissions(outputContext.getUniqueIdentifier()); } //@Test public void testLargeDataWithMixedKV() throws IOException { this.numOutputs = 1; this.initialAvailableMem = 48 *1024 * 1024; - Configuration conf = getConf(); conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT, true); PipelinedSorter sorter = new PipelinedSorter(this.outputContext, conf, numOutputs, initialAvailableMem); @@ -346,6 +348,7 @@ public void testLargeDataWithMixedKV() throws IOException { sorter.flush(); sorter.close(); + verifyOutputPermissions(outputContext.getUniqueIdentifier()); } @@ -382,7 +385,6 @@ public void testWithVariableKVLength2() throws IOException { @Test public void testWithCustomComparator() throws IOException { //Test with custom comparator - Configuration conf = getConf(); conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_COMPARATOR_CLASS, CustomComparator.class.getName()); basicTest(1, 100000, 100, (10 * 1024l * 1024l), 3 << 20); @@ -392,7 +394,6 @@ public void testWithCustomComparator() throws IOException { public void testWithPipelinedShuffle() throws IOException { this.numOutputs = 1; this.initialAvailableMem = 5 *1024 * 1024; - Configuration conf = getConf(); conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_IO_SORT_MB, 5); conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT, false); conf.setInt(TezRuntimeConfiguration @@ -411,7 +412,6 @@ public void testWithPipelinedShuffle() throws IOException { @Test public void testCountersWithMultiplePartitions() throws IOException { - Configuration conf = getConf(); conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT, true); this.numOutputs = 5; this.initialAvailableMem = 5 * 1024 * 1024; @@ -422,11 +422,11 @@ public void testCountersWithMultiplePartitions() throws IOException { writeData(sorter, 10000, 100); verifyCounters(sorter, outputContext); + verifyOutputPermissions(outputContext.getUniqueIdentifier()); } @Test public void testMultipleSpills() throws IOException { - Configuration conf = getConf(); conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT, true); this.numOutputs = 5; this.initialAvailableMem = 5 * 1024 * 1024; @@ -438,11 +438,11 @@ public void testMultipleSpills() throws IOException { writeData(sorter, 25000, 1000); assertFalse("Expecting needsRLE to be false", sorter.needsRLE()); verifyCounters(sorter, outputContext); + verifyOutputPermissions(outputContext.getUniqueIdentifier()); } @Test public void testWithCombiner() throws IOException { - Configuration conf = getConf(); conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT, true); conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_COMBINER_CLASS, DummyCombiner.class.getName()); this.numOutputs = 5; @@ -461,6 +461,7 @@ public void testWithCombiner() throws IOException { reader.close(); verifyCounters(sorter, outputContext); + verifyOutputPermissions(outputContext.getUniqueIdentifier()); } // for testWithCombiner @@ -479,7 +480,6 @@ public void combine(TezRawKeyValueIterator rawIter, IFile.Writer writer) throws @Test public void testMultipleSpills_WithRLE() throws IOException { - Configuration conf = getConf(); conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT, true); this.numOutputs = 5; this.initialAvailableMem = 5 * 1024 * 1024; @@ -491,12 +491,12 @@ public void testMultipleSpills_WithRLE() throws IOException { writeSimilarKeys(sorter, 25000, 1000, true); assertTrue("Expecting needsRLE to be true", sorter.needsRLE()); verifyCounters(sorter, outputContext); + verifyOutputPermissions(outputContext.getUniqueIdentifier()); } public void basicTest2(int partitions, int[] numkeys, int[] keysize, long initialAvailableMem, int blockSize) throws IOException { this.numOutputs = partitions; // single output - Configuration conf = getConf(); conf.setInt(TezRuntimeConfiguration .TEZ_RUNTIME_PIPELINED_SORTER_MIN_BLOCK_SIZE_IN_MB, 100); PipelinedSorter sorter = new PipelinedSorter(this.outputContext, conf, numOutputs, @@ -520,12 +520,12 @@ private void writeData2(ExternalSorter sorter, } sorter.flush(); sorter.close(); + verifyOutputPermissions(outputContext.getUniqueIdentifier()); } public void basicTest(int partitions, int numKeys, int keySize, long initialAvailableMem, int minBlockSize) throws IOException { this.numOutputs = partitions; // single output - Configuration conf = getConf(); conf.setInt(TezRuntimeConfiguration .TEZ_RUNTIME_PIPELINED_SORTER_MIN_BLOCK_SIZE_IN_MB, minBlockSize >> 20); PipelinedSorter sorter = new PipelinedSorter(this.outputContext, conf, numOutputs, @@ -543,6 +543,7 @@ public void basicTest(int partitions, int numKeys, int keySize, } verifyCounters(sorter, outputContext); + verifyOutputPermissions(outputContext.getUniqueIdentifier()); Path outputFile = sorter.finalOutputFile; FileSystem fs = outputFile.getFileSystem(conf); TezCounter finalOutputBytes = @@ -596,7 +597,6 @@ private void verifyCounters(PipelinedSorter sorter, OutputContext context) { //Its not possible to allocate > 2 GB in test environment. Carry out basic checks here. public void memTest() throws IOException { //Verify if > 2 GB can be set via config - Configuration conf = getConf(); conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_IO_SORT_MB, 3076); long size = ExternalSorter.getInitialMemoryRequirement(conf, 4096 * 1024 * 1024l); Assert.assertTrue(size == (3076l << 20)); @@ -681,7 +681,6 @@ public void memTest() throws IOException { //Intentionally not having timeout public void test_without_lazyMemAllocation() throws IOException { this.numOutputs = 10; - Configuration conf = getConf(); //128 MB. Pre-allocate. Request for default block size. Get 1 buffer conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_IO_SORT_MB, 128); @@ -722,7 +721,6 @@ public void test_without_lazyMemAllocation() throws IOException { //Intentionally not having timeout public void test_with_lazyMemAllocation() throws IOException { this.numOutputs = 10; - Configuration conf = getConf(); //128 MB. Do not pre-allocate. // Get 32 MB buffer first and the another buffer with 96 on filling up @@ -782,7 +780,6 @@ public void test_with_lazyMemAllocation() throws IOException { //Intentionally not having timeout public void testLazyAllocateMem() throws IOException { this.numOutputs = 10; - Configuration conf = getConf(); conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_IO_SORT_MB, 128); conf.setBoolean(TezRuntimeConfiguration .TEZ_RUNTIME_PIPELINED_SORTER_LAZY_ALLOCATE_MEMORY, false); @@ -830,6 +827,17 @@ public void testWithLargeKeyValueWithMinBlockSize() throws IOException { basicTest(1, 5, (2 << 20), (48 * 1024l * 1024l), 16 << 20); } + private void verifyOutputPermissions(String spillId) throws IOException { + String subpath = Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + "/" + spillId + + "/" + Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING; + Path outputPath = dirAllocator.getLocalPathToRead(subpath, conf); + Path indexPath = dirAllocator.getLocalPathToRead(subpath + Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING, conf); + Assert.assertEquals("Incorrect output permissions", (short)0640, + localFs.getFileStatus(outputPath).getPermission().toShort()); + Assert.assertEquals("Incorrect index permissions", (short)0640, + localFs.getFileStatus(indexPath).getPermission().toShort()); + } + private void writeData(ExternalSorter sorter, int numKeys, int keyLen) throws IOException { writeData(sorter, numKeys, keyLen, true); } @@ -880,7 +888,6 @@ private void verifyData(IFile.Reader reader) Text readValue = new Text(); DataInputBuffer keyIn = new DataInputBuffer(); DataInputBuffer valIn = new DataInputBuffer(); - Configuration conf = getConf(); SerializationFactory serializationFactory = new SerializationFactory(conf); Deserializer keyDeserializer = serializationFactory.getDeserializer(Text.class); Deserializer valDeserializer = serializationFactory.getDeserializer(Text.class); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java index 444ebafa05..aad232a5db 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java @@ -18,6 +18,8 @@ package org.apache.tez.runtime.library.common.sort.impl.dflt; +import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.tez.runtime.library.common.Constants; import org.junit.Assert; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -42,6 +44,7 @@ import org.apache.commons.lang.RandomStringUtils; import org.apache.commons.lang.StringUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.DataOutputBuffer; @@ -84,16 +87,19 @@ public class TestDefaultSorter { - private Configuration conf; private static final int PORT = 80; private static final String UniqueID = "UUID"; private static FileSystem localFs = null; private static Path workingDir = null; + private Configuration conf; + private LocalDirAllocator dirAllocator; + @Before public void setup() throws IOException { conf = new Configuration(); + conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "077"); conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_SORTER_CLASS, SorterImpl.LEGACY.name()); // DefaultSorter conf.set("fs.defaultFS", "file:///"); localFs = FileSystem.getLocal(conf); @@ -108,6 +114,7 @@ public void setup() throws IOException { conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_PARTITIONER_CLASS, HashPartitioner.class.getName()); conf.setStrings(TezRuntimeFrameworkConfigs.LOCAL_DIRS, localDirs); + dirAllocator = new LocalDirAllocator(TezRuntimeFrameworkConfigs.LOCAL_DIRS); } @AfterClass @@ -272,6 +279,8 @@ public void basicTest() throws IOException { } catch(IOException ioe) { fail(ioe.getMessage()); } + + verifyOutputPermissions(context.getUniqueIdentifier()); } @Test(timeout = 30000) @@ -396,6 +405,7 @@ public void testEmptyPartitionsHelper(int numKeys, boolean sendEmptyPartitionDet assertTrue(sorter.getNumSpills() == numKeys); } verifyCounters(sorter, context); + verifyOutputPermissions(context.getUniqueIdentifier()); if (sorter.indexCacheList.size() != 0) { for (int i = 0; i < sorter.getNumSpills(); i++) { TezSpillRecord record = sorter.indexCacheList.get(i); @@ -482,6 +492,7 @@ public void testWithSingleSpillWithFinalMergeDisabled() throws IOException { ShuffleUserPayloads.DataMovementEventPayloadProto shufflePayload = ShuffleUserPayloads .DataMovementEventPayloadProto.parseFrom(ByteString.copyFrom(cdme.getUserPayload())); assertTrue(shufflePayload.getPathComponent().equalsIgnoreCase(UniqueID + "_0")); + verifyOutputPermissions(shufflePayload.getPathComponent()); } } @@ -513,6 +524,7 @@ public void testWithMultipleSpillsWithFinalMergeDisabled() throws IOException { ShuffleUserPayloads.DataMovementEventPayloadProto shufflePayload = ShuffleUserPayloads .DataMovementEventPayloadProto.parseFrom(ByteString.copyFrom(cdme.getUserPayload())); assertTrue(shufflePayload.getPathComponent().equalsIgnoreCase(UniqueID + "_" + spillIndex)); + verifyOutputPermissions(shufflePayload.getPathComponent()); spillIndex++; } } @@ -520,6 +532,17 @@ public void testWithMultipleSpillsWithFinalMergeDisabled() throws IOException { verifyCounters(sorter, context); } + private void verifyOutputPermissions(String spillId) throws IOException { + String subpath = Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + "/" + spillId + + "/" + Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING; + Path outputPath = dirAllocator.getLocalPathToRead(subpath, conf); + Path indexPath = dirAllocator.getLocalPathToRead(subpath + Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING, conf); + Assert.assertEquals("Incorrect output permissions", (short)0640, + localFs.getFileStatus(outputPath).getPermission().toShort()); + Assert.assertEquals("Incorrect index permissions", (short)0640, + localFs.getFileStatus(indexPath).getPermission().toShort()); + } + private void verifyCounters(DefaultSorter sorter, OutputContext context) { TezCounter numShuffleChunks = context.getCounters().findCounter(TaskCounter.SHUFFLE_CHUNK_COUNT); TezCounter additionalSpills = context.getCounters().findCounter(TaskCounter.ADDITIONAL_SPILL_COUNT); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java index ae396cbf03..dfd807bfe4 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java @@ -51,9 +51,11 @@ import java.util.regex.Pattern; import com.google.protobuf.ByteString; +import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.runtime.api.TaskFailureType; import org.apache.tez.runtime.api.events.VertexManagerEvent; +import org.apache.tez.runtime.library.common.Constants; import org.apache.tez.runtime.library.common.writers.UnorderedPartitionedKVWriter.SpillInfo; import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.VertexManagerEventPayloadProto; import org.apache.tez.runtime.library.utils.DATA_RANGE_IN_MB; @@ -510,6 +512,10 @@ public void textTest(int numRegularRecords, int numPartitions, long availableMem if (numRecordsWritten > 0) { assertTrue(localFs.exists(outputFilePath)); assertTrue(localFs.exists(spillFilePath)); + assertEquals("Incorrect output permissions", (short)0640, + localFs.getFileStatus(outputFilePath).getPermission().toShort()); + assertEquals("Incorrect index permissions", (short)0640, + localFs.getFileStatus(spillFilePath).getPermission().toShort()); } else { return; } @@ -794,8 +800,14 @@ private void baseTestWithPipelinedTransfer(int numRecords, int numPartitions, Se if (numRecordsWritten > 0) { int numSpills = kvWriter.numSpills.get(); for (int i = 0; i < numSpills; i++) { - assertTrue(localFs.exists(taskOutput.getSpillFileForWrite(i, 10))); - assertTrue(localFs.exists(taskOutput.getSpillIndexFileForWrite(i, 10))); + Path outputFile = taskOutput.getSpillFileForWrite(i, 10); + Path indexFile = taskOutput.getSpillIndexFileForWrite(i, 10); + assertTrue(localFs.exists(outputFile)); + assertTrue(localFs.exists(indexFile)); + assertEquals("Incorrect output permissions", (short)0640, + localFs.getFileStatus(outputFile).getPermission().toShort()); + assertEquals("Incorrect index permissions", (short)0640, + localFs.getFileStatus(indexFile).getPermission().toShort()); } } else { return; @@ -1042,6 +1054,13 @@ private void baseTestWithFinalMergeDisabled(int numRecords, int numPartitions, assertEquals(2, matcher.groupCount()); assertEquals(uniqueId, matcher.group(1)); assertTrue("spill id should be present in path component", matcher.group(2) != null); + Path outputPath = new Path(outputContext.getWorkDirs()[0], + "output/" + eventProto.getPathComponent() + "/" + Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING); + Path indexPath = outputPath.suffix(Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING); + assertEquals("Incorrect output permissions", (short)0640, + localFs.getFileStatus(outputPath).getPermission().toShort()); + assertEquals("Incorrect index permissions", (short)0640, + localFs.getFileStatus(indexPath).getPermission().toShort()); } else { assertEquals(0, eventProto.getSpillId()); if (outputRecordsCounter.getValue() > 0) { @@ -1341,6 +1360,7 @@ private Configuration createConfiguration(OutputContext outputContext, boolean shouldCompress, int maxSingleBufferSizeBytes, Class partitionerClass) { Configuration conf = new Configuration(false); + conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "077"); conf.setStrings(TezRuntimeFrameworkConfigs.LOCAL_DIRS, outputContext.getWorkDirs()); conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, keyClass.getName()); conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS, valClass.getName()); From 022df7218afbb2c940ddc4447246dea5a546c759 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Wed, 14 Feb 2018 10:14:34 -0600 Subject: [PATCH 050/512] TEZ-3893. Tez Local Mode can hang for cases. (Jonathan Eagles via jlowe) --- .../dag/app/rm/LocalTaskSchedulerService.java | 87 +++++++++---------- .../dag/app/rm/TestLocalTaskScheduler.java | 13 +-- .../app/rm/TestLocalTaskSchedulerService.java | 35 ++++---- 3 files changed, 69 insertions(+), 66 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java index 3b034cdb9a..04e79a811d 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java @@ -19,8 +19,8 @@ package org.apache.tez.dag.app.rm; import java.io.IOException; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.BlockingQueue; import java.util.concurrent.PriorityBlockingQueue; import java.util.HashMap; import java.util.Iterator; @@ -51,7 +51,7 @@ public class LocalTaskSchedulerService extends TaskScheduler { private static final Logger LOG = LoggerFactory.getLogger(LocalTaskSchedulerService.class); final ContainerSignatureMatcher containerSignatureMatcher; - final PriorityBlockingQueue taskRequestQueue; + final LinkedBlockingQueue taskRequestQueue; final Configuration conf; AsyncDelegateRequestHandler taskRequestHandler; Thread asyncDelegateRequestThread; @@ -62,7 +62,7 @@ public class LocalTaskSchedulerService extends TaskScheduler { public LocalTaskSchedulerService(TaskSchedulerContext taskSchedulerContext) { super(taskSchedulerContext); - taskRequestQueue = new PriorityBlockingQueue(); + taskRequestQueue = new LinkedBlockingQueue<>(); taskAllocations = new LinkedHashMap(); this.appTrackingUrl = taskSchedulerContext.getAppTrackingUrl(); this.containerSignatureMatcher = taskSchedulerContext.getContainerSignatureMatcher(); @@ -313,29 +313,31 @@ public DeallocateTaskRequest(Object task) { } static class AsyncDelegateRequestHandler implements Runnable { - final BlockingQueue taskRequestQueue; + final LinkedBlockingQueue clientRequestQueue; + final PriorityBlockingQueue taskRequestQueue; final LocalContainerFactory localContainerFactory; final HashMap taskAllocations; final TaskSchedulerContext taskSchedulerContext; final int MAX_TASKS; - AsyncDelegateRequestHandler(BlockingQueue taskRequestQueue, + AsyncDelegateRequestHandler(LinkedBlockingQueue clientRequestQueue, LocalContainerFactory localContainerFactory, HashMap taskAllocations, TaskSchedulerContext taskSchedulerContext, Configuration conf) { - this.taskRequestQueue = taskRequestQueue; + this.clientRequestQueue = clientRequestQueue; this.localContainerFactory = localContainerFactory; this.taskAllocations = taskAllocations; this.taskSchedulerContext = taskSchedulerContext; this.MAX_TASKS = conf.getInt(TezConfiguration.TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS, TezConfiguration.TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS_DEFAULT); + this.taskRequestQueue = new PriorityBlockingQueue<>(); } public void addAllocateTaskRequest(Object task, Resource capability, Priority priority, Object clientCookie) { try { - taskRequestQueue.put(new AllocateTaskRequest(task, capability, priority, clientCookie)); + clientRequestQueue.put(new AllocateTaskRequest(task, capability, priority, clientCookie)); } catch (InterruptedException e) { Thread.currentThread().interrupt(); } @@ -343,57 +345,54 @@ public void addAllocateTaskRequest(Object task, Resource capability, Priority pr public boolean addDeallocateTaskRequest(Object task) { try { - taskRequestQueue.put(new DeallocateTaskRequest(task)); + clientRequestQueue.put(new DeallocateTaskRequest(task)); } catch (InterruptedException e) { Thread.currentThread().interrupt(); } - synchronized(taskRequestQueue) { - taskRequestQueue.notify(); - } return true; } - boolean shouldWait() { - return taskAllocations.size() >= MAX_TASKS; + boolean shouldProcess() { + return !taskRequestQueue.isEmpty() && taskAllocations.size() < MAX_TASKS; } @Override public void run() { - while(!Thread.currentThread().isInterrupted()) { - synchronized(taskRequestQueue) { - try { - if (shouldWait()) { - taskRequestQueue.wait(); - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } + while (!Thread.currentThread().isInterrupted()) { + dispatchRequest(); + while (shouldProcess()) { + allocateTask(); } - processRequest(); } } - void processRequest() { - try { - TaskRequest request = taskRequestQueue.take(); - if (request instanceof AllocateTaskRequest) { - allocateTask((AllocateTaskRequest)request); - } - else if (request instanceof DeallocateTaskRequest) { - deallocateTask((DeallocateTaskRequest)request); - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } catch (NullPointerException e) { - LOG.warn("Task request was badly constructed"); + void dispatchRequest() { + try { + TaskRequest request = clientRequestQueue.take(); + if (request instanceof AllocateTaskRequest) { + taskRequestQueue.put((AllocateTaskRequest)request); + } + else if (request instanceof DeallocateTaskRequest) { + deallocateTask((DeallocateTaskRequest)request); + } + else { + LOG.error("Unknown task request message: " + request); } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } } - void allocateTask(AllocateTaskRequest request) { - Container container = localContainerFactory.createContainer(request.capability, - request.priority); - taskAllocations.put(request.task, container); - taskSchedulerContext.taskAllocated(request.task, request.clientCookie, container); + void allocateTask() { + try { + AllocateTaskRequest request = taskRequestQueue.take(); + Container container = localContainerFactory.createContainer(request.capability, + request.priority); + taskAllocations.put(request.task, container); + taskSchedulerContext.taskAllocated(request.task, request.clientCookie, container); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } } void deallocateTask(DeallocateTaskRequest request) { @@ -403,13 +402,13 @@ void deallocateTask(DeallocateTaskRequest request) { } else { boolean deallocationBeforeAllocation = false; - Iterator iter = taskRequestQueue.iterator(); + Iterator iter = taskRequestQueue.iterator(); while (iter.hasNext()) { TaskRequest taskRequest = iter.next(); - if (taskRequest instanceof AllocateTaskRequest && taskRequest.task.equals(request.task)) { + if (taskRequest.task.equals(request.task)) { iter.remove(); deallocationBeforeAllocation = true; - LOG.info("deallcation happen before allocation for task:" + request.task); + LOG.info("Deallocation request before allocation for task:" + request.task); break; } } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskScheduler.java index 2ada2f17af..36505c2053 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskScheduler.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskScheduler.java @@ -20,7 +20,7 @@ import java.util.HashMap; import java.util.LinkedHashMap; -import java.util.concurrent.PriorityBlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; import org.apache.hadoop.conf.Configuration; import org.apache.tez.serviceplugins.api.TaskSchedulerContext; @@ -57,11 +57,11 @@ public void maxTasksAllocationsCannotBeExceeded() { LocalContainerFactory containerFactory = new LocalContainerFactory(appAttemptId, 1000); HashMap taskAllocations = new LinkedHashMap(); - PriorityBlockingQueue taskRequestQueue = new PriorityBlockingQueue(); + LinkedBlockingQueue clientRequestQueue = new LinkedBlockingQueue<>(); // Object under test AsyncDelegateRequestHandler requestHandler = - new AsyncDelegateRequestHandler(taskRequestQueue, + new AsyncDelegateRequestHandler(clientRequestQueue, containerFactory, taskAllocations, mockContext, @@ -71,17 +71,18 @@ public void maxTasksAllocationsCannotBeExceeded() { for (int i = 0; i < MAX_TASKS; i++) { Priority priority = Priority.newInstance(20); requestHandler.addAllocateTaskRequest(new Long(i), null, priority, null); - requestHandler.processRequest(); + requestHandler.dispatchRequest(); + requestHandler.allocateTask(); } // Only MAX_TASKS number of tasks should have been allocated Assert.assertEquals("Wrong number of allocate tasks", MAX_TASKS, taskAllocations.size()); - Assert.assertTrue("Another allocation should not fit", requestHandler.shouldWait()); + Assert.assertTrue("Another allocation should not fit", !requestHandler.shouldProcess()); // Deallocate down to zero for (int i = 0; i < MAX_TASKS; i++) { requestHandler.addDeallocateTaskRequest(new Long(i)); - requestHandler.processRequest(); + requestHandler.dispatchRequest(); } // All allocated tasks should have been removed diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java index 3b2de34bda..c2daf84948 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java @@ -19,7 +19,7 @@ package org.apache.tez.dag.app.rm; import java.util.HashMap; -import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; @@ -91,6 +91,9 @@ public void testDeallocationBeforeAllocation() throws InterruptedException { taskSchedulerService.initialize(); taskSchedulerService.start(); + // create a task that fills the task allocation queue + Task dummy_task = mock(Task.class); + taskSchedulerService.allocateTask(dummy_task, Resource.newInstance(1024, 1), null, null, Priority.newInstance(1), null, null); Task task = mock(Task.class); taskSchedulerService.allocateTask(task, Resource.newInstance(1024, 1), null, null, Priority.newInstance(1), null, null); taskSchedulerService.deallocateTask(task, false, null, null); @@ -98,10 +101,10 @@ public void testDeallocationBeforeAllocation() throws InterruptedException { taskSchedulerService.startRequestHandlerThread(); MockAsyncDelegateRequestHandler requestHandler = taskSchedulerService.getRequestHandler(); - requestHandler.drainRequest(1); + requestHandler.drainRequest(3); assertEquals(1, requestHandler.deallocateCount); // The corresponding AllocateTaskRequest will be removed, so won't been processed. - assertEquals(0, requestHandler.allocateCount); + assertEquals(1, requestHandler.allocateCount); taskSchedulerService.shutdown(); } @@ -170,10 +173,10 @@ static class MockAsyncDelegateRequestHandler extends AsyncDelegateRequestHandler public int allocateCount = 0; public int deallocateCount = 0; - public int processedCount =0; + public int dispatchCount = 0; MockAsyncDelegateRequestHandler( - BlockingQueue taskRequestQueue, + LinkedBlockingQueue taskRequestQueue, LocalContainerFactory localContainerFactory, HashMap taskAllocations, TaskSchedulerContext appClientDelegate, Configuration conf) { @@ -182,13 +185,19 @@ static class MockAsyncDelegateRequestHandler extends AsyncDelegateRequestHandler } @Override - void processRequest() { - super.processRequest(); - processedCount ++; + void dispatchRequest() { + super.dispatchRequest(); + dispatchCount++; + } + + @Override + void allocateTask() { + super.allocateTask(); + allocateCount++; } public void drainRequest(int count) { - while(processedCount != count || !taskRequestQueue.isEmpty()) { + while(dispatchCount != count || !clientRequestQueue.isEmpty()) { try { Thread.sleep(100); } catch (InterruptedException e) { @@ -197,16 +206,10 @@ public void drainRequest(int count) { } } - @Override - void allocateTask(AllocateTaskRequest request) { - super.allocateTask(request); - allocateCount ++; - } - @Override void deallocateTask(DeallocateTaskRequest request) { super.deallocateTask(request); - deallocateCount ++; + deallocateCount++; } } } From ab8b45eda1e8355125a56787e665db79aeec1f3c Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Fri, 16 Feb 2018 13:57:42 -0600 Subject: [PATCH 051/512] TEZ-3896. TestATSV15HistoryLoggingService#testNonSessionDomains is failing (Jason Lowe via jeagles) --- .../history/logging/ats/TestATSV15HistoryLoggingService.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSV15HistoryLoggingService.java b/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSV15HistoryLoggingService.java index 96c3c80931..65ce91aab2 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSV15HistoryLoggingService.java +++ b/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSV15HistoryLoggingService.java @@ -228,7 +228,8 @@ public void testNonSessionDomains() throws Exception { .setupSessionDAGACLs((Configuration)any(), eq(appId), eq("0"), (DAGAccessControls)any()); // All calls made with session domain id. - verify(historyACLPolicyManager, times(5)).updateTimelineEntityDomain(any(), eq("session-id")); + // NOTE: Expect 6 invocations for 5 history events because DAG_SUBMITTED becomes two separate timeline events. + verify(historyACLPolicyManager, times(6)).updateTimelineEntityDomain(any(), eq("session-id")); assertTrue(entityLog.size() > 0); service.stop(); From ff81cc4f809c8c3366e2c9318d68e78ddcf5c665 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Mon, 19 Feb 2018 10:09:48 -0600 Subject: [PATCH 052/512] TEZ-3898. TestTezCommonUtils fails when compiled against hadoop version >= 2.8 (Jason Lowe via jeagles) --- tez-api/pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tez-api/pom.xml b/tez-api/pom.xml index 763534f636..13b6c01482 100644 --- a/tez-api/pom.xml +++ b/tez-api/pom.xml @@ -225,6 +225,12 @@ hadoop-hdfs-client ${hadoop.version} + + org.apache.hadoop + hadoop-hdfs + test + ${hadoop.version} + From bb40cf5b8a49833e13760a0ffe566092eb464d88 Mon Sep 17 00:00:00 2001 From: Eric Wohlstadter Date: Thu, 22 Feb 2018 13:24:16 -0800 Subject: [PATCH 053/512] TEZ-3888: Update Jetty to org.eclipse.jetty 9.x (Eric Wohlstadter, reviewed by Rohini Palaniswamy) Signed-off-by: Gopal V --- pom.xml | 20 ++++++++++++++++---- tez-dag/pom.xml | 6 +++--- 2 files changed, 19 insertions(+), 7 deletions(-) diff --git a/pom.xml b/pom.xml index c703e2ec50..34240cf97c 100644 --- a/pom.xml +++ b/pom.xml @@ -38,7 +38,7 @@ true ${user.home}/clover.license 2.7.0 - 6.1.26 + 9.3.22.v20171030 3.6.2.Final 0.13.0 1.8 @@ -239,8 +239,8 @@ 2.6 - org.mortbay.jetty - jetty + org.eclipse.jetty + jetty-server compile ${jetty.version} @@ -251,7 +251,7 @@ ${netty.version} - org.mortbay.jetty + org.eclipse.jetty jetty-util compile ${jetty.version} @@ -284,6 +284,18 @@ commons-logging commons-logging-api + + org.eclipse.jetty + jetty-server + + + org.eclipse.jetty + jetty-util + + + org.eclipse.jetty + servlet-api-2.5 + org.mortbay.jetty jetty diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml index 210ed2bcf7..9fa9b44391 100644 --- a/tez-dag/pom.xml +++ b/tez-dag/pom.xml @@ -142,12 +142,12 @@ jettison - org.mortbay.jetty - jetty + org.eclipse.jetty + jetty-server compile - org.mortbay.jetty + org.eclipse.jetty jetty-util compile From c34e46c73218bf21a0219f3004e20cbedaad92f4 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Mon, 5 Mar 2018 09:53:11 -0600 Subject: [PATCH 054/512] TEZ-3897. Tez Local Mode hang for vertices with broadcast input. (Jonathan Eagles via jlowe) --- .../app/launcher/LocalContainerLauncher.java | 19 +- .../dag/app/rm/LocalTaskSchedulerService.java | 185 +++++++++++++----- .../dag/app/rm/TestLocalTaskScheduler.java | 8 +- .../app/rm/TestLocalTaskSchedulerService.java | 94 ++++++++- 4 files changed, 243 insertions(+), 63 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java index 9764daaaef..13e4115d9b 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java @@ -94,9 +94,9 @@ public class LocalContainerLauncher extends DagContainerLauncher { int shufflePort = TezRuntimeUtils.INVALID_PORT; private DeletionTracker deletionTracker; - private final ConcurrentHashMap + private final ConcurrentHashMap> runningContainers = - new ConcurrentHashMap(); + new ConcurrentHashMap<>(); private final ConcurrentHashMap cacheManagers = new ConcurrentHashMap<>(); @@ -281,7 +281,7 @@ private void launch(ContainerLaunchRequest event) { ListenableFuture runningTaskFuture = taskExecutorService.submit(createSubTask(tezChild, event.getContainerId())); RunningTaskCallback callback = new RunningTaskCallback(event.getContainerId()); - runningContainers.put(event.getContainerId(), callback); + runningContainers.put(event.getContainerId(), runningTaskFuture); Futures.addCallback(runningTaskFuture, callback, callbackExecutor); if (deletionTracker != null) { deletionTracker.addNodeShufflePort(event.getNodeId(), shufflePort); @@ -293,19 +293,16 @@ private void launch(ContainerLaunchRequest event) { private void stop(ContainerStopRequest event) { // A stop_request will come in when a task completes and reports back or a preemption decision - // is made. Currently the LocalTaskScheduler does not support preemption. Also preemption - // will not work in local mode till Tez supports task preemption instead of container preemption. - RunningTaskCallback callback = + // is made. + ListenableFuture future = runningContainers.get(event.getContainerId()); - if (callback == null) { + if (future == null) { LOG.info("Ignoring stop request for containerId: " + event.getContainerId()); } else { LOG.info( - "Ignoring stop request for containerId {}. Relying on regular task shutdown for it to end", + "Stopping containerId: {}", event.getContainerId()); - // Allow the tezChild thread to run it's course. It'll receive a shutdown request from the - // AM eventually since the task and container will be unregistered. - // This will need to be fixed once interrupting tasks is supported. + future.cancel(true); } // Send this event to maintain regular control flow. This isn't of much use though. getContext().containerStopRequested(event.getContainerId()); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java index 04e79a811d..cc213cb0cb 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java @@ -19,6 +19,9 @@ package org.apache.tez.dag.app.rm; import java.io.IOException; +import java.util.ArrayList; +import java.util.BitSet; +import java.util.Map; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.PriorityBlockingQueue; @@ -29,6 +32,7 @@ import com.google.common.primitives.Ints; import org.apache.tez.common.TezUtils; +import org.apache.tez.serviceplugins.api.DagInfo; import org.apache.tez.serviceplugins.api.TaskScheduler; import org.apache.tez.serviceplugins.api.TaskSchedulerContext; import org.slf4j.Logger; @@ -51,19 +55,19 @@ public class LocalTaskSchedulerService extends TaskScheduler { private static final Logger LOG = LoggerFactory.getLogger(LocalTaskSchedulerService.class); final ContainerSignatureMatcher containerSignatureMatcher; - final LinkedBlockingQueue taskRequestQueue; + final LinkedBlockingQueue taskRequestQueue; final Configuration conf; AsyncDelegateRequestHandler taskRequestHandler; Thread asyncDelegateRequestThread; - final HashMap taskAllocations; + final HashMap taskAllocations; final String appTrackingUrl; final long customContainerAppId; public LocalTaskSchedulerService(TaskSchedulerContext taskSchedulerContext) { super(taskSchedulerContext); taskRequestQueue = new LinkedBlockingQueue<>(); - taskAllocations = new LinkedHashMap(); + taskAllocations = new LinkedHashMap<>(); this.appTrackingUrl = taskSchedulerContext.getAppTrackingUrl(); this.containerSignatureMatcher = taskSchedulerContext.getContainerSignatureMatcher(); this.customContainerAppId = taskSchedulerContext.getCustomClusterIdentifier(); @@ -98,6 +102,7 @@ public int getClusterNodeCount() { @Override public void dagComplete() { + taskRequestHandler.dagComplete(); } @Override @@ -129,7 +134,7 @@ public synchronized void allocateTask(Object task, Resource capability, // in local mode every task is already container level local taskRequestHandler.addAllocateTaskRequest(task, capability, priority, clientCookie); } - + @Override public boolean deallocateTask(Object task, boolean taskSucceeded, TaskAttemptEndReason endReason, String diagnostics) { return taskRequestHandler.addDeallocateTaskRequest(task); @@ -137,6 +142,7 @@ public boolean deallocateTask(Object task, boolean taskSucceeded, TaskAttemptEnd @Override public Object deallocateContainer(ContainerId containerId) { + taskRequestHandler.addDeallocateContainerRequest(containerId); return null; } @@ -212,20 +218,14 @@ public Container createContainer(Resource capability, Priority priority) { } } - static class TaskRequest implements Comparable { - // Higher prority than Priority.UNDEFINED - static final int HIGHEST_PRIORITY = -2; - Object task; - Priority priority; + static class SchedulerRequest { + } - public TaskRequest(Object task, Priority priority) { - this.task = task; - this.priority = priority; - } + static class TaskRequest extends SchedulerRequest { + final Object task; - @Override - public int compareTo(TaskRequest request) { - return request.priority.compareTo(this.priority); + public TaskRequest(Object task) { + this.task = task; } @Override @@ -239,9 +239,6 @@ public boolean equals(Object o) { TaskRequest that = (TaskRequest) o; - if (priority != null ? !priority.equals(that.priority) : that.priority != null) { - return false; - } if (task != null ? !task.equals(that.task) : that.task != null) { return false; } @@ -251,23 +248,29 @@ public boolean equals(Object o) { @Override public int hashCode() { - int result = 1; - result = 7841 * result + (task != null ? task.hashCode() : 0); - result = 7841 * result + (priority != null ? priority.hashCode() : 0); - return result; + return 7841 + (task != null ? task.hashCode() : 0); } } - static class AllocateTaskRequest extends TaskRequest { - Resource capability; - Object clientCookie; + static class AllocateTaskRequest extends TaskRequest implements Comparable { + final Priority priority; + final Resource capability; + final Object clientCookie; + final int vertexIndex; - public AllocateTaskRequest(Object task, Resource capability, Priority priority, - Object clientCookie) { - super(task, priority); + public AllocateTaskRequest(Object task, int vertexIndex, Resource capability, Priority priority, + Object clientCookie) { + super(task); + this.priority = priority; this.capability = capability; this.clientCookie = clientCookie; + this.vertexIndex = vertexIndex; + } + + @Override + public int compareTo(AllocateTaskRequest request) { + return request.priority.compareTo(this.priority); } @Override @@ -284,6 +287,10 @@ public boolean equals(Object o) { AllocateTaskRequest that = (AllocateTaskRequest) o; + if (priority != null ? !priority.equals(that.priority) : that.priority != null) { + return false; + } + if (capability != null ? !capability.equals(that.capability) : that.capability != null) { return false; } @@ -298,6 +305,7 @@ public boolean equals(Object o) { @Override public int hashCode() { int result = super.hashCode(); + result = 12329 * result + (priority != null ? priority.hashCode() : 0); result = 12329 * result + (capability != null ? capability.hashCode() : 0); result = 12329 * result + (clientCookie != null ? clientCookie.hashCode() : 0); return result; @@ -305,24 +313,43 @@ public int hashCode() { } static class DeallocateTaskRequest extends TaskRequest { - static final Priority DEALLOCATE_PRIORITY = Priority.newInstance(HIGHEST_PRIORITY); public DeallocateTaskRequest(Object task) { - super(task, DEALLOCATE_PRIORITY); + super(task); + } + } + + static class DeallocateContainerRequest extends SchedulerRequest { + final ContainerId containerId; + + public DeallocateContainerRequest(ContainerId containerId) { + this.containerId = containerId; + } + } + + static class AllocatedTask { + final AllocateTaskRequest request; + final Container container; + + AllocatedTask(AllocateTaskRequest request, Container container) { + this.request = request; + this.container = container; } } static class AsyncDelegateRequestHandler implements Runnable { - final LinkedBlockingQueue clientRequestQueue; + final LinkedBlockingQueue clientRequestQueue; final PriorityBlockingQueue taskRequestQueue; final LocalContainerFactory localContainerFactory; - final HashMap taskAllocations; + final HashMap taskAllocations; final TaskSchedulerContext taskSchedulerContext; + private final Object descendantsLock = new Object(); + private ArrayList vertexDescendants = null; final int MAX_TASKS; - AsyncDelegateRequestHandler(LinkedBlockingQueue clientRequestQueue, + AsyncDelegateRequestHandler(LinkedBlockingQueue clientRequestQueue, LocalContainerFactory localContainerFactory, - HashMap taskAllocations, + HashMap taskAllocations, TaskSchedulerContext taskSchedulerContext, Configuration conf) { this.clientRequestQueue = clientRequestQueue; @@ -334,10 +361,33 @@ static class AsyncDelegateRequestHandler implements Runnable { this.taskRequestQueue = new PriorityBlockingQueue<>(); } + void dagComplete() { + synchronized (descendantsLock) { + vertexDescendants = null; + } + } + private void ensureVertexDescendants() { + synchronized (descendantsLock) { + if (vertexDescendants == null) { + DagInfo info = taskSchedulerContext.getCurrentDagInfo(); + if (info == null) { + throw new IllegalStateException("Scheduling tasks but no current DAG info?"); + } + int numVertices = info.getTotalVertices(); + ArrayList descendants = new ArrayList<>(numVertices); + for (int i = 0; i < numVertices; ++i) { + descendants.add(info.getVertexDescendants(i)); + } + vertexDescendants = descendants; + } + } + } + public void addAllocateTaskRequest(Object task, Resource capability, Priority priority, Object clientCookie) { try { - clientRequestQueue.put(new AllocateTaskRequest(task, capability, priority, clientCookie)); + int vertexIndex = taskSchedulerContext.getVertexIndexForTask(task); + clientRequestQueue.put(new AllocateTaskRequest(task, vertexIndex, capability, priority, clientCookie)); } catch (InterruptedException e) { Thread.currentThread().interrupt(); } @@ -352,10 +402,22 @@ public boolean addDeallocateTaskRequest(Object task) { return true; } + public void addDeallocateContainerRequest(ContainerId containerId) { + try { + clientRequestQueue.put(new DeallocateContainerRequest(containerId)); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + boolean shouldProcess() { return !taskRequestQueue.isEmpty() && taskAllocations.size() < MAX_TASKS; } + boolean shouldPreempt() { + return !taskRequestQueue.isEmpty() && taskAllocations.size() >= MAX_TASKS; + } + @Override public void run() { while (!Thread.currentThread().isInterrupted()) { @@ -368,13 +430,19 @@ public void run() { void dispatchRequest() { try { - TaskRequest request = clientRequestQueue.take(); + SchedulerRequest request = clientRequestQueue.take(); if (request instanceof AllocateTaskRequest) { taskRequestQueue.put((AllocateTaskRequest)request); + if (shouldPreempt()) { + maybePreempt((AllocateTaskRequest) request); + } } else if (request instanceof DeallocateTaskRequest) { deallocateTask((DeallocateTaskRequest)request); } + else if (request instanceof DeallocateContainerRequest) { + preemptTask((DeallocateContainerRequest)request); + } else { LOG.error("Unknown task request message: " + request); } @@ -383,12 +451,29 @@ else if (request instanceof DeallocateTaskRequest) { } } + void maybePreempt(AllocateTaskRequest request) { + Priority priority = request.priority; + for (Map.Entry entry : taskAllocations.entrySet()) { + AllocatedTask allocatedTask = entry.getValue(); + Container container = allocatedTask.container; + if (priority.compareTo(allocatedTask.container.getPriority()) > 0) { + Object task = entry.getKey(); + ensureVertexDescendants(); + if (vertexDescendants.get(request.vertexIndex).get(allocatedTask.request.vertexIndex)) { + LOG.info("Preempting task/container for task/priority:" + task + "/" + container + + " for " + request.task + "/" + priority); + taskSchedulerContext.preemptContainer(allocatedTask.container.getId()); + } + } + } + } + void allocateTask() { try { AllocateTaskRequest request = taskRequestQueue.take(); Container container = localContainerFactory.createContainer(request.capability, request.priority); - taskAllocations.put(request.task, container); + taskAllocations.put(request.task, new AllocatedTask(request, container)); taskSchedulerContext.taskAllocated(request.task, request.clientCookie, container); } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -396,24 +481,34 @@ void allocateTask() { } void deallocateTask(DeallocateTaskRequest request) { - Container container = taskAllocations.remove(request.task); - if (container != null) { - taskSchedulerContext.containerBeingReleased(container.getId()); + AllocatedTask allocatedTask = taskAllocations.remove(request.task); + if (allocatedTask != null) { + taskSchedulerContext.containerBeingReleased(allocatedTask.container.getId()); } else { - boolean deallocationBeforeAllocation = false; Iterator iter = taskRequestQueue.iterator(); while (iter.hasNext()) { TaskRequest taskRequest = iter.next(); if (taskRequest.task.equals(request.task)) { iter.remove(); - deallocationBeforeAllocation = true; LOG.info("Deallocation request before allocation for task:" + request.task); break; } } - if (!deallocationBeforeAllocation) { - throw new TezUncheckedException("Unable to find and remove task " + request.task + " from task allocations"); + } + } + + void preemptTask(DeallocateContainerRequest request) { + LOG.info("Trying to preempt: " + request.containerId); + Iterator> entries = taskAllocations.entrySet().iterator(); + while (entries.hasNext()) { + Map.Entry entry = entries.next(); + Container container = entry.getValue().container; + if (container.getId().equals(request.containerId)) { + entries.remove(); + Object task = entry.getKey(); + LOG.info("Preempting task/container:" + task + "/" + container); + taskSchedulerContext.containerBeingReleased(container.getId()); } } } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskScheduler.java index 36505c2053..d7b516add2 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskScheduler.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskScheduler.java @@ -29,13 +29,13 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.app.rm.LocalTaskSchedulerService.AllocatedTask; import org.apache.tez.dag.app.rm.LocalTaskSchedulerService.AsyncDelegateRequestHandler; import org.apache.tez.dag.app.rm.LocalTaskSchedulerService.LocalContainerFactory; -import org.apache.tez.dag.app.rm.LocalTaskSchedulerService.TaskRequest; +import org.apache.tez.dag.app.rm.LocalTaskSchedulerService.SchedulerRequest; public class TestLocalTaskScheduler { @@ -56,8 +56,8 @@ public void maxTasksAllocationsCannotBeExceeded() { LocalContainerFactory containerFactory = new LocalContainerFactory(appAttemptId, 1000); - HashMap taskAllocations = new LinkedHashMap(); - LinkedBlockingQueue clientRequestQueue = new LinkedBlockingQueue<>(); + HashMap taskAllocations = new LinkedHashMap<>(); + LinkedBlockingQueue clientRequestQueue = new LinkedBlockingQueue<>(); // Object under test AsyncDelegateRequestHandler requestHandler = diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java index c2daf84948..70e31f33e2 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java @@ -18,20 +18,25 @@ package org.apache.tez.dag.app.rm; +import java.util.BitSet; import java.util.HashMap; import java.util.concurrent.LinkedBlockingQueue; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.app.dag.Task; import org.apache.tez.dag.app.rm.TestLocalTaskSchedulerService.MockLocalTaskSchedulerSerivce.MockAsyncDelegateRequestHandler; +import org.apache.tez.serviceplugins.api.DagInfo; import org.apache.tez.serviceplugins.api.TaskSchedulerContext; import org.junit.Assert; import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; import static org.junit.Assert.*; import static org.mockito.Mockito.*; @@ -138,6 +143,82 @@ public void testDeallocationAfterAllocation() throws InterruptedException { taskSchedulerService.shutdown(); } + @Test + public void preemptDescendantsOnly() { + + final int MAX_TASKS = 2; + TezConfiguration tezConf = new TezConfiguration(); + tezConf.setInt(TezConfiguration.TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS, MAX_TASKS); + + ApplicationId appId = ApplicationId.newInstance(2000, 1); + ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1); + Long parentTask1 = new Long(1); + Long parentTask2 = new Long(2); + Long childTask1 = new Long(3); + Long grandchildTask1 = new Long(4); + + TaskSchedulerContext + mockContext = TestTaskSchedulerHelpers.setupMockTaskSchedulerContext("", 0, "", true, + appAttemptId, 1000l, null, tezConf); + when(mockContext.getVertexIndexForTask(parentTask1)).thenReturn(0); + when(mockContext.getVertexIndexForTask(parentTask2)).thenReturn(0); + when(mockContext.getVertexIndexForTask(childTask1)).thenReturn(1); + when(mockContext.getVertexIndexForTask(grandchildTask1)).thenReturn(2); + + DagInfo mockDagInfo = mock(DagInfo.class); + when(mockDagInfo.getTotalVertices()).thenReturn(3); + BitSet vertex1Descendants = new BitSet(); + vertex1Descendants.set(1); + vertex1Descendants.set(2); + BitSet vertex2Descendants = new BitSet(); + vertex2Descendants.set(2); + BitSet vertex3Descendants = new BitSet(); + when(mockDagInfo.getVertexDescendants(0)).thenReturn(vertex1Descendants); + when(mockDagInfo.getVertexDescendants(1)).thenReturn(vertex2Descendants); + when(mockDagInfo.getVertexDescendants(2)).thenReturn(vertex3Descendants); + when(mockContext.getCurrentDagInfo()).thenReturn(mockDagInfo); + + Priority priority1 = Priority.newInstance(1); + Priority priority2 = Priority.newInstance(2); + Priority priority3 = Priority.newInstance(3); + Priority priority4 = Priority.newInstance(4); + Resource resource = Resource.newInstance(1024, 1); + + MockLocalTaskSchedulerSerivce taskSchedulerService = new MockLocalTaskSchedulerSerivce(mockContext); + + // The mock context need to send a deallocate container request to the scheduler service + Answer answer = new Answer() { + @Override + public Void answer(InvocationOnMock invocation) { + ContainerId containerId = invocation.getArgumentAt(0, ContainerId.class); + taskSchedulerService.deallocateContainer(containerId); + return null; + } + }; + doAnswer(answer).when(mockContext).preemptContainer(any(ContainerId.class)); + + taskSchedulerService.initialize(); + taskSchedulerService.start(); + taskSchedulerService.startRequestHandlerThread(); + + MockAsyncDelegateRequestHandler requestHandler = taskSchedulerService.getRequestHandler(); + taskSchedulerService.allocateTask(parentTask1, resource, null, null, priority1, null, null); + taskSchedulerService.allocateTask(childTask1, resource, null, null, priority3, null, null); + taskSchedulerService.allocateTask(grandchildTask1, resource, null, null, priority4, null, null); + requestHandler.drainRequest(3); + + // We should not preempt if we have not reached max task allocations + Assert.assertEquals("Wrong number of allocate tasks", MAX_TASKS, requestHandler.allocateCount); + Assert.assertTrue("Another allocation should not fit", !requestHandler.shouldProcess()); + + // Next task allocation should preempt + taskSchedulerService.allocateTask(parentTask2, Resource.newInstance(1024, 1), null, null, priority2, null, null); + requestHandler.drainRequest(5); + + // All allocated tasks should have been removed + Assert.assertEquals("Wrong number of preempted tasks", 1, requestHandler.preemptCount); + } + static class MockLocalTaskSchedulerSerivce extends LocalTaskSchedulerService { private MockAsyncDelegateRequestHandler requestHandler; @@ -173,12 +254,13 @@ static class MockAsyncDelegateRequestHandler extends AsyncDelegateRequestHandler public int allocateCount = 0; public int deallocateCount = 0; + public int preemptCount = 0; public int dispatchCount = 0; MockAsyncDelegateRequestHandler( - LinkedBlockingQueue taskRequestQueue, + LinkedBlockingQueue taskRequestQueue, LocalContainerFactory localContainerFactory, - HashMap taskAllocations, + HashMap taskAllocations, TaskSchedulerContext appClientDelegate, Configuration conf) { super(taskRequestQueue, localContainerFactory, taskAllocations, appClientDelegate, conf); @@ -211,6 +293,12 @@ void deallocateTask(DeallocateTaskRequest request) { super.deallocateTask(request); deallocateCount++; } + + @Override + void preemptTask(DeallocateContainerRequest request) { + super.preemptTask(request); + preemptCount++; + } } } } From 82d73b380881ef8e7d6e6c963289c4f479bbea59 Mon Sep 17 00:00:00 2001 From: Eric Wohlstadter Date: Wed, 7 Mar 2018 15:50:45 -0800 Subject: [PATCH 055/512] TEZ-3892: getClient API for TezClient (Eric Wohlstadter via Gopal V) Signed-off-by: Gopal V --- .../java/org/apache/tez/client/TezClient.java | 230 ++++++++++++------ .../org/apache/tez/client/TezClientUtils.java | 21 +- .../org/apache/tez/client/TestTezClient.java | 64 ++++- .../apache/tez/examples/TezExampleBase.java | 26 +- 4 files changed, 252 insertions(+), 89 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java index 65ce0fb6ed..d2c1af4a07 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java @@ -109,6 +109,9 @@ public class TezClient { private static final Logger LOG = LoggerFactory.getLogger(TezClient.class); + + private static final String appIdStrPrefix = "application"; + private static final String APPLICATION_ID_PREFIX = appIdStrPrefix + '_'; @VisibleForTesting static final String NO_CLUSTER_DIAGNOSTICS_MSG = "No cluster diagnostics found."; @@ -377,40 +380,14 @@ public synchronized void setHistoryLogLevel(HistoryLogLevel historyLogLevel) { */ public synchronized void start() throws TezException, IOException { amConfig.setYarnConfiguration(new YarnConfiguration(amConfig.getTezConfiguration())); - - frameworkClient = createFrameworkClient(); - frameworkClient.init(amConfig.getTezConfiguration(), amConfig.getYarnConfiguration()); - frameworkClient.start(); - - if (this.amConfig.getTezConfiguration().getBoolean( - TezConfiguration.TEZ_CLIENT_JAVA_OPTS_CHECKER_ENABLED, - TezConfiguration.TEZ_CLIENT_JAVA_OPTS_CHECKER_ENABLED_DEFAULT)) { - String javaOptsCheckerClassName = this.amConfig.getTezConfiguration().get( - TezConfiguration.TEZ_CLIENT_JAVA_OPTS_CHECKER_CLASS, ""); - if (!javaOptsCheckerClassName.isEmpty()) { - try { - javaOptsChecker = ReflectionUtils.createClazzInstance(javaOptsCheckerClassName); - } catch (Exception e) { - LOG.warn("Failed to initialize configured Java Opts Checker" - + " (" + TezConfiguration.TEZ_CLIENT_JAVA_OPTS_CHECKER_CLASS - + ") , checkerClass=" + javaOptsCheckerClassName - + ". Disabling checker.", e); - javaOptsChecker = null; - } - } else { - javaOptsChecker = new JavaOptsChecker(); - } - - } - + startFrameworkClient(); + setupJavaOptsChecker(); if (isSession) { LOG.info("Session mode. Starting session."); TezClientUtils.processTezLocalCredentialsFile(sessionCredentials, amConfig.getTezConfiguration()); - Map tezJarResources = getTezJarResources(sessionCredentials); - clientTimeout = amConfig.getTezConfiguration().getInt( TezConfiguration.TEZ_SESSION_CLIENT_TIMEOUT_SECS, TezConfiguration.TEZ_SESSION_CLIENT_TIMEOUT_SECS_DEFAULT); @@ -420,23 +397,7 @@ public synchronized void start() throws TezException, IOException { sessionAppId = createApplication(); } - // Add session token for shuffle - TezClientUtils.createSessionToken(sessionAppId.toString(), - jobTokenSecretManager, sessionCredentials); - - ApplicationSubmissionContext appContext = - TezClientUtils.createApplicationSubmissionContext( - sessionAppId, - null, clientName, amConfig, - tezJarResources, sessionCredentials, usingTezArchiveDeploy, apiVersionInfo, - servicePluginsDescriptor, javaOptsChecker); - - // Set Tez Sessions to not retry on AM crashes if recovery is disabled - if (!amConfig.getTezConfiguration().getBoolean( - TezConfiguration.DAG_RECOVERY_ENABLED, - TezConfiguration.DAG_RECOVERY_ENABLED_DEFAULT)) { - appContext.setMaxAppAttempts(1); - } + ApplicationSubmissionContext appContext = setupApplicationContext(); frameworkClient.submitApplication(appContext); ApplicationReport appReport = frameworkClient.getApplicationReport(sessionAppId); LOG.info("The url to track the Tez Session: " + appReport.getTrackingUrl()); @@ -445,31 +406,136 @@ public synchronized void start() throws TezException, IOException { throw new TezException(e); } - long amClientKeepAliveTimeoutIntervalMillis = - TezCommonUtils.getAMClientHeartBeatTimeoutMillis(amConfig.getTezConfiguration()); - // Poll at minimum of 1 second interval - long pollPeriod = TezCommonUtils. - getAMClientHeartBeatPollIntervalMillis(amConfig.getTezConfiguration(), - amClientKeepAliveTimeoutIntervalMillis, 10); - - boolean isLocal = amConfig.getTezConfiguration().getBoolean( - TezConfiguration.TEZ_LOCAL_MODE, TezConfiguration.TEZ_LOCAL_MODE_DEFAULT); - if (!isLocal && amClientKeepAliveTimeoutIntervalMillis > 0) { - amKeepAliveService = Executors.newSingleThreadScheduledExecutor( - new ThreadFactoryBuilder() - .setDaemon(true).setNameFormat("AMKeepAliveThread #%d").build()); - amKeepAliveService.scheduleWithFixedDelay(new Runnable() { - - private DAGClientAMProtocolBlockingPB proxy; - - @Override - public void run() { - proxy = sendAMHeartbeat(proxy); - } - }, pollPeriod, pollPeriod, TimeUnit.MILLISECONDS); + startClientHeartbeat(); + this.stagingFs = FileSystem.get(amConfig.getTezConfiguration()); + } + } + + public synchronized TezClient getClient(String appIdStr) throws IOException, TezException { + return getClient(appIdfromString(appIdStr)); + } + + /** + * Alternative to start() that explicitly sets sessionAppId and doesn't start a new AM. + * The caller of getClient is responsible for initializing the new TezClient with a + * Configuration compatible with the existing AM. It is expected the caller has cached the + * original Configuration (e.g. in Zookeeper). + * + * In contrast to "start", no resources are localized. It is the responsibility of the caller to + * ensure that existing localized resources and staging dirs are still valid. + * + * @param appId + * @return 'this' just as a convenience for fluent style chaining + */ + public synchronized TezClient getClient(ApplicationId appId) throws TezException, IOException { + sessionAppId = appId; + amConfig.setYarnConfiguration(new YarnConfiguration(amConfig.getTezConfiguration())); + startFrameworkClient(); + setupJavaOptsChecker(); + + if (!isSession) { + String msg = "Must be in session mode to bind TezClient to existing AM"; + LOG.error(msg); + throw new IllegalStateException(msg); + } + + LOG.info("Session mode. Reconnecting to session: " + sessionAppId.toString()); + + clientTimeout = amConfig.getTezConfiguration().getInt( + TezConfiguration.TEZ_SESSION_CLIENT_TIMEOUT_SECS, + TezConfiguration.TEZ_SESSION_CLIENT_TIMEOUT_SECS_DEFAULT); + + try { + setupApplicationContext(); + ApplicationReport appReport = frameworkClient.getApplicationReport(sessionAppId); + LOG.info("The url to track the Tez Session: " + appReport.getTrackingUrl()); + sessionStarted.set(true); + } catch (YarnException e) { + throw new TezException(e); + } + + startClientHeartbeat(); + this.stagingFs = FileSystem.get(amConfig.getTezConfiguration()); + return this; + } + + private void startFrameworkClient() { + frameworkClient = createFrameworkClient(); + frameworkClient.init(amConfig.getTezConfiguration(), amConfig.getYarnConfiguration()); + frameworkClient.start(); + } + + private ApplicationSubmissionContext setupApplicationContext() throws IOException, YarnException { + TezClientUtils.processTezLocalCredentialsFile(sessionCredentials, + amConfig.getTezConfiguration()); + + Map tezJarResources = getTezJarResources(sessionCredentials); + // Add session token for shuffle + TezClientUtils.createSessionToken(sessionAppId.toString(), + jobTokenSecretManager, sessionCredentials); + + ApplicationSubmissionContext appContext = + TezClientUtils.createApplicationSubmissionContext( + sessionAppId, + null, clientName, amConfig, + tezJarResources, sessionCredentials, usingTezArchiveDeploy, apiVersionInfo, + servicePluginsDescriptor, javaOptsChecker); + + // Set Tez Sessions to not retry on AM crashes if recovery is disabled + if (!amConfig.getTezConfiguration().getBoolean( + TezConfiguration.DAG_RECOVERY_ENABLED, + TezConfiguration.DAG_RECOVERY_ENABLED_DEFAULT)) { + appContext.setMaxAppAttempts(1); + } + return appContext; + } + + private void setupJavaOptsChecker() { + if (this.amConfig.getTezConfiguration().getBoolean( + TezConfiguration.TEZ_CLIENT_JAVA_OPTS_CHECKER_ENABLED, + TezConfiguration.TEZ_CLIENT_JAVA_OPTS_CHECKER_ENABLED_DEFAULT)) { + String javaOptsCheckerClassName = this.amConfig.getTezConfiguration().get( + TezConfiguration.TEZ_CLIENT_JAVA_OPTS_CHECKER_CLASS, ""); + if (!javaOptsCheckerClassName.isEmpty()) { + try { + javaOptsChecker = ReflectionUtils.createClazzInstance(javaOptsCheckerClassName); + } catch (Exception e) { + LOG.warn("Failed to initialize configured Java Opts Checker" + + " (" + TezConfiguration.TEZ_CLIENT_JAVA_OPTS_CHECKER_CLASS + + ") , checkerClass=" + javaOptsCheckerClassName + + ". Disabling checker.", e); + javaOptsChecker = null; + } + } else { + javaOptsChecker = new JavaOptsChecker(); } - this.stagingFs = FileSystem.get(amConfig.getTezConfiguration()); + } + } + + private void startClientHeartbeat() { + long amClientKeepAliveTimeoutIntervalMillis = + TezCommonUtils.getAMClientHeartBeatTimeoutMillis(amConfig.getTezConfiguration()); + // Poll at minimum of 1 second interval + long pollPeriod = TezCommonUtils. + getAMClientHeartBeatPollIntervalMillis(amConfig.getTezConfiguration(), + amClientKeepAliveTimeoutIntervalMillis, 10); + + boolean isLocal = amConfig.getTezConfiguration().getBoolean( + TezConfiguration.TEZ_LOCAL_MODE, TezConfiguration.TEZ_LOCAL_MODE_DEFAULT); + if (!isLocal && amClientKeepAliveTimeoutIntervalMillis > 0) { + amKeepAliveService = Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder() + .setDaemon(true).setNameFormat("AMKeepAliveThread #%d").build()); + amKeepAliveService.scheduleWithFixedDelay(new Runnable() { + + private DAGClientAMProtocolBlockingPB proxy; + + @Override + public void run() { + proxy = sendAMHeartbeat(proxy); + } + }, pollPeriod, pollPeriod, TimeUnit.MILLISECONDS); } } @@ -1211,4 +1277,32 @@ public TezClient build() { servicePluginsDescriptor); } } + + //Copied this helper method from + //org.apache.hadoop.yarn.api.records.ApplicationId in Hadoop 2.8+ + //to simplify implementation on 2.7.x + @Public + @Unstable + public static ApplicationId appIdfromString(String appIdStr) { + if (!appIdStr.startsWith(APPLICATION_ID_PREFIX)) { + throw new IllegalArgumentException("Invalid ApplicationId prefix: " + + appIdStr + ". The valid ApplicationId should start with prefix " + + appIdStrPrefix); + } + try { + int pos1 = APPLICATION_ID_PREFIX.length() - 1; + int pos2 = appIdStr.indexOf('_', pos1 + 1); + if (pos2 < 0) { + throw new IllegalArgumentException("Invalid ApplicationId: " + + appIdStr); + } + long rmId = Long.parseLong(appIdStr.substring(pos1 + 1, pos2)); + int appId = Integer.parseInt(appIdStr.substring(pos2 + 1)); + ApplicationId applicationId = ApplicationId.newInstance(rmId, appId); + return applicationId; + } catch (NumberFormatException n) { + throw new IllegalArgumentException("Invalid ApplicationId: " + + appIdStr, n); + } + } } diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index f9316e5cf4..caf610db7d 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -575,16 +575,19 @@ public static ApplicationSubmissionContext createApplicationSubmissionContext( } // emit conf as PB file - ConfigurationProto finalConfProto = createFinalConfProtoForApp(tezConf, - servicePluginsDescriptor); + // don't overwrite existing conf, needed for TezClient.getClient() so existing containers have stable resource fingerprints + if(!binaryConfPath.getFileSystem(tezConf).exists(binaryConfPath)) { + ConfigurationProto finalConfProto = createFinalConfProtoForApp(tezConf, + servicePluginsDescriptor); - FSDataOutputStream amConfPBOutBinaryStream = null; - try { - amConfPBOutBinaryStream = TezCommonUtils.createFileForAM(fs, binaryConfPath); - finalConfProto.writeTo(amConfPBOutBinaryStream); - } finally { - if(amConfPBOutBinaryStream != null){ - amConfPBOutBinaryStream.close(); + FSDataOutputStream amConfPBOutBinaryStream = null; + try { + amConfPBOutBinaryStream = TezCommonUtils.createFileForAM(fs, binaryConfPath); + finalConfProto.writeTo(amConfPBOutBinaryStream); + } finally { + if (amConfPBOutBinaryStream != null) { + amConfPBOutBinaryStream.close(); + } } } diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java index 89310df83e..0cbef76c1b 100644 --- a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java +++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java @@ -168,12 +168,12 @@ TezClientForTest configureAndCreateTezClient(Map lrs, boo @Test (timeout = 5000) public void testTezclientApp() throws Exception { - testTezClient(false); + testTezClient(false, true); } @Test (timeout = 5000) public void testTezclientSession() throws Exception { - testTezClient(true); + testTezClient(true, true); } @Test (timeout = 5000) @@ -238,8 +238,51 @@ private void _testTezClientSessionLargeDAGPlan(int maxIPCMsgSize, int payloadSiz assertTrue(request.hasAdditionalAmResources()); } } + + @Test (timeout = 5000) + public void testGetClient() throws Exception { + /* BEGIN first TezClient usage without calling stop() */ + TezClientForTest client = testTezClient(true, false); + /* END first TezClient usage without calling stop() */ + + /* BEGIN reuse of AM from new TezClient */ + ArgumentCaptor captor = ArgumentCaptor.forClass(ApplicationSubmissionContext.class); + when(client.mockYarnClient.getApplicationReport(client.mockAppId).getYarnApplicationState()) + .thenReturn(YarnApplicationState.RUNNING); + + //Reuse existing appId from first TezClient + ApplicationId existingAppId = client.mockAppId; + TezClientForTest client2 = configureAndCreateTezClient(null, true, + client.amConfig.getTezConfiguration()); + String mockLR1Name = "LR1"; + Map lrDAG = Collections.singletonMap(mockLR1Name, LocalResource + .newInstance(URL.newInstance("file", "localhost", 0, "/test1"), LocalResourceType.FILE, + LocalResourceVisibility.PUBLIC, 1, 1)); + Vertex vertex = Vertex.create("Vertex", ProcessorDescriptor.create("P"), 1, + Resource.newInstance(1, 1)); + DAG dag = DAG.create("DAG").addVertex(vertex).addTaskLocalFiles(lrDAG); + + //Bind TezClient to existing app and submit a dag + DAGClient dagClient = client2.getClient(existingAppId).submitDAG(dag); + + assertTrue(dagClient.getExecutionContext().contains(existingAppId.toString())); + assertEquals(dagClient.getSessionIdentifierString(), existingAppId.toString()); + + // Validate request for new AM is not submitted to RM */ + verify(client2.mockYarnClient, times(0)).submitApplication(captor.capture()); + + // Validate dag submission from second TezClient as normal */ + verify(client2.sessionAmProxy, times(1)).submitDAG((RpcController)any(), (SubmitDAGRequestProto) any()); + + // Validate stop from new TezClient as normal */ + client2.stop(); + verify(client2.sessionAmProxy, times(1)).shutdownSession((RpcController) any(), + (ShutdownSessionRequestProto) any()); + verify(client2.mockYarnClient, times(1)).stop(); + /* END reuse of AM from new TezClient */ + } - public void testTezClient(boolean isSession) throws Exception { + public TezClientForTest testTezClient(boolean isSession, boolean shouldStop) throws Exception { Map lrs = Maps.newHashMap(); String lrName1 = "LR1"; lrs.put(lrName1, LocalResource.newInstance(URL.newInstance("file", "localhost", 0, "/test"), @@ -343,13 +386,16 @@ public void testTezClient(boolean isSession) throws Exception { assertTrue(context.getAMContainerSpec().getLocalResources().containsKey( lrName2)); } - - client.stop(); - if (isSession) { - verify(client.sessionAmProxy, times(1)).shutdownSession((RpcController) any(), - (ShutdownSessionRequestProto) any()); + + if(shouldStop) { + client.stop(); + if (isSession) { + verify(client.sessionAmProxy, times(1)).shutdownSession((RpcController) any(), + (ShutdownSessionRequestProto) any()); + } + verify(client.mockYarnClient, times(1)).stop(); } - verify(client.mockYarnClient, times(1)).stop(); + return client; } @Test (timeout=5000) diff --git a/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java b/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java index a3c0224abf..6b626b189b 100644 --- a/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java +++ b/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java @@ -63,11 +63,16 @@ public abstract class TezExampleBase extends Configured implements Tool { protected static final String LOCAL_MODE = "local"; protected static final String COUNTER_LOG = "counter"; protected static final String GENERATE_SPLIT_IN_CLIENT = "generateSplitInClient"; + protected static final String LEAVE_AM_RUNNING = "leaveAmRunning"; + protected static final String RECONNECT_APP_ID = "reconnectAppId"; + private boolean disableSplitGrouping = false; private boolean isLocalMode = false; private boolean isCountersLog = false; private boolean generateSplitInClient = false; + private boolean leaveAmRunning = false; + private String reconnectAppId; private HadoopShim hadoopShim; protected boolean isCountersLog() { @@ -88,6 +93,8 @@ private Options getExtraOptions() { options.addOption(DISABLE_SPLIT_GROUPING, false , "disable split grouping"); options.addOption(COUNTER_LOG, false , "print counter log"); options.addOption(GENERATE_SPLIT_IN_CLIENT, false, "whether generate split in client"); + options.addOption(LEAVE_AM_RUNNING, false, "whether client should stop session"); + options.addOption(RECONNECT_APP_ID, true, "appId for client reconnect"); return options; } @@ -108,6 +115,12 @@ public final int run(String[] args) throws Exception { if (optionParser.getCommandLine().hasOption(GENERATE_SPLIT_IN_CLIENT)) { generateSplitInClient = true; } + if (optionParser.getCommandLine().hasOption(LEAVE_AM_RUNNING)) { + leaveAmRunning = true; + } + if (optionParser.getCommandLine().hasOption(RECONNECT_APP_ID)) { + reconnectAppId = optionParser.getCommandLine().getOptionValue(RECONNECT_APP_ID); + } hadoopShim = new HadoopShimsLoader(conf).getHadoopShim(); return _execute(otherArgs, null, null); @@ -231,15 +244,20 @@ private int _execute(String[] otherArgs, TezConfiguration tezConf, TezClient tez try { return runJob(otherArgs, tezConf, tezClientInternal); } finally { - if (ownTezClient && tezClientInternal != null) { + if (ownTezClient && tezClientInternal != null && !leaveAmRunning) { tezClientInternal.stop(); } } } private TezClient createTezClient(TezConfiguration tezConf) throws IOException, TezException { - TezClient tezClient = TezClient.create(getClass().getSimpleName(), tezConf); - tezClient.start(); + TezClient tezClient = TezClient.create("TezExampleApplication", tezConf); + if(reconnectAppId != null) { + ApplicationId appId = TezClient.appIdfromString(reconnectAppId); + tezClient.getClient(appId); + } else { + tezClient.start(); + } return tezClient; } @@ -265,6 +283,8 @@ protected void printExtraOptionsUsage(PrintStream ps) { + " enable split grouping without this option."); ps.println("-" + COUNTER_LOG + "\t\t to print counters information"); ps.println("-" + GENERATE_SPLIT_IN_CLIENT + "\t\tgenerate input split in client"); + ps.println("-" + LEAVE_AM_RUNNING + "\t\twhether client should stop session"); + ps.println("-" + RECONNECT_APP_ID + "\t\tappId for client reconnect"); ps.println(); ps.println("The Tez example extra options usage syntax is "); ps.println("example_name [extra_options] [example_parameters]"); From 75bc7c157682cc64eccfa4722226a8ac72161f17 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Fri, 9 Mar 2018 10:00:21 -0600 Subject: [PATCH 056/512] TEZ-3874. NPE in TezClientUtils when "yarn.resourcemanager.zk-address" is present in Configuration. (Eric Wohlstadter via jlowe) --- .../org/apache/tez/client/TezClientUtils.java | 14 +++++-- .../java/org/apache/tez/common/TezUtils.java | 38 +++++++++++++------ .../main/java/org/apache/tez/dag/api/DAG.java | 15 ++------ .../org/apache/tez/client/TestTezClient.java | 13 +++++++ .../org/apache/tez/common/TestTezUtils.java | 15 ++++++++ 5 files changed, 68 insertions(+), 27 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index caf610db7d..689d94797a 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -807,10 +807,16 @@ static ConfigurationProto createFinalConfProtoForApp(Configuration amConf, assert amConf != null; ConfigurationProto.Builder builder = ConfigurationProto.newBuilder(); for (Entry entry : amConf) { - PlanKeyValuePair.Builder kvp = PlanKeyValuePair.newBuilder(); - kvp.setKey(entry.getKey()); - kvp.setValue(amConf.get(entry.getKey())); - builder.addConfKeyValues(kvp); + String key = entry.getKey(); + String val = amConf.get(key); + if(val != null) { + PlanKeyValuePair.Builder kvp = PlanKeyValuePair.newBuilder(); + kvp.setKey(key); + kvp.setValue(val); + builder.addConfKeyValues(kvp); + } else { + LOG.debug("null value in Configuration after replacement for key={}. Skipping.", key); + } } AMPluginDescriptorProto pluginDescriptorProto = diff --git a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java index dfdf9fa642..efd450290f 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java @@ -134,16 +134,8 @@ public static Configuration createConfFromUserPayload(UserPayload payload) throw private static void writeConfInPB(OutputStream dos, Configuration conf) throws IOException { - DAGProtos.ConfigurationProto.Builder confProtoBuilder = DAGProtos.ConfigurationProto - .newBuilder(); - Iterator> iter = conf.iterator(); - while (iter.hasNext()) { - Map.Entry entry = iter.next(); - DAGProtos.PlanKeyValuePair.Builder kvp = DAGProtos.PlanKeyValuePair.newBuilder(); - kvp.setKey(entry.getKey()); - kvp.setValue(entry.getValue()); - confProtoBuilder.addConfKeyValues(kvp); - } + DAGProtos.ConfigurationProto.Builder confProtoBuilder = DAGProtos.ConfigurationProto.newBuilder(); + populateConfProtoFromEntries(conf, confProtoBuilder); DAGProtos.ConfigurationProto confProto = confProtoBuilder.build(); confProto.writeTo(dos); } @@ -167,7 +159,13 @@ public static String convertToHistoryText(String description, Configuration conf Iterator> iter = conf.iterator(); while (iter.hasNext()) { Entry entry = iter.next(); - confJson.put(entry.getKey(), conf.get(entry.getKey())); + String key = entry.getKey(); + String val = conf.get(entry.getKey()); + if(val != null) { + confJson.put(key, val); + } else { + LOG.debug("null value in Configuration after replacement for key={}. Skipping.", key); + } } jsonObject.put(ATSConstants.CONFIG, confJson); } @@ -181,4 +179,22 @@ public static String convertToHistoryText(Configuration conf) { return convertToHistoryText(null, conf); } + + /* Copy each Map.Entry with non-null value to DAGProtos.ConfigurationProto */ + public static void populateConfProtoFromEntries(Iterable> params, + DAGProtos.ConfigurationProto.Builder confBuilder) { + for(Map.Entry entry : params) { + String key = entry.getKey(); + String val = entry.getValue(); + if(val != null) { + DAGProtos.PlanKeyValuePair.Builder kvp = DAGProtos.PlanKeyValuePair.newBuilder(); + kvp.setKey(key); + kvp.setValue(val); + confBuilder.addConfKeyValues(kvp); + } else { + LOG.debug("null value for key={}. Skipping.", key); + } + } + } + } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java index cdfa3b2488..735c749ee5 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java @@ -41,6 +41,7 @@ import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.tez.client.CallerContext; import org.apache.tez.common.JavaOptsChecker; +import org.apache.tez.common.TezUtils; import org.apache.tez.dag.api.Vertex.VertexExecutionContext; import org.apache.tez.dag.api.records.DAGProtos; import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor; @@ -985,12 +986,7 @@ public synchronized DAGPlan createDag(Configuration tezConf, Credentials extraCr if (vertex.getConf()!= null && vertex.getConf().size() > 0) { ConfigurationProto.Builder confBuilder = ConfigurationProto.newBuilder(); - for (Map.Entry entry : vertex.getConf().entrySet()) { - PlanKeyValuePair.Builder keyValueBuilder = PlanKeyValuePair.newBuilder(); - keyValueBuilder.setKey(entry.getKey()); - keyValueBuilder.setValue(entry.getValue()); - confBuilder.addConfKeyValues(keyValueBuilder); - } + TezUtils.populateConfProtoFromEntries(vertex.getConf().entrySet(), confBuilder); vertexBuilder.setVertexConf(confBuilder); } @@ -1091,12 +1087,7 @@ public synchronized DAGPlan createDag(Configuration tezConf, Credentials extraCr ConfigurationProto.Builder confProtoBuilder = ConfigurationProto.newBuilder(); if (!this.dagConf.isEmpty()) { - for (Entry entry : this.dagConf.entrySet()) { - PlanKeyValuePair.Builder kvp = PlanKeyValuePair.newBuilder(); - kvp.setKey(entry.getKey()); - kvp.setValue(entry.getValue()); - confProtoBuilder.addConfKeyValues(kvp); - } + TezUtils.populateConfProtoFromEntries(this.dagConf.entrySet(), confProtoBuilder); } // Copy historyLogLevel from tezConf into dagConf if its not overridden in dagConf. String logLevel = this.dagConf.get(TezConfiguration.TEZ_HISTORY_LOGGING_LOGLEVEL); diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java index 0cbef76c1b..2c0406153f 100644 --- a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java +++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java @@ -90,6 +90,7 @@ import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.ShutdownSessionRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.TezAppMasterStatusProto; +import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto; import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor; import org.hamcrest.CoreMatchers; import org.junit.Assert; @@ -927,4 +928,16 @@ public void testAMHeartbeatFailOnGetAMStatus() throws Exception { Thread.sleep(3 * amHeartBeatTimeoutSecs * 1000); assertTrue(client.getAMKeepAliveService().isTerminated()); } + + //See TEZ-3874 + @Test(timeout = 5000) + public void testYarnZkDeprecatedConf() { + Configuration conf = new Configuration(false); + String val = "hostname:2181"; + conf.set("yarn.resourcemanager.zk-address", val); + + ConfigurationProto confProto = null; + //Test that Exception is not thrown by createFinalConfProtoForApp + TezClientUtils.createFinalConfProtoForApp(conf, null); + } } diff --git a/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java b/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java index 61bb9a7c11..16efc8f52d 100644 --- a/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java +++ b/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java @@ -20,14 +20,19 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertEquals; import java.io.IOException; import java.util.BitSet; +import java.util.HashMap; +import java.util.Map; import java.util.Random; import org.apache.hadoop.conf.Configuration; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.UserPayload; +import org.apache.tez.dag.api.records.DAGProtos; import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; import org.junit.Assert; @@ -230,4 +235,14 @@ public void testConvertToHistoryTextWithReplaceVars() throws JSONException { } + @Test(timeout = 5000) + public void testPopulateConfProtoFromEntries() { + Map map = new HashMap<>(); + map.put("nonNullKey", "value"); + map.put("nullKey", null); + DAGProtos.ConfigurationProto.Builder confBuilder = DAGProtos.ConfigurationProto.newBuilder(); + TezUtils.populateConfProtoFromEntries(map.entrySet(), confBuilder); + assertEquals(confBuilder.getConfKeyValuesList().size(), 1); + } + } From 85bd772c427c0b38534bad3dbe300d5ce0395012 Mon Sep 17 00:00:00 2001 From: Eric Wohlstadter Date: Wed, 14 Mar 2018 22:27:22 -0700 Subject: [PATCH 057/512] TEZ-3905: Change BUILDING.TXT to minimum JDK 1.8 (Eric Wohlstadter via Gopal V) Signed-off-by: Gopal V --- BUILDING.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/BUILDING.txt b/BUILDING.txt index 34ed2fbff1..ff7ac51200 100644 --- a/BUILDING.txt +++ b/BUILDING.txt @@ -6,7 +6,7 @@ https://cwiki.apache.org/confluence/display/TEZ ---------------------------------------------------------------------------------- Requirements: -* JDK 1.7+ +* JDK 1.8+ * Maven 3.1 or later * Findbugs 2.0.2 or later (if running findbugs) * ProtocolBuffer 2.5.0 From 7f936f614914a456e3ace9ca5a5f0d218e7c7e01 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Mon, 26 Mar 2018 14:49:12 -0500 Subject: [PATCH 058/512] TEZ-3907. Improve log message to include the location the writers decide to spill output (Kuhu Shukla via jlowe) --- .../runtime/library/common/sort/impl/dflt/DefaultSorter.java | 3 ++- .../library/common/writers/UnorderedPartitionedKVWriter.java | 1 + 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java index cfcbd56103..557a53855f 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java @@ -977,7 +977,8 @@ protected void spill(int mstart, int mend, long sameKeyCount, long totalKeysCoun totalIndexCacheMemory += spillRec.size() * MAP_OUTPUT_INDEX_RECORD_LENGTH; } - LOG.info(outputContext.getDestinationVertexName() + ": " + "Finished spill " + numSpills); + LOG.info(outputContext.getDestinationVertexName() + ": " + "Finished spill " + numSpills + + " at " + filename.toString()); ++numSpills; if (!isFinalMergeEnabled()) { numShuffleChunks.setValue(numSpills); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java index b9f0edf9fa..948417d917 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java @@ -590,6 +590,7 @@ protected SpillResult callInternal() throws IOException { this.spillPathDetails = getSpillPathDetails(false, -1, spillNumber); this.spillIndex = spillPathDetails.spillIndex; } + LOG.info("Writing spill " + spillNumber + " to " + spillPathDetails.outputFilePath.toString()); FSDataOutputStream out = rfs.create(spillPathDetails.outputFilePath); if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { rfs.setPermission(spillPathDetails.outputFilePath, SPILL_FILE_PERMS); From dce57c83ab942e5698b7107bfa176e5efa35e269 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Tue, 3 Apr 2018 12:45:20 -0500 Subject: [PATCH 059/512] TEZ-3909. DAG can hang if vertex with no tasks is killed (Jason Lowe via jeagles) --- .../tez/dag/app/dag/impl/VertexImpl.java | 8 +++-- .../tez/dag/app/dag/impl/TestVertexImpl.java | 32 +++++++++++++++++++ 2 files changed, 38 insertions(+), 2 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index d727e39a57..ad261736c2 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -514,6 +514,11 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl EnumSet.of(VertexState.TERMINATING, VertexState.KILLED, VertexState.FAILED, VertexState.ERROR), VertexEventType.V_TASK_COMPLETED, new TaskCompletedTransition()) + .addTransition + (VertexState.TERMINATING, + EnumSet.of(VertexState.TERMINATING, VertexState.KILLED, VertexState.FAILED, VertexState.ERROR), + VertexEventType.V_COMPLETED, + new VertexNoTasksCompletedTransition()) .addTransition( VertexState.TERMINATING, VertexState.ERROR, VertexEventType.V_INTERNAL_ERROR, @@ -534,8 +539,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl VertexEventType.V_ROUTE_EVENT, VertexEventType.V_SOURCE_TASK_ATTEMPT_COMPLETED, VertexEventType.V_TASK_ATTEMPT_COMPLETED, - VertexEventType.V_TASK_RESCHEDULED, - VertexEventType.V_COMPLETED)) + VertexEventType.V_TASK_RESCHEDULED)) // Transitions from SUCCEEDED state .addTransition( diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java index d382974b29..5722406684 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java @@ -5769,6 +5769,38 @@ public void testVertexWithNoTasks() { } } + @Test(timeout = 10000) + public void testVertexNoTasksTerminated() { + TezVertexID vId = null; + try { + TezDAGID invalidDagId = TezDAGID.getInstance( + dagId.getApplicationId(), 1000); + vId = TezVertexID.getInstance(invalidDagId, 1); + VertexPlan vPlan = invalidDagPlan.getVertex(0); + EventHandler mockEventHandler = mock(EventHandler.class); + VertexImpl v = new VertexImpl(vId, vPlan, vPlan.getName(), conf, + mockEventHandler, taskCommunicatorManagerInterface, + clock, thh, true, appContext, vertexLocationHint, null, taskSpecificLaunchCmdOption, + updateTracker, new Configuration(false)); + v.setInputVertices(new HashMap()); + vertexIdMap.put(vId, v); + vertices.put(v.getName(), v); + v.handle(new VertexEvent(vId, VertexEventType.V_INIT)); + Assert.assertEquals(VertexState.INITED, v.getState()); + v.handle(new VertexEvent(vId, VertexEventType.V_START)); + Assert.assertEquals(VertexState.RUNNING, v.getState()); + v.handle(new VertexEventTermination(vId, VertexTerminationCause.OTHER_VERTEX_FAILURE)); + Assert.assertEquals(VertexState.TERMINATING, v.getState()); + v.handle(new VertexEvent(vId, VertexEventType.V_COMPLETED)); + Assert.assertEquals(VertexState.KILLED, v.getState()); + Assert.assertTrue(1.0f == v.getCompletedTaskProgress()); + } finally { + if (vId != null) { + vertexIdMap.remove(vId); + } + } + } + @SuppressWarnings("rawtypes") private static class VertexImplWithRunningInputInitializer extends VertexImpl { From 55a6b9d5cbf7062ab1f440e9d03ffedfda017170 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Wed, 4 Apr 2018 08:56:01 -0500 Subject: [PATCH 060/512] TEZ-3902. Upgrade to netty-3.10.5.Final.jar (Jason Lowe via kshukla) --- pom.xml | 2 +- tez-dist/src/main/assembly/tez-dist-minimal.xml | 1 + tez-plugins/tez-aux-services/pom.xml | 1 + 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 34240cf97c..f17b2d2df3 100644 --- a/pom.xml +++ b/pom.xml @@ -39,7 +39,7 @@ ${user.home}/clover.license 2.7.0 9.3.22.v20171030 - 3.6.2.Final + 3.10.5.Final 0.13.0 1.8 1.7.10 diff --git a/tez-dist/src/main/assembly/tez-dist-minimal.xml b/tez-dist/src/main/assembly/tez-dist-minimal.xml index 80633ffd57..fbd1782273 100644 --- a/tez-dist/src/main/assembly/tez-dist-minimal.xml +++ b/tez-dist/src/main/assembly/tez-dist-minimal.xml @@ -24,6 +24,7 @@ true org.apache.tez:tez-aux-services + org.apache.tez:tez-ext-service-tests / diff --git a/tez-plugins/tez-aux-services/pom.xml b/tez-plugins/tez-aux-services/pom.xml index ac874ae0d0..91435b163c 100644 --- a/tez-plugins/tez-aux-services/pom.xml +++ b/tez-plugins/tez-aux-services/pom.xml @@ -107,6 +107,7 @@ io.netty netty + 3.6.2.Final com.google.guava From a030800d95f5409ef7269c5fd06775e0c8295aef Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Thu, 5 Apr 2018 13:21:12 -0500 Subject: [PATCH 061/512] Revert "TEZ-3902. Upgrade to netty-3.10.5.Final.jar (Jason Lowe via kshukla)" This reverts commit 55a6b9d5cbf7062ab1f440e9d03ffedfda017170. --- pom.xml | 2 +- tez-dist/src/main/assembly/tez-dist-minimal.xml | 1 - tez-plugins/tez-aux-services/pom.xml | 1 - 3 files changed, 1 insertion(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index f17b2d2df3..34240cf97c 100644 --- a/pom.xml +++ b/pom.xml @@ -39,7 +39,7 @@ ${user.home}/clover.license 2.7.0 9.3.22.v20171030 - 3.10.5.Final + 3.6.2.Final 0.13.0 1.8 1.7.10 diff --git a/tez-dist/src/main/assembly/tez-dist-minimal.xml b/tez-dist/src/main/assembly/tez-dist-minimal.xml index fbd1782273..80633ffd57 100644 --- a/tez-dist/src/main/assembly/tez-dist-minimal.xml +++ b/tez-dist/src/main/assembly/tez-dist-minimal.xml @@ -24,7 +24,6 @@ true org.apache.tez:tez-aux-services - org.apache.tez:tez-ext-service-tests / diff --git a/tez-plugins/tez-aux-services/pom.xml b/tez-plugins/tez-aux-services/pom.xml index 91435b163c..ac874ae0d0 100644 --- a/tez-plugins/tez-aux-services/pom.xml +++ b/tez-plugins/tez-aux-services/pom.xml @@ -107,7 +107,6 @@ io.netty netty - 3.6.2.Final com.google.guava From 9b3f8c73d4268bfa04c2a1dc5cd5c027af888a06 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Mon, 9 Apr 2018 14:44:11 -0500 Subject: [PATCH 062/512] TEZ-3913. Precommit build fails to post to JIRA (Jason Lowe via jeagles) --- build-tools/test-patch.sh | 21 ++++++++++++++++++--- 1 file changed, 18 insertions(+), 3 deletions(-) diff --git a/build-tools/test-patch.sh b/build-tools/test-patch.sh index e1fa0fb00d..7a19e2ed0f 100755 --- a/build-tools/test-patch.sh +++ b/build-tools/test-patch.sh @@ -38,6 +38,7 @@ GREP=${GREP:-grep} PATCH=${PATCH:-patch} DIFF=${DIFF:-diff} JIRACLI=${JIRA:-jira} +CURL=${CURL:-curl} FINDBUGS_HOME=${FINDBUGS_HOME} ############################################################################### @@ -702,10 +703,24 @@ $comment" echo "======================================================================" echo "" echo "" + + # RESTify the comment + jsoncomment=$(echo "$comment" \ + | ${SED} -e 's,\\,\\\\,g' \ + -e 's,\",\\\",g' \ + -e 's,$,\\r\\n,g' \ + | tr -d '\n') + jsoncomment='{"body":"'"$jsoncomment"'"}' + ### Update Jira with a comment - export USER=hudson - $JIRACLI -s https://issues.apache.org/jira -a addcomment -u tezqa -p $JIRA_PASSWD --comment "$comment" --issue $defect - $JIRACLI -s https://issues.apache.org/jira -a logout -u tezqa -p $JIRA_PASSWD + ${CURL} -X POST \ + -H "Accept: application/json" \ + -H "Content-Type: application/json" \ + -u "tezqa:${JIRA_PASSWD}" \ + -d "$jsoncomment" \ + --silent --location \ + "https://issues.apache.org/jira/rest/api/2/issue/${defect}/comment" \ + >/dev/null fi } From 9431ce6c358dbefa8973c2ebe4c0bed17645ddba Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Mon, 9 Apr 2018 16:36:53 -0500 Subject: [PATCH 063/512] Revert "TEZ-3913. Precommit build fails to post to JIRA (Jason Lowe via jeagles)" This reverts commit 9b3f8c73d4268bfa04c2a1dc5cd5c027af888a06. --- build-tools/test-patch.sh | 21 +++------------------ 1 file changed, 3 insertions(+), 18 deletions(-) diff --git a/build-tools/test-patch.sh b/build-tools/test-patch.sh index 7a19e2ed0f..e1fa0fb00d 100755 --- a/build-tools/test-patch.sh +++ b/build-tools/test-patch.sh @@ -38,7 +38,6 @@ GREP=${GREP:-grep} PATCH=${PATCH:-patch} DIFF=${DIFF:-diff} JIRACLI=${JIRA:-jira} -CURL=${CURL:-curl} FINDBUGS_HOME=${FINDBUGS_HOME} ############################################################################### @@ -703,24 +702,10 @@ $comment" echo "======================================================================" echo "" echo "" - - # RESTify the comment - jsoncomment=$(echo "$comment" \ - | ${SED} -e 's,\\,\\\\,g' \ - -e 's,\",\\\",g' \ - -e 's,$,\\r\\n,g' \ - | tr -d '\n') - jsoncomment='{"body":"'"$jsoncomment"'"}' - ### Update Jira with a comment - ${CURL} -X POST \ - -H "Accept: application/json" \ - -H "Content-Type: application/json" \ - -u "tezqa:${JIRA_PASSWD}" \ - -d "$jsoncomment" \ - --silent --location \ - "https://issues.apache.org/jira/rest/api/2/issue/${defect}/comment" \ - >/dev/null + export USER=hudson + $JIRACLI -s https://issues.apache.org/jira -a addcomment -u tezqa -p $JIRA_PASSWD --comment "$comment" --issue $defect + $JIRACLI -s https://issues.apache.org/jira -a logout -u tezqa -p $JIRA_PASSWD fi } From 871ea80c04eccd6eb985fc2dd5cf46e0ecfc59cf Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Mon, 9 Apr 2018 16:42:29 -0500 Subject: [PATCH 064/512] TEZ-3913. Precommit build fails to post to JIRA (Jason Lowe via jeagles) --- build-tools/test-patch.sh | 22 +++++++++++++++++++--- 1 file changed, 19 insertions(+), 3 deletions(-) diff --git a/build-tools/test-patch.sh b/build-tools/test-patch.sh index e1fa0fb00d..9b457af221 100755 --- a/build-tools/test-patch.sh +++ b/build-tools/test-patch.sh @@ -38,6 +38,8 @@ GREP=${GREP:-grep} PATCH=${PATCH:-patch} DIFF=${DIFF:-diff} JIRACLI=${JIRA:-jira} +SED=${SED:-sed} +CURL=${CURL:-curl} FINDBUGS_HOME=${FINDBUGS_HOME} ############################################################################### @@ -702,10 +704,24 @@ $comment" echo "======================================================================" echo "" echo "" + + # RESTify the comment + jsoncomment=$(echo "$comment" \ + | ${SED} -e 's,\\,\\\\,g' \ + -e 's,\",\\\",g' \ + -e 's,$,\\r\\n,g' \ + | tr -d '\n') + jsoncomment='{"body":"'"$jsoncomment"'"}' + ### Update Jira with a comment - export USER=hudson - $JIRACLI -s https://issues.apache.org/jira -a addcomment -u tezqa -p $JIRA_PASSWD --comment "$comment" --issue $defect - $JIRACLI -s https://issues.apache.org/jira -a logout -u tezqa -p $JIRA_PASSWD + ${CURL} -X POST \ + -H "Accept: application/json" \ + -H "Content-Type: application/json" \ + -u "tezqa:${JIRA_PASSWD}" \ + -d "$jsoncomment" \ + --silent --location \ + "https://issues.apache.org/jira/rest/api/2/issue/${defect}/comment" \ + >/dev/null fi } From 24b872a7fc7a9bca11cbf4b5da80198386485547 Mon Sep 17 00:00:00 2001 From: Harish JP Date: Fri, 20 Apr 2018 13:41:37 +0530 Subject: [PATCH 065/512] TEZ-3915. Create protobuf based history event logger. (Harish Jaiprakash, reviewed by Gunther Hagleitner) --- pom.xml | 2 +- .../apache/tez/dag/api/TezConfiguration.java | 21 + tez-dist/pom.xml | 10 + tez-plugins/pom.xml | 2 + .../findbugs-exclude.xml | 50 ++ .../tez-protobuf-history-plugin/pom.xml | 92 +++ .../logging/proto/DagManifesFileScanner.java | 150 ++++ .../logging/proto/DatePartitionedLogger.java | 167 ++++ .../proto/HistoryEventProtoConverter.java | 498 ++++++++++++ .../proto/ProtoHistoryLoggingService.java | 211 ++++++ .../logging/proto/ProtoMessageReader.java | 66 ++ .../logging/proto/ProtoMessageWritable.java | 101 +++ .../logging/proto/ProtoMessageWriter.java | 71 ++ .../logging/proto/TezProtoLoggers.java | 64 ++ .../src/main/proto/HistoryLogger.proto | 49 ++ .../proto/TestDagManifestFileScanner.java | 118 +++ .../proto/TestHistoryEventProtoConverter.java | 716 ++++++++++++++++++ .../proto/TestProtoHistoryLoggingService.java | 195 +++++ 18 files changed, 2582 insertions(+), 1 deletion(-) create mode 100644 tez-plugins/tez-protobuf-history-plugin/findbugs-exclude.xml create mode 100644 tez-plugins/tez-protobuf-history-plugin/pom.xml create mode 100644 tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DagManifesFileScanner.java create mode 100644 tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DatePartitionedLogger.java create mode 100644 tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java create mode 100644 tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java create mode 100644 tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageReader.java create mode 100644 tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWritable.java create mode 100644 tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWriter.java create mode 100644 tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/TezProtoLoggers.java create mode 100644 tez-plugins/tez-protobuf-history-plugin/src/main/proto/HistoryLogger.proto create mode 100644 tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestDagManifestFileScanner.java create mode 100644 tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestHistoryEventProtoConverter.java create mode 100644 tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestProtoHistoryLoggingService.java diff --git a/pom.xml b/pom.xml index 34240cf97c..16745f0d57 100644 --- a/pom.xml +++ b/pom.xml @@ -37,7 +37,7 @@ true ${user.home}/clover.license - 2.7.0 + 2.7.2 9.3.22.v20171030 3.6.2.Final 0.13.0 diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 6d3050d745..243f278127 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -1438,6 +1438,27 @@ public TezConfiguration(boolean loadDefaults) { TEZ_PREFIX + "simple.history.max.errors"; public static final int TEZ_SIMPLE_HISTORY_LOGGING_MAX_ERRORS_DEFAULT = 10; + /** + * String value. The base directory into which history data will be written when proto history + * logging service is used for {@link TezConfiguration#TEZ_HISTORY_LOGGING_SERVICE_CLASS}. + * If this is not set, then logging is disabled for ProtoHistoryLoggingService. + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty + public static final String TEZ_HISTORY_LOGGING_PROTO_BASE_DIR = + TEZ_PREFIX + "history.logging.proto-base-dir"; + + /** + * Long value. The amount of time in seconds to wait to ensure all events for a day is synced + * to disk. This should be maximum time variation b/w machines + maximum time to sync file + * content and metadata. + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty(type="long") + public static final String TEZ_HISTORY_LOGGING_PROTO_SYNC_WINDOWN_SECS = + TEZ_PREFIX + "history.logging.proto-sync-window-secs"; + public static final long TEZ_HISTORY_LOGGING_PROTO_SYNC_WINDOWN_SECS_DEFAULT = 60L; + /** * Int value. Time, in milliseconds, to wait while flushing YARN ATS data during shutdown. * Expert level setting. diff --git a/tez-dist/pom.xml b/tez-dist/pom.xml index 854a548a31..9447fe748d 100644 --- a/tez-dist/pom.xml +++ b/tez-dist/pom.xml @@ -67,6 +67,11 @@ tez-yarn-timeline-history-with-acls ${project.version} + + org.apache.tez + tez-protobuf-history-plugin + ${project.version} + org.apache.tez hadoop-shim-2.7 @@ -95,6 +100,11 @@ tez-yarn-timeline-cache-plugin ${project.version} + + org.apache.tez + tez-protobuf-history-plugin + ${project.version} + org.apache.tez hadoop-shim-2.8 diff --git a/tez-plugins/pom.xml b/tez-plugins/pom.xml index fb0691a0a6..f90cae733a 100644 --- a/tez-plugins/pom.xml +++ b/tez-plugins/pom.xml @@ -35,6 +35,7 @@ + tez-protobuf-history-plugin tez-yarn-timeline-history tez-yarn-timeline-history-with-acls tez-history-parser @@ -47,6 +48,7 @@ false + tez-protobuf-history-plugin tez-yarn-timeline-history tez-yarn-timeline-history-with-acls tez-yarn-timeline-cache-plugin diff --git a/tez-plugins/tez-protobuf-history-plugin/findbugs-exclude.xml b/tez-plugins/tez-protobuf-history-plugin/findbugs-exclude.xml new file mode 100644 index 0000000000..c91265d650 --- /dev/null +++ b/tez-plugins/tez-protobuf-history-plugin/findbugs-exclude.xml @@ -0,0 +1,50 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/tez-plugins/tez-protobuf-history-plugin/pom.xml b/tez-plugins/tez-protobuf-history-plugin/pom.xml new file mode 100644 index 0000000000..880aca952b --- /dev/null +++ b/tez-plugins/tez-protobuf-history-plugin/pom.xml @@ -0,0 +1,92 @@ + + + + 4.0.0 + + org.apache.tez + tez-plugins + 0.9.2-SNAPSHOT + + tez-protobuf-history-plugin + + + + org.apache.tez + tez-common + + + org.apache.tez + tez-dag + + + org.apache.hadoop + hadoop-common + + + com.google.protobuf + protobuf-java + + + junit + junit + test + + + org.mockito + mockito-all + test + + + + + + + org.apache.rat + apache-rat-plugin + + + org.apache.hadoop + hadoop-maven-plugins + + + compile-protoc + generate-sources + + protoc + + + ${protobuf.version} + ${protoc.path} + + ${basedir}/src/main/proto + + + ${basedir}/src/main/proto + + HistoryLogger.proto + + + ${project.build.directory}/generated-sources/java + + + + + + + + + diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DagManifesFileScanner.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DagManifesFileScanner.java new file mode 100644 index 0000000000..c8ea02f188 --- /dev/null +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DagManifesFileScanner.java @@ -0,0 +1,150 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.dag.history.logging.proto; + +import java.io.Closeable; +import java.io.IOException; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.Path; +import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.history.logging.proto.HistoryLoggerProtos.ManifestEntryProto; +import org.codehaus.jackson.map.ObjectMapper; + +/** + * Helper class to scan all the dag manifest files to get manifest entries. + */ +public class DagManifesFileScanner implements Closeable { + private static final int OFFSET_VERSION = 1; + + private final ObjectMapper mapper = new ObjectMapper(); + private final DatePartitionedLogger manifestLogger; + private final long syncTime; + + private String scanDir; + private Map offsets; + private List newFiles; + + private ProtoMessageReader reader; + + public DagManifesFileScanner(DatePartitionedLogger manifestLogger) { + this.manifestLogger = manifestLogger; + this.syncTime = manifestLogger.getConfig().getLong( + TezConfiguration.TEZ_HISTORY_LOGGING_PROTO_SYNC_WINDOWN_SECS, + TezConfiguration.TEZ_HISTORY_LOGGING_PROTO_SYNC_WINDOWN_SECS_DEFAULT); + this.setOffset(LocalDate.ofEpochDay(0)); + } + + // All public to simplify json conversion. + public static class DagManifestOffset { + public int version; + public String scanDir; + public Map offsets; + } + + public void setOffset(String offset) { + try { + DagManifestOffset dagOffset = mapper.readValue(offset, DagManifestOffset.class); + if (dagOffset.version != OFFSET_VERSION) { + throw new IllegalArgumentException("Version mismatch: " + dagOffset.version); + } + this.scanDir = dagOffset.scanDir; + this.offsets = dagOffset.offsets; + this.newFiles = new ArrayList<>(); + } catch (IOException e) { + throw new IllegalArgumentException("Invalid offset", e); + } + } + + public void setOffset(LocalDate date) { + this.scanDir = manifestLogger.getDirForDate(date); + this.offsets = new HashMap<>(); + this.newFiles = new ArrayList<>(); + } + + public String getOffset() { + try { + DagManifestOffset offset = new DagManifestOffset(); + offset.version = OFFSET_VERSION; + offset.scanDir = scanDir; + offset.offsets = offsets; + return mapper.writeValueAsString(offset); + } catch (IOException e) { + throw new RuntimeException("Unexpected exception while converting to json.", e); + } + } + + public ManifestEntryProto getNext() throws IOException { + while (true) { + if (reader != null) { + ManifestEntryProto evt = reader.readEvent(); + if (evt != null) { + offsets.put(reader.getFilePath().getName(), reader.getOffset()); + return evt; + } else { + IOUtils.closeQuietly(reader); + reader = null; + } + } + if (!newFiles.isEmpty()) { + this.reader = manifestLogger.getReader(newFiles.remove(0)); + } else { + if (!loadMore()) { + return null; + } + } + } + } + + @Override + public void close() throws IOException { + if (reader != null) { + reader.close(); + reader = null; + } + } + + private boolean loadMore() throws IOException { + newFiles = manifestLogger.scanForChangedFiles(scanDir, offsets); + while (newFiles.isEmpty()) { + LocalDateTime utcNow = manifestLogger.getNow(); + if (utcNow.getHour() * 3600 + utcNow.getMinute() * 60 + utcNow.getSecond() < syncTime) { + // We are in the delay window for today, do not advance date if we are moving from + // yesterday. + String yesterDir = manifestLogger.getDirForDate(utcNow.toLocalDate().minusDays(1)); + if (yesterDir.equals(scanDir)) { + return false; + } + } + String nextDir = manifestLogger.getNextDirectory(scanDir); + if (nextDir == null) { + return false; + } + scanDir = nextDir; + offsets = new HashMap<>(); + newFiles = manifestLogger.scanForChangedFiles(scanDir, offsets); + } + return true; + } +} diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DatePartitionedLogger.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DatePartitionedLogger.java new file mode 100644 index 0000000000..8f89b2e37b --- /dev/null +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DatePartitionedLogger.java @@ -0,0 +1,167 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.history.logging.proto; + +import java.io.IOException; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.yarn.util.Clock; + +import com.google.protobuf.MessageLite; +import com.google.protobuf.Parser; + +/** + * Class to create proto reader and writer for a date partitioned directory structure. + * + * @param The proto message type. + */ +public class DatePartitionedLogger { + // Everyone has permission to write, but with sticky set so that delete is restricted. + // This is required, since the path is same for all users and everyone writes into it. + private static final FsPermission DIR_PERMISSION = FsPermission.createImmutable((short)01777); + + private final Parser parser; + private final Path basePath; + private final Configuration conf; + private final Clock clock; + private final FileSystem fileSystem; + + public DatePartitionedLogger(Parser parser, Path baseDir, Configuration conf, Clock clock) + throws IOException { + this.conf = conf; + this.clock = clock; + this.parser = parser; + this.fileSystem = baseDir.getFileSystem(conf); + if (!fileSystem.exists(baseDir)) { + fileSystem.mkdirs(baseDir); + fileSystem.setPermission(baseDir, DIR_PERMISSION); + } + this.basePath = fileSystem.resolvePath(baseDir); + } + + /** + * Creates a writer for the given fileName, with date as today. + */ + public ProtoMessageWriter getWriter(String fileName) throws IOException { + Path filePath = getPathForDate(getNow().toLocalDate(), fileName); + return new ProtoMessageWriter<>(conf, filePath, parser); + } + + /** + * Creates a reader for the given filePath, no validation is done. + */ + public ProtoMessageReader getReader(Path filePath) throws IOException { + return new ProtoMessageReader<>(conf, filePath, parser); + } + + /** + * Create a path for the given date and fileName. This can be used to create a reader. + */ + public Path getPathForDate(LocalDate date, String fileName) throws IOException { + Path path = new Path(basePath, getDirForDate(date)); + if (!fileSystem.exists(path)) { + fileSystem.mkdirs(path); + fileSystem.setPermission(path, DIR_PERMISSION); + } + return new Path(path, fileName); + } + + /** + * Extract the date from the directory name, this should be a directory created by this class. + */ + public LocalDate getDateFromDir(String dirName) { + if (!dirName.startsWith("date=")) { + throw new IllegalArgumentException("Invalid directory: "+ dirName); + } + return LocalDate.parse(dirName.substring(5), DateTimeFormatter.ISO_LOCAL_DATE); + } + + /** + * Returns the directory name for a given date. + */ + public String getDirForDate(LocalDate date) { + return "date=" + DateTimeFormatter.ISO_LOCAL_DATE.format(date); + } + + /** + * Find next available directory, after the given directory. + */ + public String getNextDirectory(String currentDir) throws IOException { + // Fast check, if the next day directory exists return it. + String nextDate = getDirForDate(getDateFromDir(currentDir).plusDays(1)); + if (fileSystem.exists(new Path(basePath, nextDate))) { + return nextDate; + } + // Have to scan the directory to find min date greater than currentDir. + String dirName = null; + for (FileStatus status : fileSystem.listStatus(basePath)) { + String name = status.getPath().getName(); + // String comparison is good enough, since its of form date=yyyy-MM-dd + if (name.compareTo(currentDir) > 0 && (dirName == null || name.compareTo(dirName) < 0)) { + dirName = name; + } + } + return dirName; + } + + /** + * Returns new or changed files in the given directory. The offsets are used to find + * changed files. + */ + public List scanForChangedFiles(String subDir, Map currentOffsets) + throws IOException { + Path dirPath = new Path(basePath, subDir); + List newFiles = new ArrayList<>(); + if (!fileSystem.exists(dirPath)) { + return newFiles; + } + for (FileStatus status : fileSystem.listStatus(dirPath)) { + String fileName = status.getPath().getName(); + Long offset = currentOffsets.get(fileName); + // If the offset was never added or offset < fileSize. + if (offset == null || offset < status.getLen()) { + newFiles.add(new Path(dirPath, fileName)); + } + } + return newFiles; + } + + /** + * Returns the current time, using the underlying clock in UTC time. + */ + public LocalDateTime getNow() { + // Use UTC date to ensure reader date is same on all timezones. + return LocalDateTime.ofEpochSecond(clock.getTime() / 1000, 0, ZoneOffset.UTC); + } + + public Configuration getConfig() { + return conf; + } +} diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java new file mode 100644 index 0000000000..44dccb6d29 --- /dev/null +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java @@ -0,0 +1,498 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.history.logging.proto; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; +import java.util.TreeMap; + +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.tez.common.ATSConstants; +import org.apache.tez.dag.api.EdgeProperty; +import org.apache.tez.dag.api.TezUncheckedException; +import org.apache.tez.dag.api.oldrecords.TaskAttemptState; +import org.apache.tez.dag.api.records.DAGProtos.CallerContextProto; +import org.apache.tez.dag.app.web.AMWebController; +import org.apache.tez.dag.history.HistoryEvent; +import org.apache.tez.dag.history.events.AMLaunchedEvent; +import org.apache.tez.dag.history.events.AMStartedEvent; +import org.apache.tez.dag.history.events.AppLaunchedEvent; +import org.apache.tez.dag.history.events.ContainerLaunchedEvent; +import org.apache.tez.dag.history.events.ContainerStoppedEvent; +import org.apache.tez.dag.history.events.DAGFinishedEvent; +import org.apache.tez.dag.history.events.DAGInitializedEvent; +import org.apache.tez.dag.history.events.DAGRecoveredEvent; +import org.apache.tez.dag.history.events.DAGStartedEvent; +import org.apache.tez.dag.history.events.DAGSubmittedEvent; +import org.apache.tez.dag.history.events.TaskAttemptFinishedEvent; +import org.apache.tez.dag.history.events.TaskAttemptStartedEvent; +import org.apache.tez.dag.history.events.TaskFinishedEvent; +import org.apache.tez.dag.history.events.TaskStartedEvent; +import org.apache.tez.dag.history.events.VertexConfigurationDoneEvent; +import org.apache.tez.dag.history.events.VertexFinishedEvent; +import org.apache.tez.dag.history.events.VertexInitializedEvent; +import org.apache.tez.dag.history.events.VertexStartedEvent; +import org.apache.tez.dag.history.logging.proto.HistoryLoggerProtos.HistoryEventProto; +import org.apache.tez.dag.history.logging.proto.HistoryLoggerProtos.KVPair; +import org.apache.tez.dag.history.utils.DAGUtils; +import org.apache.tez.dag.records.TezDAGID; +import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.dag.records.TezTaskID; +import org.apache.tez.dag.records.TezVertexID; +import org.codehaus.jackson.map.ObjectMapper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Convert history event into HistoryEventProto message. + */ +public class HistoryEventProtoConverter { + private static final Logger log = + LoggerFactory.getLogger(HistoryEventProtoConverter.class); + + private final ObjectMapper mapper = new ObjectMapper(); + + /** + * Convert a given history event to HistoryEventProto message. + */ + public HistoryEventProto convert(HistoryEvent historyEvent) { + validateEvent(historyEvent); + switch (historyEvent.getEventType()) { + case APP_LAUNCHED: + return convertAppLaunchedEvent((AppLaunchedEvent) historyEvent); + case AM_LAUNCHED: + return convertAMLaunchedEvent((AMLaunchedEvent) historyEvent); + case AM_STARTED: + return convertAMStartedEvent((AMStartedEvent) historyEvent); + case CONTAINER_LAUNCHED: + return convertContainerLaunchedEvent((ContainerLaunchedEvent) historyEvent); + case CONTAINER_STOPPED: + return convertContainerStoppedEvent((ContainerStoppedEvent) historyEvent); + case DAG_SUBMITTED: + return convertDAGSubmittedEvent((DAGSubmittedEvent) historyEvent); + case DAG_INITIALIZED: + return convertDAGInitializedEvent((DAGInitializedEvent) historyEvent); + case DAG_STARTED: + return convertDAGStartedEvent((DAGStartedEvent) historyEvent); + case DAG_FINISHED: + return convertDAGFinishedEvent((DAGFinishedEvent) historyEvent); + case VERTEX_INITIALIZED: + return convertVertexInitializedEvent((VertexInitializedEvent) historyEvent); + case VERTEX_STARTED: + return convertVertexStartedEvent((VertexStartedEvent) historyEvent); + case VERTEX_FINISHED: + return convertVertexFinishedEvent((VertexFinishedEvent) historyEvent); + case TASK_STARTED: + return convertTaskStartedEvent((TaskStartedEvent) historyEvent); + case TASK_FINISHED: + return convertTaskFinishedEvent((TaskFinishedEvent) historyEvent); + case TASK_ATTEMPT_STARTED: + return convertTaskAttemptStartedEvent((TaskAttemptStartedEvent) historyEvent); + case TASK_ATTEMPT_FINISHED: + return convertTaskAttemptFinishedEvent((TaskAttemptFinishedEvent) historyEvent); + case VERTEX_CONFIGURE_DONE: + return convertVertexReconfigureDoneEvent((VertexConfigurationDoneEvent) historyEvent); + case DAG_RECOVERED: + return convertDAGRecoveredEvent((DAGRecoveredEvent) historyEvent); + case VERTEX_COMMIT_STARTED: + case VERTEX_GROUP_COMMIT_STARTED: + case VERTEX_GROUP_COMMIT_FINISHED: + case DAG_COMMIT_STARTED: + case DAG_KILL_REQUEST: + throw new UnsupportedOperationException("Invalid Event, does not support history, eventType=" + + historyEvent.getEventType()); + // Do not add default, if a new event type is added, we'll get a warning for the + // switch. + } + throw new UnsupportedOperationException( + "Unhandled Event, eventType=" + historyEvent.getEventType()); + } + + private void validateEvent(HistoryEvent event) { + if (!event.isHistoryEvent()) { + throw new UnsupportedOperationException( + "Invalid Event, does not support history" + ", eventType=" + event.getEventType()); + } + } + + private HistoryEventProto.Builder makeBuilderForEvent(HistoryEvent event, long time, + TezDAGID dagId, ApplicationId appId, ApplicationAttemptId appAttemptId, TezVertexID vertexId, + TezTaskID taskId, TezTaskAttemptID taskAttemptId, String user) { + HistoryEventProto.Builder builder = HistoryEventProto.newBuilder(); + builder.setEventType(event.getEventType().name()); + builder.setEventTime(time); + if (taskAttemptId != null) { + builder.setTaskAttemptId(taskAttemptId.toString()); + taskId = taskAttemptId.getTaskID(); + } + if (taskId != null) { + builder.setTaskId(taskId.toString()); + vertexId = taskId.getVertexID(); + } + if (vertexId != null) { + builder.setVertexId(vertexId.toString()); + dagId = vertexId.getDAGId(); + } + if (dagId != null) { + builder.setDagId(dagId.toString()); + if (appId == null) { + appId = dagId.getApplicationId(); + } + } + if (appAttemptId != null) { + builder.setAppAttemptId(appAttemptId.toString()); + if (appId == null) { + appId = appAttemptId.getApplicationId(); + } + } + if (appId != null) { + builder.setAppId(appId.toString()); + } + if (user != null) { + builder.setUser(user); + } + return builder; + } + + private void addEventData(HistoryEventProto.Builder builder, String key, String value) { + if (value == null) { + return; + } + builder.addEventData(KVPair.newBuilder().setKey(key).setValue(value)); + } + + private void addEventData(HistoryEventProto.Builder builder, String key, Number value) { + builder.addEventData(KVPair.newBuilder().setKey(key).setValue(value.toString())); + } + + private void addEventData(HistoryEventProto.Builder builder, String key, + Map value) { + try { + builder.addEventData( + KVPair.newBuilder().setKey(key).setValue(mapper.writeValueAsString(value))); + } catch (IOException e) { + log.error("Error converting value for key {} to json: ", key, e); + } + } + + private HistoryEventProto convertAppLaunchedEvent(AppLaunchedEvent event) { + HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getLaunchTime(), null, + event.getApplicationId(), null, null, null, null, event.getUser()); + // This is ok as long as we do not modify the underlying map. + @SuppressWarnings({ "unchecked", "rawtypes" }) + Map confMap = (Map)DAGUtils.convertConfigurationToATSMap(event.getConf()); + addEventData(builder, ATSConstants.CONFIG, confMap); + if (event.getVersion() != null) { + addEventData(builder, ATSConstants.TEZ_VERSION, + DAGUtils.convertTezVersionToATSMap(event.getVersion())); + } + addEventData(builder, ATSConstants.DAG_AM_WEB_SERVICE_VERSION, AMWebController.VERSION); + return builder.build(); + } + + private HistoryEventProto convertAMLaunchedEvent(AMLaunchedEvent event) { + HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getLaunchTime(), null, + null, event.getApplicationAttemptId(), null, null, null, event.getUser()); + addEventData(builder, ATSConstants.APP_SUBMIT_TIME, event.getAppSubmitTime()); + return builder.build(); + } + + private HistoryEventProto convertAMStartedEvent(AMStartedEvent event) { + HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getStartTime(), null, + null, event.getApplicationAttemptId(), null, null, null, event.getUser()); + return builder.build(); + } + + private HistoryEventProto convertContainerLaunchedEvent(ContainerLaunchedEvent event) { + HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getLaunchTime(), null, + null, event.getApplicationAttemptId(), null, null, null, null); + addEventData(builder, ATSConstants.CONTAINER_ID, event.getContainerId().toString()); + return builder.build(); + } + + private HistoryEventProto convertContainerStoppedEvent(ContainerStoppedEvent event) { + HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getStoppedTime(), null, + null, event.getApplicationAttemptId(), null, null, null, null); + addEventData(builder, ATSConstants.CONTAINER_ID, event.getContainerId().toString()); + addEventData(builder, ATSConstants.EXIT_STATUS, event.getExitStatus()); + addEventData(builder, ATSConstants.FINISH_TIME, event.getStoppedTime()); + return builder.build(); + } + + private HistoryEventProto convertDAGSubmittedEvent(DAGSubmittedEvent event) { + HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getSubmitTime(), + event.getDagID(), null, event.getApplicationAttemptId(), null, null, null, + event.getUser()); + addEventData(builder, ATSConstants.DAG_NAME, event.getDAGName()); + if (event.getDAGPlan().hasCallerContext() && + event.getDAGPlan().getCallerContext().hasCallerId()) { + CallerContextProto callerContext = event.getDagPlan().getCallerContext(); + addEventData(builder, ATSConstants.CALLER_CONTEXT_ID, callerContext.getCallerId()); + addEventData(builder, ATSConstants.CALLER_CONTEXT_TYPE, callerContext.getCallerType()); + addEventData(builder, ATSConstants.CALLER_CONTEXT, callerContext.getContext()); + } + if (event.getQueueName() != null) { + addEventData(builder, ATSConstants.DAG_QUEUE_NAME, event.getQueueName()); + } + addEventData(builder, ATSConstants.DAG_AM_WEB_SERVICE_VERSION, AMWebController.VERSION); + addEventData(builder, ATSConstants.IN_PROGRESS_LOGS_URL + "_" + + event.getApplicationAttemptId().getAttemptId(), event.getContainerLogs()); + try { + addEventData(builder, ATSConstants.DAG_PLAN, + DAGUtils.convertDAGPlanToATSMap(event.getDAGPlan())); + } catch (IOException e) { + throw new TezUncheckedException(e); + } + return builder.build(); + } + + private HistoryEventProto convertDAGInitializedEvent(DAGInitializedEvent event) { + HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getInitTime(), + event.getDagID(), null, null, null, null, null, event.getUser()); + addEventData(builder, ATSConstants.DAG_NAME, event.getDagName()); + + if (event.getVertexNameIDMap() != null) { + Map nameIdStrMap = new TreeMap(); + for (Entry entry : event.getVertexNameIDMap().entrySet()) { + nameIdStrMap.put(entry.getKey(), entry.getValue().toString()); + } + addEventData(builder, ATSConstants.VERTEX_NAME_ID_MAPPING, nameIdStrMap); + } + return builder.build(); + } + + private HistoryEventProto convertDAGStartedEvent(DAGStartedEvent event) { + HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getStartTime(), + event.getDagID(), null, null, null, null, null, event.getUser()); + + addEventData(builder, ATSConstants.DAG_NAME, event.getDagName()); + addEventData(builder, ATSConstants.STATUS, event.getDagState().name()); + + return builder.build(); + } + + private HistoryEventProto convertDAGFinishedEvent(DAGFinishedEvent event) { + HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getFinishTime(), + event.getDagID(), null, event.getApplicationAttemptId(), null, null, null, + event.getUser()); + addEventData(builder, ATSConstants.DAG_NAME, event.getDagName()); + if (event.getDAGPlan().hasCallerContext()) { + if (event.getDAGPlan().getCallerContext().hasCallerType()) { + addEventData(builder, ATSConstants.CALLER_CONTEXT_TYPE, + event.getDAGPlan().getCallerContext().getCallerType()); + } + if (event.getDAGPlan().getCallerContext().hasCallerId()) { + addEventData(builder, ATSConstants.CALLER_CONTEXT_ID, + event.getDAGPlan().getCallerContext().getCallerId()); + } + } + addEventData(builder, ATSConstants.START_TIME, event.getStartTime()); + addEventData(builder, ATSConstants.TIME_TAKEN, (event.getFinishTime() - event.getStartTime())); + addEventData(builder, ATSConstants.STATUS, event.getState().name()); + addEventData(builder, ATSConstants.DIAGNOSTICS, event.getDiagnostics()); + addEventData(builder, ATSConstants.COMPLETION_APPLICATION_ATTEMPT_ID, + event.getApplicationAttemptId().toString()); + addEventData(builder, ATSConstants.COUNTERS, + DAGUtils.convertCountersToATSMap(event.getTezCounters())); + Map dagTaskStats = event.getDagTaskStats(); + if (dagTaskStats != null) { + for (Entry entry : dagTaskStats.entrySet()) { + addEventData(builder, entry.getKey(), entry.getValue()); + } + } + return builder.build(); + } + + private HistoryEventProto convertTaskAttemptStartedEvent(TaskAttemptStartedEvent event) { + HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getStartTime(), + null, null, null, null, null, event.getTaskAttemptID(), null); + if (event.getInProgressLogsUrl() != null) { + addEventData(builder, ATSConstants.IN_PROGRESS_LOGS_URL, event.getInProgressLogsUrl()); + } + if (event.getCompletedLogsUrl() != null) { + addEventData(builder, ATSConstants.COMPLETED_LOGS_URL, event.getCompletedLogsUrl()); + } + addEventData(builder, ATSConstants.NODE_ID, event.getNodeId().toString()); + addEventData(builder, ATSConstants.NODE_HTTP_ADDRESS, event.getNodeHttpAddress()); + addEventData(builder, ATSConstants.CONTAINER_ID, event.getContainerId().toString()); + addEventData(builder, ATSConstants.STATUS, TaskAttemptState.RUNNING.name()); + + return builder.build(); + } + + private HistoryEventProto convertTaskAttemptFinishedEvent(TaskAttemptFinishedEvent event) { + HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getFinishTime(), + null, null, null, null, null, event.getTaskAttemptID(), null); + + addEventData(builder, ATSConstants.STATUS, event.getState().name()); + if (event.getTaskFailureType() != null) { + addEventData(builder, ATSConstants.TASK_FAILURE_TYPE, event.getTaskFailureType().name()); + } + + addEventData(builder, ATSConstants.CREATION_TIME, event.getCreationTime()); + addEventData(builder, ATSConstants.ALLOCATION_TIME, event.getAllocationTime()); + addEventData(builder, ATSConstants.START_TIME, event.getStartTime()); + if (event.getCreationCausalTA() != null) { + addEventData(builder, ATSConstants.CREATION_CAUSAL_ATTEMPT, + event.getCreationCausalTA().toString()); + } + addEventData(builder, ATSConstants.TIME_TAKEN, (event.getFinishTime() - event.getStartTime())); + if (event.getTaskAttemptError() != null) { + addEventData(builder, ATSConstants.TASK_ATTEMPT_ERROR_ENUM, + event.getTaskAttemptError().name()); + } + addEventData(builder, ATSConstants.DIAGNOSTICS, event.getDiagnostics()); + addEventData(builder, ATSConstants.COUNTERS, + DAGUtils.convertCountersToATSMap(event.getCounters())); + if (event.getDataEvents() != null && !event.getDataEvents().isEmpty()) { + addEventData(builder, ATSConstants.LAST_DATA_EVENTS, + DAGUtils.convertDataEventDependecyInfoToATS(event.getDataEvents())); + } + if (event.getNodeId() != null) { + addEventData(builder, ATSConstants.NODE_ID, event.getNodeId().toString()); + } + if (event.getContainerId() != null) { + addEventData(builder, ATSConstants.CONTAINER_ID, event.getContainerId().toString()); + } + if (event.getInProgressLogsUrl() != null) { + addEventData(builder, ATSConstants.IN_PROGRESS_LOGS_URL, event.getInProgressLogsUrl()); + } + if (event.getCompletedLogsUrl() != null) { + addEventData(builder, ATSConstants.COMPLETED_LOGS_URL, event.getCompletedLogsUrl()); + } + if (event.getNodeHttpAddress() != null) { + addEventData(builder, ATSConstants.NODE_HTTP_ADDRESS, event.getNodeHttpAddress()); + } + + return builder.build(); + } + + private HistoryEventProto convertTaskFinishedEvent(TaskFinishedEvent event) { + HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getFinishTime(), + null, null, null, null, event.getTaskID(), null, null); + + addEventData(builder, ATSConstants.TIME_TAKEN, (event.getFinishTime() - event.getStartTime())); + addEventData(builder, ATSConstants.STATUS, event.getState().name()); + addEventData(builder, ATSConstants.NUM_FAILED_TASKS_ATTEMPTS, event.getNumFailedAttempts()); + if (event.getSuccessfulAttemptID() != null) { + addEventData(builder, ATSConstants.SUCCESSFUL_ATTEMPT_ID, + event.getSuccessfulAttemptID().toString()); + } + + addEventData(builder, ATSConstants.DIAGNOSTICS, event.getDiagnostics()); + addEventData(builder, ATSConstants.COUNTERS, + DAGUtils.convertCountersToATSMap(event.getTezCounters())); + + return builder.build(); + } + + private HistoryEventProto convertTaskStartedEvent(TaskStartedEvent event) { + HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getStartTime(), + null, null, null, null, event.getTaskID(), null, null); + + addEventData(builder, ATSConstants.SCHEDULED_TIME, event.getScheduledTime()); + addEventData(builder, ATSConstants.STATUS, event.getState().name()); + + return builder.build(); + } + + private HistoryEventProto convertVertexFinishedEvent(VertexFinishedEvent event) { + HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getFinishTime(), + null, null, null, event.getVertexID(), null, null, null); + + addEventData(builder, ATSConstants.STATUS, event.getState().name()); + addEventData(builder, ATSConstants.VERTEX_NAME, event.getVertexName()); + addEventData(builder, ATSConstants.TIME_TAKEN, (event.getFinishTime() - event.getStartTime())); + addEventData(builder, ATSConstants.DIAGNOSTICS, event.getDiagnostics()); + addEventData(builder, ATSConstants.COUNTERS, + DAGUtils.convertCountersToATSMap(event.getTezCounters())); + addEventData(builder, ATSConstants.STATS, + DAGUtils.convertVertexStatsToATSMap(event.getVertexStats())); + if (event.getServicePluginInfo() != null) { + addEventData(builder, ATSConstants.SERVICE_PLUGIN, + DAGUtils.convertServicePluginToATSMap(event.getServicePluginInfo())); + } + + final Map vertexTaskStats = event.getVertexTaskStats(); + if (vertexTaskStats != null) { + for (Entry entry : vertexTaskStats.entrySet()) { + addEventData(builder, entry.getKey(), entry.getValue()); + } + } + + return builder.build(); + } + + private HistoryEventProto convertVertexInitializedEvent(VertexInitializedEvent event) { + HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getInitedTime(), + null, null, null, event.getVertexID(), null, null, null); + addEventData(builder, ATSConstants.VERTEX_NAME, event.getVertexName()); + addEventData(builder, ATSConstants.INIT_REQUESTED_TIME, event.getInitRequestedTime()); + addEventData(builder, ATSConstants.NUM_TASKS, event.getNumTasks()); + addEventData(builder, ATSConstants.PROCESSOR_CLASS_NAME, event.getProcessorName()); + if (event.getServicePluginInfo() != null) { + addEventData(builder, ATSConstants.SERVICE_PLUGIN, + DAGUtils.convertServicePluginToATSMap(event.getServicePluginInfo())); + } + + return builder.build(); + } + + private HistoryEventProto convertVertexStartedEvent(VertexStartedEvent event) { + HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getStartTime(), + null, null, null, event.getVertexID(), null, null, null); + addEventData(builder, ATSConstants.START_REQUESTED_TIME, event.getStartRequestedTime()); + addEventData(builder, ATSConstants.STATUS, event.getVertexState().name()); + return builder.build(); + } + + private HistoryEventProto convertVertexReconfigureDoneEvent(VertexConfigurationDoneEvent event) { + HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getReconfigureDoneTime(), + null, null, null, event.getVertexID(), null, null, null); + if (event.getSourceEdgeProperties() != null && !event.getSourceEdgeProperties().isEmpty()) { + Map updatedEdgeManagers = new HashMap(); + for (Entry entry : event.getSourceEdgeProperties().entrySet()) { + updatedEdgeManagers.put(entry.getKey(), DAGUtils.convertEdgeProperty(entry.getValue())); + } + addEventData(builder, ATSConstants.UPDATED_EDGE_MANAGERS, updatedEdgeManagers); + } + addEventData(builder, ATSConstants.NUM_TASKS, event.getNumTasks()); + return builder.build(); + } + + private HistoryEventProto convertDAGRecoveredEvent(DAGRecoveredEvent event) { + HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getRecoveredTime(), + event.getDagID(), null, event.getApplicationAttemptId(), null, null, null, + event.getUser()); + addEventData(builder, ATSConstants.DAG_NAME, event.getDagName()); + if (event.getRecoveredDagState() != null) { + addEventData(builder, ATSConstants.DAG_STATE, event.getRecoveredDagState().name()); + } + if (event.getRecoveryFailureReason() != null) { + addEventData(builder, ATSConstants.RECOVERY_FAILURE_REASON, + event.getRecoveryFailureReason()); + } + addEventData(builder, ATSConstants.IN_PROGRESS_LOGS_URL + "_" + + event.getApplicationAttemptId().getAttemptId(), event.getContainerLogs()); + return builder.build(); + } +} diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java new file mode 100644 index 0000000000..60cbda5edd --- /dev/null +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java @@ -0,0 +1,211 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.history.logging.proto; + +import java.io.IOException; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.history.DAGHistoryEvent; +import org.apache.tez.dag.history.HistoryEvent; +import org.apache.tez.dag.history.HistoryEventType; +import org.apache.tez.dag.history.events.DAGFinishedEvent; +import org.apache.tez.dag.history.logging.HistoryLoggingService; +import org.apache.tez.dag.history.logging.proto.HistoryLoggerProtos.HistoryEventProto; +import org.apache.tez.dag.history.logging.proto.HistoryLoggerProtos.ManifestEntryProto; +import org.apache.tez.dag.records.TezDAGID; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Logging service to write history events serialized using protobuf into sequence files. + * This can be used as external tables in hive. Or the reader can be used independently to + * read the data from these files. + */ +public class ProtoHistoryLoggingService extends HistoryLoggingService { + private static final Logger LOG = LoggerFactory.getLogger(ProtoHistoryLoggingService.class); + private final HistoryEventProtoConverter converter = + new HistoryEventProtoConverter(); + private boolean loggingDisabled = false; + + private final LinkedBlockingQueue eventQueue = + new LinkedBlockingQueue(10000); + private Thread eventHandlingThread; + private final AtomicBoolean stopped = new AtomicBoolean(false); + + private TezProtoLoggers loggers; + private ProtoMessageWriter appEventsWriter; + private ProtoMessageWriter dagEventsWriter; + private TezDAGID currentDagId; + private long dagSubmittedEventOffset = -1; + + private String appEventsFile; + private long appLaunchedEventOffset; + + public ProtoHistoryLoggingService() { + super(ProtoHistoryLoggingService.class.getName()); + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + LOG.info("Initing ProtoHistoryLoggingService"); + setConfig(conf); + loggingDisabled = !conf.getBoolean(TezConfiguration.TEZ_AM_HISTORY_LOGGING_ENABLED, + TezConfiguration.TEZ_AM_HISTORY_LOGGING_ENABLED_DEFAULT); + LOG.info("Inited ProtoHistoryLoggingService"); + } + + @Override + protected void serviceStart() throws Exception { + LOG.info("Starting ProtoHistoryLoggingService"); + if (!loggingDisabled) { + loggers = new TezProtoLoggers(); + if (!loggers.setup(getConfig(), appContext.getClock())) { + LOG.warn("Log file location for ProtoHistoryLoggingService not specified, " + + "logging disabled"); + loggingDisabled = true; + return; + } + appEventsWriter = loggers.getAppEventsLogger().getWriter( + appContext.getApplicationAttemptId().toString()); + eventHandlingThread = new Thread(this::loop, "HistoryEventHandlingThread"); + eventHandlingThread.start(); + } + LOG.info("Started ProtoHistoryLoggingService"); + } + + @Override + protected void serviceStop() throws Exception { + LOG.info("Stopping ProtoHistoryLoggingService, eventQueueBacklog=" + eventQueue.size()); + stopped.set(true); + eventHandlingThread.join(); + IOUtils.closeQuietly(appEventsWriter); + IOUtils.closeQuietly(dagEventsWriter); + LOG.info("Stopped ProtoHistoryLoggingService"); + } + + @Override + public void handle(DAGHistoryEvent event) { + if (loggingDisabled || stopped.get()) { + return; + } + try { + eventQueue.add(event); + } catch (IllegalStateException e) { + LOG.error("Queue capacity filled up, ignoring event: " + + event.getHistoryEvent().getEventType()); + if (LOG.isDebugEnabled()) { + LOG.debug("Queue capacity filled up, ignoring event: {}", event.getHistoryEvent()); + } + } + } + + private void loop() { + // Keep looping while the service is not stopped. + // Drain any left over events after the service has been stopped. + while (!stopped.get() || !eventQueue.isEmpty()) { + DAGHistoryEvent evt = null; + try { + evt = eventQueue.poll(100, TimeUnit.MILLISECONDS); + if (evt != null) { + handleEvent(evt); + } + } catch (InterruptedException e) { + LOG.info("EventQueue poll interrupted, ignoring it.", e); + } catch (IOException e) { + TezDAGID dagid = evt.getDagID(); + HistoryEventType type = evt.getHistoryEvent().getEventType(); + // Retry is hard, because there are several places where this exception can happen + // the state will get messed up a lot. + LOG.error("Got exception while handling event {} for dag {}.", type, dagid, e); + } + } + } + + private void handleEvent(DAGHistoryEvent event) throws IOException { + if (loggingDisabled) { + return; + } + HistoryEvent historyEvent = event.getHistoryEvent(); + if (event.getDagID() == null) { + if (historyEvent.getEventType() == HistoryEventType.APP_LAUNCHED) { + appEventsFile = appEventsWriter.getPath().toString(); + appLaunchedEventOffset = appEventsWriter.getOffset(); + } + appEventsWriter.writeProto(converter.convert(historyEvent)); + } else { + HistoryEventType type = historyEvent.getEventType(); + TezDAGID dagId = event.getDagID(); + if (type == HistoryEventType.DAG_FINISHED) { + finishCurrentDag((DAGFinishedEvent)historyEvent); + } else if (type == HistoryEventType.DAG_SUBMITTED) { + finishCurrentDag(null); + currentDagId = dagId; + dagEventsWriter = loggers.getDagEventsLogger().getWriter(dagId.toString()); + dagSubmittedEventOffset = dagEventsWriter.getOffset(); + dagEventsWriter.writeProto(converter.convert(historyEvent)); + } else if (dagEventsWriter != null) { + dagEventsWriter.writeProto(converter.convert(historyEvent)); + } + } + } + + private void finishCurrentDag(DAGFinishedEvent event) throws IOException { + if (dagEventsWriter == null) { + return; + } + ProtoMessageWriter writer = null; + try { + long finishEventOffset = -1; + if (event != null) { + finishEventOffset = dagEventsWriter.getOffset(); + dagEventsWriter.writeProto(converter.convert(event)); + } + // Do not cache this writer, it should be created at the time of writing + writer = loggers.getManifestEventsLogger() + .getWriter(appContext.getApplicationAttemptId().toString()); + ManifestEntryProto.Builder entry = ManifestEntryProto.newBuilder() + .setDagId(currentDagId.toString()) + .setAppId(currentDagId.getApplicationId().toString()) + .setDagSubmittedEventOffset(dagSubmittedEventOffset) + .setDagFinishedEventOffset(finishEventOffset) + .setDagFilePath(dagEventsWriter.getPath().toString()) + .setAppFilePath(appEventsFile) + .setAppLaunchedEventOffset(appLaunchedEventOffset) + .setWriteTime(System.currentTimeMillis()); + if (event != null) { + entry.setDagId(event.getDagID().toString()); + } + writer.writeProto(entry.build()); + appEventsWriter.hflush(); + } finally { + // On an error, cleanup everything this will ensure, we do not use one dag's writer + // into another dag. + IOUtils.closeQuietly(dagEventsWriter); + IOUtils.closeQuietly(writer); + dagEventsWriter = null; + currentDagId = null; + dagSubmittedEventOffset = -1; + } + } +} diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageReader.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageReader.java new file mode 100644 index 0000000000..e5f5e6befa --- /dev/null +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageReader.java @@ -0,0 +1,66 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.history.logging.proto; + +import java.io.Closeable; +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.SequenceFile; + +import com.google.protobuf.MessageLite; +import com.google.protobuf.Parser; + +public class ProtoMessageReader implements Closeable { + private final Path filePath; + private final SequenceFile.Reader reader; + private final ProtoMessageWritable writable; + + ProtoMessageReader(Configuration conf, Path filePath, Parser parser) throws IOException { + this.filePath = filePath; + this.reader = new SequenceFile.Reader(conf, SequenceFile.Reader.file(filePath)); + this.writable = new ProtoMessageWritable<>(parser); + } + + public Path getFilePath() { + return filePath; + } + + public void setOffset(long offset) throws IOException { + reader.seek(offset); + } + + public long getOffset() throws IOException { + return reader.getPosition(); + } + + public T readEvent() throws IOException { + if (!reader.next(NullWritable.get(), writable)) { + return null; + } + return writable.getMessage(); + } + + @Override + public void close() throws IOException { + reader.close(); + } +} diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWritable.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWritable.java new file mode 100644 index 0000000000..34e47014e1 --- /dev/null +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWritable.java @@ -0,0 +1,101 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.history.logging.proto; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +import org.apache.hadoop.io.Writable; + +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.MessageLite; +import com.google.protobuf.Parser; + +public class ProtoMessageWritable implements Writable { + private T message; + private final Parser parser; + private DataOutputStream dos; + private CodedOutputStream cos; + private DataInputStream din; + private CodedInputStream cin; + + ProtoMessageWritable(Parser parser) { + this.parser = parser; + } + + public T getMessage() { + return message; + } + + public void setMessage(T message) { + this.message = message; + } + + private static class DataOutputStream extends OutputStream { + DataOutput out; + @Override + public void write(int b) throws IOException { + out.write(b); + } + + @Override + public void write(byte b[], int off, int len) throws IOException { + out.write(b, off, len); + } + } + + @Override + public void write(DataOutput out) throws IOException { + if (dos == null) { + dos = new DataOutputStream(); + cos = CodedOutputStream.newInstance(dos); + } + dos.out = out; + cos.writeMessageNoTag(message); + cos.flush(); + } + + private static class DataInputStream extends InputStream { + DataInput in; + @Override + public int read() throws IOException { + try { + return in.readUnsignedByte(); + } catch (EOFException e) { + return -1; + } + } + } + + @Override + public void readFields(DataInput in) throws IOException { + if (din == null) { + din = new DataInputStream(); + cin = CodedInputStream.newInstance(din); + cin.setSizeLimit(Integer.MAX_VALUE); + } + din.in = in; + message = cin.readMessage(parser, null); + } +} diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWriter.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWriter.java new file mode 100644 index 0000000000..ca9ba61e02 --- /dev/null +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWriter.java @@ -0,0 +1,71 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.history.logging.proto; + +import java.io.Closeable; +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.SequenceFile.CompressionType; + +import com.google.protobuf.MessageLite; +import com.google.protobuf.Parser; + +public class ProtoMessageWriter implements Closeable { + private final Path filePath; + private final SequenceFile.Writer writer; + private final ProtoMessageWritable writable; + + ProtoMessageWriter(Configuration conf, Path filePath, Parser parser) throws IOException { + this.filePath = filePath; + this.writer = SequenceFile.createWriter( + conf, + SequenceFile.Writer.file(filePath), + SequenceFile.Writer.keyClass(NullWritable.class), + SequenceFile.Writer.valueClass(ProtoMessageWritable.class), + SequenceFile.Writer.appendIfExists(true), + SequenceFile.Writer.compression(CompressionType.RECORD)); + this.writable = new ProtoMessageWritable<>(parser); + } + + public Path getPath() { + return filePath; + } + + public long getOffset() throws IOException { + return writer.getLength(); + } + + public void writeProto(T message) throws IOException { + writable.setMessage(message); + writer.append(NullWritable.get(), writable); + } + + public void hflush() throws IOException { + writer.hflush(); + } + + @Override + public void close() throws IOException { + writer.close(); + } +} diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/TezProtoLoggers.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/TezProtoLoggers.java new file mode 100644 index 0000000000..44390fc063 --- /dev/null +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/TezProtoLoggers.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.history.logging.proto; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.util.Clock; +import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.history.logging.proto.HistoryLoggerProtos.HistoryEventProto; +import org.apache.tez.dag.history.logging.proto.HistoryLoggerProtos.ManifestEntryProto; + +/** + * Helper class to create the logger for tez, we would use this to read the events outside + * tez and hence isolating the configuration and the paths in this. + */ +public class TezProtoLoggers { + private DatePartitionedLogger appEventsLogger; + private DatePartitionedLogger dagEventsLogger; + private DatePartitionedLogger manifestEventsLogger; + + public boolean setup(Configuration conf, Clock clock) throws IOException { + String logDir = conf.get(TezConfiguration.TEZ_HISTORY_LOGGING_PROTO_BASE_DIR); + if (logDir == null) { + return false; + } + appEventsLogger = new DatePartitionedLogger<>(HistoryEventProto.PARSER, + new Path(logDir, "app_data"), conf, clock); + dagEventsLogger = new DatePartitionedLogger<>(HistoryEventProto.PARSER, + new Path(logDir, "dag_data"), conf, clock); + manifestEventsLogger = new DatePartitionedLogger<>(ManifestEntryProto.PARSER, + new Path(logDir, "dag_meta"), conf, clock); + return true; + } + + public DatePartitionedLogger getAppEventsLogger() { + return appEventsLogger; + } + + public DatePartitionedLogger getDagEventsLogger() { + return dagEventsLogger; + } + + public DatePartitionedLogger getManifestEventsLogger() { + return manifestEventsLogger; + } +} diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/proto/HistoryLogger.proto b/tez-plugins/tez-protobuf-history-plugin/src/main/proto/HistoryLogger.proto new file mode 100644 index 0000000000..a5bbe3492e --- /dev/null +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/proto/HistoryLogger.proto @@ -0,0 +1,49 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +option java_package = "org.apache.tez.dag.history.logging.proto"; +option java_outer_classname = "HistoryLoggerProtos"; + +message KVPair { + optional string key = 1; + optional string value = 2; +} + +message HistoryEventProto { + optional string event_type = 1; + optional int64 event_time = 2; + optional string user = 3; + optional string app_id = 4; + optional string app_attempt_id = 5; + optional string dag_id = 6; + optional string vertex_id = 7; + optional string task_id = 8; + optional string task_attempt_id = 9; + repeated KVPair event_data = 10; +} + +message ManifestEntryProto { + optional string dag_id = 1; + optional string app_id = 2; + optional int64 dag_submitted_event_offset = 3; + optional int64 dag_finished_event_offset = 4; + optional string dag_file_path = 5; + optional int64 writeTime = 6; + optional string app_file_path = 7; + optional int64 app_launched_event_offset = 8; +} diff --git a/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestDagManifestFileScanner.java b/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestDagManifestFileScanner.java new file mode 100644 index 0000000000..fcaa315f34 --- /dev/null +++ b/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestDagManifestFileScanner.java @@ -0,0 +1,118 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.dag.history.logging.proto; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.util.Clock; +import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.history.logging.proto.HistoryLoggerProtos.ManifestEntryProto; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestDagManifestFileScanner { + private MockClock clock; + private DatePartitionedLogger manifestLogger; + + @Rule + public TemporaryFolder tempFolder = new TemporaryFolder(); + + @Before + public void setupTest() throws Exception { + String basePath = tempFolder.newFolder().getAbsolutePath(); + clock = new MockClock(); + Configuration conf = new Configuration(false); + conf.set(TezConfiguration.TEZ_HISTORY_LOGGING_PROTO_BASE_DIR, basePath); + TezProtoLoggers loggers = new TezProtoLoggers(); + loggers.setup(conf, clock); + manifestLogger = loggers.getManifestEventsLogger(); + } + + @Test + public void testNormal() throws Exception { + clock.setTime(0); // 0th day. + createManifestEvents(0, 8); + clock.setTime((24 * 60 * 60 + 1) * 1000); // 1 day 1 sec. + createManifestEvents(24 * 3600, 5); + DagManifesFileScanner scanner = new DagManifesFileScanner(manifestLogger); + int count = 0; + while (scanner.getNext() != null) { + ++count; + } + Assert.assertEquals(8, count); + + // Save offset for later use. + String offset = scanner.getOffset(); + + // Move time outside the window, no changes and it will give more events. + clock.setTime((24 * 60 * 60 + 61) * 1000); // 1 day 61 sec. + count = 0; + while (scanner.getNext() != null) { + ++count; + } + Assert.assertEquals(5, count); + + // Reset the offset + scanner.setOffset(offset); + count = 0; + while (scanner.getNext() != null) { + ++count; + } + Assert.assertEquals(5, count); + + scanner.close(); + + // Not able to test append since the LocalFileSystem does not implement append. + } + + private void createManifestEvents(long time, int numEvents) throws IOException { + for (int i = 0; i < numEvents; ++i) { + ApplicationId appId = ApplicationId.newInstance(1000l, i); + ManifestEntryProto proto = ManifestEntryProto.newBuilder() + .setAppId(appId.toString()) + .setDagFilePath("dummy_dag_path_" + i) + .setDagSubmittedEventOffset(0) + .setDagFinishedEventOffset(1) + .setAppFilePath("dummp_app_path_" + i) + .setAppLaunchedEventOffset(2) + .setWriteTime(clock.getTime()) + .build(); + ProtoMessageWriter writer = manifestLogger.getWriter(appId.toString()); + writer.writeProto(proto); + writer.close(); + } + } + + private static class MockClock implements Clock { + private long time = 0; + + void setTime(long time) { + this.time = time; + } + + @Override + public long getTime() { + return time; + } + } +} diff --git a/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestHistoryEventProtoConverter.java b/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestHistoryEventProtoConverter.java new file mode 100644 index 0000000000..92d3e300d7 --- /dev/null +++ b/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestHistoryEventProtoConverter.java @@ -0,0 +1,716 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.history.logging.proto; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.tez.common.ATSConstants; +import org.apache.tez.common.VersionInfo; +import org.apache.tez.common.counters.TezCounters; +import org.apache.tez.dag.api.EdgeManagerPluginDescriptor; +import org.apache.tez.dag.api.EdgeProperty; +import org.apache.tez.dag.api.EdgeProperty.DataSourceType; +import org.apache.tez.dag.api.EdgeProperty.SchedulingType; +import org.apache.tez.dag.api.InputDescriptor; +import org.apache.tez.dag.api.OutputDescriptor; +import org.apache.tez.dag.api.oldrecords.TaskAttemptState; +import org.apache.tez.dag.api.oldrecords.TaskState; +import org.apache.tez.dag.api.records.DAGProtos.CallerContextProto; +import org.apache.tez.dag.api.records.DAGProtos.DAGPlan; +import org.apache.tez.dag.app.dag.DAGState; +import org.apache.tez.dag.app.dag.VertexState; +import org.apache.tez.dag.app.dag.impl.ServicePluginInfo; +import org.apache.tez.dag.app.dag.impl.TaskAttemptImpl.DataEventDependencyInfo; +import org.apache.tez.dag.app.dag.impl.VertexStats; +import org.apache.tez.dag.app.web.AMWebController; +import org.apache.tez.dag.history.HistoryEvent; +import org.apache.tez.dag.history.HistoryEventType; +import org.apache.tez.dag.history.events.AMLaunchedEvent; +import org.apache.tez.dag.history.events.AMStartedEvent; +import org.apache.tez.dag.history.events.AppLaunchedEvent; +import org.apache.tez.dag.history.events.ContainerLaunchedEvent; +import org.apache.tez.dag.history.events.ContainerStoppedEvent; +import org.apache.tez.dag.history.events.DAGCommitStartedEvent; +import org.apache.tez.dag.history.events.DAGFinishedEvent; +import org.apache.tez.dag.history.events.DAGInitializedEvent; +import org.apache.tez.dag.history.events.DAGKillRequestEvent; +import org.apache.tez.dag.history.events.DAGRecoveredEvent; +import org.apache.tez.dag.history.events.DAGStartedEvent; +import org.apache.tez.dag.history.events.DAGSubmittedEvent; +import org.apache.tez.dag.history.events.TaskAttemptFinishedEvent; +import org.apache.tez.dag.history.events.TaskAttemptStartedEvent; +import org.apache.tez.dag.history.events.TaskFinishedEvent; +import org.apache.tez.dag.history.events.TaskStartedEvent; +import org.apache.tez.dag.history.events.VertexCommitStartedEvent; +import org.apache.tez.dag.history.events.VertexConfigurationDoneEvent; +import org.apache.tez.dag.history.events.VertexFinishedEvent; +import org.apache.tez.dag.history.events.VertexGroupCommitFinishedEvent; +import org.apache.tez.dag.history.events.VertexGroupCommitStartedEvent; +import org.apache.tez.dag.history.events.VertexInitializedEvent; +import org.apache.tez.dag.history.events.VertexStartedEvent; +import org.apache.tez.dag.history.logging.EntityTypes; +import org.apache.tez.dag.history.logging.proto.HistoryLoggerProtos.HistoryEventProto; +import org.apache.tez.dag.history.logging.proto.HistoryLoggerProtos.KVPair; +import org.apache.tez.dag.records.TaskAttemptTerminationCause; +import org.apache.tez.dag.records.TezDAGID; +import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.dag.records.TezTaskID; +import org.apache.tez.dag.records.TezVertexID; +import org.apache.tez.runtime.api.TaskFailureType; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import com.google.common.collect.Lists; + +public class TestHistoryEventProtoConverter { + private ApplicationAttemptId applicationAttemptId; + private ApplicationId applicationId; + private String user = "user"; + private Random random = new Random(); + private TezDAGID tezDAGID; + private TezVertexID tezVertexID; + private TezTaskID tezTaskID; + private TezTaskAttemptID tezTaskAttemptID; + private DAGPlan dagPlan; + private ContainerId containerId; + private NodeId nodeId; + private String containerLogs = "containerLogs"; + private HistoryEventProtoConverter converter = new HistoryEventProtoConverter(); + + @Before + public void setup() { + applicationId = ApplicationId.newInstance(9999l, 1); + applicationAttemptId = ApplicationAttemptId.newInstance(applicationId, 1); + tezDAGID = TezDAGID.getInstance(applicationId, random.nextInt()); + tezVertexID = TezVertexID.getInstance(tezDAGID, random.nextInt()); + tezTaskID = TezTaskID.getInstance(tezVertexID, random.nextInt()); + tezTaskAttemptID = TezTaskAttemptID.getInstance(tezTaskID, random.nextInt()); + CallerContextProto.Builder callerContextProto = CallerContextProto.newBuilder(); + callerContextProto.setContext("ctxt"); + callerContextProto.setCallerId("Caller_ID"); + callerContextProto.setCallerType("Caller_Type"); + callerContextProto.setBlob("Desc_1"); + dagPlan = DAGPlan.newBuilder().setName("DAGPlanMock") + .setCallerContext(callerContextProto).build(); + containerId = ContainerId.newContainerId(applicationAttemptId, 111); + nodeId = NodeId.newInstance("node", 13435); + } + + @Test(timeout = 5000) + public void testHandlerExists() { + for (HistoryEventType eventType : HistoryEventType.values()) { + HistoryEvent event = null; + switch (eventType) { + case APP_LAUNCHED: + event = new AppLaunchedEvent(applicationId, random.nextInt(), random.nextInt(), + user, new Configuration(false), null); + break; + case AM_LAUNCHED: + event = new AMLaunchedEvent(applicationAttemptId, random.nextInt(), random.nextInt(), + user); + break; + case AM_STARTED: + event = new AMStartedEvent(applicationAttemptId, random.nextInt(), user); + break; + case DAG_SUBMITTED: + event = new DAGSubmittedEvent(tezDAGID, random.nextInt(), dagPlan, applicationAttemptId, + null, user, null, containerLogs, null); + break; + case DAG_INITIALIZED: + event = new DAGInitializedEvent(tezDAGID, random.nextInt(), user, dagPlan.getName(), null); + break; + case DAG_STARTED: + event = new DAGStartedEvent(tezDAGID, random.nextInt(), user, dagPlan.getName()); + break; + case DAG_FINISHED: + event = new DAGFinishedEvent(tezDAGID, random.nextInt(), random.nextInt(), DAGState.ERROR, + null, null, user, dagPlan.getName(), null, applicationAttemptId, dagPlan); + break; + case VERTEX_INITIALIZED: + event = new VertexInitializedEvent(tezVertexID, "v1", random.nextInt(), random.nextInt(), + random.nextInt(), "proc", null, null, null); + break; + case VERTEX_STARTED: + event = new VertexStartedEvent(tezVertexID, random.nextInt(), random.nextInt()); + break; + case VERTEX_CONFIGURE_DONE: + event = new VertexConfigurationDoneEvent(tezVertexID, 0L, 1, null, null, null, true); + break; + case VERTEX_FINISHED: + event = new VertexFinishedEvent(tezVertexID, "v1", 1, random.nextInt(), random.nextInt(), + random.nextInt(), random.nextInt(), random.nextInt(), VertexState.ERROR, + null, null, null, null, null); + break; + case TASK_STARTED: + event = new TaskStartedEvent(tezTaskID, "v1", random.nextInt(), random.nextInt()); + break; + case TASK_FINISHED: + event = new TaskFinishedEvent(tezTaskID, "v1", random.nextInt(), random.nextInt(), + tezTaskAttemptID, TaskState.FAILED, null, null, 0); + break; + case TASK_ATTEMPT_STARTED: + event = new TaskAttemptStartedEvent(tezTaskAttemptID, "v1", random.nextInt(), containerId, + nodeId, null, null, "nodeHttpAddress"); + break; + case TASK_ATTEMPT_FINISHED: + event = new TaskAttemptFinishedEvent(tezTaskAttemptID, "v1", random.nextInt(), + random.nextInt(), TaskAttemptState.FAILED, TaskFailureType.NON_FATAL, TaskAttemptTerminationCause.OUTPUT_LOST, + null, null, null, null, 0, null, 0, + containerId, nodeId, null, null, "nodeHttpAddress"); + break; + case CONTAINER_LAUNCHED: + event = new ContainerLaunchedEvent(containerId, random.nextInt(), + applicationAttemptId); + break; + case CONTAINER_STOPPED: + event = new ContainerStoppedEvent(containerId, random.nextInt(), -1, applicationAttemptId); + break; + case DAG_COMMIT_STARTED: + event = new DAGCommitStartedEvent(); + break; + case VERTEX_COMMIT_STARTED: + event = new VertexCommitStartedEvent(); + break; + case VERTEX_GROUP_COMMIT_STARTED: + event = new VertexGroupCommitStartedEvent(); + break; + case VERTEX_GROUP_COMMIT_FINISHED: + event = new VertexGroupCommitFinishedEvent(); + break; + case DAG_RECOVERED: + event = new DAGRecoveredEvent(applicationAttemptId, tezDAGID, dagPlan.getName(), + user, random.nextLong(), containerLogs); + break; + case DAG_KILL_REQUEST: + event = new DAGKillRequestEvent(); + break; + default: + Assert.fail("Unhandled event type " + eventType); + } + if (event == null || !event.isHistoryEvent()) { + continue; + } + converter.convert(event); + } + } + + static class MockVersionInfo extends VersionInfo { + MockVersionInfo() { + super("component", "1.1.0", "rev1", "20120101", "git.apache.org"); + } + } + + private String findEventData(HistoryEventProto proto, String key) { + for (KVPair data : proto.getEventDataList()) { + if (data.getKey().equals(key)) { + return data.getValue(); + } + } + return null; + } + + private void assertEventData(HistoryEventProto proto, String key, String value) { + String evtVal = findEventData(proto, key); + if (evtVal == null) { + Assert.fail("Cannot find kv pair: " + key); + } + if (value != null) { + Assert.assertEquals(value, evtVal); + } + } + + private void assertNoEventData(HistoryEventProto proto, String key) { + for (KVPair data : proto.getEventDataList()) { + if (data.getKey().equals(key)) { + Assert.fail("Found find kv pair: " + key); + } + } + } + + private String safeToString(Object obj) { + return obj == null ? "" : obj.toString(); + } + + private void assertCommon(HistoryEventProto proto, HistoryEventType type, long eventTime, + EntityTypes entityType, ApplicationAttemptId appAttemptId, String user, int numData) { + Assert.assertEquals(type.name(), proto.getEventType()); + Assert.assertEquals(eventTime, proto.getEventTime()); + // Assert.assertEquals(safeToString(appId), proto.getAppId()); + Assert.assertEquals(safeToString(appAttemptId), proto.getAppAttemptId()); + Assert.assertEquals(safeToString(user), proto.getUser()); + if (entityType != null) { + switch (entityType) { // Intentional fallthrough. + case TEZ_TASK_ATTEMPT_ID: + Assert.assertEquals(tezTaskAttemptID.toString(), proto.getTaskAttemptId()); + case TEZ_TASK_ID: + Assert.assertEquals(tezTaskID.toString(), proto.getTaskId()); + case TEZ_VERTEX_ID: + Assert.assertEquals(tezVertexID.toString(), proto.getVertexId()); + case TEZ_DAG_ID: + Assert.assertEquals(tezDAGID.toString(), proto.getDagId()); + case TEZ_APPLICATION: + Assert.assertEquals(applicationId.toString(), proto.getAppId()); + break; + default: + Assert.fail("Invalid type: " + entityType.name()); + } + } + Assert.assertEquals(numData, proto.getEventDataCount()); + } + + @Test(timeout = 5000) + public void testConvertAppLaunchedEvent() { + long launchTime = random.nextLong(); + long submitTime = random.nextLong(); + Configuration conf = new Configuration(false); + conf.set("foo", "bar"); + conf.set("applicationId", "1234"); + + MockVersionInfo mockVersionInfo = new MockVersionInfo(); + AppLaunchedEvent event = new AppLaunchedEvent(applicationId, launchTime, submitTime, user, + conf, mockVersionInfo); + HistoryEventProto proto = converter.convert(event); + + assertCommon(proto, HistoryEventType.APP_LAUNCHED, launchTime, EntityTypes.TEZ_APPLICATION, + null, user, 3); + assertEventData(proto, ATSConstants.CONFIG, null); + assertEventData(proto, ATSConstants.TEZ_VERSION, null); + assertEventData(proto, ATSConstants.DAG_AM_WEB_SERVICE_VERSION, AMWebController.VERSION); + } + + @Test(timeout = 5000) + public void testConvertAMLaunchedEvent() { + long launchTime = random.nextLong(); + long submitTime = random.nextLong(); + AMLaunchedEvent event = new AMLaunchedEvent(applicationAttemptId, launchTime, submitTime, + user); + HistoryEventProto proto = converter.convert(event); + assertCommon(proto, HistoryEventType.AM_LAUNCHED, launchTime, EntityTypes.TEZ_APPLICATION, + applicationAttemptId, user, 1); + assertEventData(proto, ATSConstants.APP_SUBMIT_TIME, String.valueOf(submitTime)); + } + + @Test(timeout = 5000) + public void testConvertAMStartedEvent() { + long startTime = random.nextLong(); + AMStartedEvent event = new AMStartedEvent(applicationAttemptId, startTime, user); + HistoryEventProto proto = converter.convert(event); + assertCommon(proto, HistoryEventType.AM_STARTED, startTime, EntityTypes.TEZ_APPLICATION, + applicationAttemptId, user, 0); + } + + @Test(timeout = 5000) + public void testConvertContainerLaunchedEvent() { + long launchTime = random.nextLong(); + ContainerLaunchedEvent event = new ContainerLaunchedEvent(containerId, launchTime, + applicationAttemptId); + HistoryEventProto proto = converter.convert(event); + assertCommon(proto, HistoryEventType.CONTAINER_LAUNCHED, launchTime, EntityTypes.TEZ_APPLICATION, + applicationAttemptId, null, 1); + assertEventData(proto, ATSConstants.CONTAINER_ID, containerId.toString()); + } + + @Test(timeout = 5000) + public void testConvertContainerStoppedEvent() { + long stopTime = random.nextLong(); + int exitStatus = random.nextInt(); + ContainerStoppedEvent event = new ContainerStoppedEvent(containerId, stopTime, exitStatus, + applicationAttemptId); + HistoryEventProto proto = converter.convert(event); + assertCommon(proto, HistoryEventType.CONTAINER_STOPPED, stopTime, EntityTypes.TEZ_APPLICATION, + applicationAttemptId, null, 3); + assertEventData(proto, ATSConstants.CONTAINER_ID, containerId.toString()); + assertEventData(proto, ATSConstants.EXIT_STATUS, String.valueOf(exitStatus)); + assertEventData(proto, ATSConstants.FINISH_TIME, String.valueOf(stopTime)); + } + + @Test(timeout = 5000) + public void testConvertDAGStartedEvent() { + long startTime = random.nextLong(); + String dagName = "testDagName"; + DAGStartedEvent event = new DAGStartedEvent(tezDAGID, startTime, user, dagName); + HistoryEventProto proto = converter.convert(event); + assertCommon(proto, HistoryEventType.DAG_STARTED, startTime, EntityTypes.TEZ_DAG_ID, null, + user, 2); + assertEventData(proto, ATSConstants.DAG_NAME, dagName); + assertEventData(proto, ATSConstants.STATUS, DAGState.RUNNING.name()); + } + + @Test(timeout = 5000) + public void testConvertDAGSubmittedEvent() { + long submitTime = random.nextLong(); + + final String queueName = "TEST_DAG_SUBMITTED"; + DAGSubmittedEvent event = new DAGSubmittedEvent(tezDAGID, submitTime, dagPlan, + applicationAttemptId, null, user, null, containerLogs, queueName); + HistoryEventProto proto = converter.convert(event); + assertCommon(proto, HistoryEventType.DAG_SUBMITTED, submitTime, EntityTypes.TEZ_DAG_ID, + applicationAttemptId, user, 8); + + assertEventData(proto, ATSConstants.DAG_NAME, dagPlan.getName()); + assertEventData(proto, ATSConstants.DAG_QUEUE_NAME, event.getQueueName()); + assertEventData(proto, ATSConstants.DAG_AM_WEB_SERVICE_VERSION, AMWebController.VERSION); + assertEventData(proto, ATSConstants.IN_PROGRESS_LOGS_URL + "_" + + applicationAttemptId.getAttemptId(), containerLogs); + assertEventData(proto, ATSConstants.CALLER_CONTEXT_ID, + dagPlan.getCallerContext().getCallerId()); + assertEventData(proto, ATSConstants.CALLER_CONTEXT_TYPE, + dagPlan.getCallerContext().getCallerType()); + assertEventData(proto, ATSConstants.CALLER_CONTEXT, dagPlan.getCallerContext().getContext()); + assertEventData(proto, ATSConstants.DAG_PLAN, null); + } + + @Test(timeout = 5000) + public void testConvertTaskAttemptFinishedEvent() { + String vertexName = "testVertex"; + long creationTime = random.nextLong(); + long startTime = creationTime + 1000; + long allocationTime = creationTime + 1001; + long finishTime = startTime + 1002; + TaskAttemptState state = TaskAttemptState + .values()[random.nextInt(TaskAttemptState.values().length)]; + TaskAttemptTerminationCause error = TaskAttemptTerminationCause + .values()[random.nextInt(TaskAttemptTerminationCause.values().length)]; + String diagnostics = "random diagnostics message"; + TezCounters counters = new TezCounters(); + long lastDataEventTime = finishTime - 1; + List events = Lists.newArrayList(); + events.add(new DataEventDependencyInfo(lastDataEventTime, tezTaskAttemptID)); + events.add(new DataEventDependencyInfo(lastDataEventTime, tezTaskAttemptID)); + + TaskAttemptFinishedEvent event = new TaskAttemptFinishedEvent(tezTaskAttemptID, vertexName, + startTime, finishTime, state, TaskFailureType.FATAL, error, diagnostics, counters, events, + null, creationTime, tezTaskAttemptID, allocationTime, containerId, nodeId, "inProgressURL", + "logsURL", "nodeHttpAddress"); + HistoryEventProto proto = converter.convert(event); + assertCommon(proto, HistoryEventType.TASK_ATTEMPT_FINISHED, finishTime, + EntityTypes.TEZ_DAG_ID, null, null, 16); + + assertEventData(proto, ATSConstants.STATUS, state.name()); + assertEventData(proto, ATSConstants.CREATION_CAUSAL_ATTEMPT, tezTaskAttemptID.toString()); + assertEventData(proto, ATSConstants.CREATION_TIME, String.valueOf(creationTime)); + assertEventData(proto, ATSConstants.ALLOCATION_TIME, String.valueOf(allocationTime)); + assertEventData(proto, ATSConstants.START_TIME, String.valueOf(startTime)); + assertEventData(proto, ATSConstants.TIME_TAKEN, String.valueOf(finishTime - startTime)); + assertEventData(proto, ATSConstants.TASK_FAILURE_TYPE, TaskFailureType.FATAL.name()); + assertEventData(proto, ATSConstants.TASK_ATTEMPT_ERROR_ENUM, error.name()); + assertEventData(proto, ATSConstants.DIAGNOSTICS, diagnostics); + assertEventData(proto, ATSConstants.LAST_DATA_EVENTS, null); + assertEventData(proto, ATSConstants.COUNTERS, null); + assertEventData(proto, ATSConstants.IN_PROGRESS_LOGS_URL, "inProgressURL"); + assertEventData(proto, ATSConstants.COMPLETED_LOGS_URL, "logsURL"); + assertEventData(proto, ATSConstants.NODE_ID, nodeId.toString()); + assertEventData(proto, ATSConstants.CONTAINER_ID, containerId.toString()); + assertEventData(proto, ATSConstants.NODE_HTTP_ADDRESS, "nodeHttpAddress"); + + TaskAttemptFinishedEvent eventWithNullFailureType = + new TaskAttemptFinishedEvent(tezTaskAttemptID, vertexName, + startTime, finishTime, state, null, error, diagnostics, counters, events, null, + creationTime, + tezTaskAttemptID, allocationTime, containerId, nodeId, "inProgressURL", "logsURL", + "nodeHttpAddress"); + proto = converter.convert(eventWithNullFailureType); + assertNoEventData(proto, ATSConstants.TASK_FAILURE_TYPE); + } + + @Test(timeout = 5000) + public void testConvertDAGInitializedEvent() { + long initTime = random.nextLong(); + + Map nameIdMap = new HashMap(); + nameIdMap.put("foo", tezVertexID); + + DAGInitializedEvent event = new DAGInitializedEvent(tezDAGID, initTime, "user", "dagName", + nameIdMap); + HistoryEventProto proto = converter.convert(event); + assertCommon(proto, HistoryEventType.DAG_INITIALIZED, initTime, + EntityTypes.TEZ_DAG_ID, null, user, 2); + assertEventData(proto, ATSConstants.DAG_NAME, "dagName"); + assertEventData(proto, ATSConstants.VERTEX_NAME_ID_MAPPING, null); + } + + @Test(timeout = 5000) + public void testConvertDAGFinishedEvent() { + long finishTime = random.nextLong(); + long startTime = random.nextLong(); + Map taskStats = new HashMap(); + taskStats.put("FOO", 100); + taskStats.put("BAR", 200); + + DAGFinishedEvent event = new DAGFinishedEvent(tezDAGID, startTime, finishTime, DAGState.ERROR, + "diagnostics", null, user, dagPlan.getName(), taskStats, applicationAttemptId, dagPlan); + HistoryEventProto proto = converter.convert(event); + assertCommon(proto, HistoryEventType.DAG_FINISHED, finishTime, + EntityTypes.TEZ_DAG_ID, applicationAttemptId, user, 11); + + assertEventData(proto, ATSConstants.DAG_NAME, dagPlan.getName()); + assertEventData(proto, ATSConstants.STATUS, DAGState.ERROR.name()); + assertEventData(proto, ATSConstants.CALLER_CONTEXT_ID, + dagPlan.getCallerContext().getCallerId()); + assertEventData(proto, ATSConstants.CALLER_CONTEXT_TYPE, + dagPlan.getCallerContext().getCallerType()); + assertEventData(proto, ATSConstants.START_TIME, String.valueOf(startTime)); + assertEventData(proto, ATSConstants.TIME_TAKEN, String.valueOf(finishTime - startTime)); + assertEventData(proto, ATSConstants.DIAGNOSTICS, "diagnostics"); + assertEventData(proto, ATSConstants.COMPLETION_APPLICATION_ATTEMPT_ID, + applicationAttemptId.toString()); + assertEventData(proto, "FOO", String.valueOf(100)); + assertEventData(proto, "BAR", String.valueOf(200)); + assertEventData(proto, ATSConstants.COUNTERS, null); + } + + @Test(timeout = 5000) + public void testConvertVertexInitializedEvent() { + long initRequestedTime = random.nextLong(); + long initedTime = random.nextLong(); + int numTasks = random.nextInt(); + VertexInitializedEvent event = new VertexInitializedEvent(tezVertexID, "v1", initRequestedTime, + initedTime, numTasks, "proc", null, null, + new ServicePluginInfo().setContainerLauncherName("abc") + .setTaskSchedulerName("def").setTaskCommunicatorName("ghi") + .setContainerLauncherClassName("abc1") + .setTaskSchedulerClassName("def1") + .setTaskCommunicatorClassName("ghi1")); + + HistoryEventProto proto = converter.convert(event); + assertCommon(proto, HistoryEventType.VERTEX_INITIALIZED, initedTime, + EntityTypes.TEZ_VERTEX_ID, null, null, 5); + + assertEventData(proto, ATSConstants.VERTEX_NAME, "v1"); + assertEventData(proto, ATSConstants.PROCESSOR_CLASS_NAME, "proc"); + assertEventData(proto, ATSConstants.INIT_REQUESTED_TIME, String.valueOf(initRequestedTime)); + assertEventData(proto, ATSConstants.NUM_TASKS, String.valueOf(numTasks)); + assertEventData(proto, ATSConstants.SERVICE_PLUGIN, null); + + /* + Assert.assertNotNull(timelineEntity.getOtherInfo().get(ATSConstants.SERVICE_PLUGIN)); + Assert.assertEquals("abc", + ((Map)timelineEntity.getOtherInfo().get(ATSConstants.SERVICE_PLUGIN)).get( + ATSConstants.CONTAINER_LAUNCHER_NAME)); + Assert.assertEquals("def", + ((Map)timelineEntity.getOtherInfo().get(ATSConstants.SERVICE_PLUGIN)).get( + ATSConstants.TASK_SCHEDULER_NAME)); + Assert.assertEquals("ghi", + ((Map)timelineEntity.getOtherInfo().get(ATSConstants.SERVICE_PLUGIN)).get( + ATSConstants.TASK_COMMUNICATOR_NAME)); + Assert.assertEquals("abc1", + ((Map)timelineEntity.getOtherInfo().get(ATSConstants.SERVICE_PLUGIN)).get( + ATSConstants.CONTAINER_LAUNCHER_CLASS_NAME)); + Assert.assertEquals("def1", + ((Map)timelineEntity.getOtherInfo().get(ATSConstants.SERVICE_PLUGIN)).get( + ATSConstants.TASK_SCHEDULER_CLASS_NAME)); + Assert.assertEquals("ghi1", + ((Map)timelineEntity.getOtherInfo().get(ATSConstants.SERVICE_PLUGIN)).get( + ATSConstants.TASK_COMMUNICATOR_CLASS_NAME)); + */ + } + + @Test(timeout = 5000) + public void testConvertVertexStartedEvent() { + long startRequestedTime = random.nextLong(); + long startTime = random.nextLong(); + + VertexStartedEvent event = new VertexStartedEvent(tezVertexID, startRequestedTime, startTime); + HistoryEventProto proto = converter.convert(event); + assertCommon(proto, HistoryEventType.VERTEX_STARTED, startTime, + EntityTypes.TEZ_VERTEX_ID, null, null, 2); + assertEventData(proto, ATSConstants.START_REQUESTED_TIME, String.valueOf(startRequestedTime)); + assertEventData(proto, ATSConstants.STATUS, VertexState.RUNNING.name()); + } + + @Test(timeout = 5000) + public void testConvertVertexFinishedEvent() { + String vertexName = "v1"; + long initRequestedTime = random.nextLong(); + long initedTime = random.nextLong(); + long startRequestedTime = random.nextLong(); + long startTime = random.nextLong(); + long finishTime = random.nextLong(); + Map taskStats = new HashMap(); + taskStats.put("FOO", 100); + taskStats.put("BAR", 200); + VertexStats vertexStats = new VertexStats(); + + VertexFinishedEvent event = new VertexFinishedEvent(tezVertexID, vertexName, 1, + initRequestedTime, initedTime, startRequestedTime, startTime, finishTime, + VertexState.ERROR, "diagnostics", null, vertexStats, taskStats, + new ServicePluginInfo().setContainerLauncherName("abc") + .setTaskSchedulerName("def").setTaskCommunicatorName("ghi") + .setContainerLauncherClassName("abc1") + .setTaskSchedulerClassName("def1") + .setTaskCommunicatorClassName("ghi1")); + HistoryEventProto proto = converter.convert(event); + assertCommon(proto, HistoryEventType.VERTEX_FINISHED, finishTime, + EntityTypes.TEZ_VERTEX_ID, null, null, 9); + + assertEventData(proto, ATSConstants.VERTEX_NAME, vertexName); + assertEventData(proto, ATSConstants.STATUS, VertexState.ERROR.name()); + + assertEventData(proto, ATSConstants.TIME_TAKEN, String.valueOf(finishTime - startTime)); + assertEventData(proto, ATSConstants.DIAGNOSTICS, "diagnostics"); + assertEventData(proto, ATSConstants.COUNTERS, null); + assertEventData(proto, ATSConstants.STATS, null); + assertEventData(proto, "FOO", "100"); + assertEventData(proto, "BAR", "200"); + + assertEventData(proto, ATSConstants.SERVICE_PLUGIN, null); + /* + Assert.assertEquals("abc", + ((Map)timelineEntity.getOtherInfo().get(ATSConstants.SERVICE_PLUGIN)).get( + ATSConstants.CONTAINER_LAUNCHER_NAME)); + Assert.assertEquals("def", + ((Map)timelineEntity.getOtherInfo().get(ATSConstants.SERVICE_PLUGIN)).get( + ATSConstants.TASK_SCHEDULER_NAME)); + Assert.assertEquals("ghi", + ((Map)timelineEntity.getOtherInfo().get(ATSConstants.SERVICE_PLUGIN)).get( + ATSConstants.TASK_COMMUNICATOR_NAME)); + Assert.assertEquals("abc1", + ((Map)timelineEntity.getOtherInfo().get(ATSConstants.SERVICE_PLUGIN)).get( + ATSConstants.CONTAINER_LAUNCHER_CLASS_NAME)); + Assert.assertEquals("def1", + ((Map)timelineEntity.getOtherInfo().get(ATSConstants.SERVICE_PLUGIN)).get( + ATSConstants.TASK_SCHEDULER_CLASS_NAME)); + Assert.assertEquals("ghi1", + ((Map)timelineEntity.getOtherInfo().get(ATSConstants.SERVICE_PLUGIN)).get( + ATSConstants.TASK_COMMUNICATOR_CLASS_NAME)); + */ + } + + @Test(timeout = 5000) + public void testConvertTaskStartedEvent() { + long scheduleTime = random.nextLong(); + long startTime = random.nextLong(); + TaskStartedEvent event = new TaskStartedEvent(tezTaskID, "v1", scheduleTime, startTime); + HistoryEventProto proto = converter.convert(event); + assertCommon(proto, HistoryEventType.TASK_STARTED, startTime, + EntityTypes.TEZ_TASK_ID, null, null, 2); + + assertEventData(proto, ATSConstants.SCHEDULED_TIME, String.valueOf(scheduleTime)); + assertEventData(proto, ATSConstants.STATUS, TaskState.SCHEDULED.name()); + } + + @Test(timeout = 5000) + public void testConvertTaskAttemptStartedEvent() { + long startTime = random.nextLong(); + TaskAttemptStartedEvent event = new TaskAttemptStartedEvent(tezTaskAttemptID, "v1", + startTime, containerId, nodeId, "inProgressURL", "logsURL", "nodeHttpAddress"); + HistoryEventProto proto = converter.convert(event); + assertCommon(proto, HistoryEventType.TASK_ATTEMPT_STARTED, startTime, + EntityTypes.TEZ_TASK_ATTEMPT_ID, null, null, 6); + + assertEventData(proto, ATSConstants.STATUS, TaskAttemptState.RUNNING.name()); + assertEventData(proto, ATSConstants.IN_PROGRESS_LOGS_URL, "inProgressURL"); + assertEventData(proto, ATSConstants.COMPLETED_LOGS_URL, "logsURL"); + assertEventData(proto, ATSConstants.NODE_ID, nodeId.toString()); + assertEventData(proto, ATSConstants.CONTAINER_ID, containerId.toString()); + assertEventData(proto, ATSConstants.NODE_HTTP_ADDRESS, "nodeHttpAddress"); + } + + @Test(timeout = 5000) + public void testConvertTaskFinishedEvent() { + String vertexName = "testVertexName"; + long startTime = random.nextLong(); + long finishTime = random.nextLong(); + TaskState state = TaskState.values()[random.nextInt(TaskState.values().length)]; + String diagnostics = "diagnostics message"; + TezCounters counters = new TezCounters(); + + TaskFinishedEvent event = new TaskFinishedEvent(tezTaskID, vertexName, startTime, finishTime, + tezTaskAttemptID, state, diagnostics, counters, 3); + HistoryEventProto proto = converter.convert(event); + assertCommon(proto, HistoryEventType.TASK_FINISHED, finishTime, + EntityTypes.TEZ_TASK_ID, null, null, 6); + + assertEventData(proto, ATSConstants.STATUS, state.name()); + assertEventData(proto, ATSConstants.TIME_TAKEN, String.valueOf(finishTime - startTime)); + assertEventData(proto, ATSConstants.SUCCESSFUL_ATTEMPT_ID, tezTaskAttemptID.toString()); + assertEventData(proto, ATSConstants.NUM_FAILED_TASKS_ATTEMPTS, "3"); + assertEventData(proto, ATSConstants.DIAGNOSTICS, diagnostics); + assertEventData(proto, ATSConstants.COUNTERS, null); + } + + @Test(timeout = 5000) + public void testConvertVertexReconfigreDoneEvent() { + TezVertexID vId = tezVertexID; + Map edgeMgrs = + new HashMap(); + + edgeMgrs.put("a", EdgeProperty.create(EdgeManagerPluginDescriptor.create("a.class") + .setHistoryText("text"), DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, + OutputDescriptor.create("Out"), InputDescriptor.create("In"))); + VertexConfigurationDoneEvent event = new VertexConfigurationDoneEvent(vId, 0L, 1, null, + edgeMgrs, null, true); + HistoryEventProto proto = converter.convert(event); + assertCommon(proto, HistoryEventType.VERTEX_CONFIGURE_DONE, 0L, + EntityTypes.TEZ_VERTEX_ID, null, null, 2); + assertEventData(proto, ATSConstants.NUM_TASKS, "1"); + assertEventData(proto, ATSConstants.UPDATED_EDGE_MANAGERS, null); + + /* + Map updatedEdgeMgrs = (Map) + evt.getEventInfo().get(ATSConstants.UPDATED_EDGE_MANAGERS); + Assert.assertEquals(1, updatedEdgeMgrs.size()); + Assert.assertTrue(updatedEdgeMgrs.containsKey("a")); + Map updatedEdgeMgr = (Map) updatedEdgeMgrs.get("a"); + + Assert.assertEquals(DataMovementType.CUSTOM.name(), + updatedEdgeMgr.get(DAGUtils.DATA_MOVEMENT_TYPE_KEY)); + Assert.assertEquals("In", updatedEdgeMgr.get(DAGUtils.EDGE_DESTINATION_CLASS_KEY)); + Assert.assertEquals("a.class", updatedEdgeMgr.get(DAGUtils.EDGE_MANAGER_CLASS_KEY)); + */ + } + + @Test(timeout = 5000) + public void testConvertDAGRecoveredEvent() { + long recoverTime = random.nextLong(); + DAGRecoveredEvent event = new DAGRecoveredEvent(applicationAttemptId, tezDAGID, + dagPlan.getName(), user, recoverTime, containerLogs); + HistoryEventProto proto = converter.convert(event); + assertCommon(proto, HistoryEventType.DAG_RECOVERED, recoverTime, + EntityTypes.TEZ_DAG_ID, applicationAttemptId, user, 2); + assertEventData(proto, ATSConstants.IN_PROGRESS_LOGS_URL + "_" + + applicationAttemptId.getAttemptId(), containerLogs); + assertEventData(proto, ATSConstants.DAG_NAME, dagPlan.getName()); + } + + @Test(timeout = 5000) + public void testConvertDAGRecoveredEvent2() { + long recoverTime = random.nextLong(); + + DAGRecoveredEvent event = new DAGRecoveredEvent(applicationAttemptId, tezDAGID, + dagPlan.getName(), user, recoverTime, DAGState.ERROR, "mock reason", containerLogs); + HistoryEventProto proto = converter.convert(event); + assertCommon(proto, HistoryEventType.DAG_RECOVERED, recoverTime, + EntityTypes.TEZ_DAG_ID, applicationAttemptId, user, 4); + assertEventData(proto, ATSConstants.DAG_STATE, DAGState.ERROR.name()); + assertEventData(proto, ATSConstants.RECOVERY_FAILURE_REASON, "mock reason"); + assertEventData(proto, ATSConstants.IN_PROGRESS_LOGS_URL + "_" + + applicationAttemptId.getAttemptId(), containerLogs); + assertEventData(proto, ATSConstants.DAG_NAME, dagPlan.getName()); + } +} diff --git a/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestProtoHistoryLoggingService.java b/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestProtoHistoryLoggingService.java new file mode 100644 index 0000000000..4bd5d4e401 --- /dev/null +++ b/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestProtoHistoryLoggingService.java @@ -0,0 +1,195 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.history.logging.proto; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.time.LocalDate; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.util.Clock; +import org.apache.hadoop.yarn.util.SystemClock; +import org.apache.tez.common.VersionInfo; +import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.api.records.DAGProtos.DAGPlan; +import org.apache.tez.dag.app.AppContext; +import org.apache.tez.dag.app.dag.DAGState; +import org.apache.tez.dag.history.DAGHistoryEvent; +import org.apache.tez.dag.history.HistoryEventType; +import org.apache.tez.dag.history.events.AppLaunchedEvent; +import org.apache.tez.dag.history.events.DAGFinishedEvent; +import org.apache.tez.dag.history.events.DAGSubmittedEvent; +import org.apache.tez.dag.history.events.TaskAttemptStartedEvent; +import org.apache.tez.dag.history.events.TaskStartedEvent; +import org.apache.tez.dag.history.events.VertexStartedEvent; +import org.apache.tez.dag.history.logging.proto.HistoryLoggerProtos.HistoryEventProto; +import org.apache.tez.dag.history.logging.proto.HistoryLoggerProtos.ManifestEntryProto; +import org.apache.tez.dag.records.TezDAGID; +import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.dag.records.TezTaskID; +import org.apache.tez.dag.records.TezVertexID; +import org.apache.tez.hadoop.shim.HadoopShim; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestProtoHistoryLoggingService { + private static ApplicationId appId = ApplicationId.newInstance(1000l, 1); + private static ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1); + private static String user = "TEST_USER"; + private Clock clock; + + @Rule + public TemporaryFolder tempFolder = new TemporaryFolder(); + + @Test + public void testService() throws Exception { + ProtoHistoryLoggingService service = createService(); + service.start(); + TezDAGID dagId = TezDAGID.getInstance(appId, 0); + List protos = new ArrayList<>(); + for (DAGHistoryEvent event : makeHistoryEvents(dagId, service)) { + protos.add(new HistoryEventProtoConverter().convert(event.getHistoryEvent())); + service.handle(event); + } + service.stop(); + + TezProtoLoggers loggers = new TezProtoLoggers(); + Assert.assertTrue(loggers.setup(service.getConfig(), clock)); + + // Verify dag events are logged. + DatePartitionedLogger dagLogger = loggers.getDagEventsLogger(); + Path dagFilePath = dagLogger.getPathForDate(LocalDate.ofEpochDay(0), dagId.toString()); + ProtoMessageReader reader = dagLogger.getReader(dagFilePath); + HistoryEventProto evt = reader.readEvent(); + int ind = 1; + while (evt != null) { + Assert.assertEquals(protos.get(ind), evt); + ind++; + evt = reader.readEvent(); + } + reader.close(); + + // Verify app events are logged. + DatePartitionedLogger appLogger = loggers.getAppEventsLogger(); + Path appFilePath = appLogger.getPathForDate(LocalDate.ofEpochDay(0), attemptId.toString()); + ProtoMessageReader appReader = appLogger.getReader(appFilePath); + long appOffset = appReader.getOffset(); + Assert.assertEquals(protos.get(0), appReader.readEvent()); + reader.close(); + + // Verify manifest events are logged. + DatePartitionedLogger manifestLogger = loggers.getManifestEventsLogger(); + Path manifestFilePath = manifestLogger.getPathForDate( + LocalDate.ofEpochDay(0), attemptId.toString()); + ProtoMessageReader reader2 = manifestLogger.getReader(manifestFilePath); + ManifestEntryProto manifest = reader2.readEvent(); + Assert.assertEquals(appId.toString(), manifest.getAppId()); + Assert.assertEquals(dagId.toString(), manifest.getDagId()); + Assert.assertEquals(dagFilePath.toString(), manifest.getDagFilePath()); + Assert.assertEquals(appFilePath.toString(), manifest.getAppFilePath()); + Assert.assertEquals(appOffset, manifest.getAppLaunchedEventOffset()); + + // Verify offsets in manifest logger. + reader = dagLogger.getReader(new Path(manifest.getDagFilePath())); + reader.setOffset(manifest.getDagSubmittedEventOffset()); + evt = reader.readEvent(); + Assert.assertNotNull(evt); + Assert.assertEquals(HistoryEventType.DAG_SUBMITTED.name(), evt.getEventType()); + + reader.setOffset(manifest.getDagFinishedEventOffset()); + evt = reader.readEvent(); + Assert.assertNotNull(evt); + Assert.assertEquals(HistoryEventType.DAG_FINISHED.name(), evt.getEventType()); + + // Verify manifest file scanner. + DagManifesFileScanner scanner = new DagManifesFileScanner(manifestLogger); + Assert.assertEquals(manifest, scanner.getNext()); + Assert.assertNull(scanner.getNext()); + scanner.close(); + } + + private List makeHistoryEvents(TezDAGID dagId, + ProtoHistoryLoggingService service) { + List historyEvents = new ArrayList<>(); + DAGPlan dagPlan = DAGPlan.newBuilder().setName("DAGPlanMock").build(); + + long time = System.currentTimeMillis(); + Configuration conf = new Configuration(service.getConfig()); + historyEvents.add(new DAGHistoryEvent(null, new AppLaunchedEvent(appId, time, time, user, conf, + new VersionInfo("component", "1.1.0", "rev1", "20120101", "git.apache.org") {}))); + historyEvents.add(new DAGHistoryEvent(dagId, new DAGSubmittedEvent(dagId, time, + DAGPlan.getDefaultInstance(), attemptId, null, user, conf, null, "default"))); + TezVertexID vertexID = TezVertexID.getInstance(dagId, 1); + historyEvents.add(new DAGHistoryEvent(dagId, new VertexStartedEvent(vertexID, time, time))); + TezTaskID tezTaskID = TezTaskID.getInstance(vertexID, 1); + historyEvents + .add(new DAGHistoryEvent(dagId, new TaskStartedEvent(tezTaskID, "test", time, time))); + historyEvents.add(new DAGHistoryEvent(dagId, + new TaskAttemptStartedEvent(TezTaskAttemptID.getInstance(tezTaskID, 1), "test", time, + ContainerId.newContainerId(attemptId, 1), NodeId.newInstance("localhost", 8765), null, + null, null))); + historyEvents.add(new DAGHistoryEvent(dagId, new DAGFinishedEvent(dagId, time, time, + DAGState.ERROR, "diagnostics", null, user, dagPlan.getName(), + new HashMap(), attemptId, dagPlan))); + return historyEvents; + } + + private static class FixedClock implements Clock { + final Clock clock = new SystemClock(); + final long diff; + + public FixedClock(long startTime) { + diff = clock.getTime() - startTime; + } + + @Override + public long getTime() { + return clock.getTime() - diff; + } + } + + private ProtoHistoryLoggingService createService() throws IOException { + ProtoHistoryLoggingService service = new ProtoHistoryLoggingService(); + clock = new FixedClock(0); // Start time is always first day, easier to write tests. + AppContext appContext = mock(AppContext.class); + when(appContext.getApplicationID()).thenReturn(appId); + when(appContext.getApplicationAttemptId()).thenReturn(attemptId); + when(appContext.getUser()).thenReturn(user); + when(appContext.getHadoopShim()).thenReturn(new HadoopShim() {}); + when(appContext.getClock()).thenReturn(clock); + service.setAppContext(appContext); + Configuration conf = new Configuration(false); + String basePath = tempFolder.newFolder().getAbsolutePath(); + conf.set(TezConfiguration.TEZ_HISTORY_LOGGING_PROTO_BASE_DIR, basePath); + service.init(conf); + return service; + } +} From 2e66f3cb2ef082889551f6a0830c7014317d9680 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Mon, 23 Apr 2018 16:52:55 -0500 Subject: [PATCH 066/512] TEZ-3817. DAGs can hang after more than one uncaught Exception during doTransition. (kshukla) --- .../apache/tez/dag/app/dag/impl/DAGImpl.java | 63 ++++++++++--------- .../tez/dag/app/dag/impl/TestDAGImpl.java | 48 ++++++++++++++ 2 files changed, 81 insertions(+), 30 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java index 0a775a6db8..ecd8d17048 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java @@ -1388,41 +1388,44 @@ private void updateCpuCounters() { } private DAGState finished(DAGState finalState) { - if (finishTime == 0) { - setFinishTime(); - } - entityUpdateTracker.stop(); - - boolean recoveryError = false; - - // update cpu time counters before finishing the dag - updateCpuCounters(); - TezCounters counters = null; + boolean dagError = false; try { - counters = getAllCounters(); - } catch (LimitExceededException e) { - addDiagnostic("Counters limit exceeded: " + e.getMessage()); - finalState = DAGState.FAILED; - } + if (finishTime == 0) { + setFinishTime(); + } + entityUpdateTracker.stop(); - try { - if (finalState == DAGState.SUCCEEDED) { - logJobHistoryFinishedEvent(counters); - } else { - logJobHistoryUnsuccesfulEvent(finalState, counters); + // update cpu time counters before finishing the dag + updateCpuCounters(); + TezCounters counters = null; + try { + counters = getAllCounters(); + } catch (LimitExceededException e) { + addDiagnostic("Counters limit exceeded: " + e.getMessage()); + finalState = DAGState.FAILED; } - } catch (IOException e) { - LOG.warn("Failed to persist recovery event for DAG completion" - + ", dagId=" + dagId - + ", finalState=" + finalState); - recoveryError = true; - } - if (finalState != DAGState.SUCCEEDED) { - abortOutputs(); - } + try { + if (finalState == DAGState.SUCCEEDED) { + logJobHistoryFinishedEvent(counters); + } else { + logJobHistoryUnsuccesfulEvent(finalState, counters); + } + } catch (IOException e) { + LOG.warn("Failed to persist recovery event for DAG completion" + + ", dagId=" + dagId + + ", finalState=" + finalState, e); + dagError = true; + } - if (recoveryError) { + if (finalState != DAGState.SUCCEEDED) { + abortOutputs(); + } + } catch (Exception e) { + dagError = true; + LOG.warn("Encountered exception while DAG finish", e); + } + if (dagError) { eventHandler.handle(new DAGAppMasterEventDAGFinished(getID(), DAGState.ERROR)); } else { eventHandler.handle(new DAGAppMasterEventDAGFinished(getID(), finalState)); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java index 966b464539..c0506de68d 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java @@ -21,6 +21,7 @@ import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyInt; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -111,6 +112,7 @@ import org.apache.tez.dag.app.dag.event.DAGAppMasterEventType; import org.apache.tez.dag.app.dag.event.DAGEvent; import org.apache.tez.dag.app.dag.event.DAGEventStartDag; +import org.apache.tez.dag.app.dag.event.DAGEventCommitCompleted; import org.apache.tez.dag.app.dag.event.DAGEventType; import org.apache.tez.dag.app.dag.event.DAGEventVertexCompleted; import org.apache.tez.dag.app.dag.event.DAGEventVertexReRunning; @@ -140,6 +142,7 @@ import org.apache.tez.runtime.api.impl.EventMetaData; import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType; import org.apache.tez.runtime.api.impl.TezEvent; +import org.apache.tez.state.StateMachineTez; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -1929,6 +1932,51 @@ private void _testDAGTerminate(DAGTerminationCause terminationCause) { Assert.assertEquals(1, dagFinishEventHandler.dagFinishEvents); } + @Test (timeout = 5000L) + @SuppressWarnings("unchecked") + public void testDAGHang() throws Exception { + conf.setBoolean( + TezConfiguration.TEZ_AM_COMMIT_ALL_OUTPUTS_ON_DAG_SUCCESS, + false); + dag = Mockito.spy(new DAGImpl(dagId, conf, dagPlan, + dispatcher.getEventHandler(), taskCommunicatorManagerInterface, + fsTokens, clock, "user", thh, appContext)); + StateMachineTez spyStateMachine = + Mockito.spy(new StateMachineTez( + dag.stateMachineFactory.make(dag), dag)); + when(dag.getStateMachine()).thenReturn(spyStateMachine); + dag.entityUpdateTracker = new StateChangeNotifierForTest(dag); + doReturn(dag).when(appContext).getCurrentDAG(); + DAGImpl.OutputKey outputKey = Mockito.mock(DAGImpl.OutputKey.class); + ListenableFuture future = Mockito.mock(ListenableFuture.class); + dag.commitFutures.put(outputKey, future); + initDAG(dag); + startDAG(dag); + dispatcher.await(); + + dispatcher.getEventHandler().handle(new DAGEventVertexCompleted( + TezVertexID.getInstance(dagId, 0), VertexState.SUCCEEDED)); + dispatcher.getEventHandler().handle(new DAGEventVertexCompleted( + TezVertexID.getInstance(dagId, 1), VertexState.SUCCEEDED)); + dispatcher.getEventHandler().handle(new DAGEventVertexCompleted( + TezVertexID.getInstance(dagId, 2), VertexState.SUCCEEDED)); + dispatcher.getEventHandler().handle(new DAGEventVertexCompleted( + TezVertexID.getInstance(dagId, 3), VertexState.SUCCEEDED)); + dispatcher.getEventHandler().handle(new DAGEventVertexCompleted( + TezVertexID.getInstance(dagId, 4), VertexState.SUCCEEDED)); + dispatcher.getEventHandler().handle(new DAGEventVertexCompleted( + TezVertexID.getInstance(dagId, 5), VertexState.SUCCEEDED)); + dispatcher.await(); + Assert.assertEquals(DAGState.COMMITTING, dag.getState()); + DAGEventCommitCompleted dagEvent = new DAGEventCommitCompleted( + dagId, outputKey, false , new RuntimeException("test")); + doThrow(new RuntimeException("test")).when( + dag).logJobHistoryUnsuccesfulEvent(any(DAGState.class), any(TezCounters.class)); + dag.handle(dagEvent); + dispatcher.await(); + Assert.assertTrue("DAG did not terminate!", dag.getInternalState() == DAGState.FAILED); + } + @Test(timeout = 5000) public void testDAGKillVertexSuccessAfterTerminated() { _testDAGKillVertexSuccessAfterTerminated(DAGTerminationCause.DAG_KILL); From ebc9f4f6dee1badeca39cac26c00818be3e4d77d Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Fri, 27 Apr 2018 11:29:08 -0500 Subject: [PATCH 067/512] TEZ-3914. Recovering a large DAG fails to size limit exceeded (Jonathan Eagles via jlowe) --- .../apache/tez/dag/app/RecoveryParser.java | 19 +++-- .../tez/dag/app/dag/impl/VertexImpl.java | 11 ++- .../apache/tez/dag/history/HistoryEvent.java | 9 +- .../dag/history/events/AMLaunchedEvent.java | 12 +-- .../dag/history/events/AMStartedEvent.java | 12 +-- .../dag/history/events/AppLaunchedEvent.java | 8 +- .../events/ContainerLaunchedEvent.java | 13 ++- .../history/events/ContainerStoppedEvent.java | 13 ++- .../history/events/DAGCommitStartedEvent.java | 11 +-- .../dag/history/events/DAGFinishedEvent.java | 11 +-- .../history/events/DAGInitializedEvent.java | 14 ++-- .../history/events/DAGKillRequestEvent.java | 17 ++-- .../dag/history/events/DAGRecoveredEvent.java | 8 +- .../dag/history/events/DAGStartedEvent.java | 12 +-- .../dag/history/events/DAGSubmittedEvent.java | 11 +-- .../events/TaskAttemptFinishedEvent.java | 13 ++- .../events/TaskAttemptStartedEvent.java | 12 +-- .../dag/history/events/TaskFinishedEvent.java | 12 +-- .../dag/history/events/TaskStartedEvent.java | 12 +-- .../events/VertexCommitStartedEvent.java | 11 +-- .../events/VertexConfigurationDoneEvent.java | 12 +-- .../history/events/VertexFinishedEvent.java | 11 +-- .../VertexGroupCommitFinishedEvent.java | 11 +-- .../events/VertexGroupCommitStartedEvent.java | 11 +-- .../events/VertexInitializedEvent.java | 13 ++- .../history/events/VertexStartedEvent.java | 12 +-- .../dag/history/recovery/RecoveryService.java | 58 +++++++++---- .../tez/dag/app/TestRecoveryParser.java | 82 ++++++++++++++++++- .../TestHistoryEventsProtoConversion.java | 8 +- .../org/apache/tez/test/MiniTezCluster.java | 6 ++ .../RecoveryServiceWithEventHandlingHook.java | 9 +- .../org/apache/tez/test/TestAMRecovery.java | 4 + .../org/apache/tez/test/TestDAGRecovery.java | 8 +- .../org/apache/tez/test/TestRecovery.java | 4 + 34 files changed, 317 insertions(+), 173 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java index 368dd17f47..99ac2837ae 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java @@ -27,6 +27,7 @@ import java.util.Map; import java.util.Map.Entry; +import com.google.protobuf.CodedInputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -251,11 +252,15 @@ private static void parseSummaryFile(FSDataInputStream inputStream) } } - private static HistoryEvent getNextEvent(FSDataInputStream inputStream) + private static HistoryEvent getNextEvent(CodedInputStream inputStream) throws IOException { + boolean isAtEnd = inputStream.isAtEnd(); + if (isAtEnd) { + return null; + } int eventTypeOrdinal = -1; try { - eventTypeOrdinal = inputStream.readInt(); + eventTypeOrdinal = inputStream.readFixed32(); } catch (EOFException eof) { return null; } @@ -353,13 +358,15 @@ private static HistoryEvent getNextEvent(FSDataInputStream inputStream) public static List parseDAGRecoveryFile(FSDataInputStream inputStream) throws IOException { List historyEvents = new ArrayList(); + CodedInputStream codedInputStream = CodedInputStream.newInstance(inputStream); + codedInputStream.setSizeLimit(Integer.MAX_VALUE); while (true) { - HistoryEvent historyEvent = getNextEvent(inputStream); + HistoryEvent historyEvent = getNextEvent(codedInputStream); if (historyEvent == null) { LOG.info("Reached end of stream"); break; } - LOG.debug("Read HistoryEvent, eventType=" + historyEvent.getEventType() + ", event=" + historyEvent); + LOG.debug("Read HistoryEvent, eventType={}, event={}", historyEvent.getEventType(), historyEvent); historyEvents.add(historyEvent); } return historyEvents; @@ -745,10 +752,12 @@ public DAGRecoveryData parseRecoveryData() throws IOException { + ", dagRecoveryFile=" + dagRecoveryFile + ", len=" + fileStatus.getLen()); FSDataInputStream dagRecoveryStream = recoveryFS.open(dagRecoveryFile, recoveryBufferSize); + CodedInputStream codedInputStream = CodedInputStream.newInstance(dagRecoveryStream); + codedInputStream.setSizeLimit(Integer.MAX_VALUE); while (true) { HistoryEvent event; try { - event = getNextEvent(dagRecoveryStream); + event = getNextEvent(codedInputStream); if (event == null) { LOG.info("Reached end of dag recovery stream"); break; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index ad261736c2..f3fc2694b6 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -42,6 +42,8 @@ import javax.annotation.Nullable; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.hadoop.classification.InterfaceAudience.Private; @@ -63,7 +65,6 @@ import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.common.counters.LimitExceededException; import org.apache.tez.common.counters.TezCounters; -import org.apache.tez.common.io.NonSyncByteArrayInputStream; import org.apache.tez.common.io.NonSyncByteArrayOutputStream; import org.apache.tez.dag.api.DagTypeConverters; import org.apache.tez.dag.api.EdgeManagerPluginDescriptor; @@ -2695,9 +2696,11 @@ private void assignVertexManager() throws TezException { } NonSyncByteArrayOutputStream out = new NonSyncByteArrayOutputStream(); try { - reconfigureDoneEvent.toProtoStream(out); + CodedOutputStream codedOutputStream = CodedOutputStream.newInstance(out); + reconfigureDoneEvent.toProtoStream(codedOutputStream); + codedOutputStream.flush(); } catch (IOException e) { - throw new TezUncheckedException("Unable to deserilize VertexReconfigureDoneEvent"); + throw new TezUncheckedException("Unable to deserialize VertexReconfigureDoneEvent"); } this.vertexManager = new VertexManager( VertexManagerPluginDescriptor.create(NoOpVertexManager.class.getName()) @@ -4589,7 +4592,7 @@ public void initialize() throws Exception { LOG.debug("initialize NoOpVertexManager"); } configurationDoneEvent = new VertexConfigurationDoneEvent(); - configurationDoneEvent.fromProtoStream(new NonSyncByteArrayInputStream(getContext().getUserPayload().deepCopyAsArray())); + configurationDoneEvent.fromProtoStream(CodedInputStream.newInstance(getContext().getUserPayload().deepCopyAsArray())); String vertexName = getContext().getVertexName(); if (getContext().getVertexNumTasks(vertexName) == -1) { Preconditions.checkArgument(configurationDoneEvent.isSetParallelismCalled(), "SetParallelism must be called " diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEvent.java index 1ca0d5f069..5b077e98ec 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEvent.java @@ -18,9 +18,10 @@ package org.apache.tez.dag.history; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; + import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; public interface HistoryEvent { @@ -30,8 +31,8 @@ public interface HistoryEvent { public boolean isHistoryEvent(); - public void toProtoStream(OutputStream outputStream) throws IOException; + public void toProtoStream(CodedOutputStream outputStream) throws IOException; - public void fromProtoStream(InputStream inputStream) throws IOException; + public void fromProtoStream(CodedInputStream inputStream) throws IOException; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMLaunchedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMLaunchedEvent.java index fa332d68eb..001cbf0104 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMLaunchedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMLaunchedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.tez.dag.history.HistoryEvent; @@ -84,13 +84,13 @@ public void fromProto(AMLaunchedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - AMLaunchedProto proto = AMLaunchedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + AMLaunchedProto proto = inputStream.readMessage(AMLaunchedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMStartedEvent.java index 8a59d84769..87daba68b8 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMStartedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.tez.dag.history.HistoryEvent; @@ -79,13 +79,13 @@ public void fromProto(AMStartedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - AMStartedProto proto = AMStartedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + AMStartedProto proto = inputStream.readMessage(AMStartedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AppLaunchedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AppLaunchedEvent.java index 08d2aff021..0b812f0184 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AppLaunchedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AppLaunchedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.tez.common.VersionInfo; @@ -67,12 +67,12 @@ public boolean isHistoryEvent() { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { + public void toProtoStream(CodedOutputStream outputStream) throws IOException { throw new UnsupportedOperationException("Not a recovery event"); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { + public void fromProtoStream(CodedInputStream inputStream) throws IOException { throw new UnsupportedOperationException("Not a recovery event"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerLaunchedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerLaunchedEvent.java index 45d0261adf..11528e2f8c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerLaunchedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerLaunchedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.util.ConverterUtils; @@ -77,14 +77,13 @@ public void fromProto(ContainerLaunchedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - ContainerLaunchedProto proto = - ContainerLaunchedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + ContainerLaunchedProto proto = inputStream.readMessage(ContainerLaunchedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerStoppedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerStoppedEvent.java index 86971ce520..528f6293b6 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerStoppedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerStoppedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.util.ConverterUtils; @@ -82,14 +82,13 @@ public void fromProto(ContainerStoppedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - ContainerStoppedProto proto = - ContainerStoppedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + ContainerStoppedProto proto = inputStream.readMessage(ContainerStoppedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGCommitStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGCommitStartedEvent.java index 016bb60b09..241dada2d6 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGCommitStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGCommitStartedEvent.java @@ -19,9 +19,10 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; @@ -69,13 +70,13 @@ public void fromProto(DAGCommitStartedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - DAGCommitStartedProto proto = DAGCommitStartedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + DAGCommitStartedProto proto = inputStream.readMessage(DAGCommitStartedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java index c395297965..0a7ef561f7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java @@ -19,10 +19,11 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.util.Map; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.DagTypeConverters; @@ -121,13 +122,13 @@ public void fromProto(DAGFinishedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - DAGFinishedProto proto = DAGFinishedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + DAGFinishedProto proto = inputStream.readMessage(DAGFinishedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java index 98d64d3da8..9e6c8b282b 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java @@ -19,15 +19,16 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.util.Map; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.dag.recovery.records.RecoveryProtos; +import org.apache.tez.dag.recovery.records.RecoveryProtos.DAGInitializedProto; public class DAGInitializedEvent implements HistoryEvent { @@ -83,14 +84,13 @@ public void fromProto(RecoveryProtos.DAGInitializedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - RecoveryProtos.DAGInitializedProto proto = - RecoveryProtos.DAGInitializedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + DAGInitializedProto proto = inputStream.readMessage(DAGInitializedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGKillRequestEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGKillRequestEvent.java index 525e361364..c87f5ce7e7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGKillRequestEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGKillRequestEvent.java @@ -18,14 +18,16 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.recovery.records.RecoveryProtos; +import org.apache.tez.dag.recovery.records.RecoveryProtos.DAGKillRequestProto; import org.apache.tez.dag.recovery.records.RecoveryProtos.SummaryEventProto; import org.apache.tez.dag.utils.ProtoUtils; @@ -60,12 +62,12 @@ public boolean isHistoryEvent() { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } - public RecoveryProtos.DAGKillRequestProto toProto() { - return RecoveryProtos.DAGKillRequestProto.newBuilder() + public DAGKillRequestProto toProto() { + return DAGKillRequestProto.newBuilder() .setDagId(dagID.toString()) .setKillRequestTime(killRequestTime) .setIsSessionStopped(isSessionStopped) @@ -73,9 +75,8 @@ public RecoveryProtos.DAGKillRequestProto toProto() { } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - RecoveryProtos.DAGKillRequestProto proto = - RecoveryProtos.DAGKillRequestProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + DAGKillRequestProto proto = inputStream.readMessage(DAGKillRequestProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGRecoveredEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGRecoveredEvent.java index 2bfa43b894..e5f5614819 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGRecoveredEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGRecoveredEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.tez.dag.app.dag.DAGState; import org.apache.tez.dag.history.HistoryEvent; @@ -76,13 +76,13 @@ public boolean isHistoryEvent() { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { + public void toProtoStream(CodedOutputStream outputStream) throws IOException { throw new UnsupportedOperationException("Invalid operation for eventType " + getEventType().name()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { + public void fromProtoStream(CodedInputStream inputStream) throws IOException { throw new UnsupportedOperationException("Invalid operation for eventType " + getEventType().name()); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java index d0e0e693a5..f1fdcac809 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.app.dag.DAGState; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; @@ -74,13 +74,13 @@ public void fromProto(DAGStartedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - DAGStartedProto proto = DAGStartedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + DAGStartedProto proto = inputStream.readMessage(DAGStartedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java index 1b1fdf38a9..e04ee80188 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java @@ -19,10 +19,11 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.util.Map; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -126,13 +127,13 @@ public void fromProto(DAGSubmittedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - DAGSubmittedProto proto = DAGSubmittedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + DAGSubmittedProto proto = inputStream.readMessage(DAGSubmittedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java index e9100e8ab9..96dc0997eb 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java @@ -20,10 +20,10 @@ import javax.annotation.Nullable; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.util.List; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.common.TezConverterUtils; import org.apache.tez.runtime.api.TaskFailureType; import org.slf4j.Logger; @@ -226,14 +226,13 @@ public void fromProto(TaskAttemptFinishedProto proto) throws IOException { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - TaskAttemptFinishedProto proto = - TaskAttemptFinishedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + TaskAttemptFinishedProto proto = inputStream.readMessage(TaskAttemptFinishedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java index 71d4419ee7..a49e47c97f 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.util.ConverterUtils; @@ -91,13 +91,13 @@ public void fromProto(TaskAttemptStartedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - TaskAttemptStartedProto proto = TaskAttemptStartedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + TaskAttemptStartedProto proto = inputStream.readMessage(TaskAttemptStartedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java index 71ff6c84b7..6befa1a399 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.tez.common.counters.TezCounters; @@ -107,13 +107,13 @@ public void fromProto(TaskFinishedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - TaskFinishedProto proto = TaskFinishedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + TaskFinishedProto proto = inputStream.readMessage(TaskFinishedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java index 07dc2f9d99..cc629699a4 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.api.oldrecords.TaskState; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; @@ -83,13 +83,13 @@ public void fromProto(TaskStartedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - TaskStartedProto proto = TaskStartedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + TaskStartedProto proto = inputStream.readMessage(TaskStartedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java index c4521870e6..8ff86b8af1 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java @@ -19,10 +19,11 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.nio.charset.Charset; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; @@ -73,13 +74,13 @@ public void fromProto(VertexCommitStartedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - VertexCommitStartedProto proto = VertexCommitStartedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + VertexCommitStartedProto proto = inputStream.readMessage(VertexCommitStartedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java index 137342cf21..a2e20397cf 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java @@ -18,12 +18,12 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.util.HashMap; import java.util.Map; import java.util.Map.Entry; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.api.DagTypeConverters; import org.apache.tez.dag.api.EdgeProperty; import org.apache.tez.dag.api.VertexLocationHint; @@ -155,13 +155,13 @@ public void fromProto(VertexConfigurationDoneProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - VertexConfigurationDoneProto proto = VertexConfigurationDoneProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + VertexConfigurationDoneProto proto = inputStream.readMessage(VertexConfigurationDoneProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java index a2cdae2de6..58cb628b28 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java @@ -19,10 +19,11 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.util.Map; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.app.dag.impl.ServicePluginInfo; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -123,13 +124,13 @@ public void fromProto(VertexFinishedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - VertexFinishedProto proto = VertexFinishedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + VertexFinishedProto proto = inputStream.readMessage(VertexFinishedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java index ec8f3e1a4f..c9d5aae770 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java @@ -19,10 +19,11 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.util.Collection; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; @@ -94,13 +95,13 @@ public TezVertexID apply(String input) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - VertexGroupCommitFinishedProto proto = VertexGroupCommitFinishedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + VertexGroupCommitFinishedProto proto = inputStream.readMessage(VertexGroupCommitFinishedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java index 3de355cc64..cdd11bcc4c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java @@ -19,10 +19,11 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.util.Collection; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; @@ -94,13 +95,13 @@ public TezVertexID apply(String input) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - VertexGroupCommitStartedProto proto = VertexGroupCommitStartedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + VertexGroupCommitStartedProto proto = inputStream.readMessage(VertexGroupCommitStartedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java index 90099fce22..e7452e6770 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java @@ -19,12 +19,12 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.api.DagTypeConverters; import org.apache.tez.dag.api.InputDescriptor; import org.apache.tez.dag.api.InputInitializerDescriptor; @@ -151,14 +151,13 @@ public void fromProto(RecoveryProtos.VertexInitializedProto proto) throws IOExce } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - RecoveryProtos.VertexInitializedProto proto = - RecoveryProtos.VertexInitializedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + VertexInitializedProto proto = inputStream.readMessage(VertexInitializedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java index a8bd21eacf..4a3e05f1f4 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.app.dag.VertexState; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; @@ -74,13 +74,13 @@ public void fromProto(VertexStartedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - VertexStartedProto proto = VertexStartedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + VertexStartedProto proto = inputStream.readMessage(VertexStartedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java index 8c291722e2..d874e0a6d1 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java @@ -28,6 +28,7 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; +import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -84,8 +85,7 @@ public class RecoveryService extends AbstractService { private FileSystem recoveryDirFS; // FS where staging dir exists Path recoveryPath; @VisibleForTesting - public Map outputStreamMap = new - HashMap(); + public Map outputStreamMap = new HashMap<>(); private int bufferSize; @VisibleForTesting public FSDataOutputStream summaryStream; @@ -101,6 +101,31 @@ public class RecoveryService extends AbstractService { private volatile boolean drained = true; private Object waitForDrained = new Object(); + @VisibleForTesting + public static class RecoveryStream { + private final FSDataOutputStream outputStream; + private final CodedOutputStream codedOutputStream; + + RecoveryStream(FSDataOutputStream outputStream) { + this.outputStream = outputStream; + this.codedOutputStream = CodedOutputStream.newInstance(outputStream); + } + + public void write(byte[] bytes) throws IOException { + codedOutputStream.writeRawBytes(bytes); + } + + public void flush() throws IOException { + codedOutputStream.flush(); + outputStream.hflush(); + } + + public void close() throws IOException { + flush(); + outputStream.close(); + } + } + public RecoveryService(AppContext appContext) { super(RecoveryService.class.getName()); this.appContext = appContext; @@ -231,10 +256,9 @@ public void serviceStop() throws Exception { } } } - for (Entry entry : outputStreamMap.entrySet()) { + for (Entry entry : outputStreamMap.entrySet()) { try { LOG.info("Closing Output Stream for DAG " + entry.getKey()); - entry.getValue().hflush(); entry.getValue().close(); } catch (IOException ioe) { if (!recoveryDirFS.exists(recoveryPath)) { @@ -303,7 +327,7 @@ public void handle(DAGHistoryEvent event) throws IOException { if (event.getHistoryEvent() instanceof SummaryEvent) { synchronized (lock) { if (stopped.get()) { - LOG.warn("Igoring event as service stopped, eventType" + LOG.warn("Ignoring event as service stopped, eventType" + event.getHistoryEvent().getEventType()); return; } @@ -429,9 +453,9 @@ protected void handleRecoveryEvent(DAGHistoryEvent event) throws IOException { return; } - if (!outputStreamMap.containsKey(dagID)) { + RecoveryStream recoveryStream = outputStreamMap.get(dagID); + if (recoveryStream == null) { Path dagFilePath = TezCommonUtils.getDAGRecoveryPath(recoveryPath, dagID.toString()); - FSDataOutputStream outputStream; if (recoveryDirFS.exists(dagFilePath)) { createFatalErrorFlagDir(); return; @@ -440,12 +464,12 @@ protected void handleRecoveryEvent(DAGHistoryEvent event) throws IOException { LOG.debug("Opening DAG recovery file in create mode" + ", filePath=" + dagFilePath); } - outputStream = recoveryDirFS.create(dagFilePath, false, bufferSize); + FSDataOutputStream outputStream = recoveryDirFS.create(dagFilePath, false, bufferSize); + recoveryStream = new RecoveryStream(outputStream); } - outputStreamMap.put(dagID, outputStream); + outputStreamMap.put(dagID, recoveryStream); } - FSDataOutputStream outputStream = outputStreamMap.get(dagID); if (LOG.isDebugEnabled()) { LOG.debug("Writing recovery event to output stream" @@ -453,15 +477,15 @@ protected void handleRecoveryEvent(DAGHistoryEvent event) throws IOException { + ", eventType=" + eventType); } ++unflushedEventsCount; - outputStream.writeInt(event.getHistoryEvent().getEventType().ordinal()); - event.getHistoryEvent().toProtoStream(outputStream); + recoveryStream.codedOutputStream.writeFixed32NoTag(event.getHistoryEvent().getEventType().ordinal()); + event.getHistoryEvent().toProtoStream(recoveryStream.codedOutputStream); if (!EnumSet.of(HistoryEventType.DAG_SUBMITTED, HistoryEventType.DAG_FINISHED).contains(eventType)) { - maybeFlush(outputStream); + maybeFlush(recoveryStream); } } - private void maybeFlush(FSDataOutputStream outputStream) throws IOException { + private void maybeFlush(RecoveryStream recoveryStream) throws IOException { long currentTime = appContext.getClock().getTime(); boolean doFlush = false; if (maxUnflushedEvents >=0 @@ -482,12 +506,12 @@ private void maybeFlush(FSDataOutputStream outputStream) throws IOException { if (!doFlush) { return; } - doFlush(outputStream, currentTime); + doFlush(recoveryStream, currentTime); } - private void doFlush(FSDataOutputStream outputStream, + private void doFlush(RecoveryStream recoveryStream, long currentTime) throws IOException { - outputStream.hflush(); + recoveryStream.flush(); if (LOG.isDebugEnabled()) { LOG.debug("Flushing output stream" diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java index 6673b39e5b..1c09d5d9e8 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java @@ -20,12 +20,15 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Random; +import com.google.common.collect.Sets; +import com.google.protobuf.CodedInputStream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -36,6 +39,8 @@ import org.apache.hadoop.yarn.util.SystemClock; import org.apache.log4j.Level; import org.apache.log4j.LogManager; +import org.apache.tez.dag.api.TaskLocationHint; +import org.apache.tez.dag.api.VertexLocationHint; import org.apache.tez.dag.api.oldrecords.TaskAttemptState; import org.apache.tez.dag.api.oldrecords.TaskState; import org.apache.tez.dag.api.records.DAGProtos.DAGPlan; @@ -59,6 +64,7 @@ import org.apache.tez.dag.history.events.TaskFinishedEvent; import org.apache.tez.dag.history.events.TaskStartedEvent; import org.apache.tez.dag.history.events.VertexCommitStartedEvent; +import org.apache.tez.dag.history.events.VertexConfigurationDoneEvent; import org.apache.tez.dag.history.events.VertexFinishedEvent; import org.apache.tez.dag.history.events.VertexGroupCommitFinishedEvent; import org.apache.tez.dag.history.events.VertexGroupCommitStartedEvent; @@ -91,6 +97,8 @@ public class TestRecoveryParser { private Path recoveryPath; private DAGAppMaster mockAppMaster; private DAGImpl mockDAGImpl; + // Protobuf message limit is 64 MB by default + private static final int PROTOBUF_DEFAULT_SIZE_LIMIT = 64 << 20; @Before public void setUp() throws IllegalArgumentException, IOException { @@ -105,7 +113,6 @@ public void setUp() throws IllegalArgumentException, IOException { mockDAGImpl = mock(DAGImpl.class); when(mockAppMaster.createDAG(any(DAGPlan.class), any(TezDAGID.class))).thenReturn(mockDAGImpl); parser = new RecoveryParser(mockAppMaster, localFS, recoveryPath, 3); - LogManager.getRootLogger().setLevel(Level.DEBUG); } private DAGSummaryData createDAGSummaryData(TezDAGID dagId, boolean completed) { @@ -267,7 +274,7 @@ public void testLastCorruptedRecoveryRecord() throws IOException { null, "user", new Configuration(), null, null))); // wait until DAGSubmittedEvent is handled in the RecoveryEventHandling thread rService.await(); - rService.outputStreamMap.get(dagID).writeUTF("INVALID_DATA"); + rService.outputStreamMap.get(dagID).write("INVALID_DATA".getBytes("UTF-8")); rService.stop(); // write data in attempt_2 @@ -278,7 +285,7 @@ public void testLastCorruptedRecoveryRecord() throws IOException { rService.handle(new DAGHistoryEvent(dagID, new DAGInitializedEvent(dagID, 1L, "user", dagPlan.getName(), null))); rService.await(); - rService.outputStreamMap.get(dagID).writeUTF("INVALID_DATA"); + rService.outputStreamMap.get(dagID).write("INVALID_DATA".getBytes("UTF-8")); rService.stop(); // corrupted last records will be skipped but the whole recovery logs will be read @@ -618,6 +625,75 @@ public void testRecoverableNonSummary2() throws IOException { + ", but its full recovery events are not seen")); } + @Test(timeout=20000) + public void testRecoveryLargeEventData() throws IOException { + ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1); + TezDAGID dagID = TezDAGID.getInstance(appId, 1); + AppContext appContext = mock(AppContext.class); + when(appContext.getCurrentRecoveryDir()).thenReturn(new Path(recoveryPath+"/1")); + when(appContext.getClock()).thenReturn(new SystemClock()); + when(mockDAGImpl.getID()).thenReturn(dagID); + when(appContext.getHadoopShim()).thenReturn(new DefaultHadoopShim()); + when(appContext.getApplicationID()).thenReturn(appId); + + RecoveryService rService = new RecoveryService(appContext); + Configuration conf = new Configuration(); + conf.setBoolean(RecoveryService.TEZ_TEST_RECOVERY_DRAIN_EVENTS_WHEN_STOPPED, true); + rService.init(conf); + rService.start(); + + DAGPlan dagPlan = TestDAGImpl.createTestDAGPlan(); + // DAG DAGSubmittedEvent -> DAGInitializedEvent -> DAGStartedEvent + rService.handle(new DAGHistoryEvent(dagID, + new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1), + null, "user", new Configuration(), null, null))); + DAGInitializedEvent dagInitedEvent = new DAGInitializedEvent(dagID, 100L, + "user", "dagName", null); + DAGStartedEvent dagStartedEvent = new DAGStartedEvent(dagID, 0L, "user", "dagName"); + rService.handle(new DAGHistoryEvent(dagID, dagInitedEvent)); + rService.handle(new DAGHistoryEvent(dagID, dagStartedEvent)); + + // Create a Recovery event larger than 64 MB to verify default max protobuf size + ArrayList taskLocationHints = new ArrayList<>(100000); + TaskLocationHint taskLocationHint = TaskLocationHint.createTaskLocationHint( + Sets.newHashSet("aaaaaaaaaaaaaaa.aaaaaaaaaaaaaaa.aaaaaaaaaaaaaaa", + "bbbbbbbbbbbbbbb.bbbbbbbbbbbbbbb.bbbbbbbbbbbbbbb", + "ccccccccccccccc.ccccccccccccccc.ccccccccccccccc", + "ddddddddddddddd.ddddddddddddddd.ddddddddddddddd", + "eeeeeeeeeeeeeee.eeeeeeeeeeeeeee.eeeeeeeeeeeeeee", + "fffffffffffffff.fffffffffffffff.fffffffffffffff", + "ggggggggggggggg.ggggggggggggggg.ggggggggggggggg", + "hhhhhhhhhhhhhhh.hhhhhhhhhhhhhhh.hhhhhhhhhhhhhhh", + "iiiiiiiiiiiiiii.iiiiiiiiiiiiiii.iiiiiiiiiiiiiii", + "jjjjjjjjjjjjjjj.jjjjjjjjjjjjjjj.jjjjjjjjjjjjjjj", + "kkkkkkkkkkkkkkk.kkkkkkkkkkkkkkk.kkkkkkkkkkkkkkk", + "lllllllllllllll.lllllllllllllll.lllllllllllllll", + "mmmmmmmmmmmmmmm.mmmmmmmmmmmmmmm.mmmmmmmmmmmmmmm", + "nnnnnnnnnnnnnnn.nnnnnnnnnnnnnnn.nnnnnnnnnnnnnnn"), + Sets.newHashSet("rack1", "rack2", "rack3")); + for (int i = 0; i < 100000; i++) { + taskLocationHints.add(taskLocationHint); + } + + TezVertexID v0Id = TezVertexID.getInstance(dagID, 0); + VertexLocationHint vertexLocationHint = VertexLocationHint.create(taskLocationHints); + VertexConfigurationDoneEvent vertexConfigurationDoneEvent = new VertexConfigurationDoneEvent( + v0Id, 0, 100000, vertexLocationHint, null, null, false); + // Verify large protobuf message + assertTrue(vertexConfigurationDoneEvent.toProto().getSerializedSize() > PROTOBUF_DEFAULT_SIZE_LIMIT ); + rService.handle(new DAGHistoryEvent(dagID, vertexConfigurationDoneEvent)); + rService.stop(); + + DAGRecoveryData dagData = parser.parseRecoveryData(); + VertexRecoveryData v0data = dagData.getVertexRecoveryData(v0Id); + assertNotNull("Vertex Recovery Data should be non-null", v0data); + VertexConfigurationDoneEvent parsedVertexConfigurationDoneEvent = v0data.getVertexConfigurationDoneEvent(); + assertNotNull("Vertex Configuration Done Event should be non-null", parsedVertexConfigurationDoneEvent); + VertexLocationHint parsedVertexLocationHint = parsedVertexConfigurationDoneEvent.getVertexLocationHint(); + assertNotNull("Vertex Location Hint should be non-null", parsedVertexLocationHint); + assertEquals(parsedVertexLocationHint.getTaskLocationHints().size(), 100000); + } + @Test(timeout=5000) public void testRecoveryData() throws IOException { ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java index 47d8389e34..50a80cb5f5 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java @@ -23,6 +23,8 @@ import java.nio.ByteBuffer; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.runtime.api.TaskFailureType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -89,7 +91,9 @@ public class TestHistoryEventsProtoConversion { private HistoryEvent testProtoConversion(HistoryEvent event) throws IOException, TezException { ByteArrayOutputStream os = new ByteArrayOutputStream(); HistoryEvent deserializedEvent = null; - event.toProtoStream(os); + CodedOutputStream codedOutputStream = CodedOutputStream.newInstance(os); + event.toProtoStream(codedOutputStream); + codedOutputStream.flush(); os.flush(); os.close(); deserializedEvent = ReflectionUtils.createClazzInstance( @@ -98,7 +102,7 @@ private HistoryEvent testProtoConversion(HistoryEvent event) throws IOException, + ", eventType=" + event.getEventType() + ", bufLen=" + os.toByteArray().length); deserializedEvent.fromProtoStream( - new ByteArrayInputStream(os.toByteArray())); + CodedInputStream.newInstance(os.toByteArray())); return deserializedEvent; } diff --git a/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java b/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java index c727a8fc38..bac0e8c5cc 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java +++ b/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.Set; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -135,6 +136,11 @@ public void serviceInit(Configuration conf) throws Exception { conf.setBoolean(YarnConfiguration.NM_VMEM_CHECK_ENABLED, false); conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "000"); + conf.setInt(CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 1); + conf.setInt(CommonConfigurationKeys.IPC_CLIENT_CONNECT_TIMEOUT_KEY, 1000); + conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,0); + conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY, 0); + conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_TIMEOUT_KEY,1000); try { Path stagingPath = FileContext.getFileContext(conf).makeQualified( diff --git a/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java b/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java index c08780f473..50c5a667ac 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java +++ b/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java @@ -24,6 +24,8 @@ import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.conf.Configuration; import org.apache.tez.common.ReflectionUtils; @@ -222,15 +224,16 @@ public HistoryEvent getHistoryEvent() { private String encodeHistoryEvent(HistoryEvent event) throws IOException { ByteArrayOutputStream out = new ByteArrayOutputStream(); - event.toProtoStream(out); + CodedOutputStream codedOutputStream = CodedOutputStream.newInstance(out); + event.toProtoStream(codedOutputStream); + codedOutputStream.flush(); return event.getClass().getName() + "," + Base64.encodeBase64String(out.toByteArray()); } private HistoryEvent decodeHistoryEvent(String eventClass, String base64) throws IOException { - ByteArrayInputStream in = new ByteArrayInputStream( - Base64.decodeBase64(base64)); + CodedInputStream in = CodedInputStream.newInstance(Base64.decodeBase64(base64)); try { HistoryEvent event = ReflectionUtils.createClazzInstance(eventClass); event.fromProtoStream(in); diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java b/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java index f00ae5cb11..6d3ab1c711 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java @@ -26,6 +26,7 @@ import java.util.List; import java.util.Random; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -175,6 +176,9 @@ public void setup() throws Exception { tezConf.setBoolean( RecoveryService.TEZ_TEST_RECOVERY_DRAIN_EVENTS_WHEN_STOPPED, true); + tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,0); + tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY, 0); + tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_TIMEOUT_KEY,1000); tezSession = TezClient.create("TestDAGRecovery", tezConf); tezSession.start(); } diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java b/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java index b0c9ccc40a..cf4744b2a2 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java @@ -18,6 +18,9 @@ package org.apache.tez.test; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.log4j.Level; +import org.apache.log4j.LogManager; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -133,6 +136,9 @@ public void setup() throws Exception { tezConf.set(TezConfiguration.TEZ_AM_LAUNCH_CMD_OPTS, " -Xmx256m"); tezConf.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, true); tezConf.set(TezConfiguration.TEZ_AM_STAGING_SCRATCH_DATA_AUTO_DELETE, "false"); + tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,0); + tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY, 0); + tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_TIMEOUT_KEY,1000); tezSession = TezClient.create("TestDAGRecovery", tezConf); tezSession.start(); @@ -154,7 +160,7 @@ public void teardown() throws InterruptedException { void runDAGAndVerify(DAG dag, DAGStatus.State finalState) throws Exception { tezSession.waitTillReady(); DAGClient dagClient = tezSession.submitDAG(dag); - DAGStatus dagStatus = dagClient.getDAGStatus(null); + DAGStatus dagStatus = dagClient.getDAGStatus(null, 10); while (!dagStatus.isCompleted()) { LOG.info("Waiting for dag to complete. Sleeping for 500ms." + " DAG name: " + dag.getName() diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestRecovery.java b/tez-tests/src/test/java/org/apache/tez/test/TestRecovery.java index 93fd972161..c7b1fb9639 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestRecovery.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestRecovery.java @@ -32,6 +32,7 @@ import java.util.Set; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; @@ -478,6 +479,9 @@ private void testHashJoinExample(SimpleShutdownCondition shutdownCondition, RecoveryService.TEZ_TEST_RECOVERY_DRAIN_EVENTS_WHEN_STOPPED, false); tezConf.setBoolean( TezConfiguration.TEZ_AM_STAGING_SCRATCH_DATA_AUTO_DELETE, false); + tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,0); + tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY, 0); + tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_TIMEOUT_KEY,1000); tezConf.set(TezConfiguration.TEZ_AM_LOG_LEVEL, "INFO;org.apache.tez=DEBUG"); hashJoinExample.setConf(tezConf); From 5bf3e2d45a84bcfa4f90b7f8b03b4ec8875bee57 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Fri, 27 Apr 2018 11:25:01 -0500 Subject: [PATCH 068/512] TEZ-3887. Tez Shuffle Handler should support Index Cache configuration (Jonathan Eagles via kshukla) --- .../apache/tez/auxservices/IndexCache.java | 3 +- .../tez/auxservices/TestIndexCache.java | 335 ++++++++++++++++++ 2 files changed, 337 insertions(+), 1 deletion(-) create mode 100644 tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestIndexCache.java diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/IndexCache.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/IndexCache.java index 5a945c422f..1a9cfb2b12 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/IndexCache.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/IndexCache.java @@ -38,10 +38,11 @@ class IndexCache { private final LinkedBlockingQueue queue = new LinkedBlockingQueue(); + public static final String INDEX_CACHE_MB = "tez.shuffle.indexcache.mb"; public IndexCache(Configuration conf) { this.conf = conf; - totalMemoryAllowed = 10 * 1024 * 1024; + totalMemoryAllowed = conf.getInt(INDEX_CACHE_MB, 10) * 1024 * 1024; LOG.info("IndexCache created with max memory = " + totalMemoryAllowed); } diff --git a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestIndexCache.java b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestIndexCache.java new file mode 100644 index 0000000000..802fdd6f60 --- /dev/null +++ b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestIndexCache.java @@ -0,0 +1,335 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.auxservices; + +import java.io.DataOutputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Random; +import java.util.zip.CRC32; +import java.util.zip.CheckedOutputStream; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.ChecksumException; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.security.UserGroupInformation; + +import org.apache.tez.runtime.library.common.Constants; +import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord; +import org.junit.Before; +import org.junit.Test; + +import static org.apache.tez.auxservices.IndexCache.INDEX_CACHE_MB; +import static org.junit.Assert.*; + +public class TestIndexCache { + private Configuration conf; + private FileSystem fs; + private Path p; + + @Before + public void setUp() throws IOException { + conf = new Configuration(); + fs = FileSystem.getLocal(conf).getRaw(); + p = new Path(System.getProperty("test.build.data", "/tmp"), + "cache").makeQualified(fs.getUri(), fs.getWorkingDirectory()); + } + + @Test + public void testLRCPolicy() throws Exception { + Random r = new Random(); + long seed = r.nextLong(); + r.setSeed(seed); + System.out.println("seed: " + seed); + fs.delete(p, true); + conf.setInt(INDEX_CACHE_MB, 1); + final int partsPerMap = 1000; + final int bytesPerFile = partsPerMap * 24; + IndexCache cache = new IndexCache(conf); + + // fill cache + int totalsize = bytesPerFile; + for (; totalsize < 1024 * 1024; totalsize += bytesPerFile) { + Path f = new Path(p, Integer.toString(totalsize, 36)); + writeFile(fs, f, totalsize, partsPerMap); + TezIndexRecord rec = cache.getIndexInformation( + Integer.toString(totalsize, 36), r.nextInt(partsPerMap), f, + UserGroupInformation.getCurrentUser().getShortUserName()); + checkRecord(rec, totalsize); + } + + // delete files, ensure cache retains all elem + for (FileStatus stat : fs.listStatus(p)) { + fs.delete(stat.getPath(),true); + } + for (int i = bytesPerFile; i < 1024 * 1024; i += bytesPerFile) { + Path f = new Path(p, Integer.toString(i, 36)); + TezIndexRecord rec = cache.getIndexInformation(Integer.toString(i, 36), + r.nextInt(partsPerMap), f, + UserGroupInformation.getCurrentUser().getShortUserName()); + checkRecord(rec, i); + } + + // push oldest (bytesPerFile) out of cache + Path f = new Path(p, Integer.toString(totalsize, 36)); + writeFile(fs, f, totalsize, partsPerMap); + cache.getIndexInformation(Integer.toString(totalsize, 36), + r.nextInt(partsPerMap), f, + UserGroupInformation.getCurrentUser().getShortUserName()); + fs.delete(f, false); + + // oldest fails to read, or error + boolean fnf = false; + try { + cache.getIndexInformation(Integer.toString(bytesPerFile, 36), + r.nextInt(partsPerMap), new Path(p, Integer.toString(bytesPerFile)), + UserGroupInformation.getCurrentUser().getShortUserName()); + } catch (IOException e) { + if (e.getCause() == null || + !(e.getCause() instanceof FileNotFoundException)) { + throw e; + } + else { + fnf = true; + } + } + if (!fnf) + fail("Failed to push out last entry"); + // should find all the other entries + for (int i = bytesPerFile << 1; i < 1024 * 1024; i += bytesPerFile) { + TezIndexRecord rec = cache.getIndexInformation(Integer.toString(i, 36), + r.nextInt(partsPerMap), new Path(p, Integer.toString(i, 36)), + UserGroupInformation.getCurrentUser().getShortUserName()); + checkRecord(rec, i); + } + TezIndexRecord rec = cache.getIndexInformation(Integer.toString(totalsize, 36), + r.nextInt(partsPerMap), f, + UserGroupInformation.getCurrentUser().getShortUserName()); + + checkRecord(rec, totalsize); + } + + @Test + public void testBadIndex() throws Exception { + final int parts = 30; + fs.delete(p, true); + conf.setInt(INDEX_CACHE_MB, 1); + IndexCache cache = new IndexCache(conf); + + Path f = new Path(p, "badindex"); + FSDataOutputStream out = fs.create(f, false); + CheckedOutputStream iout = new CheckedOutputStream(out, new CRC32()); + DataOutputStream dout = new DataOutputStream(iout); + for (int i = 0; i < parts; ++i) { + for (int j = 0; j < Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH / 8; ++j) { + if (0 == (i % 3)) { + dout.writeLong(i); + } else { + out.writeLong(i); + } + } + } + out.writeLong(iout.getChecksum().getValue()); + dout.close(); + try { + cache.getIndexInformation("badindex", 7, f, + UserGroupInformation.getCurrentUser().getShortUserName()); + fail("Did not detect bad checksum"); + } catch (IOException e) { + if (!(e.getCause() instanceof ChecksumException)) { + throw e; + } + } + } + + @Test + public void testInvalidReduceNumberOrLength() throws Exception { + fs.delete(p, true); + conf.setInt(INDEX_CACHE_MB, 1); + final int partsPerMap = 1000; + final int bytesPerFile = partsPerMap * 24; + IndexCache cache = new IndexCache(conf); + + // fill cache + Path feq = new Path(p, "invalidReduceOrPartsPerMap"); + writeFile(fs, feq, bytesPerFile, partsPerMap); + + // Number of reducers should always be less than partsPerMap as reducer + // numbers start from 0 and there cannot be more reducer than parts + + try { + // Number of reducers equal to partsPerMap + cache.getIndexInformation("reduceEqualPartsPerMap", + partsPerMap, // reduce number == partsPerMap + feq, UserGroupInformation.getCurrentUser().getShortUserName()); + fail("Number of reducers equal to partsPerMap did not fail"); + } catch (Exception e) { + if (!(e instanceof IOException)) { + throw e; + } + } + + try { + // Number of reducers more than partsPerMap + cache.getIndexInformation( + "reduceMorePartsPerMap", + partsPerMap + 1, // reduce number > partsPerMap + feq, UserGroupInformation.getCurrentUser().getShortUserName()); + fail("Number of reducers more than partsPerMap did not fail"); + } catch (Exception e) { + if (!(e instanceof IOException)) { + throw e; + } + } + } + + @Test + public void testRemoveMap() throws Exception { + // This test case use two thread to call getIndexInformation and + // removeMap concurrently, in order to construct race condition. + // This test case may not repeatable. But on my macbook this test + // fails with probability of 100% on code before MAPREDUCE-2541, + // so it is repeatable in practice. + fs.delete(p, true); + conf.setInt(INDEX_CACHE_MB, 10); + // Make a big file so removeMapThread almost surely runs faster than + // getInfoThread + final int partsPerMap = 100000; + final int bytesPerFile = partsPerMap * 24; + final IndexCache cache = new IndexCache(conf); + + final Path big = new Path(p, "bigIndex"); + final String user = + UserGroupInformation.getCurrentUser().getShortUserName(); + writeFile(fs, big, bytesPerFile, partsPerMap); + + // run multiple times + for (int i = 0; i < 20; ++i) { + Thread getInfoThread = new Thread() { + @Override + public void run() { + try { + cache.getIndexInformation("bigIndex", partsPerMap, big, user); + } catch (Exception e) { + // should not be here + } + } + }; + Thread removeMapThread = new Thread() { + @Override + public void run() { + cache.removeMap("bigIndex"); + } + }; + if (i%2==0) { + getInfoThread.start(); + removeMapThread.start(); + } else { + removeMapThread.start(); + getInfoThread.start(); + } + getInfoThread.join(); + removeMapThread.join(); + assertEquals(true, cache.checkTotalMemoryUsed()); + } + } + + @Test + public void testCreateRace() throws Exception { + fs.delete(p, true); + conf.setInt(INDEX_CACHE_MB, 1); + final int partsPerMap = 1000; + final int bytesPerFile = partsPerMap * 24; + final IndexCache cache = new IndexCache(conf); + + final Path racy = new Path(p, "racyIndex"); + final String user = + UserGroupInformation.getCurrentUser().getShortUserName(); + writeFile(fs, racy, bytesPerFile, partsPerMap); + + // run multiple instances + Thread[] getInfoThreads = new Thread[50]; + for (int i = 0; i < 50; i++) { + getInfoThreads[i] = new Thread() { + @Override + public void run() { + try { + cache.getIndexInformation("racyIndex", partsPerMap, racy, user); + cache.removeMap("racyIndex"); + } catch (Exception e) { + // should not be here + } + } + }; + } + + for (int i = 0; i < 50; i++) { + getInfoThreads[i].start(); + } + + final Thread mainTestThread = Thread.currentThread(); + + Thread timeoutThread = new Thread() { + @Override + public void run() { + try { + Thread.sleep(15000); + mainTestThread.interrupt(); + } catch (InterruptedException ie) { + // we are done; + } + } + }; + + for (int i = 0; i < 50; i++) { + try { + getInfoThreads[i].join(); + } catch (InterruptedException ie) { + // we haven't finished in time. Potential deadlock/race. + fail("Unexpectedly long delay during concurrent cache entry creations"); + } + } + // stop the timeoutThread. If we get interrupted before stopping, there + // must be something wrong, although it wasn't a deadlock. No need to + // catch and swallow. + timeoutThread.interrupt(); + } + + private static void checkRecord(TezIndexRecord rec, long fill) { + assertEquals(fill, rec.getStartOffset()); + assertEquals(fill, rec.getRawLength()); + assertEquals(fill, rec.getPartLength()); + } + + private static void writeFile(FileSystem fs, Path f, long fill, int parts) + throws IOException { + FSDataOutputStream out = fs.create(f, false); + CheckedOutputStream iout = new CheckedOutputStream(out, new CRC32()); + DataOutputStream dout = new DataOutputStream(iout); + for (int i = 0; i < parts; ++i) { + for (int j = 0; j < Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH / 8; ++j) { + dout.writeLong(fill); + } + } + out.writeLong(iout.getChecksum().getValue()); + dout.close(); + } +} From 086d7bad24fcd74c61180353735297114c9974af Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Fri, 27 Apr 2018 13:14:36 -0500 Subject: [PATCH 069/512] TEZ-3873. A maven enforcer plugin dependency error in pom.xml (Jinjiang Ling via jeagles) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 16745f0d57..18b5a208f1 100644 --- a/pom.xml +++ b/pom.xml @@ -778,7 +778,7 @@ org.apache.maven.plugins - maven-enforce-plugin + maven-enforcer-plugin 1.4 From 21cd0241776c46821c83a67aac2c979e405651b6 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Fri, 27 Apr 2018 14:14:40 -0500 Subject: [PATCH 070/512] Revert "TEZ-3914. Recovering a large DAG fails to size limit exceeded (Jonathan Eagles via jlowe)" This reverts commit ebc9f4f6dee1badeca39cac26c00818be3e4d77d. --- .../apache/tez/dag/app/RecoveryParser.java | 19 ++--- .../tez/dag/app/dag/impl/VertexImpl.java | 11 +-- .../apache/tez/dag/history/HistoryEvent.java | 9 +- .../dag/history/events/AMLaunchedEvent.java | 12 +-- .../dag/history/events/AMStartedEvent.java | 12 +-- .../dag/history/events/AppLaunchedEvent.java | 8 +- .../events/ContainerLaunchedEvent.java | 13 +-- .../history/events/ContainerStoppedEvent.java | 13 +-- .../history/events/DAGCommitStartedEvent.java | 11 ++- .../dag/history/events/DAGFinishedEvent.java | 11 ++- .../history/events/DAGInitializedEvent.java | 14 ++-- .../history/events/DAGKillRequestEvent.java | 17 ++-- .../dag/history/events/DAGRecoveredEvent.java | 8 +- .../dag/history/events/DAGStartedEvent.java | 12 +-- .../dag/history/events/DAGSubmittedEvent.java | 11 ++- .../events/TaskAttemptFinishedEvent.java | 13 +-- .../events/TaskAttemptStartedEvent.java | 12 +-- .../dag/history/events/TaskFinishedEvent.java | 12 +-- .../dag/history/events/TaskStartedEvent.java | 12 +-- .../events/VertexCommitStartedEvent.java | 11 ++- .../events/VertexConfigurationDoneEvent.java | 12 +-- .../history/events/VertexFinishedEvent.java | 11 ++- .../VertexGroupCommitFinishedEvent.java | 11 ++- .../events/VertexGroupCommitStartedEvent.java | 11 ++- .../events/VertexInitializedEvent.java | 13 +-- .../history/events/VertexStartedEvent.java | 12 +-- .../dag/history/recovery/RecoveryService.java | 58 ++++--------- .../tez/dag/app/TestRecoveryParser.java | 82 +------------------ .../TestHistoryEventsProtoConversion.java | 8 +- .../org/apache/tez/test/MiniTezCluster.java | 6 -- .../RecoveryServiceWithEventHandlingHook.java | 9 +- .../org/apache/tez/test/TestAMRecovery.java | 4 - .../org/apache/tez/test/TestDAGRecovery.java | 8 +- .../org/apache/tez/test/TestRecovery.java | 4 - 34 files changed, 173 insertions(+), 317 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java index 99ac2837ae..368dd17f47 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java @@ -27,7 +27,6 @@ import java.util.Map; import java.util.Map.Entry; -import com.google.protobuf.CodedInputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -252,15 +251,11 @@ private static void parseSummaryFile(FSDataInputStream inputStream) } } - private static HistoryEvent getNextEvent(CodedInputStream inputStream) + private static HistoryEvent getNextEvent(FSDataInputStream inputStream) throws IOException { - boolean isAtEnd = inputStream.isAtEnd(); - if (isAtEnd) { - return null; - } int eventTypeOrdinal = -1; try { - eventTypeOrdinal = inputStream.readFixed32(); + eventTypeOrdinal = inputStream.readInt(); } catch (EOFException eof) { return null; } @@ -358,15 +353,13 @@ private static HistoryEvent getNextEvent(CodedInputStream inputStream) public static List parseDAGRecoveryFile(FSDataInputStream inputStream) throws IOException { List historyEvents = new ArrayList(); - CodedInputStream codedInputStream = CodedInputStream.newInstance(inputStream); - codedInputStream.setSizeLimit(Integer.MAX_VALUE); while (true) { - HistoryEvent historyEvent = getNextEvent(codedInputStream); + HistoryEvent historyEvent = getNextEvent(inputStream); if (historyEvent == null) { LOG.info("Reached end of stream"); break; } - LOG.debug("Read HistoryEvent, eventType={}, event={}", historyEvent.getEventType(), historyEvent); + LOG.debug("Read HistoryEvent, eventType=" + historyEvent.getEventType() + ", event=" + historyEvent); historyEvents.add(historyEvent); } return historyEvents; @@ -752,12 +745,10 @@ public DAGRecoveryData parseRecoveryData() throws IOException { + ", dagRecoveryFile=" + dagRecoveryFile + ", len=" + fileStatus.getLen()); FSDataInputStream dagRecoveryStream = recoveryFS.open(dagRecoveryFile, recoveryBufferSize); - CodedInputStream codedInputStream = CodedInputStream.newInstance(dagRecoveryStream); - codedInputStream.setSizeLimit(Integer.MAX_VALUE); while (true) { HistoryEvent event; try { - event = getNextEvent(codedInputStream); + event = getNextEvent(dagRecoveryStream); if (event == null) { LOG.info("Reached end of dag recovery stream"); break; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index f3fc2694b6..ad261736c2 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -42,8 +42,6 @@ import javax.annotation.Nullable; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.hadoop.classification.InterfaceAudience.Private; @@ -65,6 +63,7 @@ import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.common.counters.LimitExceededException; import org.apache.tez.common.counters.TezCounters; +import org.apache.tez.common.io.NonSyncByteArrayInputStream; import org.apache.tez.common.io.NonSyncByteArrayOutputStream; import org.apache.tez.dag.api.DagTypeConverters; import org.apache.tez.dag.api.EdgeManagerPluginDescriptor; @@ -2696,11 +2695,9 @@ private void assignVertexManager() throws TezException { } NonSyncByteArrayOutputStream out = new NonSyncByteArrayOutputStream(); try { - CodedOutputStream codedOutputStream = CodedOutputStream.newInstance(out); - reconfigureDoneEvent.toProtoStream(codedOutputStream); - codedOutputStream.flush(); + reconfigureDoneEvent.toProtoStream(out); } catch (IOException e) { - throw new TezUncheckedException("Unable to deserialize VertexReconfigureDoneEvent"); + throw new TezUncheckedException("Unable to deserilize VertexReconfigureDoneEvent"); } this.vertexManager = new VertexManager( VertexManagerPluginDescriptor.create(NoOpVertexManager.class.getName()) @@ -4592,7 +4589,7 @@ public void initialize() throws Exception { LOG.debug("initialize NoOpVertexManager"); } configurationDoneEvent = new VertexConfigurationDoneEvent(); - configurationDoneEvent.fromProtoStream(CodedInputStream.newInstance(getContext().getUserPayload().deepCopyAsArray())); + configurationDoneEvent.fromProtoStream(new NonSyncByteArrayInputStream(getContext().getUserPayload().deepCopyAsArray())); String vertexName = getContext().getVertexName(); if (getContext().getVertexNumTasks(vertexName) == -1) { Preconditions.checkArgument(configurationDoneEvent.isSetParallelismCalled(), "SetParallelism must be called " diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEvent.java index 5b077e98ec..1ca0d5f069 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEvent.java @@ -18,10 +18,9 @@ package org.apache.tez.dag.history; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; - import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; public interface HistoryEvent { @@ -31,8 +30,8 @@ public interface HistoryEvent { public boolean isHistoryEvent(); - public void toProtoStream(CodedOutputStream outputStream) throws IOException; + public void toProtoStream(OutputStream outputStream) throws IOException; - public void fromProtoStream(CodedInputStream inputStream) throws IOException; + public void fromProtoStream(InputStream inputStream) throws IOException; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMLaunchedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMLaunchedEvent.java index 001cbf0104..fa332d68eb 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMLaunchedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMLaunchedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.tez.dag.history.HistoryEvent; @@ -84,13 +84,13 @@ public void fromProto(AMLaunchedProto proto) { } @Override - public void toProtoStream(CodedOutputStream outputStream) throws IOException { - outputStream.writeMessageNoTag(toProto()); + public void toProtoStream(OutputStream outputStream) throws IOException { + toProto().writeDelimitedTo(outputStream); } @Override - public void fromProtoStream(CodedInputStream inputStream) throws IOException { - AMLaunchedProto proto = inputStream.readMessage(AMLaunchedProto.PARSER, null); + public void fromProtoStream(InputStream inputStream) throws IOException { + AMLaunchedProto proto = AMLaunchedProto.parseDelimitedFrom(inputStream); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMStartedEvent.java index 87daba68b8..8a59d84769 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMStartedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.tez.dag.history.HistoryEvent; @@ -79,13 +79,13 @@ public void fromProto(AMStartedProto proto) { } @Override - public void toProtoStream(CodedOutputStream outputStream) throws IOException { - outputStream.writeMessageNoTag(toProto()); + public void toProtoStream(OutputStream outputStream) throws IOException { + toProto().writeDelimitedTo(outputStream); } @Override - public void fromProtoStream(CodedInputStream inputStream) throws IOException { - AMStartedProto proto = inputStream.readMessage(AMStartedProto.PARSER, null); + public void fromProtoStream(InputStream inputStream) throws IOException { + AMStartedProto proto = AMStartedProto.parseDelimitedFrom(inputStream); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AppLaunchedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AppLaunchedEvent.java index 0b812f0184..08d2aff021 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AppLaunchedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AppLaunchedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.tez.common.VersionInfo; @@ -67,12 +67,12 @@ public boolean isHistoryEvent() { } @Override - public void toProtoStream(CodedOutputStream outputStream) throws IOException { + public void toProtoStream(OutputStream outputStream) throws IOException { throw new UnsupportedOperationException("Not a recovery event"); } @Override - public void fromProtoStream(CodedInputStream inputStream) throws IOException { + public void fromProtoStream(InputStream inputStream) throws IOException { throw new UnsupportedOperationException("Not a recovery event"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerLaunchedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerLaunchedEvent.java index 11528e2f8c..45d0261adf 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerLaunchedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerLaunchedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.util.ConverterUtils; @@ -77,13 +77,14 @@ public void fromProto(ContainerLaunchedProto proto) { } @Override - public void toProtoStream(CodedOutputStream outputStream) throws IOException { - outputStream.writeMessageNoTag(toProto()); + public void toProtoStream(OutputStream outputStream) throws IOException { + toProto().writeDelimitedTo(outputStream); } @Override - public void fromProtoStream(CodedInputStream inputStream) throws IOException { - ContainerLaunchedProto proto = inputStream.readMessage(ContainerLaunchedProto.PARSER, null); + public void fromProtoStream(InputStream inputStream) throws IOException { + ContainerLaunchedProto proto = + ContainerLaunchedProto.parseDelimitedFrom(inputStream); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerStoppedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerStoppedEvent.java index 528f6293b6..86971ce520 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerStoppedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerStoppedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.util.ConverterUtils; @@ -82,13 +82,14 @@ public void fromProto(ContainerStoppedProto proto) { } @Override - public void toProtoStream(CodedOutputStream outputStream) throws IOException { - outputStream.writeMessageNoTag(toProto()); + public void toProtoStream(OutputStream outputStream) throws IOException { + toProto().writeDelimitedTo(outputStream); } @Override - public void fromProtoStream(CodedInputStream inputStream) throws IOException { - ContainerStoppedProto proto = inputStream.readMessage(ContainerStoppedProto.PARSER, null); + public void fromProtoStream(InputStream inputStream) throws IOException { + ContainerStoppedProto proto = + ContainerStoppedProto.parseDelimitedFrom(inputStream); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGCommitStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGCommitStartedEvent.java index 241dada2d6..016bb60b09 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGCommitStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGCommitStartedEvent.java @@ -19,10 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; +import java.io.InputStream; import java.io.OutputStream; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; @@ -70,13 +69,13 @@ public void fromProto(DAGCommitStartedProto proto) { } @Override - public void toProtoStream(CodedOutputStream outputStream) throws IOException { - outputStream.writeMessageNoTag(toProto()); + public void toProtoStream(OutputStream outputStream) throws IOException { + toProto().writeDelimitedTo(outputStream); } @Override - public void fromProtoStream(CodedInputStream inputStream) throws IOException { - DAGCommitStartedProto proto = inputStream.readMessage(DAGCommitStartedProto.PARSER, null); + public void fromProtoStream(InputStream inputStream) throws IOException { + DAGCommitStartedProto proto = DAGCommitStartedProto.parseDelimitedFrom(inputStream); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java index 0a7ef561f7..c395297965 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java @@ -19,11 +19,10 @@ package org.apache.tez.dag.history.events; import java.io.IOException; +import java.io.InputStream; import java.io.OutputStream; import java.util.Map; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.DagTypeConverters; @@ -122,13 +121,13 @@ public void fromProto(DAGFinishedProto proto) { } @Override - public void toProtoStream(CodedOutputStream outputStream) throws IOException { - outputStream.writeMessageNoTag(toProto()); + public void toProtoStream(OutputStream outputStream) throws IOException { + toProto().writeDelimitedTo(outputStream); } @Override - public void fromProtoStream(CodedInputStream inputStream) throws IOException { - DAGFinishedProto proto = inputStream.readMessage(DAGFinishedProto.PARSER, null); + public void fromProtoStream(InputStream inputStream) throws IOException { + DAGFinishedProto proto = DAGFinishedProto.parseDelimitedFrom(inputStream); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java index 9e6c8b282b..98d64d3da8 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java @@ -19,16 +19,15 @@ package org.apache.tez.dag.history.events; import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; import java.util.Map; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.dag.recovery.records.RecoveryProtos; -import org.apache.tez.dag.recovery.records.RecoveryProtos.DAGInitializedProto; public class DAGInitializedEvent implements HistoryEvent { @@ -84,13 +83,14 @@ public void fromProto(RecoveryProtos.DAGInitializedProto proto) { } @Override - public void toProtoStream(CodedOutputStream outputStream) throws IOException { - outputStream.writeMessageNoTag(toProto()); + public void toProtoStream(OutputStream outputStream) throws IOException { + toProto().writeDelimitedTo(outputStream); } @Override - public void fromProtoStream(CodedInputStream inputStream) throws IOException { - DAGInitializedProto proto = inputStream.readMessage(DAGInitializedProto.PARSER, null); + public void fromProtoStream(InputStream inputStream) throws IOException { + RecoveryProtos.DAGInitializedProto proto = + RecoveryProtos.DAGInitializedProto.parseDelimitedFrom(inputStream); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGKillRequestEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGKillRequestEvent.java index c87f5ce7e7..525e361364 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGKillRequestEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGKillRequestEvent.java @@ -18,16 +18,14 @@ package org.apache.tez.dag.history.events; import java.io.IOException; +import java.io.InputStream; import java.io.OutputStream; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.recovery.records.RecoveryProtos; -import org.apache.tez.dag.recovery.records.RecoveryProtos.DAGKillRequestProto; import org.apache.tez.dag.recovery.records.RecoveryProtos.SummaryEventProto; import org.apache.tez.dag.utils.ProtoUtils; @@ -62,12 +60,12 @@ public boolean isHistoryEvent() { } @Override - public void toProtoStream(CodedOutputStream outputStream) throws IOException { - outputStream.writeMessageNoTag(toProto()); + public void toProtoStream(OutputStream outputStream) throws IOException { + toProto().writeDelimitedTo(outputStream); } - public DAGKillRequestProto toProto() { - return DAGKillRequestProto.newBuilder() + public RecoveryProtos.DAGKillRequestProto toProto() { + return RecoveryProtos.DAGKillRequestProto.newBuilder() .setDagId(dagID.toString()) .setKillRequestTime(killRequestTime) .setIsSessionStopped(isSessionStopped) @@ -75,8 +73,9 @@ public DAGKillRequestProto toProto() { } @Override - public void fromProtoStream(CodedInputStream inputStream) throws IOException { - DAGKillRequestProto proto = inputStream.readMessage(DAGKillRequestProto.PARSER, null); + public void fromProtoStream(InputStream inputStream) throws IOException { + RecoveryProtos.DAGKillRequestProto proto = + RecoveryProtos.DAGKillRequestProto.parseDelimitedFrom(inputStream); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGRecoveredEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGRecoveredEvent.java index e5f5614819..2bfa43b894 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGRecoveredEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGRecoveredEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.tez.dag.app.dag.DAGState; import org.apache.tez.dag.history.HistoryEvent; @@ -76,13 +76,13 @@ public boolean isHistoryEvent() { } @Override - public void toProtoStream(CodedOutputStream outputStream) throws IOException { + public void toProtoStream(OutputStream outputStream) throws IOException { throw new UnsupportedOperationException("Invalid operation for eventType " + getEventType().name()); } @Override - public void fromProtoStream(CodedInputStream inputStream) throws IOException { + public void fromProtoStream(InputStream inputStream) throws IOException { throw new UnsupportedOperationException("Invalid operation for eventType " + getEventType().name()); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java index f1fdcac809..d0e0e693a5 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.app.dag.DAGState; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; @@ -74,13 +74,13 @@ public void fromProto(DAGStartedProto proto) { } @Override - public void toProtoStream(CodedOutputStream outputStream) throws IOException { - outputStream.writeMessageNoTag(toProto()); + public void toProtoStream(OutputStream outputStream) throws IOException { + toProto().writeDelimitedTo(outputStream); } @Override - public void fromProtoStream(CodedInputStream inputStream) throws IOException { - DAGStartedProto proto = inputStream.readMessage(DAGStartedProto.PARSER, null); + public void fromProtoStream(InputStream inputStream) throws IOException { + DAGStartedProto proto = DAGStartedProto.parseDelimitedFrom(inputStream); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java index e04ee80188..1b1fdf38a9 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java @@ -19,11 +19,10 @@ package org.apache.tez.dag.history.events; import java.io.IOException; +import java.io.InputStream; import java.io.OutputStream; import java.util.Map; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -127,13 +126,13 @@ public void fromProto(DAGSubmittedProto proto) { } @Override - public void toProtoStream(CodedOutputStream outputStream) throws IOException { - outputStream.writeMessageNoTag(toProto()); + public void toProtoStream(OutputStream outputStream) throws IOException { + toProto().writeDelimitedTo(outputStream); } @Override - public void fromProtoStream(CodedInputStream inputStream) throws IOException { - DAGSubmittedProto proto = inputStream.readMessage(DAGSubmittedProto.PARSER, null); + public void fromProtoStream(InputStream inputStream) throws IOException { + DAGSubmittedProto proto = DAGSubmittedProto.parseDelimitedFrom(inputStream); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java index 96dc0997eb..e9100e8ab9 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java @@ -20,10 +20,10 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; import java.util.List; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.apache.tez.common.TezConverterUtils; import org.apache.tez.runtime.api.TaskFailureType; import org.slf4j.Logger; @@ -226,13 +226,14 @@ public void fromProto(TaskAttemptFinishedProto proto) throws IOException { } @Override - public void toProtoStream(CodedOutputStream outputStream) throws IOException { - outputStream.writeMessageNoTag(toProto()); + public void toProtoStream(OutputStream outputStream) throws IOException { + toProto().writeDelimitedTo(outputStream); } @Override - public void fromProtoStream(CodedInputStream inputStream) throws IOException { - TaskAttemptFinishedProto proto = inputStream.readMessage(TaskAttemptFinishedProto.PARSER, null); + public void fromProtoStream(InputStream inputStream) throws IOException { + TaskAttemptFinishedProto proto = + TaskAttemptFinishedProto.parseDelimitedFrom(inputStream); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java index a49e47c97f..71d4419ee7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.util.ConverterUtils; @@ -91,13 +91,13 @@ public void fromProto(TaskAttemptStartedProto proto) { } @Override - public void toProtoStream(CodedOutputStream outputStream) throws IOException { - outputStream.writeMessageNoTag(toProto()); + public void toProtoStream(OutputStream outputStream) throws IOException { + toProto().writeDelimitedTo(outputStream); } @Override - public void fromProtoStream(CodedInputStream inputStream) throws IOException { - TaskAttemptStartedProto proto = inputStream.readMessage(TaskAttemptStartedProto.PARSER, null); + public void fromProtoStream(InputStream inputStream) throws IOException { + TaskAttemptStartedProto proto = TaskAttemptStartedProto.parseDelimitedFrom(inputStream); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java index 6befa1a399..71ff6c84b7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.tez.common.counters.TezCounters; @@ -107,13 +107,13 @@ public void fromProto(TaskFinishedProto proto) { } @Override - public void toProtoStream(CodedOutputStream outputStream) throws IOException { - outputStream.writeMessageNoTag(toProto()); + public void toProtoStream(OutputStream outputStream) throws IOException { + toProto().writeDelimitedTo(outputStream); } @Override - public void fromProtoStream(CodedInputStream inputStream) throws IOException { - TaskFinishedProto proto = inputStream.readMessage(TaskFinishedProto.PARSER, null); + public void fromProtoStream(InputStream inputStream) throws IOException { + TaskFinishedProto proto = TaskFinishedProto.parseDelimitedFrom(inputStream); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java index cc629699a4..07dc2f9d99 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.api.oldrecords.TaskState; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; @@ -83,13 +83,13 @@ public void fromProto(TaskStartedProto proto) { } @Override - public void toProtoStream(CodedOutputStream outputStream) throws IOException { - outputStream.writeMessageNoTag(toProto()); + public void toProtoStream(OutputStream outputStream) throws IOException { + toProto().writeDelimitedTo(outputStream); } @Override - public void fromProtoStream(CodedInputStream inputStream) throws IOException { - TaskStartedProto proto = inputStream.readMessage(TaskStartedProto.PARSER, null); + public void fromProtoStream(InputStream inputStream) throws IOException { + TaskStartedProto proto = TaskStartedProto.parseDelimitedFrom(inputStream); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java index 8ff86b8af1..c4521870e6 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java @@ -19,11 +19,10 @@ package org.apache.tez.dag.history.events; import java.io.IOException; +import java.io.InputStream; import java.io.OutputStream; import java.nio.charset.Charset; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; @@ -74,13 +73,13 @@ public void fromProto(VertexCommitStartedProto proto) { } @Override - public void toProtoStream(CodedOutputStream outputStream) throws IOException { - outputStream.writeMessageNoTag(toProto()); + public void toProtoStream(OutputStream outputStream) throws IOException { + toProto().writeDelimitedTo(outputStream); } @Override - public void fromProtoStream(CodedInputStream inputStream) throws IOException { - VertexCommitStartedProto proto = inputStream.readMessage(VertexCommitStartedProto.PARSER, null); + public void fromProtoStream(InputStream inputStream) throws IOException { + VertexCommitStartedProto proto = VertexCommitStartedProto.parseDelimitedFrom(inputStream); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java index a2e20397cf..137342cf21 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java @@ -18,12 +18,12 @@ package org.apache.tez.dag.history.events; import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; import java.util.HashMap; import java.util.Map; import java.util.Map.Entry; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.api.DagTypeConverters; import org.apache.tez.dag.api.EdgeProperty; import org.apache.tez.dag.api.VertexLocationHint; @@ -155,13 +155,13 @@ public void fromProto(VertexConfigurationDoneProto proto) { } @Override - public void toProtoStream(CodedOutputStream outputStream) throws IOException { - outputStream.writeMessageNoTag(toProto()); + public void toProtoStream(OutputStream outputStream) throws IOException { + toProto().writeDelimitedTo(outputStream); } @Override - public void fromProtoStream(CodedInputStream inputStream) throws IOException { - VertexConfigurationDoneProto proto = inputStream.readMessage(VertexConfigurationDoneProto.PARSER, null); + public void fromProtoStream(InputStream inputStream) throws IOException { + VertexConfigurationDoneProto proto = VertexConfigurationDoneProto.parseDelimitedFrom(inputStream); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java index 58cb628b28..a2cdae2de6 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java @@ -19,11 +19,10 @@ package org.apache.tez.dag.history.events; import java.io.IOException; +import java.io.InputStream; import java.io.OutputStream; import java.util.Map; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.app.dag.impl.ServicePluginInfo; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -124,13 +123,13 @@ public void fromProto(VertexFinishedProto proto) { } @Override - public void toProtoStream(CodedOutputStream outputStream) throws IOException { - outputStream.writeMessageNoTag(toProto()); + public void toProtoStream(OutputStream outputStream) throws IOException { + toProto().writeDelimitedTo(outputStream); } @Override - public void fromProtoStream(CodedInputStream inputStream) throws IOException { - VertexFinishedProto proto = inputStream.readMessage(VertexFinishedProto.PARSER, null); + public void fromProtoStream(InputStream inputStream) throws IOException { + VertexFinishedProto proto = VertexFinishedProto.parseDelimitedFrom(inputStream); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java index c9d5aae770..ec8f3e1a4f 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java @@ -19,11 +19,10 @@ package org.apache.tez.dag.history.events; import java.io.IOException; +import java.io.InputStream; import java.io.OutputStream; import java.util.Collection; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; @@ -95,13 +94,13 @@ public TezVertexID apply(String input) { } @Override - public void toProtoStream(CodedOutputStream outputStream) throws IOException { - outputStream.writeMessageNoTag(toProto()); + public void toProtoStream(OutputStream outputStream) throws IOException { + toProto().writeDelimitedTo(outputStream); } @Override - public void fromProtoStream(CodedInputStream inputStream) throws IOException { - VertexGroupCommitFinishedProto proto = inputStream.readMessage(VertexGroupCommitFinishedProto.PARSER, null); + public void fromProtoStream(InputStream inputStream) throws IOException { + VertexGroupCommitFinishedProto proto = VertexGroupCommitFinishedProto.parseDelimitedFrom(inputStream); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java index cdd11bcc4c..3de355cc64 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java @@ -19,11 +19,10 @@ package org.apache.tez.dag.history.events; import java.io.IOException; +import java.io.InputStream; import java.io.OutputStream; import java.util.Collection; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; @@ -95,13 +94,13 @@ public TezVertexID apply(String input) { } @Override - public void toProtoStream(CodedOutputStream outputStream) throws IOException { - outputStream.writeMessageNoTag(toProto()); + public void toProtoStream(OutputStream outputStream) throws IOException { + toProto().writeDelimitedTo(outputStream); } @Override - public void fromProtoStream(CodedInputStream inputStream) throws IOException { - VertexGroupCommitStartedProto proto = inputStream.readMessage(VertexGroupCommitStartedProto.PARSER, null); + public void fromProtoStream(InputStream inputStream) throws IOException { + VertexGroupCommitStartedProto proto = VertexGroupCommitStartedProto.parseDelimitedFrom(inputStream); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java index e7452e6770..90099fce22 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java @@ -19,12 +19,12 @@ package org.apache.tez.dag.history.events; import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.api.DagTypeConverters; import org.apache.tez.dag.api.InputDescriptor; import org.apache.tez.dag.api.InputInitializerDescriptor; @@ -151,13 +151,14 @@ public void fromProto(RecoveryProtos.VertexInitializedProto proto) throws IOExce } @Override - public void toProtoStream(CodedOutputStream outputStream) throws IOException { - outputStream.writeMessageNoTag(toProto()); + public void toProtoStream(OutputStream outputStream) throws IOException { + toProto().writeDelimitedTo(outputStream); } @Override - public void fromProtoStream(CodedInputStream inputStream) throws IOException { - VertexInitializedProto proto = inputStream.readMessage(VertexInitializedProto.PARSER, null); + public void fromProtoStream(InputStream inputStream) throws IOException { + RecoveryProtos.VertexInitializedProto proto = + RecoveryProtos.VertexInitializedProto.parseDelimitedFrom(inputStream); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java index 4a3e05f1f4..a8bd21eacf 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.app.dag.VertexState; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; @@ -74,13 +74,13 @@ public void fromProto(VertexStartedProto proto) { } @Override - public void toProtoStream(CodedOutputStream outputStream) throws IOException { - outputStream.writeMessageNoTag(toProto()); + public void toProtoStream(OutputStream outputStream) throws IOException { + toProto().writeDelimitedTo(outputStream); } @Override - public void fromProtoStream(CodedInputStream inputStream) throws IOException { - VertexStartedProto proto = inputStream.readMessage(VertexStartedProto.PARSER, null); + public void fromProtoStream(InputStream inputStream) throws IOException { + VertexStartedProto proto = VertexStartedProto.parseDelimitedFrom(inputStream); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java index d874e0a6d1..8c291722e2 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java @@ -28,7 +28,6 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; -import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -85,7 +84,8 @@ public class RecoveryService extends AbstractService { private FileSystem recoveryDirFS; // FS where staging dir exists Path recoveryPath; @VisibleForTesting - public Map outputStreamMap = new HashMap<>(); + public Map outputStreamMap = new + HashMap(); private int bufferSize; @VisibleForTesting public FSDataOutputStream summaryStream; @@ -101,31 +101,6 @@ public class RecoveryService extends AbstractService { private volatile boolean drained = true; private Object waitForDrained = new Object(); - @VisibleForTesting - public static class RecoveryStream { - private final FSDataOutputStream outputStream; - private final CodedOutputStream codedOutputStream; - - RecoveryStream(FSDataOutputStream outputStream) { - this.outputStream = outputStream; - this.codedOutputStream = CodedOutputStream.newInstance(outputStream); - } - - public void write(byte[] bytes) throws IOException { - codedOutputStream.writeRawBytes(bytes); - } - - public void flush() throws IOException { - codedOutputStream.flush(); - outputStream.hflush(); - } - - public void close() throws IOException { - flush(); - outputStream.close(); - } - } - public RecoveryService(AppContext appContext) { super(RecoveryService.class.getName()); this.appContext = appContext; @@ -256,9 +231,10 @@ public void serviceStop() throws Exception { } } } - for (Entry entry : outputStreamMap.entrySet()) { + for (Entry entry : outputStreamMap.entrySet()) { try { LOG.info("Closing Output Stream for DAG " + entry.getKey()); + entry.getValue().hflush(); entry.getValue().close(); } catch (IOException ioe) { if (!recoveryDirFS.exists(recoveryPath)) { @@ -327,7 +303,7 @@ public void handle(DAGHistoryEvent event) throws IOException { if (event.getHistoryEvent() instanceof SummaryEvent) { synchronized (lock) { if (stopped.get()) { - LOG.warn("Ignoring event as service stopped, eventType" + LOG.warn("Igoring event as service stopped, eventType" + event.getHistoryEvent().getEventType()); return; } @@ -453,9 +429,9 @@ protected void handleRecoveryEvent(DAGHistoryEvent event) throws IOException { return; } - RecoveryStream recoveryStream = outputStreamMap.get(dagID); - if (recoveryStream == null) { + if (!outputStreamMap.containsKey(dagID)) { Path dagFilePath = TezCommonUtils.getDAGRecoveryPath(recoveryPath, dagID.toString()); + FSDataOutputStream outputStream; if (recoveryDirFS.exists(dagFilePath)) { createFatalErrorFlagDir(); return; @@ -464,12 +440,12 @@ protected void handleRecoveryEvent(DAGHistoryEvent event) throws IOException { LOG.debug("Opening DAG recovery file in create mode" + ", filePath=" + dagFilePath); } - FSDataOutputStream outputStream = recoveryDirFS.create(dagFilePath, false, bufferSize); - recoveryStream = new RecoveryStream(outputStream); + outputStream = recoveryDirFS.create(dagFilePath, false, bufferSize); } - outputStreamMap.put(dagID, recoveryStream); + outputStreamMap.put(dagID, outputStream); } + FSDataOutputStream outputStream = outputStreamMap.get(dagID); if (LOG.isDebugEnabled()) { LOG.debug("Writing recovery event to output stream" @@ -477,15 +453,15 @@ protected void handleRecoveryEvent(DAGHistoryEvent event) throws IOException { + ", eventType=" + eventType); } ++unflushedEventsCount; - recoveryStream.codedOutputStream.writeFixed32NoTag(event.getHistoryEvent().getEventType().ordinal()); - event.getHistoryEvent().toProtoStream(recoveryStream.codedOutputStream); + outputStream.writeInt(event.getHistoryEvent().getEventType().ordinal()); + event.getHistoryEvent().toProtoStream(outputStream); if (!EnumSet.of(HistoryEventType.DAG_SUBMITTED, HistoryEventType.DAG_FINISHED).contains(eventType)) { - maybeFlush(recoveryStream); + maybeFlush(outputStream); } } - private void maybeFlush(RecoveryStream recoveryStream) throws IOException { + private void maybeFlush(FSDataOutputStream outputStream) throws IOException { long currentTime = appContext.getClock().getTime(); boolean doFlush = false; if (maxUnflushedEvents >=0 @@ -506,12 +482,12 @@ private void maybeFlush(RecoveryStream recoveryStream) throws IOException { if (!doFlush) { return; } - doFlush(recoveryStream, currentTime); + doFlush(outputStream, currentTime); } - private void doFlush(RecoveryStream recoveryStream, + private void doFlush(FSDataOutputStream outputStream, long currentTime) throws IOException { - recoveryStream.flush(); + outputStream.hflush(); if (LOG.isDebugEnabled()) { LOG.debug("Flushing output stream" diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java index 1c09d5d9e8..6673b39e5b 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java @@ -20,15 +20,12 @@ import java.io.IOException; import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Random; -import com.google.common.collect.Sets; -import com.google.protobuf.CodedInputStream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -39,8 +36,6 @@ import org.apache.hadoop.yarn.util.SystemClock; import org.apache.log4j.Level; import org.apache.log4j.LogManager; -import org.apache.tez.dag.api.TaskLocationHint; -import org.apache.tez.dag.api.VertexLocationHint; import org.apache.tez.dag.api.oldrecords.TaskAttemptState; import org.apache.tez.dag.api.oldrecords.TaskState; import org.apache.tez.dag.api.records.DAGProtos.DAGPlan; @@ -64,7 +59,6 @@ import org.apache.tez.dag.history.events.TaskFinishedEvent; import org.apache.tez.dag.history.events.TaskStartedEvent; import org.apache.tez.dag.history.events.VertexCommitStartedEvent; -import org.apache.tez.dag.history.events.VertexConfigurationDoneEvent; import org.apache.tez.dag.history.events.VertexFinishedEvent; import org.apache.tez.dag.history.events.VertexGroupCommitFinishedEvent; import org.apache.tez.dag.history.events.VertexGroupCommitStartedEvent; @@ -97,8 +91,6 @@ public class TestRecoveryParser { private Path recoveryPath; private DAGAppMaster mockAppMaster; private DAGImpl mockDAGImpl; - // Protobuf message limit is 64 MB by default - private static final int PROTOBUF_DEFAULT_SIZE_LIMIT = 64 << 20; @Before public void setUp() throws IllegalArgumentException, IOException { @@ -113,6 +105,7 @@ public void setUp() throws IllegalArgumentException, IOException { mockDAGImpl = mock(DAGImpl.class); when(mockAppMaster.createDAG(any(DAGPlan.class), any(TezDAGID.class))).thenReturn(mockDAGImpl); parser = new RecoveryParser(mockAppMaster, localFS, recoveryPath, 3); + LogManager.getRootLogger().setLevel(Level.DEBUG); } private DAGSummaryData createDAGSummaryData(TezDAGID dagId, boolean completed) { @@ -274,7 +267,7 @@ public void testLastCorruptedRecoveryRecord() throws IOException { null, "user", new Configuration(), null, null))); // wait until DAGSubmittedEvent is handled in the RecoveryEventHandling thread rService.await(); - rService.outputStreamMap.get(dagID).write("INVALID_DATA".getBytes("UTF-8")); + rService.outputStreamMap.get(dagID).writeUTF("INVALID_DATA"); rService.stop(); // write data in attempt_2 @@ -285,7 +278,7 @@ public void testLastCorruptedRecoveryRecord() throws IOException { rService.handle(new DAGHistoryEvent(dagID, new DAGInitializedEvent(dagID, 1L, "user", dagPlan.getName(), null))); rService.await(); - rService.outputStreamMap.get(dagID).write("INVALID_DATA".getBytes("UTF-8")); + rService.outputStreamMap.get(dagID).writeUTF("INVALID_DATA"); rService.stop(); // corrupted last records will be skipped but the whole recovery logs will be read @@ -625,75 +618,6 @@ public void testRecoverableNonSummary2() throws IOException { + ", but its full recovery events are not seen")); } - @Test(timeout=20000) - public void testRecoveryLargeEventData() throws IOException { - ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1); - TezDAGID dagID = TezDAGID.getInstance(appId, 1); - AppContext appContext = mock(AppContext.class); - when(appContext.getCurrentRecoveryDir()).thenReturn(new Path(recoveryPath+"/1")); - when(appContext.getClock()).thenReturn(new SystemClock()); - when(mockDAGImpl.getID()).thenReturn(dagID); - when(appContext.getHadoopShim()).thenReturn(new DefaultHadoopShim()); - when(appContext.getApplicationID()).thenReturn(appId); - - RecoveryService rService = new RecoveryService(appContext); - Configuration conf = new Configuration(); - conf.setBoolean(RecoveryService.TEZ_TEST_RECOVERY_DRAIN_EVENTS_WHEN_STOPPED, true); - rService.init(conf); - rService.start(); - - DAGPlan dagPlan = TestDAGImpl.createTestDAGPlan(); - // DAG DAGSubmittedEvent -> DAGInitializedEvent -> DAGStartedEvent - rService.handle(new DAGHistoryEvent(dagID, - new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1), - null, "user", new Configuration(), null, null))); - DAGInitializedEvent dagInitedEvent = new DAGInitializedEvent(dagID, 100L, - "user", "dagName", null); - DAGStartedEvent dagStartedEvent = new DAGStartedEvent(dagID, 0L, "user", "dagName"); - rService.handle(new DAGHistoryEvent(dagID, dagInitedEvent)); - rService.handle(new DAGHistoryEvent(dagID, dagStartedEvent)); - - // Create a Recovery event larger than 64 MB to verify default max protobuf size - ArrayList taskLocationHints = new ArrayList<>(100000); - TaskLocationHint taskLocationHint = TaskLocationHint.createTaskLocationHint( - Sets.newHashSet("aaaaaaaaaaaaaaa.aaaaaaaaaaaaaaa.aaaaaaaaaaaaaaa", - "bbbbbbbbbbbbbbb.bbbbbbbbbbbbbbb.bbbbbbbbbbbbbbb", - "ccccccccccccccc.ccccccccccccccc.ccccccccccccccc", - "ddddddddddddddd.ddddddddddddddd.ddddddddddddddd", - "eeeeeeeeeeeeeee.eeeeeeeeeeeeeee.eeeeeeeeeeeeeee", - "fffffffffffffff.fffffffffffffff.fffffffffffffff", - "ggggggggggggggg.ggggggggggggggg.ggggggggggggggg", - "hhhhhhhhhhhhhhh.hhhhhhhhhhhhhhh.hhhhhhhhhhhhhhh", - "iiiiiiiiiiiiiii.iiiiiiiiiiiiiii.iiiiiiiiiiiiiii", - "jjjjjjjjjjjjjjj.jjjjjjjjjjjjjjj.jjjjjjjjjjjjjjj", - "kkkkkkkkkkkkkkk.kkkkkkkkkkkkkkk.kkkkkkkkkkkkkkk", - "lllllllllllllll.lllllllllllllll.lllllllllllllll", - "mmmmmmmmmmmmmmm.mmmmmmmmmmmmmmm.mmmmmmmmmmmmmmm", - "nnnnnnnnnnnnnnn.nnnnnnnnnnnnnnn.nnnnnnnnnnnnnnn"), - Sets.newHashSet("rack1", "rack2", "rack3")); - for (int i = 0; i < 100000; i++) { - taskLocationHints.add(taskLocationHint); - } - - TezVertexID v0Id = TezVertexID.getInstance(dagID, 0); - VertexLocationHint vertexLocationHint = VertexLocationHint.create(taskLocationHints); - VertexConfigurationDoneEvent vertexConfigurationDoneEvent = new VertexConfigurationDoneEvent( - v0Id, 0, 100000, vertexLocationHint, null, null, false); - // Verify large protobuf message - assertTrue(vertexConfigurationDoneEvent.toProto().getSerializedSize() > PROTOBUF_DEFAULT_SIZE_LIMIT ); - rService.handle(new DAGHistoryEvent(dagID, vertexConfigurationDoneEvent)); - rService.stop(); - - DAGRecoveryData dagData = parser.parseRecoveryData(); - VertexRecoveryData v0data = dagData.getVertexRecoveryData(v0Id); - assertNotNull("Vertex Recovery Data should be non-null", v0data); - VertexConfigurationDoneEvent parsedVertexConfigurationDoneEvent = v0data.getVertexConfigurationDoneEvent(); - assertNotNull("Vertex Configuration Done Event should be non-null", parsedVertexConfigurationDoneEvent); - VertexLocationHint parsedVertexLocationHint = parsedVertexConfigurationDoneEvent.getVertexLocationHint(); - assertNotNull("Vertex Location Hint should be non-null", parsedVertexLocationHint); - assertEquals(parsedVertexLocationHint.getTaskLocationHints().size(), 100000); - } - @Test(timeout=5000) public void testRecoveryData() throws IOException { ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java index 50a80cb5f5..47d8389e34 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java @@ -23,8 +23,6 @@ import java.nio.ByteBuffer; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.apache.tez.runtime.api.TaskFailureType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -91,9 +89,7 @@ public class TestHistoryEventsProtoConversion { private HistoryEvent testProtoConversion(HistoryEvent event) throws IOException, TezException { ByteArrayOutputStream os = new ByteArrayOutputStream(); HistoryEvent deserializedEvent = null; - CodedOutputStream codedOutputStream = CodedOutputStream.newInstance(os); - event.toProtoStream(codedOutputStream); - codedOutputStream.flush(); + event.toProtoStream(os); os.flush(); os.close(); deserializedEvent = ReflectionUtils.createClazzInstance( @@ -102,7 +98,7 @@ private HistoryEvent testProtoConversion(HistoryEvent event) throws IOException, + ", eventType=" + event.getEventType() + ", bufLen=" + os.toByteArray().length); deserializedEvent.fromProtoStream( - CodedInputStream.newInstance(os.toByteArray())); + new ByteArrayInputStream(os.toByteArray())); return deserializedEvent; } diff --git a/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java b/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java index bac0e8c5cc..c727a8fc38 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java +++ b/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java @@ -28,7 +28,6 @@ import java.util.List; import java.util.Set; -import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -136,11 +135,6 @@ public void serviceInit(Configuration conf) throws Exception { conf.setBoolean(YarnConfiguration.NM_VMEM_CHECK_ENABLED, false); conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "000"); - conf.setInt(CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 1); - conf.setInt(CommonConfigurationKeys.IPC_CLIENT_CONNECT_TIMEOUT_KEY, 1000); - conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,0); - conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY, 0); - conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_TIMEOUT_KEY,1000); try { Path stagingPath = FileContext.getFileContext(conf).makeQualified( diff --git a/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java b/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java index 50c5a667ac..c08780f473 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java +++ b/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java @@ -24,8 +24,6 @@ import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; -import com.google.protobuf.CodedInputStream; -import com.google.protobuf.CodedOutputStream; import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.conf.Configuration; import org.apache.tez.common.ReflectionUtils; @@ -224,16 +222,15 @@ public HistoryEvent getHistoryEvent() { private String encodeHistoryEvent(HistoryEvent event) throws IOException { ByteArrayOutputStream out = new ByteArrayOutputStream(); - CodedOutputStream codedOutputStream = CodedOutputStream.newInstance(out); - event.toProtoStream(codedOutputStream); - codedOutputStream.flush(); + event.toProtoStream(out); return event.getClass().getName() + "," + Base64.encodeBase64String(out.toByteArray()); } private HistoryEvent decodeHistoryEvent(String eventClass, String base64) throws IOException { - CodedInputStream in = CodedInputStream.newInstance(Base64.decodeBase64(base64)); + ByteArrayInputStream in = new ByteArrayInputStream( + Base64.decodeBase64(base64)); try { HistoryEvent event = ReflectionUtils.createClazzInstance(eventClass); event.fromProtoStream(in); diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java b/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java index 6d3ab1c711..f00ae5cb11 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java @@ -26,7 +26,6 @@ import java.util.List; import java.util.Random; -import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -176,9 +175,6 @@ public void setup() throws Exception { tezConf.setBoolean( RecoveryService.TEZ_TEST_RECOVERY_DRAIN_EVENTS_WHEN_STOPPED, true); - tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,0); - tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY, 0); - tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_TIMEOUT_KEY,1000); tezSession = TezClient.create("TestDAGRecovery", tezConf); tezSession.start(); } diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java b/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java index cf4744b2a2..b0c9ccc40a 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java @@ -18,9 +18,6 @@ package org.apache.tez.test; -import org.apache.hadoop.fs.CommonConfigurationKeysPublic; -import org.apache.log4j.Level; -import org.apache.log4j.LogManager; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -136,9 +133,6 @@ public void setup() throws Exception { tezConf.set(TezConfiguration.TEZ_AM_LAUNCH_CMD_OPTS, " -Xmx256m"); tezConf.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, true); tezConf.set(TezConfiguration.TEZ_AM_STAGING_SCRATCH_DATA_AUTO_DELETE, "false"); - tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,0); - tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY, 0); - tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_TIMEOUT_KEY,1000); tezSession = TezClient.create("TestDAGRecovery", tezConf); tezSession.start(); @@ -160,7 +154,7 @@ public void teardown() throws InterruptedException { void runDAGAndVerify(DAG dag, DAGStatus.State finalState) throws Exception { tezSession.waitTillReady(); DAGClient dagClient = tezSession.submitDAG(dag); - DAGStatus dagStatus = dagClient.getDAGStatus(null, 10); + DAGStatus dagStatus = dagClient.getDAGStatus(null); while (!dagStatus.isCompleted()) { LOG.info("Waiting for dag to complete. Sleeping for 500ms." + " DAG name: " + dag.getName() diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestRecovery.java b/tez-tests/src/test/java/org/apache/tez/test/TestRecovery.java index c7b1fb9639..93fd972161 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestRecovery.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestRecovery.java @@ -32,7 +32,6 @@ import java.util.Set; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; @@ -479,9 +478,6 @@ private void testHashJoinExample(SimpleShutdownCondition shutdownCondition, RecoveryService.TEZ_TEST_RECOVERY_DRAIN_EVENTS_WHEN_STOPPED, false); tezConf.setBoolean( TezConfiguration.TEZ_AM_STAGING_SCRATCH_DATA_AUTO_DELETE, false); - tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,0); - tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY, 0); - tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_TIMEOUT_KEY,1000); tezConf.set(TezConfiguration.TEZ_AM_LOG_LEVEL, "INFO;org.apache.tez=DEBUG"); hashJoinExample.setConf(tezConf); From fa6bc2acf1b548e481b236ab50903e0ed0269da4 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Mon, 30 Apr 2018 14:09:23 -0500 Subject: [PATCH 071/512] TEZ-3914. Recovering a large DAG fails to size limit exceeded (Jonathan Eagles via jlowe) --- .../apache/tez/dag/app/RecoveryParser.java | 19 +++-- .../tez/dag/app/dag/impl/VertexImpl.java | 11 ++- .../apache/tez/dag/history/HistoryEvent.java | 9 +- .../dag/history/events/AMLaunchedEvent.java | 12 +-- .../dag/history/events/AMStartedEvent.java | 12 +-- .../dag/history/events/AppLaunchedEvent.java | 8 +- .../events/ContainerLaunchedEvent.java | 13 ++- .../history/events/ContainerStoppedEvent.java | 13 ++- .../history/events/DAGCommitStartedEvent.java | 11 +-- .../dag/history/events/DAGFinishedEvent.java | 11 +-- .../history/events/DAGInitializedEvent.java | 14 ++-- .../history/events/DAGKillRequestEvent.java | 17 ++-- .../dag/history/events/DAGRecoveredEvent.java | 8 +- .../dag/history/events/DAGStartedEvent.java | 12 +-- .../dag/history/events/DAGSubmittedEvent.java | 11 +-- .../events/TaskAttemptFinishedEvent.java | 13 ++- .../events/TaskAttemptStartedEvent.java | 12 +-- .../dag/history/events/TaskFinishedEvent.java | 12 +-- .../dag/history/events/TaskStartedEvent.java | 12 +-- .../events/VertexCommitStartedEvent.java | 11 +-- .../events/VertexConfigurationDoneEvent.java | 12 +-- .../history/events/VertexFinishedEvent.java | 11 +-- .../VertexGroupCommitFinishedEvent.java | 11 +-- .../events/VertexGroupCommitStartedEvent.java | 11 +-- .../events/VertexInitializedEvent.java | 13 ++- .../history/events/VertexStartedEvent.java | 12 +-- .../dag/history/recovery/RecoveryService.java | 58 +++++++++---- .../tez/dag/app/TestRecoveryParser.java | 82 ++++++++++++++++++- .../TestHistoryEventsProtoConversion.java | 8 +- .../history/ats/acls/TestATSHistoryV15.java | 13 +-- .../org/apache/tez/test/MiniTezCluster.java | 6 ++ .../RecoveryServiceWithEventHandlingHook.java | 9 +- .../org/apache/tez/test/TestAMRecovery.java | 4 + .../org/apache/tez/test/TestDAGRecovery.java | 8 +- .../org/apache/tez/test/TestRecovery.java | 4 + 35 files changed, 324 insertions(+), 179 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java index 368dd17f47..99ac2837ae 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java @@ -27,6 +27,7 @@ import java.util.Map; import java.util.Map.Entry; +import com.google.protobuf.CodedInputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -251,11 +252,15 @@ private static void parseSummaryFile(FSDataInputStream inputStream) } } - private static HistoryEvent getNextEvent(FSDataInputStream inputStream) + private static HistoryEvent getNextEvent(CodedInputStream inputStream) throws IOException { + boolean isAtEnd = inputStream.isAtEnd(); + if (isAtEnd) { + return null; + } int eventTypeOrdinal = -1; try { - eventTypeOrdinal = inputStream.readInt(); + eventTypeOrdinal = inputStream.readFixed32(); } catch (EOFException eof) { return null; } @@ -353,13 +358,15 @@ private static HistoryEvent getNextEvent(FSDataInputStream inputStream) public static List parseDAGRecoveryFile(FSDataInputStream inputStream) throws IOException { List historyEvents = new ArrayList(); + CodedInputStream codedInputStream = CodedInputStream.newInstance(inputStream); + codedInputStream.setSizeLimit(Integer.MAX_VALUE); while (true) { - HistoryEvent historyEvent = getNextEvent(inputStream); + HistoryEvent historyEvent = getNextEvent(codedInputStream); if (historyEvent == null) { LOG.info("Reached end of stream"); break; } - LOG.debug("Read HistoryEvent, eventType=" + historyEvent.getEventType() + ", event=" + historyEvent); + LOG.debug("Read HistoryEvent, eventType={}, event={}", historyEvent.getEventType(), historyEvent); historyEvents.add(historyEvent); } return historyEvents; @@ -745,10 +752,12 @@ public DAGRecoveryData parseRecoveryData() throws IOException { + ", dagRecoveryFile=" + dagRecoveryFile + ", len=" + fileStatus.getLen()); FSDataInputStream dagRecoveryStream = recoveryFS.open(dagRecoveryFile, recoveryBufferSize); + CodedInputStream codedInputStream = CodedInputStream.newInstance(dagRecoveryStream); + codedInputStream.setSizeLimit(Integer.MAX_VALUE); while (true) { HistoryEvent event; try { - event = getNextEvent(dagRecoveryStream); + event = getNextEvent(codedInputStream); if (event == null) { LOG.info("Reached end of dag recovery stream"); break; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index ad261736c2..f3fc2694b6 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -42,6 +42,8 @@ import javax.annotation.Nullable; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.hadoop.classification.InterfaceAudience.Private; @@ -63,7 +65,6 @@ import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.common.counters.LimitExceededException; import org.apache.tez.common.counters.TezCounters; -import org.apache.tez.common.io.NonSyncByteArrayInputStream; import org.apache.tez.common.io.NonSyncByteArrayOutputStream; import org.apache.tez.dag.api.DagTypeConverters; import org.apache.tez.dag.api.EdgeManagerPluginDescriptor; @@ -2695,9 +2696,11 @@ private void assignVertexManager() throws TezException { } NonSyncByteArrayOutputStream out = new NonSyncByteArrayOutputStream(); try { - reconfigureDoneEvent.toProtoStream(out); + CodedOutputStream codedOutputStream = CodedOutputStream.newInstance(out); + reconfigureDoneEvent.toProtoStream(codedOutputStream); + codedOutputStream.flush(); } catch (IOException e) { - throw new TezUncheckedException("Unable to deserilize VertexReconfigureDoneEvent"); + throw new TezUncheckedException("Unable to deserialize VertexReconfigureDoneEvent"); } this.vertexManager = new VertexManager( VertexManagerPluginDescriptor.create(NoOpVertexManager.class.getName()) @@ -4589,7 +4592,7 @@ public void initialize() throws Exception { LOG.debug("initialize NoOpVertexManager"); } configurationDoneEvent = new VertexConfigurationDoneEvent(); - configurationDoneEvent.fromProtoStream(new NonSyncByteArrayInputStream(getContext().getUserPayload().deepCopyAsArray())); + configurationDoneEvent.fromProtoStream(CodedInputStream.newInstance(getContext().getUserPayload().deepCopyAsArray())); String vertexName = getContext().getVertexName(); if (getContext().getVertexNumTasks(vertexName) == -1) { Preconditions.checkArgument(configurationDoneEvent.isSetParallelismCalled(), "SetParallelism must be called " diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEvent.java index 1ca0d5f069..5b077e98ec 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEvent.java @@ -18,9 +18,10 @@ package org.apache.tez.dag.history; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; + import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; public interface HistoryEvent { @@ -30,8 +31,8 @@ public interface HistoryEvent { public boolean isHistoryEvent(); - public void toProtoStream(OutputStream outputStream) throws IOException; + public void toProtoStream(CodedOutputStream outputStream) throws IOException; - public void fromProtoStream(InputStream inputStream) throws IOException; + public void fromProtoStream(CodedInputStream inputStream) throws IOException; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMLaunchedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMLaunchedEvent.java index fa332d68eb..001cbf0104 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMLaunchedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMLaunchedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.tez.dag.history.HistoryEvent; @@ -84,13 +84,13 @@ public void fromProto(AMLaunchedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - AMLaunchedProto proto = AMLaunchedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + AMLaunchedProto proto = inputStream.readMessage(AMLaunchedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMStartedEvent.java index 8a59d84769..87daba68b8 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMStartedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.tez.dag.history.HistoryEvent; @@ -79,13 +79,13 @@ public void fromProto(AMStartedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - AMStartedProto proto = AMStartedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + AMStartedProto proto = inputStream.readMessage(AMStartedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AppLaunchedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AppLaunchedEvent.java index 08d2aff021..0b812f0184 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AppLaunchedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AppLaunchedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.tez.common.VersionInfo; @@ -67,12 +67,12 @@ public boolean isHistoryEvent() { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { + public void toProtoStream(CodedOutputStream outputStream) throws IOException { throw new UnsupportedOperationException("Not a recovery event"); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { + public void fromProtoStream(CodedInputStream inputStream) throws IOException { throw new UnsupportedOperationException("Not a recovery event"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerLaunchedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerLaunchedEvent.java index 45d0261adf..11528e2f8c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerLaunchedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerLaunchedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.util.ConverterUtils; @@ -77,14 +77,13 @@ public void fromProto(ContainerLaunchedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - ContainerLaunchedProto proto = - ContainerLaunchedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + ContainerLaunchedProto proto = inputStream.readMessage(ContainerLaunchedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerStoppedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerStoppedEvent.java index 86971ce520..528f6293b6 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerStoppedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerStoppedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.util.ConverterUtils; @@ -82,14 +82,13 @@ public void fromProto(ContainerStoppedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - ContainerStoppedProto proto = - ContainerStoppedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + ContainerStoppedProto proto = inputStream.readMessage(ContainerStoppedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGCommitStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGCommitStartedEvent.java index 016bb60b09..241dada2d6 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGCommitStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGCommitStartedEvent.java @@ -19,9 +19,10 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; @@ -69,13 +70,13 @@ public void fromProto(DAGCommitStartedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - DAGCommitStartedProto proto = DAGCommitStartedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + DAGCommitStartedProto proto = inputStream.readMessage(DAGCommitStartedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java index c395297965..0a7ef561f7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java @@ -19,10 +19,11 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.util.Map; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.DagTypeConverters; @@ -121,13 +122,13 @@ public void fromProto(DAGFinishedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - DAGFinishedProto proto = DAGFinishedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + DAGFinishedProto proto = inputStream.readMessage(DAGFinishedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java index 98d64d3da8..9e6c8b282b 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java @@ -19,15 +19,16 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.util.Map; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.dag.recovery.records.RecoveryProtos; +import org.apache.tez.dag.recovery.records.RecoveryProtos.DAGInitializedProto; public class DAGInitializedEvent implements HistoryEvent { @@ -83,14 +84,13 @@ public void fromProto(RecoveryProtos.DAGInitializedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - RecoveryProtos.DAGInitializedProto proto = - RecoveryProtos.DAGInitializedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + DAGInitializedProto proto = inputStream.readMessage(DAGInitializedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGKillRequestEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGKillRequestEvent.java index 525e361364..c87f5ce7e7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGKillRequestEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGKillRequestEvent.java @@ -18,14 +18,16 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.recovery.records.RecoveryProtos; +import org.apache.tez.dag.recovery.records.RecoveryProtos.DAGKillRequestProto; import org.apache.tez.dag.recovery.records.RecoveryProtos.SummaryEventProto; import org.apache.tez.dag.utils.ProtoUtils; @@ -60,12 +62,12 @@ public boolean isHistoryEvent() { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } - public RecoveryProtos.DAGKillRequestProto toProto() { - return RecoveryProtos.DAGKillRequestProto.newBuilder() + public DAGKillRequestProto toProto() { + return DAGKillRequestProto.newBuilder() .setDagId(dagID.toString()) .setKillRequestTime(killRequestTime) .setIsSessionStopped(isSessionStopped) @@ -73,9 +75,8 @@ public RecoveryProtos.DAGKillRequestProto toProto() { } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - RecoveryProtos.DAGKillRequestProto proto = - RecoveryProtos.DAGKillRequestProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + DAGKillRequestProto proto = inputStream.readMessage(DAGKillRequestProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGRecoveredEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGRecoveredEvent.java index 2bfa43b894..e5f5614819 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGRecoveredEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGRecoveredEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.tez.dag.app.dag.DAGState; import org.apache.tez.dag.history.HistoryEvent; @@ -76,13 +76,13 @@ public boolean isHistoryEvent() { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { + public void toProtoStream(CodedOutputStream outputStream) throws IOException { throw new UnsupportedOperationException("Invalid operation for eventType " + getEventType().name()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { + public void fromProtoStream(CodedInputStream inputStream) throws IOException { throw new UnsupportedOperationException("Invalid operation for eventType " + getEventType().name()); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java index d0e0e693a5..f1fdcac809 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.app.dag.DAGState; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; @@ -74,13 +74,13 @@ public void fromProto(DAGStartedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - DAGStartedProto proto = DAGStartedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + DAGStartedProto proto = inputStream.readMessage(DAGStartedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java index 1b1fdf38a9..e04ee80188 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java @@ -19,10 +19,11 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.util.Map; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -126,13 +127,13 @@ public void fromProto(DAGSubmittedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - DAGSubmittedProto proto = DAGSubmittedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + DAGSubmittedProto proto = inputStream.readMessage(DAGSubmittedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java index e9100e8ab9..96dc0997eb 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java @@ -20,10 +20,10 @@ import javax.annotation.Nullable; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.util.List; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.common.TezConverterUtils; import org.apache.tez.runtime.api.TaskFailureType; import org.slf4j.Logger; @@ -226,14 +226,13 @@ public void fromProto(TaskAttemptFinishedProto proto) throws IOException { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - TaskAttemptFinishedProto proto = - TaskAttemptFinishedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + TaskAttemptFinishedProto proto = inputStream.readMessage(TaskAttemptFinishedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java index 71d4419ee7..a49e47c97f 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.util.ConverterUtils; @@ -91,13 +91,13 @@ public void fromProto(TaskAttemptStartedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - TaskAttemptStartedProto proto = TaskAttemptStartedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + TaskAttemptStartedProto proto = inputStream.readMessage(TaskAttemptStartedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java index 71ff6c84b7..6befa1a399 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.tez.common.counters.TezCounters; @@ -107,13 +107,13 @@ public void fromProto(TaskFinishedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - TaskFinishedProto proto = TaskFinishedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + TaskFinishedProto proto = inputStream.readMessage(TaskFinishedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java index 07dc2f9d99..cc629699a4 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.api.oldrecords.TaskState; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; @@ -83,13 +83,13 @@ public void fromProto(TaskStartedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - TaskStartedProto proto = TaskStartedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + TaskStartedProto proto = inputStream.readMessage(TaskStartedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java index c4521870e6..8ff86b8af1 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java @@ -19,10 +19,11 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.nio.charset.Charset; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; @@ -73,13 +74,13 @@ public void fromProto(VertexCommitStartedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - VertexCommitStartedProto proto = VertexCommitStartedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + VertexCommitStartedProto proto = inputStream.readMessage(VertexCommitStartedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java index 137342cf21..a2e20397cf 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java @@ -18,12 +18,12 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.util.HashMap; import java.util.Map; import java.util.Map.Entry; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.api.DagTypeConverters; import org.apache.tez.dag.api.EdgeProperty; import org.apache.tez.dag.api.VertexLocationHint; @@ -155,13 +155,13 @@ public void fromProto(VertexConfigurationDoneProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - VertexConfigurationDoneProto proto = VertexConfigurationDoneProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + VertexConfigurationDoneProto proto = inputStream.readMessage(VertexConfigurationDoneProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java index a2cdae2de6..58cb628b28 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java @@ -19,10 +19,11 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.util.Map; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.app.dag.impl.ServicePluginInfo; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -123,13 +124,13 @@ public void fromProto(VertexFinishedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - VertexFinishedProto proto = VertexFinishedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + VertexFinishedProto proto = inputStream.readMessage(VertexFinishedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java index ec8f3e1a4f..c9d5aae770 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java @@ -19,10 +19,11 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.util.Collection; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; @@ -94,13 +95,13 @@ public TezVertexID apply(String input) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - VertexGroupCommitFinishedProto proto = VertexGroupCommitFinishedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + VertexGroupCommitFinishedProto proto = inputStream.readMessage(VertexGroupCommitFinishedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java index 3de355cc64..cdd11bcc4c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java @@ -19,10 +19,11 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.util.Collection; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; @@ -94,13 +95,13 @@ public TezVertexID apply(String input) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - VertexGroupCommitStartedProto proto = VertexGroupCommitStartedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + VertexGroupCommitStartedProto proto = inputStream.readMessage(VertexGroupCommitStartedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java index 90099fce22..e7452e6770 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java @@ -19,12 +19,12 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.api.DagTypeConverters; import org.apache.tez.dag.api.InputDescriptor; import org.apache.tez.dag.api.InputInitializerDescriptor; @@ -151,14 +151,13 @@ public void fromProto(RecoveryProtos.VertexInitializedProto proto) throws IOExce } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - RecoveryProtos.VertexInitializedProto proto = - RecoveryProtos.VertexInitializedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + VertexInitializedProto proto = inputStream.readMessage(VertexInitializedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java index a8bd21eacf..4a3e05f1f4 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java @@ -19,9 +19,9 @@ package org.apache.tez.dag.history.events; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.app.dag.VertexState; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; @@ -74,13 +74,13 @@ public void fromProto(VertexStartedProto proto) { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { - toProto().writeDelimitedTo(outputStream); + public void toProtoStream(CodedOutputStream outputStream) throws IOException { + outputStream.writeMessageNoTag(toProto()); } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { - VertexStartedProto proto = VertexStartedProto.parseDelimitedFrom(inputStream); + public void fromProtoStream(CodedInputStream inputStream) throws IOException { + VertexStartedProto proto = inputStream.readMessage(VertexStartedProto.PARSER, null); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java index 8c291722e2..d874e0a6d1 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java @@ -28,6 +28,7 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; +import com.google.protobuf.CodedOutputStream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -84,8 +85,7 @@ public class RecoveryService extends AbstractService { private FileSystem recoveryDirFS; // FS where staging dir exists Path recoveryPath; @VisibleForTesting - public Map outputStreamMap = new - HashMap(); + public Map outputStreamMap = new HashMap<>(); private int bufferSize; @VisibleForTesting public FSDataOutputStream summaryStream; @@ -101,6 +101,31 @@ public class RecoveryService extends AbstractService { private volatile boolean drained = true; private Object waitForDrained = new Object(); + @VisibleForTesting + public static class RecoveryStream { + private final FSDataOutputStream outputStream; + private final CodedOutputStream codedOutputStream; + + RecoveryStream(FSDataOutputStream outputStream) { + this.outputStream = outputStream; + this.codedOutputStream = CodedOutputStream.newInstance(outputStream); + } + + public void write(byte[] bytes) throws IOException { + codedOutputStream.writeRawBytes(bytes); + } + + public void flush() throws IOException { + codedOutputStream.flush(); + outputStream.hflush(); + } + + public void close() throws IOException { + flush(); + outputStream.close(); + } + } + public RecoveryService(AppContext appContext) { super(RecoveryService.class.getName()); this.appContext = appContext; @@ -231,10 +256,9 @@ public void serviceStop() throws Exception { } } } - for (Entry entry : outputStreamMap.entrySet()) { + for (Entry entry : outputStreamMap.entrySet()) { try { LOG.info("Closing Output Stream for DAG " + entry.getKey()); - entry.getValue().hflush(); entry.getValue().close(); } catch (IOException ioe) { if (!recoveryDirFS.exists(recoveryPath)) { @@ -303,7 +327,7 @@ public void handle(DAGHistoryEvent event) throws IOException { if (event.getHistoryEvent() instanceof SummaryEvent) { synchronized (lock) { if (stopped.get()) { - LOG.warn("Igoring event as service stopped, eventType" + LOG.warn("Ignoring event as service stopped, eventType" + event.getHistoryEvent().getEventType()); return; } @@ -429,9 +453,9 @@ protected void handleRecoveryEvent(DAGHistoryEvent event) throws IOException { return; } - if (!outputStreamMap.containsKey(dagID)) { + RecoveryStream recoveryStream = outputStreamMap.get(dagID); + if (recoveryStream == null) { Path dagFilePath = TezCommonUtils.getDAGRecoveryPath(recoveryPath, dagID.toString()); - FSDataOutputStream outputStream; if (recoveryDirFS.exists(dagFilePath)) { createFatalErrorFlagDir(); return; @@ -440,12 +464,12 @@ protected void handleRecoveryEvent(DAGHistoryEvent event) throws IOException { LOG.debug("Opening DAG recovery file in create mode" + ", filePath=" + dagFilePath); } - outputStream = recoveryDirFS.create(dagFilePath, false, bufferSize); + FSDataOutputStream outputStream = recoveryDirFS.create(dagFilePath, false, bufferSize); + recoveryStream = new RecoveryStream(outputStream); } - outputStreamMap.put(dagID, outputStream); + outputStreamMap.put(dagID, recoveryStream); } - FSDataOutputStream outputStream = outputStreamMap.get(dagID); if (LOG.isDebugEnabled()) { LOG.debug("Writing recovery event to output stream" @@ -453,15 +477,15 @@ protected void handleRecoveryEvent(DAGHistoryEvent event) throws IOException { + ", eventType=" + eventType); } ++unflushedEventsCount; - outputStream.writeInt(event.getHistoryEvent().getEventType().ordinal()); - event.getHistoryEvent().toProtoStream(outputStream); + recoveryStream.codedOutputStream.writeFixed32NoTag(event.getHistoryEvent().getEventType().ordinal()); + event.getHistoryEvent().toProtoStream(recoveryStream.codedOutputStream); if (!EnumSet.of(HistoryEventType.DAG_SUBMITTED, HistoryEventType.DAG_FINISHED).contains(eventType)) { - maybeFlush(outputStream); + maybeFlush(recoveryStream); } } - private void maybeFlush(FSDataOutputStream outputStream) throws IOException { + private void maybeFlush(RecoveryStream recoveryStream) throws IOException { long currentTime = appContext.getClock().getTime(); boolean doFlush = false; if (maxUnflushedEvents >=0 @@ -482,12 +506,12 @@ private void maybeFlush(FSDataOutputStream outputStream) throws IOException { if (!doFlush) { return; } - doFlush(outputStream, currentTime); + doFlush(recoveryStream, currentTime); } - private void doFlush(FSDataOutputStream outputStream, + private void doFlush(RecoveryStream recoveryStream, long currentTime) throws IOException { - outputStream.hflush(); + recoveryStream.flush(); if (LOG.isDebugEnabled()) { LOG.debug("Flushing output stream" diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java index 6673b39e5b..1c09d5d9e8 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java @@ -20,12 +20,15 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Random; +import com.google.common.collect.Sets; +import com.google.protobuf.CodedInputStream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -36,6 +39,8 @@ import org.apache.hadoop.yarn.util.SystemClock; import org.apache.log4j.Level; import org.apache.log4j.LogManager; +import org.apache.tez.dag.api.TaskLocationHint; +import org.apache.tez.dag.api.VertexLocationHint; import org.apache.tez.dag.api.oldrecords.TaskAttemptState; import org.apache.tez.dag.api.oldrecords.TaskState; import org.apache.tez.dag.api.records.DAGProtos.DAGPlan; @@ -59,6 +64,7 @@ import org.apache.tez.dag.history.events.TaskFinishedEvent; import org.apache.tez.dag.history.events.TaskStartedEvent; import org.apache.tez.dag.history.events.VertexCommitStartedEvent; +import org.apache.tez.dag.history.events.VertexConfigurationDoneEvent; import org.apache.tez.dag.history.events.VertexFinishedEvent; import org.apache.tez.dag.history.events.VertexGroupCommitFinishedEvent; import org.apache.tez.dag.history.events.VertexGroupCommitStartedEvent; @@ -91,6 +97,8 @@ public class TestRecoveryParser { private Path recoveryPath; private DAGAppMaster mockAppMaster; private DAGImpl mockDAGImpl; + // Protobuf message limit is 64 MB by default + private static final int PROTOBUF_DEFAULT_SIZE_LIMIT = 64 << 20; @Before public void setUp() throws IllegalArgumentException, IOException { @@ -105,7 +113,6 @@ public void setUp() throws IllegalArgumentException, IOException { mockDAGImpl = mock(DAGImpl.class); when(mockAppMaster.createDAG(any(DAGPlan.class), any(TezDAGID.class))).thenReturn(mockDAGImpl); parser = new RecoveryParser(mockAppMaster, localFS, recoveryPath, 3); - LogManager.getRootLogger().setLevel(Level.DEBUG); } private DAGSummaryData createDAGSummaryData(TezDAGID dagId, boolean completed) { @@ -267,7 +274,7 @@ public void testLastCorruptedRecoveryRecord() throws IOException { null, "user", new Configuration(), null, null))); // wait until DAGSubmittedEvent is handled in the RecoveryEventHandling thread rService.await(); - rService.outputStreamMap.get(dagID).writeUTF("INVALID_DATA"); + rService.outputStreamMap.get(dagID).write("INVALID_DATA".getBytes("UTF-8")); rService.stop(); // write data in attempt_2 @@ -278,7 +285,7 @@ public void testLastCorruptedRecoveryRecord() throws IOException { rService.handle(new DAGHistoryEvent(dagID, new DAGInitializedEvent(dagID, 1L, "user", dagPlan.getName(), null))); rService.await(); - rService.outputStreamMap.get(dagID).writeUTF("INVALID_DATA"); + rService.outputStreamMap.get(dagID).write("INVALID_DATA".getBytes("UTF-8")); rService.stop(); // corrupted last records will be skipped but the whole recovery logs will be read @@ -618,6 +625,75 @@ public void testRecoverableNonSummary2() throws IOException { + ", but its full recovery events are not seen")); } + @Test(timeout=20000) + public void testRecoveryLargeEventData() throws IOException { + ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1); + TezDAGID dagID = TezDAGID.getInstance(appId, 1); + AppContext appContext = mock(AppContext.class); + when(appContext.getCurrentRecoveryDir()).thenReturn(new Path(recoveryPath+"/1")); + when(appContext.getClock()).thenReturn(new SystemClock()); + when(mockDAGImpl.getID()).thenReturn(dagID); + when(appContext.getHadoopShim()).thenReturn(new DefaultHadoopShim()); + when(appContext.getApplicationID()).thenReturn(appId); + + RecoveryService rService = new RecoveryService(appContext); + Configuration conf = new Configuration(); + conf.setBoolean(RecoveryService.TEZ_TEST_RECOVERY_DRAIN_EVENTS_WHEN_STOPPED, true); + rService.init(conf); + rService.start(); + + DAGPlan dagPlan = TestDAGImpl.createTestDAGPlan(); + // DAG DAGSubmittedEvent -> DAGInitializedEvent -> DAGStartedEvent + rService.handle(new DAGHistoryEvent(dagID, + new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1), + null, "user", new Configuration(), null, null))); + DAGInitializedEvent dagInitedEvent = new DAGInitializedEvent(dagID, 100L, + "user", "dagName", null); + DAGStartedEvent dagStartedEvent = new DAGStartedEvent(dagID, 0L, "user", "dagName"); + rService.handle(new DAGHistoryEvent(dagID, dagInitedEvent)); + rService.handle(new DAGHistoryEvent(dagID, dagStartedEvent)); + + // Create a Recovery event larger than 64 MB to verify default max protobuf size + ArrayList taskLocationHints = new ArrayList<>(100000); + TaskLocationHint taskLocationHint = TaskLocationHint.createTaskLocationHint( + Sets.newHashSet("aaaaaaaaaaaaaaa.aaaaaaaaaaaaaaa.aaaaaaaaaaaaaaa", + "bbbbbbbbbbbbbbb.bbbbbbbbbbbbbbb.bbbbbbbbbbbbbbb", + "ccccccccccccccc.ccccccccccccccc.ccccccccccccccc", + "ddddddddddddddd.ddddddddddddddd.ddddddddddddddd", + "eeeeeeeeeeeeeee.eeeeeeeeeeeeeee.eeeeeeeeeeeeeee", + "fffffffffffffff.fffffffffffffff.fffffffffffffff", + "ggggggggggggggg.ggggggggggggggg.ggggggggggggggg", + "hhhhhhhhhhhhhhh.hhhhhhhhhhhhhhh.hhhhhhhhhhhhhhh", + "iiiiiiiiiiiiiii.iiiiiiiiiiiiiii.iiiiiiiiiiiiiii", + "jjjjjjjjjjjjjjj.jjjjjjjjjjjjjjj.jjjjjjjjjjjjjjj", + "kkkkkkkkkkkkkkk.kkkkkkkkkkkkkkk.kkkkkkkkkkkkkkk", + "lllllllllllllll.lllllllllllllll.lllllllllllllll", + "mmmmmmmmmmmmmmm.mmmmmmmmmmmmmmm.mmmmmmmmmmmmmmm", + "nnnnnnnnnnnnnnn.nnnnnnnnnnnnnnn.nnnnnnnnnnnnnnn"), + Sets.newHashSet("rack1", "rack2", "rack3")); + for (int i = 0; i < 100000; i++) { + taskLocationHints.add(taskLocationHint); + } + + TezVertexID v0Id = TezVertexID.getInstance(dagID, 0); + VertexLocationHint vertexLocationHint = VertexLocationHint.create(taskLocationHints); + VertexConfigurationDoneEvent vertexConfigurationDoneEvent = new VertexConfigurationDoneEvent( + v0Id, 0, 100000, vertexLocationHint, null, null, false); + // Verify large protobuf message + assertTrue(vertexConfigurationDoneEvent.toProto().getSerializedSize() > PROTOBUF_DEFAULT_SIZE_LIMIT ); + rService.handle(new DAGHistoryEvent(dagID, vertexConfigurationDoneEvent)); + rService.stop(); + + DAGRecoveryData dagData = parser.parseRecoveryData(); + VertexRecoveryData v0data = dagData.getVertexRecoveryData(v0Id); + assertNotNull("Vertex Recovery Data should be non-null", v0data); + VertexConfigurationDoneEvent parsedVertexConfigurationDoneEvent = v0data.getVertexConfigurationDoneEvent(); + assertNotNull("Vertex Configuration Done Event should be non-null", parsedVertexConfigurationDoneEvent); + VertexLocationHint parsedVertexLocationHint = parsedVertexConfigurationDoneEvent.getVertexLocationHint(); + assertNotNull("Vertex Location Hint should be non-null", parsedVertexLocationHint); + assertEquals(parsedVertexLocationHint.getTaskLocationHints().size(), 100000); + } + @Test(timeout=5000) public void testRecoveryData() throws IOException { ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java index 47d8389e34..50a80cb5f5 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java @@ -23,6 +23,8 @@ import java.nio.ByteBuffer; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.tez.runtime.api.TaskFailureType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -89,7 +91,9 @@ public class TestHistoryEventsProtoConversion { private HistoryEvent testProtoConversion(HistoryEvent event) throws IOException, TezException { ByteArrayOutputStream os = new ByteArrayOutputStream(); HistoryEvent deserializedEvent = null; - event.toProtoStream(os); + CodedOutputStream codedOutputStream = CodedOutputStream.newInstance(os); + event.toProtoStream(codedOutputStream); + codedOutputStream.flush(); os.flush(); os.close(); deserializedEvent = ReflectionUtils.createClazzInstance( @@ -98,7 +102,7 @@ private HistoryEvent testProtoConversion(HistoryEvent event) throws IOException, + ", eventType=" + event.getEventType() + ", bufLen=" + os.toByteArray().length); deserializedEvent.fromProtoStream( - new ByteArrayInputStream(os.toByteArray())); + CodedInputStream.newInstance(os.toByteArray())); return deserializedEvent; } diff --git a/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryV15.java b/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryV15.java index a690a1903d..54abd4465b 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryV15.java +++ b/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryV15.java @@ -20,9 +20,10 @@ import static org.junit.Assert.assertEquals; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; + import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.util.Random; import org.apache.hadoop.fs.LocatedFileStatus; @@ -32,8 +33,6 @@ import org.apache.tez.dag.app.AppContext; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.logging.ats.ATSV15HistoryLoggingService; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -60,6 +59,8 @@ import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -288,12 +289,12 @@ public boolean isHistoryEvent() { } @Override - public void toProtoStream(OutputStream outputStream) throws IOException { + public void toProtoStream(CodedOutputStream outputStream) throws IOException { } @Override - public void fromProtoStream(InputStream inputStream) throws IOException { + public void fromProtoStream(CodedInputStream inputStream) throws IOException { } }; diff --git a/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java b/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java index c727a8fc38..bac0e8c5cc 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java +++ b/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.Set; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -135,6 +136,11 @@ public void serviceInit(Configuration conf) throws Exception { conf.setBoolean(YarnConfiguration.NM_VMEM_CHECK_ENABLED, false); conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "000"); + conf.setInt(CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 1); + conf.setInt(CommonConfigurationKeys.IPC_CLIENT_CONNECT_TIMEOUT_KEY, 1000); + conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,0); + conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY, 0); + conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_TIMEOUT_KEY,1000); try { Path stagingPath = FileContext.getFileContext(conf).makeQualified( diff --git a/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java b/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java index c08780f473..50c5a667ac 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java +++ b/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java @@ -24,6 +24,8 @@ import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.CodedOutputStream; import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.conf.Configuration; import org.apache.tez.common.ReflectionUtils; @@ -222,15 +224,16 @@ public HistoryEvent getHistoryEvent() { private String encodeHistoryEvent(HistoryEvent event) throws IOException { ByteArrayOutputStream out = new ByteArrayOutputStream(); - event.toProtoStream(out); + CodedOutputStream codedOutputStream = CodedOutputStream.newInstance(out); + event.toProtoStream(codedOutputStream); + codedOutputStream.flush(); return event.getClass().getName() + "," + Base64.encodeBase64String(out.toByteArray()); } private HistoryEvent decodeHistoryEvent(String eventClass, String base64) throws IOException { - ByteArrayInputStream in = new ByteArrayInputStream( - Base64.decodeBase64(base64)); + CodedInputStream in = CodedInputStream.newInstance(Base64.decodeBase64(base64)); try { HistoryEvent event = ReflectionUtils.createClazzInstance(eventClass); event.fromProtoStream(in); diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java b/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java index f00ae5cb11..6d3ab1c711 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java @@ -26,6 +26,7 @@ import java.util.List; import java.util.Random; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -175,6 +176,9 @@ public void setup() throws Exception { tezConf.setBoolean( RecoveryService.TEZ_TEST_RECOVERY_DRAIN_EVENTS_WHEN_STOPPED, true); + tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,0); + tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY, 0); + tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_TIMEOUT_KEY,1000); tezSession = TezClient.create("TestDAGRecovery", tezConf); tezSession.start(); } diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java b/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java index b0c9ccc40a..cf4744b2a2 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java @@ -18,6 +18,9 @@ package org.apache.tez.test; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.log4j.Level; +import org.apache.log4j.LogManager; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -133,6 +136,9 @@ public void setup() throws Exception { tezConf.set(TezConfiguration.TEZ_AM_LAUNCH_CMD_OPTS, " -Xmx256m"); tezConf.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, true); tezConf.set(TezConfiguration.TEZ_AM_STAGING_SCRATCH_DATA_AUTO_DELETE, "false"); + tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,0); + tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY, 0); + tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_TIMEOUT_KEY,1000); tezSession = TezClient.create("TestDAGRecovery", tezConf); tezSession.start(); @@ -154,7 +160,7 @@ public void teardown() throws InterruptedException { void runDAGAndVerify(DAG dag, DAGStatus.State finalState) throws Exception { tezSession.waitTillReady(); DAGClient dagClient = tezSession.submitDAG(dag); - DAGStatus dagStatus = dagClient.getDAGStatus(null); + DAGStatus dagStatus = dagClient.getDAGStatus(null, 10); while (!dagStatus.isCompleted()) { LOG.info("Waiting for dag to complete. Sleeping for 500ms." + " DAG name: " + dag.getName() diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestRecovery.java b/tez-tests/src/test/java/org/apache/tez/test/TestRecovery.java index 93fd972161..c7b1fb9639 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestRecovery.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestRecovery.java @@ -32,6 +32,7 @@ import java.util.Set; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; @@ -478,6 +479,9 @@ private void testHashJoinExample(SimpleShutdownCondition shutdownCondition, RecoveryService.TEZ_TEST_RECOVERY_DRAIN_EVENTS_WHEN_STOPPED, false); tezConf.setBoolean( TezConfiguration.TEZ_AM_STAGING_SCRATCH_DATA_AUTO_DELETE, false); + tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,0); + tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY, 0); + tezConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_TIMEOUT_KEY,1000); tezConf.set(TezConfiguration.TEZ_AM_LOG_LEVEL, "INFO;org.apache.tez=DEBUG"); hashJoinExample.setConf(tezConf); From c5c26c655d43c1c6602e50d1bdf3a69e9b40fd50 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Wed, 2 May 2018 12:46:12 -0500 Subject: [PATCH 072/512] TEZ-3926. Changes to master for 0.10.x line and 0.9 release branch (jeagles) --- docs/pom.xml | 2 +- hadoop-shim-impls/hadoop-shim-2.7/pom.xml | 2 +- hadoop-shim-impls/hadoop-shim-2.8/pom.xml | 2 +- hadoop-shim-impls/pom.xml | 2 +- hadoop-shim/pom.xml | 2 +- pom.xml | 2 +- tez-api/pom.xml | 2 +- tez-common/pom.xml | 2 +- tez-dag/pom.xml | 2 +- tez-dist/pom.xml | 2 +- tez-examples/pom.xml | 2 +- tez-ext-service-tests/pom.xml | 2 +- tez-mapreduce/pom.xml | 2 +- tez-plugins/pom.xml | 2 +- tez-plugins/tez-aux-services/pom.xml | 2 +- tez-plugins/tez-history-parser/pom.xml | 2 +- tez-plugins/tez-protobuf-history-plugin/pom.xml | 2 +- tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml | 2 +- tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml | 2 +- tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml | 2 +- tez-plugins/tez-yarn-timeline-history/pom.xml | 2 +- tez-runtime-internals/pom.xml | 2 +- tez-runtime-library/pom.xml | 2 +- tez-tests/pom.xml | 2 +- tez-tools/analyzers/job-analyzer/pom.xml | 2 +- tez-tools/analyzers/pom.xml | 2 +- tez-tools/pom.xml | 2 +- tez-tools/tez-javadoc-tools/pom.xml | 2 +- tez-tools/tez-tfile-parser/pom.xml | 2 +- tez-ui/pom.xml | 2 +- 30 files changed, 30 insertions(+), 30 deletions(-) diff --git a/docs/pom.xml b/docs/pom.xml index b5fc0f62db..045222057c 100644 --- a/docs/pom.xml +++ b/docs/pom.xml @@ -27,7 +27,7 @@ org.apache.tez tez - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-docs pom diff --git a/hadoop-shim-impls/hadoop-shim-2.7/pom.xml b/hadoop-shim-impls/hadoop-shim-2.7/pom.xml index a4b0a5bdae..08c2255905 100644 --- a/hadoop-shim-impls/hadoop-shim-2.7/pom.xml +++ b/hadoop-shim-impls/hadoop-shim-2.7/pom.xml @@ -19,7 +19,7 @@ hadoop-shim-impls org.apache.tez - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT 4.0.0 hadoop-shim-2.7 diff --git a/hadoop-shim-impls/hadoop-shim-2.8/pom.xml b/hadoop-shim-impls/hadoop-shim-2.8/pom.xml index 64ce4a959b..bf4865c597 100644 --- a/hadoop-shim-impls/hadoop-shim-2.8/pom.xml +++ b/hadoop-shim-impls/hadoop-shim-2.8/pom.xml @@ -19,7 +19,7 @@ hadoop-shim-impls org.apache.tez - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT 4.0.0 hadoop-shim-2.8 diff --git a/hadoop-shim-impls/pom.xml b/hadoop-shim-impls/pom.xml index 5eb304c9de..c0371f1dcf 100644 --- a/hadoop-shim-impls/pom.xml +++ b/hadoop-shim-impls/pom.xml @@ -20,7 +20,7 @@ tez org.apache.tez - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT hadoop-shim-impls pom diff --git a/hadoop-shim/pom.xml b/hadoop-shim/pom.xml index ab612b2929..df565c387a 100644 --- a/hadoop-shim/pom.xml +++ b/hadoop-shim/pom.xml @@ -20,7 +20,7 @@ tez org.apache.tez - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT hadoop-shim diff --git a/pom.xml b/pom.xml index 18b5a208f1..1c8caeb91d 100644 --- a/pom.xml +++ b/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez pom - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez diff --git a/tez-api/pom.xml b/tez-api/pom.xml index 13b6c01482..d84ed64374 100644 --- a/tez-api/pom.xml +++ b/tez-api/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-api diff --git a/tez-common/pom.xml b/tez-common/pom.xml index 76fdfb894e..b3c2892943 100644 --- a/tez-common/pom.xml +++ b/tez-common/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-common diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml index 9fa9b44391..ee48b23050 100644 --- a/tez-dag/pom.xml +++ b/tez-dag/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-dag diff --git a/tez-dist/pom.xml b/tez-dist/pom.xml index 9447fe748d..4c0f5b82fe 100644 --- a/tez-dist/pom.xml +++ b/tez-dist/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-dist diff --git a/tez-examples/pom.xml b/tez-examples/pom.xml index 6490ef4b7c..2586869e14 100644 --- a/tez-examples/pom.xml +++ b/tez-examples/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-examples diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml index 9cc73362fe..a09e382206 100644 --- a/tez-ext-service-tests/pom.xml +++ b/tez-ext-service-tests/pom.xml @@ -20,7 +20,7 @@ tez org.apache.tez - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-ext-service-tests diff --git a/tez-mapreduce/pom.xml b/tez-mapreduce/pom.xml index e72cc66f9b..7e50fcc54c 100644 --- a/tez-mapreduce/pom.xml +++ b/tez-mapreduce/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-mapreduce diff --git a/tez-plugins/pom.xml b/tez-plugins/pom.xml index f90cae733a..31c7834860 100644 --- a/tez-plugins/pom.xml +++ b/tez-plugins/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-plugins pom diff --git a/tez-plugins/tez-aux-services/pom.xml b/tez-plugins/tez-aux-services/pom.xml index ac874ae0d0..38f05f9631 100644 --- a/tez-plugins/tez-aux-services/pom.xml +++ b/tez-plugins/tez-aux-services/pom.xml @@ -20,7 +20,7 @@ tez-plugins org.apache.tez - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-aux-services diff --git a/tez-plugins/tez-history-parser/pom.xml b/tez-plugins/tez-history-parser/pom.xml index 9ccf2fe9eb..6f46047efe 100644 --- a/tez-plugins/tez-history-parser/pom.xml +++ b/tez-plugins/tez-history-parser/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez-plugins - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-history-parser diff --git a/tez-plugins/tez-protobuf-history-plugin/pom.xml b/tez-plugins/tez-protobuf-history-plugin/pom.xml index 880aca952b..5ed61974eb 100644 --- a/tez-plugins/tez-protobuf-history-plugin/pom.xml +++ b/tez-plugins/tez-protobuf-history-plugin/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-protobuf-history-plugin diff --git a/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml b/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml index 2b40ac974c..9488fe319c 100644 --- a/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml +++ b/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-yarn-timeline-cache-plugin diff --git a/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml b/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml index 692b9158f3..3f5792fa29 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-yarn-timeline-history-with-acls diff --git a/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml b/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml index 0866e3d980..d151a36d93 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-yarn-timeline-history-with-fs diff --git a/tez-plugins/tez-yarn-timeline-history/pom.xml b/tez-plugins/tez-yarn-timeline-history/pom.xml index b43d49b7cb..6137e986e3 100644 --- a/tez-plugins/tez-yarn-timeline-history/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-yarn-timeline-history diff --git a/tez-runtime-internals/pom.xml b/tez-runtime-internals/pom.xml index 4f4cd6e2dc..851923eb1c 100644 --- a/tez-runtime-internals/pom.xml +++ b/tez-runtime-internals/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-runtime-internals diff --git a/tez-runtime-library/pom.xml b/tez-runtime-library/pom.xml index cd70f6662f..4ba6ad7eb9 100644 --- a/tez-runtime-library/pom.xml +++ b/tez-runtime-library/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-runtime-library diff --git a/tez-tests/pom.xml b/tez-tests/pom.xml index 9bacb438ba..fd9b7bd277 100644 --- a/tez-tests/pom.xml +++ b/tez-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-tests diff --git a/tez-tools/analyzers/job-analyzer/pom.xml b/tez-tools/analyzers/job-analyzer/pom.xml index 3acf2bfdfa..ec305e7296 100644 --- a/tez-tools/analyzers/job-analyzer/pom.xml +++ b/tez-tools/analyzers/job-analyzer/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-perf-analyzer - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-job-analyzer diff --git a/tez-tools/analyzers/pom.xml b/tez-tools/analyzers/pom.xml index 7f6920b84f..a571a03ee1 100644 --- a/tez-tools/analyzers/pom.xml +++ b/tez-tools/analyzers/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-tools - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-perf-analyzer pom diff --git a/tez-tools/pom.xml b/tez-tools/pom.xml index 8d1a5b6dbd..d6b67c8163 100644 --- a/tez-tools/pom.xml +++ b/tez-tools/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-tools pom diff --git a/tez-tools/tez-javadoc-tools/pom.xml b/tez-tools/tez-javadoc-tools/pom.xml index 56f390ba5e..b7cdbd4eea 100644 --- a/tez-tools/tez-javadoc-tools/pom.xml +++ b/tez-tools/tez-javadoc-tools/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-tools - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-javadoc-tools diff --git a/tez-tools/tez-tfile-parser/pom.xml b/tez-tools/tez-tfile-parser/pom.xml index 4ada794af5..74a8c558ac 100644 --- a/tez-tools/tez-tfile-parser/pom.xml +++ b/tez-tools/tez-tfile-parser/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-tools - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-tfile-parser diff --git a/tez-ui/pom.xml b/tez-ui/pom.xml index 1dcffe96d1..59040a5dda 100644 --- a/tez-ui/pom.xml +++ b/tez-ui/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.9.2-SNAPSHOT + 0.10.0-SNAPSHOT tez-ui war From cf6ea5f62aa4a5957028e6c074a62747324abaa1 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Fri, 4 May 2018 15:47:41 -0500 Subject: [PATCH 073/512] TEZ-3924. TestDefaultSorter fails intermittently due random keys and interaction with RLE and partition collisions (Jonathan Eagles via kshukla) --- .../sort/impl/dflt/TestDefaultSorter.java | 191 +++++++++++++----- 1 file changed, 140 insertions(+), 51 deletions(-) diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java index aad232a5db..e0fb15320c 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java @@ -19,7 +19,9 @@ package org.apache.tez.runtime.library.common.sort.impl.dflt; import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.tez.runtime.library.api.Partitioner; import org.apache.tez.runtime.library.common.Constants; +import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.junit.Assert; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -269,11 +271,17 @@ public void basicTest() throws IOException { conf.setLong(TezRuntimeConfiguration.TEZ_RUNTIME_IO_SORT_MB, 1); context.requestInitialMemory(ExternalSorter.getInitialMemoryRequirement(conf, context.getTotalMemoryAvailableToTask()), handler); - DefaultSorter sorter = new DefaultSorter(context, conf, 5, handler.getMemoryAssigned()); + SorterWrapper sorterWrapper = new SorterWrapper(context, conf, 5, handler.getMemoryAssigned()); + DefaultSorter sorter = sorterWrapper.getSorter(); //Write 1000 keys each of size 1000, (> 1 spill should happen) try { - writeData(sorter, 1000, 1000); + Text[] keys = generateData(1000, 1000); + Text[] values = generateData(1000, 1000); + for (int i = 0; i < keys.length; i++) { + sorterWrapper.writeKeyValue(keys[i], values[i]); + } + sorterWrapper.close(); assertTrue(sorter.getNumSpills() > 2); verifyCounters(sorter, context); } catch(IOException ioe) { @@ -285,11 +293,13 @@ public void basicTest() throws IOException { @Test(timeout = 30000) public void testEmptyCaseFileLengths() throws IOException { - testEmptyCaseFileLengthsHelper(50, 2, 1, 48); - testEmptyCaseFileLengthsHelper(1, 1, 10, 0); + testEmptyCaseFileLengthsHelper(50, new String[] {"a", "b"}, new String[] {"1", "2"}); + testEmptyCaseFileLengthsHelper(50, new String[] {"a", "a"}, new String[] {"1", "2"}); + testEmptyCaseFileLengthsHelper(50, new String[] {"aaa", "bbb", "aaa"}, new String[] {"1", "2", "3"}); + testEmptyCaseFileLengthsHelper(1, new String[] {"abcdefghij"}, new String[] {"1234567890"}); } - public void testEmptyCaseFileLengthsHelper(int numPartitions, int numKeys, int keyLen, int expectedEmptyPartitions) + public void testEmptyCaseFileLengthsHelper(int numPartitions, String[] keys, String[] values) throws IOException { OutputContext context = createTezOutputContext(); @@ -298,39 +308,49 @@ public void testEmptyCaseFileLengthsHelper(int numPartitions, int numKeys, int k context.requestInitialMemory(ExternalSorter.getInitialMemoryRequirement(conf, context.getTotalMemoryAvailableToTask()), handler); String auxService = conf.get(TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID, TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID_DEFAULT); - DefaultSorter sorter = new DefaultSorter(context, conf, numPartitions, handler.getMemoryAssigned()); - try { - writeData(sorter, numKeys, keyLen); - List events = new ArrayList(); - String pathComponent = (context.getUniqueIdentifier() + "_" + 0); - ShuffleUtils.generateEventOnSpill(events, true, true, context, 0, - sorter.indexCacheList.get(0), 0, true, pathComponent, sorter.getPartitionStats(), - sorter.reportDetailedPartitionStats(), auxService, TezCommonUtils.newBestCompressionDeflater()); - - CompositeDataMovementEvent compositeDataMovementEvent = - (CompositeDataMovementEvent) events.get(1); - ByteBuffer bb = compositeDataMovementEvent.getUserPayload(); - ShuffleUserPayloads.DataMovementEventPayloadProto shufflePayload = - ShuffleUserPayloads.DataMovementEventPayloadProto.parseFrom(ByteString.copyFrom(bb)); - - if (shufflePayload.hasEmptyPartitions()) { - byte[] emptyPartitionsBytesString = - TezCommonUtils.decompressByteStringToByteArray( - shufflePayload.getEmptyPartitions()); - BitSet emptyPartitionBitSet = TezUtilsInternal.fromByteArray(emptyPartitionsBytesString); - Assert.assertTrue("Number of empty partitions did not match!", - emptyPartitionBitSet.cardinality() == expectedEmptyPartitions); - } else { - Assert.assertTrue(expectedEmptyPartitions == 0); - } - //4 bytes of header + numKeys* 2 *(keydata.length + keyLength.length) + 2 * 1 byte of EOF_MARKER + 4 bytes of checksum - assertEquals("Unexpected Output File Size!", - localFs.getFileStatus(sorter.getFinalOutputFile()).getLen(), numKeys * (4 + (2 * (2 + keyLen)) + 2 + 4)); - assertTrue(sorter.getNumSpills() == 1); - verifyCounters(sorter, context); - } catch(IOException ioe) { - fail(ioe.getMessage()); + SorterWrapper sorterWrapper = new SorterWrapper(context, conf, numPartitions, handler.getMemoryAssigned()); + DefaultSorter sorter = sorterWrapper.getSorter(); + assertEquals("Key and Values must have the same number of elements", keys.length, values.length); + BitSet keyRLEs = new BitSet(keys.length); + for (int i = 0; i < keys.length; i++) { + boolean isRLE = sorterWrapper.writeKeyValue(new Text(keys[i]), new Text(values[i])); + keyRLEs.set(i, isRLE); + } + sorterWrapper.close(); + + List events = new ArrayList<>(); + String pathComponent = (context.getUniqueIdentifier() + "_" + 0); + ShuffleUtils.generateEventOnSpill(events, true, true, context, 0, + sorter.indexCacheList.get(0), 0, true, pathComponent, sorter.getPartitionStats(), + sorter.reportDetailedPartitionStats(), auxService, TezCommonUtils.newBestCompressionDeflater()); + + CompositeDataMovementEvent compositeDataMovementEvent = + (CompositeDataMovementEvent) events.get(1); + ByteBuffer bb = compositeDataMovementEvent.getUserPayload(); + ShuffleUserPayloads.DataMovementEventPayloadProto shufflePayload = + ShuffleUserPayloads.DataMovementEventPayloadProto.parseFrom(ByteString.copyFrom(bb)); + + if (shufflePayload.hasEmptyPartitions()) { + byte[] emptyPartitionsBytesString = + TezCommonUtils.decompressByteStringToByteArray( + shufflePayload.getEmptyPartitions()); + BitSet emptyPartitionBitSet = TezUtilsInternal.fromByteArray(emptyPartitionsBytesString); + Assert.assertEquals("Number of empty partitions did not match!", + emptyPartitionBitSet.cardinality(), sorterWrapper.getEmptyPartitionsCount()); + } else { + Assert.assertEquals(sorterWrapper.getEmptyPartitionsCount(), 0); + } + // Each non-empty partition adds 4 bytes for header, 2 bytes for EOF_MARKER, 4 bytes for checksum + int expectedFileOutLength = sorterWrapper.getNonEmptyPartitionsCount() * 10; + for (int i = 0; i < keys.length; i++) { + // Each Record adds 1 byte for key length, 1 byte Text overhead (length), key.length bytes for key + expectedFileOutLength += keys[i].length() + 2; + // Each Record adds 1 byte for value length, 1 byte Text overhead (length), value.length bytes for value + expectedFileOutLength += values[i].length() + 2; } + assertEquals("Unexpected Output File Size!", localFs.getFileStatus(sorter.getFinalOutputFile()).getLen(), expectedFileOutLength); + assertEquals(sorter.getNumSpills(), 1); + verifyCounters(sorter, context); } @Test @@ -396,9 +416,15 @@ public void testEmptyPartitionsHelper(int numKeys, boolean sendEmptyPartitionDet context.requestInitialMemory(ExternalSorter.getInitialMemoryRequirement(conf, context.getTotalMemoryAvailableToTask()), handler); int partitions = 50; - DefaultSorter sorter = new DefaultSorter(context, conf, partitions, handler.getMemoryAssigned()); + SorterWrapper sorterWrapper = new SorterWrapper(context, conf, partitions, handler.getMemoryAssigned()); + DefaultSorter sorter = sorterWrapper.getSorter(); - writeData(sorter, numKeys, 1000000); + Text[] keys = generateData(numKeys, 1000000); + Text[] values = generateData(numKeys, 1000000); + for (int i = 0; i < keys.length; i++) { + sorterWrapper.writeKeyValue(keys[i], values[i]); + } + sorterWrapper.close(); if (numKeys == 0) { assertTrue(sorter.getNumSpills() == 1); } else { @@ -446,9 +472,15 @@ void testPartitionStats(boolean withStats) throws IOException { MemoryUpdateCallbackHandler handler = new MemoryUpdateCallbackHandler(); context.requestInitialMemory(ExternalSorter.getInitialMemoryRequirement(conf, context.getTotalMemoryAvailableToTask()), handler); - DefaultSorter sorter = new DefaultSorter(context, conf, 1, handler.getMemoryAssigned()); + SorterWrapper sorterWrapper = new SorterWrapper(context, conf, 1, handler.getMemoryAssigned()); + DefaultSorter sorter = sorterWrapper.getSorter(); - writeData(sorter, 1000, 10); + Text[] keys = generateData(1000, 10); + Text[] values = generateData(1000, 10); + for (int i = 0; i < keys.length; i++) { + sorterWrapper.writeKeyValue(keys[i], values[i]); + } + sorterWrapper.close(); assertTrue(sorter.getNumSpills() == 1); verifyCounters(sorter, context); @@ -479,9 +511,16 @@ public void testWithSingleSpillWithFinalMergeDisabled() throws IOException { MemoryUpdateCallbackHandler handler = new MemoryUpdateCallbackHandler(); context.requestInitialMemory(ExternalSorter.getInitialMemoryRequirement(conf, context.getTotalMemoryAvailableToTask()), handler); - DefaultSorter sorter = new DefaultSorter(context, conf, 1, handler.getMemoryAssigned()); - writeData(sorter, 1000, 10); + SorterWrapper sorterWrapper = new SorterWrapper(context, conf, 1, handler.getMemoryAssigned()); + DefaultSorter sorter = sorterWrapper.getSorter(); + + Text[] keys = generateData(1000, 10); + Text[] values = generateData(1000, 10); + for (int i = 0; i < keys.length; i++) { + sorterWrapper.writeKeyValue(keys[i], values[i]); + } + sorterWrapper.close(); assertTrue(sorter.getNumSpills() == 1); ArgumentCaptor eventCaptor = ArgumentCaptor.forClass(List.class); verify(context, times(1)).sendEvents(eventCaptor.capture()); @@ -510,9 +549,16 @@ public void testWithMultipleSpillsWithFinalMergeDisabled() throws IOException { MemoryUpdateCallbackHandler handler = new MemoryUpdateCallbackHandler(); context.requestInitialMemory(ExternalSorter.getInitialMemoryRequirement(conf, context.getTotalMemoryAvailableToTask()), handler); - DefaultSorter sorter = new DefaultSorter(context, conf, 1, handler.getMemoryAssigned()); + SorterWrapper sorterWrapper = new SorterWrapper(context, conf, 1, handler.getMemoryAssigned()); + DefaultSorter sorter = sorterWrapper.getSorter(); + + Text[] keys = generateData(10000, 1000); + Text[] values = generateData(10000, 1000); + for (int i = 0; i < keys.length; i++) { + sorterWrapper.writeKeyValue(keys[i], values[i]); + } + sorterWrapper.close(); - writeData(sorter, 10000, 1000); int spillCount = sorter.getNumSpills(); ArgumentCaptor eventCaptor = ArgumentCaptor.forClass(List.class); verify(context, times(1)).sendEvents(eventCaptor.capture()); @@ -574,14 +620,57 @@ private void verifyCounters(DefaultSorter sorter, OutputContext context) { verify(context, atLeastOnce()).notifyProgress(); } - private void writeData(ExternalSorter sorter, int numKeys, int keyLen) throws IOException { - for (int i = 0; i < numKeys; i++) { - Text key = new Text(RandomStringUtils.randomAlphanumeric(keyLen)); - Text value = new Text(RandomStringUtils.randomAlphanumeric(keyLen)); + private static class SorterWrapper { + + private final DefaultSorter sorter; + private final Partitioner partitioner; + private final BitSet nonEmptyPartitions; + private final Object[] lastKeys; + private final int numPartitions; + + + public SorterWrapper(OutputContext context, Configuration conf, int numPartitions, long memoryAssigned) throws IOException { + sorter = new DefaultSorter(context, conf, numPartitions, memoryAssigned); + partitioner = TezRuntimeUtils.instantiatePartitioner(conf); + nonEmptyPartitions = new BitSet(numPartitions); + lastKeys = new Object[numPartitions]; + this.numPartitions = numPartitions; + } + + public boolean writeKeyValue(Object key, Object value) throws IOException { + int partition = partitioner.getPartition(key, value, this.numPartitions); + nonEmptyPartitions.set(partition); sorter.write(key, value); + + boolean isRLE = key.equals(lastKeys[partition]); + lastKeys[partition] = key; + return isRLE; + } + + public int getNonEmptyPartitionsCount() { + return nonEmptyPartitions.cardinality(); + } + + public int getEmptyPartitionsCount() { + return numPartitions - nonEmptyPartitions.cardinality(); + } + + public void close () throws IOException { + sorter.flush(); + sorter.close(); + } + + public DefaultSorter getSorter() { + return sorter; + } + } + + private static Text[] generateData(int numKeys, int keyLen) { + Text[] ret = new Text[numKeys]; + for (int i = 0; i < numKeys; i++) { + ret[i] = new Text(RandomStringUtils.randomAlphanumeric(keyLen)); } - sorter.flush(); - sorter.close(); + return ret; } private OutputContext createTezOutputContext() throws IOException { From bb2c42b7c5fbe465dbec78590cff0e0020b3a861 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Fri, 4 May 2018 16:09:39 -0500 Subject: [PATCH 074/512] TEZ-3927. TestReduceProcessor fails on Hadoop 3.x (Jonathan Eagles via kshukla) --- .../main/java/org/apache/tez/mapreduce/hadoop/MRJobConfig.java | 2 ++ .../tez/mapreduce/processor/reduce/TestReduceProcessor.java | 1 + 2 files changed, 3 insertions(+) diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRJobConfig.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRJobConfig.java index 02c74b278d..334a7dba3d 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRJobConfig.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRJobConfig.java @@ -122,6 +122,8 @@ public interface MRJobConfig { public static final String CACHE_ARCHIVES_VISIBILITIES = "mapreduce.job.cache.archives.visibilities"; + public static final String FILEOUTPUTCOMMITTER_ALGORITHM_VERSION = "mapreduce.fileoutputcommitter.algorithm.version"; + /** * @deprecated Symlinks are always on and cannot be disabled. */ diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java index 17c790341f..0223482da6 100644 --- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java @@ -117,6 +117,7 @@ public void setUpJobConf(JobConf job) { TezTaskOutput.class); job.set(TezRuntimeConfiguration.TEZ_RUNTIME_PARTITIONER_CLASS, MRPartitioner.class.getName()); job.setNumReduceTasks(1); + job.setInt(MRJobConfig.FILEOUTPUTCOMMITTER_ALGORITHM_VERSION, 1); } @Before From 46bb527fcbcb82a6d7a2a4b8c5ce56438b4fbf2c Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Tue, 8 May 2018 09:17:44 -0500 Subject: [PATCH 075/512] TEZ-3930. TestDagAwareYarnTaskScheduler fails on Hadoop 3 (Jason Lowe via jeagles) --- .../tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java index 553306f534..529f65c368 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java @@ -1336,6 +1336,14 @@ static class AMRMClientAsyncWrapperForTest extends AMRMClientAsyncWrapper { RegisterApplicationMasterResponse getRegistrationResponse() { return ((MockAMRMClient) client).getRegistrationResponse(); } + + @Override + protected void serviceStart() { + } + + @Override + protected void serviceStop() { + } } static class MockAMRMClient extends AMRMClientImpl { From 081a64f716a73a5f65c8d74e6ab729e57b0747bc Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Tue, 8 May 2018 11:03:38 -0500 Subject: [PATCH 076/512] TEZ-3931. TestExternalTezServices fails on Hadoop3 (Jonathan Eagles via kshukla) --- tez-ext-service-tests/pom.xml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml index a09e382206..d6d85732b6 100644 --- a/tez-ext-service-tests/pom.xml +++ b/tez-ext-service-tests/pom.xml @@ -96,8 +96,7 @@ org.apache.hadoop - hadoop-mapreduce-client-jobclient - test-jar + hadoop-mapreduce-client-shuffle test From 72c458a431545565d38681862a883065f4efa44e Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Wed, 9 May 2018 10:25:32 -0500 Subject: [PATCH 077/512] TEZ-3932. TaskSchedulerManager can throw NullPointerException during DAGAppMaster container cleanup race (Jonathan Eagles via jlowe) --- .../tez/dag/app/rm/TaskSchedulerManager.java | 41 +++++-- .../dag/app/rm/TestTaskSchedulerManager.java | 108 +++++++++++++++++- 2 files changed, 136 insertions(+), 13 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java index 5777a2a4fe..61e3702eaf 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java @@ -420,10 +420,14 @@ private void handleTAUnsuccessfulEnd(AMSchedulerEventTAEnded event) { // Inform the Node - the task has asked to be STOPPED / has already // stopped. // AMNodeImpl blacklisting logic does not account for KILLED attempts. - sendEvent(new AMNodeEventTaskAttemptEnded(appContext.getAllContainers(). - get(attemptContainerId).getContainer().getNodeId(), event.getSchedulerId(), - attemptContainerId, - attempt.getID(), event.getState() == TaskAttemptState.FAILED)); + AMContainer amContainer = appContext.getAllContainers().get(attemptContainerId); + // DAG can be shutting down so protect against container cleanup race + if (amContainer != null) { + Container container = amContainer.getContainer(); + sendEvent(new AMNodeEventTaskAttemptEnded(container.getNodeId(), event.getSchedulerId(), + attemptContainerId, + attempt.getID(), event.getState() == TaskAttemptState.FAILED)); + } } } @@ -436,9 +440,14 @@ private void handleTASucceeded(AMSchedulerEventTAEnded event) { if (event.getUsedContainerId() != null) { sendEvent(new AMContainerEventTASucceeded(usedContainerId, event.getAttemptID())); - sendEvent(new AMNodeEventTaskAttemptSucceeded(appContext.getAllContainers(). - get(usedContainerId).getContainer().getNodeId(), event.getSchedulerId(), usedContainerId, - event.getAttemptID())); + AMContainer amContainer = appContext.getAllContainers().get(usedContainerId); + // DAG can be shutting down so protect against container cleanup race + if (amContainer != null) { + Container container = amContainer.getContainer(); + sendEvent(new AMNodeEventTaskAttemptSucceeded(container.getNodeId(), event.getSchedulerId(), + usedContainerId, + event.getAttemptID())); + } } boolean wasContainerAllocated = false; @@ -742,10 +751,15 @@ public synchronized void taskAllocated(int schedulerId, Object task, // because the deallocateTask downcall may have raced with the // taskAllocated() upcall assert task.equals(taskAttempt); - - if (appContext.getAllContainers().get(containerId).getState() == AMContainerState.ALLOCATED) { - sendEvent(new AMContainerEventLaunchRequest(containerId, taskAttempt.getVertexID(), - event.getContainerContext(), event.getLauncherId(), event.getTaskCommId())); + + AMContainer amContainer = appContext.getAllContainers().get(containerId); + // Even though we just added this container, + // DAG can be shutting down so protect against container cleanup race + if (amContainer != null) { + if (amContainer.getState() == AMContainerState.ALLOCATED) { + sendEvent(new AMContainerEventLaunchRequest(containerId, taskAttempt.getVertexID(), + event.getContainerContext(), event.getLauncherId(), event.getTaskCommId())); + } } sendEvent(new AMContainerEventAssignTA(containerId, taskAttempt.getID(), event.getRemoteTaskSpec(), event.getContainerContext().getLocalResources(), event @@ -951,7 +965,10 @@ public void preemptContainer(int schedulerId, ContainerId containerId) { // An AMContainer instance should already exist if an attempt is being made to preempt it AMContainer amContainer = appContext.getAllContainers().get(containerId); try { - taskSchedulers[amContainer.getTaskSchedulerIdentifier()].deallocateContainer(containerId); + // DAG can be shutting down so protect against container cleanup race + if (amContainer != null) { + taskSchedulers[amContainer.getTaskSchedulerIdentifier()].deallocateContainer(containerId); + } } catch (Exception e) { String msg = "Error in TaskScheduler when preempting container" + ", scheduler=" + Utils.getTaskSchedulerIdentifierString(amContainer.getTaskSchedulerIdentifier(), appContext) diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java index 5df25de963..dcf9a5dd69 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java @@ -70,6 +70,7 @@ import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.api.UserPayload; import org.apache.tez.dag.api.client.DAGClientServer; +import org.apache.tez.dag.api.oldrecords.TaskAttemptState; import org.apache.tez.dag.app.AppContext; import org.apache.tez.dag.app.ContainerContext; import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService; @@ -84,6 +85,7 @@ import org.apache.tez.dag.app.rm.container.AMContainer; import org.apache.tez.dag.app.rm.container.AMContainerEventAssignTA; import org.apache.tez.dag.app.rm.container.AMContainerEventCompleted; +import org.apache.tez.dag.app.rm.container.AMContainerEventStopRequest; import org.apache.tez.dag.app.rm.container.AMContainerEventType; import org.apache.tez.dag.app.rm.container.AMContainerMap; import org.apache.tez.dag.app.rm.container.AMContainerState; @@ -217,6 +219,79 @@ public void testSimpleAllocate() throws Exception { assertEquals(mockAttemptId, assignEvent.getTaskAttemptId()); } + @Test(timeout = 5000) + public void testTASucceededAfterContainerCleanup() throws Exception { + Configuration conf = new Configuration(false); + schedulerHandler.init(conf); + schedulerHandler.start(); + + TaskAttemptImpl mockTaskAttempt = mock(TaskAttemptImpl.class); + TezTaskAttemptID mockAttemptId = mock(TezTaskAttemptID.class); + when(mockAttemptId.getId()).thenReturn(0); + when(mockTaskAttempt.getID()).thenReturn(mockAttemptId); + Resource resource = Resource.newInstance(1024, 1); + ContainerContext containerContext = + new ContainerContext(new HashMap(), new Credentials(), + new HashMap(), ""); + int priority = 10; + TaskLocationHint locHint = TaskLocationHint.createTaskLocationHint(new HashSet(), null); + + ContainerId mockCId = mock(ContainerId.class); + Container container = mock(Container.class); + when(container.getId()).thenReturn(mockCId); + + AMContainer mockAMContainer = mock(AMContainer.class); + when(mockAMContainer.getContainerId()).thenReturn(mockCId); + when(mockAMContainer.getState()).thenReturn(AMContainerState.IDLE); + + // Returning null container will replicate container cleanup scenario + when(mockAMContainerMap.get(mockCId)).thenReturn(null); + + AMSchedulerEventTALaunchRequest lr = + new AMSchedulerEventTALaunchRequest(mockAttemptId, resource, null, mockTaskAttempt, locHint, + priority, containerContext, 0, 0, 0); + schedulerHandler.taskAllocated(0, mockTaskAttempt, lr, container); + assertEquals(1, mockEventHandler.events.size()); + assertTrue(mockEventHandler.events.get(0) instanceof AMContainerEventAssignTA); + AMContainerEventAssignTA assignEvent = + (AMContainerEventAssignTA) mockEventHandler.events.get(0); + assertEquals(priority, assignEvent.getPriority()); + assertEquals(mockAttemptId, assignEvent.getTaskAttemptId()); + } + + @Test(timeout = 5000) + public void testTAUnsuccessfulAfterContainerCleanup() throws Exception { + Configuration conf = new Configuration(false); + schedulerHandler.init(conf); + schedulerHandler.start(); + + TaskAttemptImpl mockTaskAttempt = mock(TaskAttemptImpl.class); + TezTaskAttemptID mockAttemptId = mock(TezTaskAttemptID.class); + when(mockAttemptId.getId()).thenReturn(0); + when(mockTaskAttempt.getID()).thenReturn(mockAttemptId); + + ContainerId mockCId = mock(ContainerId.class); + Container container = mock(Container.class); + when(container.getId()).thenReturn(mockCId); + + AMContainer mockAMContainer = mock(AMContainer.class); + when(mockAMContainer.getContainerId()).thenReturn(mockCId); + when(mockAMContainer.getState()).thenReturn(AMContainerState.IDLE); + when(mockTaskAttempt.getAssignedContainerID()).thenReturn(mockCId); + + // Returning null container will replicate container cleanup scenario + when(mockAMContainerMap.get(mockCId)).thenReturn(null); + + schedulerHandler.handleEvent( + new AMSchedulerEventTAEnded( + mockTaskAttempt, mockCId, TaskAttemptState.KILLED, null, null, 0)); + assertEquals(1, mockEventHandler.events.size()); + assertTrue(mockEventHandler.events.get(0) instanceof AMContainerEventStopRequest); + AMContainerEventStopRequest stopEvent = + (AMContainerEventStopRequest) mockEventHandler.events.get(0); + assertEquals(mockCId, stopEvent.getContainerId()); + } + @Test (timeout = 5000) public void testTaskBasedAffinity() throws Exception { Configuration conf = new Configuration(false); @@ -288,7 +363,7 @@ public void testContainerPreempted() throws IOException { schedulerHandler.stop(); schedulerHandler.close(); } - + @Test (timeout = 5000) public void testContainerInternalPreempted() throws IOException, ServicePluginException { Configuration conf = new Configuration(false); @@ -318,6 +393,37 @@ public void testContainerInternalPreempted() throws IOException, ServicePluginEx schedulerHandler.stop(); schedulerHandler.close(); } + + @Test(timeout = 5000) + public void testContainerInternalPreemptedAfterContainerCleanup() throws IOException, ServicePluginException { + Configuration conf = new Configuration(false); + schedulerHandler.init(conf); + schedulerHandler.start(); + + AMContainer mockAmContainer = mock(AMContainer.class); + when(mockAmContainer.getTaskSchedulerIdentifier()).thenReturn(0); + when(mockAmContainer.getContainerLauncherIdentifier()).thenReturn(0); + when(mockAmContainer.getTaskCommunicatorIdentifier()).thenReturn(0); + ContainerId mockCId = mock(ContainerId.class); + verify(mockTaskScheduler, times(0)).deallocateContainer((ContainerId) any()); + // Returning null container will replicate container cleanup scenario + when(mockAMContainerMap.get(mockCId)).thenReturn(null); + schedulerHandler.preemptContainer(0, mockCId); + verify(mockTaskScheduler, times(0)).deallocateContainer(mockCId); + assertEquals(1, mockEventHandler.events.size()); + Event event = mockEventHandler.events.get(0); + assertEquals(AMContainerEventType.C_COMPLETED, event.getType()); + AMContainerEventCompleted completedEvent = (AMContainerEventCompleted) event; + assertEquals(mockCId, completedEvent.getContainerId()); + assertEquals("Container preempted internally", completedEvent.getDiagnostics()); + assertTrue(completedEvent.isPreempted()); + Assert.assertFalse(completedEvent.isDiskFailed()); + assertEquals(TaskAttemptTerminationCause.INTERNAL_PREEMPTION, + completedEvent.getTerminationCause()); + + schedulerHandler.stop(); + schedulerHandler.close(); + } @Test (timeout = 5000) public void testContainerDiskFailed() throws IOException { From d0abd3d54cbd864a388b7f1e66018c46db328e16 Mon Sep 17 00:00:00 2001 From: Vineet Garg Date: Wed, 9 May 2018 12:02:15 -0700 Subject: [PATCH 078/512] TEZ-3911: Optional min/max/avg aggr. task counters reported to HistoryLoggingService at final counter aggr (Vineet Garg, via Gopal V) Signed-off-by: Gopal V --- .../org/apache/tez/common/ATSConstants.java | 4 + .../common/counters/AbstractCounterGroup.java | 7 +- .../tez/common/counters/AbstractCounters.java | 11 +- .../counters/AggregateFrameworkCounter.java | 85 +++++++++++++ .../common/counters/AggregateTezCounter.java | 31 +++++ .../counters/AggregateTezCounterDelegate.java | 118 +++++++++++++++++ .../common/counters/AggregateTezCounters.java | 119 ++++++++++++++++++ .../tez/common/counters/CounterGroupBase.java | 9 ++ .../counters/FileSystemCounterGroup.java | 9 +- .../counters/FrameworkCounterGroup.java | 10 +- .../tez/common/counters/TezCounter.java | 8 ++ .../tez/common/counters/TezCounters.java | 12 +- .../apache/tez/dag/app/dag/impl/DAGImpl.java | 22 ++-- .../tez/dag/app/dag/impl/VertexImpl.java | 30 ++--- .../tez/dag/history/utils/DAGUtils.java | 11 ++ .../tez/dag/app/TestMockDAGAppMaster.java | 84 +++++++++++++ 16 files changed, 539 insertions(+), 31 deletions(-) create mode 100644 tez-api/src/main/java/org/apache/tez/common/counters/AggregateFrameworkCounter.java create mode 100644 tez-api/src/main/java/org/apache/tez/common/counters/AggregateTezCounter.java create mode 100644 tez-api/src/main/java/org/apache/tez/common/counters/AggregateTezCounterDelegate.java create mode 100644 tez-api/src/main/java/org/apache/tez/common/counters/AggregateTezCounters.java diff --git a/tez-api/src/main/java/org/apache/tez/common/ATSConstants.java b/tez-api/src/main/java/org/apache/tez/common/ATSConstants.java index 6e07849341..47d536fdd9 100644 --- a/tez-api/src/main/java/org/apache/tez/common/ATSConstants.java +++ b/tez-api/src/main/java/org/apache/tez/common/ATSConstants.java @@ -107,6 +107,10 @@ public class ATSConstants { public static final String COUNTER_NAME = "counterName"; public static final String COUNTER_DISPLAY_NAME = "counterDisplayName"; public static final String COUNTER_VALUE = "counterValue"; + public static final String COUNTER_MIN_VALUE = "counterMinValue"; + public static final String COUNTER_MAX_VALUE = "counterMaxValue"; + public static final String COUNTER_INSTANCE_COUNT = "counterInstanceCount"; + /* Url related */ public static final String RESOURCE_URI_BASE = "/ws/v1/timeline"; diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounterGroup.java b/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounterGroup.java index a4b153f6cd..1d1b56d062 100644 --- a/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounterGroup.java +++ b/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounterGroup.java @@ -194,10 +194,15 @@ public synchronized int hashCode() { @Override public void incrAllCounters(CounterGroupBase rightGroup) { + aggrAllCounters(rightGroup); + } + + @Override + public void aggrAllCounters(CounterGroupBase rightGroup) { try { for (TezCounter right : rightGroup) { TezCounter left = findCounter(right.getName(), right.getDisplayName()); - left.increment(right.getValue()); + left.aggregate(right); } } catch (LimitExceededException e) { counters.clear(); diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounters.java b/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounters.java index 470cb78fdd..5910164318 100644 --- a/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounters.java +++ b/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounters.java @@ -354,13 +354,22 @@ public synchronized String toString() { * @param other the other Counters instance */ public synchronized void incrAllCounters(AbstractCounters other) { + aggrAllCounters(other); + } + + /** + * Increments multiple counters by their amounts in another Counters + * instance. + * @param other the other Counters instance + */ + public synchronized void aggrAllCounters(AbstractCounters other) { for(G right : other) { String groupName = right.getName(); G left = (isFrameworkGroup(groupName) ? fgroups : groups).get(groupName); if (left == null) { left = addGroup(groupName, right.getDisplayName()); } - left.incrAllCounters(right); + left.aggrAllCounters(right); } } diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/AggregateFrameworkCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/AggregateFrameworkCounter.java new file mode 100644 index 0000000000..aa7d4462cf --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/common/counters/AggregateFrameworkCounter.java @@ -0,0 +1,85 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.common.counters; + +import org.apache.tez.common.counters.FrameworkCounterGroup.FrameworkCounter; + +@SuppressWarnings("rawtypes") +public class AggregateFrameworkCounter> extends FrameworkCounter implements AggregateTezCounter { + + private long min = Long.MAX_VALUE; + private long max = Long.MIN_VALUE; + private long count = 0; + + @SuppressWarnings("unchecked") + public AggregateFrameworkCounter(Enum ref, String groupName) { + super(ref, groupName); + } + + @Override + public void increment(long incr) { + throw new IllegalArgumentException("Cannot increment an aggregate counter directly"); + } + + @Override + public void aggregate(TezCounter other) { + final long val = other.getValue(); + final long othermax; + final long othermin; + final long othercount; + if (other instanceof AggregateTezCounter) { + othermax = ((AggregateTezCounter) other).getMax(); + othermin = ((AggregateTezCounter) other).getMin(); + othercount = ((AggregateTezCounter) other).getCount(); + } else { + othermin = othermax = val; + othercount = 1; + } + this.count += othercount; + super.increment(val); + if (this.min == Long.MAX_VALUE) { + this.min = othermin; + this.max = othermax; + return; + } + this.min = Math.min(this.min, othermin); + this.max = Math.max(this.max, othermax); + } + + @Override + public long getMin() { + return min; + } + + @Override + public long getMax() { + return max; + } + + @SuppressWarnings("unchecked") + public FrameworkCounter asFrameworkCounter() { + return ((FrameworkCounter)this); + } + + @Override + public long getCount() { + return count; + } + +} diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/AggregateTezCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/AggregateTezCounter.java new file mode 100644 index 0000000000..bf711dae86 --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/common/counters/AggregateTezCounter.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.common.counters; + +public interface AggregateTezCounter { + + public abstract void aggregate(TezCounter other); + + public abstract long getMin(); + + public abstract long getMax(); + + public abstract long getCount(); + +} \ No newline at end of file diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/AggregateTezCounterDelegate.java b/tez-api/src/main/java/org/apache/tez/common/counters/AggregateTezCounterDelegate.java new file mode 100644 index 0000000000..ae2ca7b0c6 --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/common/counters/AggregateTezCounterDelegate.java @@ -0,0 +1,118 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.common.counters; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +public class AggregateTezCounterDelegate extends AbstractCounter implements AggregateTezCounter { + + private final T child; + private long min = Long.MAX_VALUE; + private long max = Long.MIN_VALUE; + private long count = 0; + + public AggregateTezCounterDelegate(T child) { + this.child = child; + } + + @Override + public String getName() { + return child.getName(); // this is a pass-through + } + + @Override + public String getDisplayName() { + return child.getDisplayName(); + } + + @Override + public long getValue() { + return child.getValue(); + } + + @Override + public void setValue(long value) { + this.child.setValue(value); + } + + @Override + public void increment(long incr) { + throw new UnsupportedOperationException("Cannot increment an aggregate counter"); + } + + /* (non-Javadoc) + * @see org.apache.tez.common.counters.AggregateTezCounter#aggregate(org.apache.tez.common.counters.TezCounter) + */ + @Override + public void aggregate(TezCounter other) { + final long val = other.getValue(); + final long othermax; + final long othermin; + final long othercount; + if (other instanceof AggregateTezCounter) { + othermax = ((AggregateTezCounter) other).getMax(); + othermin = ((AggregateTezCounter) other).getMin(); + othercount = ((AggregateTezCounter) other).getCount(); + } else { + othermin = othermax = val; + othercount = 1; + } + this.count += othercount; + this.child.increment(val); + if (this.min == Long.MAX_VALUE) { + this.min = othermin; + this.max = othermax; + return; + } + this.min = Math.min(this.min, othermin); + this.max = Math.max(this.max, othermax); + } + + @Override + public TezCounter getUnderlyingCounter() { + return this.child; + } + + @Override + public void readFields(DataInput arg0) throws IOException { + throw new UnsupportedOperationException("Cannot deserialize an aggregate counter"); + } + + @Override + public void write(DataOutput arg0) throws IOException { + throw new UnsupportedOperationException("Cannot deserialize an aggregate counter"); + } + + @Override + public long getMin() { + return min; + } + + @Override + public long getMax() { + return max; + } + + @Override + public long getCount() { + return count; + } +} diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/AggregateTezCounters.java b/tez-api/src/main/java/org/apache/tez/common/counters/AggregateTezCounters.java new file mode 100644 index 0000000000..332c24a960 --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/common/counters/AggregateTezCounters.java @@ -0,0 +1,119 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.common.counters; + +public class AggregateTezCounters extends TezCounters { + + private static final GroupFactory groupFactory = new GroupFactory(); + + public AggregateTezCounters() { + super(groupFactory); + } + + // Mix framework group implementation into CounterGroup interface + private static class AggregateFrameworkGroupImpl> + extends FrameworkCounterGroup implements CounterGroup { + + AggregateFrameworkGroupImpl(Class cls) { + super(cls); + } + + @Override + protected FrameworkCounter newCounter(T key) { + return (new AggregateFrameworkCounter(key, getName())) + .asFrameworkCounter(); + } + + @Override + public CounterGroupBase getUnderlyingGroup() { + return this; + } + } + + // Mix generic group implementation into CounterGroup interface + // and provide some mandatory group factory methods. + private static class AggregateGenericGroup extends AbstractCounterGroup + implements CounterGroup { + + AggregateGenericGroup(String name, String displayName, Limits limits) { + super(name, displayName, limits); + } + + @Override + protected TezCounter newCounter(String name, String displayName, long value) { + return new AggregateTezCounterDelegate(new GenericCounter(name, displayName, value)); + } + + @Override + protected TezCounter newCounter() { + return new AggregateTezCounterDelegate(new GenericCounter()); + } + + @Override + public CounterGroupBase getUnderlyingGroup() { + return this; + } + } + + // Mix file system group implementation into the CounterGroup interface + private static class AggregateFileSystemGroup extends FileSystemCounterGroup + implements CounterGroup { + + @Override + protected TezCounter newCounter(String scheme, FileSystemCounter key) { + return new AggregateTezCounterDelegate(new FSCounter(scheme, key)); + } + + @Override + public CounterGroupBase getUnderlyingGroup() { + return this; + } + } + + /** + * Provide factory methods for counter group factory implementation. + * See also the GroupFactory in + * {@link org.apache.hadoop.TezCounters.Counters mapred.Counters} + */ + private static class GroupFactory + extends CounterGroupFactory { + + @Override + protected > + FrameworkGroupFactory + newFrameworkGroupFactory(final Class cls) { + return new FrameworkGroupFactory() { + @Override public CounterGroup newGroup(String name) { + return new AggregateFrameworkGroupImpl(cls); // impl in this package + } + }; + } + + @Override + protected CounterGroup newGenericGroup(String name, String displayName, + Limits limits) { + return new AggregateGenericGroup(name, displayName, limits); + } + + @Override + protected CounterGroup newFileSystemGroup() { + return new AggregateFileSystemGroup(); + } + } +} diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroupBase.java b/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroupBase.java index be4bf77e83..216d2f4c75 100644 --- a/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroupBase.java +++ b/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroupBase.java @@ -97,8 +97,17 @@ public interface CounterGroupBase * Increment all counters by a group of counters * @param rightGroup the group to be added to this group */ + @Deprecated void incrAllCounters(CounterGroupBase rightGroup); + /** + * Aggregate all counters by a group of counters + * @param rightGroup the group to be added to this group + */ + public default void aggrAllCounters(CounterGroupBase rightGroup) { + incrAllCounters(rightGroup); + } + @Private /** * Exposes the underlying group type if a facade. diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounterGroup.java b/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounterGroup.java index 502415484a..3ea4acd153 100644 --- a/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounterGroup.java +++ b/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounterGroup.java @@ -225,12 +225,17 @@ public int size() { } @Override - public void incrAllCounters(CounterGroupBase other) { + public void incrAllCounters(CounterGroupBase rightGroup) { + aggrAllCounters(rightGroup); + } + + @Override + public void aggrAllCounters(CounterGroupBase other) { if (checkNotNull(other.getUnderlyingGroup(), "other group") instanceof FileSystemCounterGroup) { for (TezCounter counter : other) { FSCounter c = (FSCounter) ((TezCounter)counter).getUnderlyingCounter(); - findCounter(c.scheme, c.key) .increment(counter.getValue()); + findCounter(c.scheme, c.key) .aggregate(counter); } } } diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/FrameworkCounterGroup.java b/tez-api/src/main/java/org/apache/tez/common/counters/FrameworkCounterGroup.java index 3a4aa9754e..bcb6454146 100644 --- a/tez-api/src/main/java/org/apache/tez/common/counters/FrameworkCounterGroup.java +++ b/tez-api/src/main/java/org/apache/tez/common/counters/FrameworkCounterGroup.java @@ -190,14 +190,20 @@ public int size() { return n; } + @Override + @SuppressWarnings("deprecation") + public void incrAllCounters(CounterGroupBase rightGroup) { + aggrAllCounters(rightGroup); + } + @SuppressWarnings("rawtypes") @Override - public void incrAllCounters(CounterGroupBase other) { + public void aggrAllCounters(CounterGroupBase other) { if (checkNotNull(other, "other counter group") instanceof FrameworkCounterGroup) { for (TezCounter counter : other) { findCounter(((FrameworkCounter) counter).key.name()) - .increment(counter.getValue()); + .aggregate(counter); } } } diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/TezCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/TezCounter.java index 2b40ed2dbc..4cb1ae94e0 100644 --- a/tez-api/src/main/java/org/apache/tez/common/counters/TezCounter.java +++ b/tez-api/src/main/java/org/apache/tez/common/counters/TezCounter.java @@ -73,6 +73,14 @@ public interface TezCounter extends Writable { * @param incr the value to increase this counter by */ void increment(long incr); + + /** + * Aggregate this counter with another counter + * @param other TezCounter to aggregate with, by default this is incr(other.getValue()) + */ + public default void aggregate(TezCounter other) { + increment(other.getValue()); + }; /** * Return the underlying object if this is a facade. diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/TezCounters.java b/tez-api/src/main/java/org/apache/tez/common/counters/TezCounters.java index ca03f415ba..a1205b9fdd 100644 --- a/tez-api/src/main/java/org/apache/tez/common/counters/TezCounters.java +++ b/tez-api/src/main/java/org/apache/tez/common/counters/TezCounters.java @@ -128,7 +128,17 @@ protected CounterGroup newFileSystemGroup() { * Default constructor */ public TezCounters() { - super(groupFactory); + this(groupFactory); + } + + /** + * Construct the Counters object from the another counters object + * @param the type of counter + * @param the type of counter group + */ + public > TezCounters( + CounterGroupFactory customGroupFactory) { + super(customGroupFactory); } /** diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java index ecd8d17048..bd5e0ff019 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java @@ -64,6 +64,7 @@ import org.apache.hadoop.yarn.util.Clock; import org.apache.tez.common.ATSConstants; import org.apache.tez.common.ReflectionUtils; +import org.apache.tez.common.counters.AggregateTezCounters; import org.apache.tez.common.counters.DAGCounter; import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.DagTypeConverters; @@ -700,7 +701,7 @@ public TezCounters getAllCounters() { updateCpuCounters(); TezCounters counters = new TezCounters(); counters.incrAllCounters(dagCounters); - return incrTaskCounters(counters, vertices.values()); + return aggrTaskCounters(counters, vertices.values()); } finally { readLock.unlock(); @@ -732,7 +733,7 @@ public TezCounters getCachedCounters() { updateCpuCounters(); TezCounters counters = new TezCounters(); counters.incrAllCounters(dagCounters); - return incrTaskCounters(counters, vertices.values()); + return aggrTaskCounters(counters, vertices.values()); } finally { readLock.unlock(); @@ -748,10 +749,10 @@ boolean inTerminalState() { return false; } - public static TezCounters incrTaskCounters( + public static TezCounters aggrTaskCounters( TezCounters counters, Collection vertices) { for (Vertex vertex : vertices) { - counters.incrAllCounters(vertex.getAllCounters()); + counters.aggrAllCounters(vertex.getAllCounters()); } return counters; } @@ -1399,7 +1400,7 @@ private DAGState finished(DAGState finalState) { updateCpuCounters(); TezCounters counters = null; try { - counters = getAllCounters(); + counters = constructFinalFullcounters(); } catch (LimitExceededException e) { addDiagnostic("Counters limit exceeded: " + e.getMessage()); finalState = DAGState.FAILED; @@ -1868,17 +1869,18 @@ private void mayBeConstructFinalFullCounters() { // Already constructed. Just return. return; } - this.constructFinalFullcounters(); + this.fullCounters = this.constructFinalFullcounters(); } } @Private - public void constructFinalFullcounters() { - this.fullCounters = new TezCounters(); - this.fullCounters.incrAllCounters(dagCounters); + public TezCounters constructFinalFullcounters() { + final AggregateTezCounters aggregateTezCounters = new AggregateTezCounters(); + aggregateTezCounters.aggrAllCounters(dagCounters); for (Vertex v : this.vertices.values()) { - this.fullCounters.incrAllCounters(v.getAllCounters()); + aggregateTezCounters.aggrAllCounters(v.getAllCounters()); } + return aggregateTezCounters; } /** diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index f3fc2694b6..0184657c2a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -63,6 +63,7 @@ import org.apache.tez.common.ATSConstants; import org.apache.tez.common.ReflectionUtils; import org.apache.tez.common.TezUtilsInternal; +import org.apache.tez.common.counters.AggregateTezCounters; import org.apache.tez.common.counters.LimitExceededException; import org.apache.tez.common.counters.TezCounters; import org.apache.tez.common.io.NonSyncByteArrayOutputStream; @@ -1197,8 +1198,8 @@ public TezCounters getAllCounters() { } TezCounters counters = new TezCounters(); - counters.incrAllCounters(this.counters); - return incrTaskCounters(counters, tasks.values()); + counters.aggrAllCounters(this.counters); + return aggrTaskCounters(counters, tasks.values()); } finally { readLock.unlock(); @@ -1226,8 +1227,8 @@ public TezCounters getCachedCounters() { } TezCounters counters = new TezCounters(); - counters.incrAllCounters(this.counters); - cachedCounters = incrTaskCounters(counters, tasks.values()); + counters.aggrAllCounters(this.counters); + cachedCounters = aggrTaskCounters(counters, tasks.values()); return cachedCounters; } finally { readLock.unlock(); @@ -1236,7 +1237,7 @@ public TezCounters getCachedCounters() { @Override public void addCounters(final TezCounters tezCounters) { - counters.incrAllCounters(tezCounters); + counters.aggrAllCounters(tezCounters); } @Override @@ -1335,10 +1336,10 @@ boolean inTerminalState() { return false; } - public static TezCounters incrTaskCounters( + public static TezCounters aggrTaskCounters( TezCounters counters, Collection tasks) { for (Task task : tasks) { - counters.incrAllCounters(task.getCounters()); + counters.aggrAllCounters(task.getCounters()); } return counters; } @@ -2057,7 +2058,7 @@ void logJobHistoryVertexFinishedEvent() throws IOException { || !recoveryData.isVertexSucceeded()) { logJobHistoryVertexCompletedHelper(VertexState.SUCCEEDED, finishTime, logSuccessDiagnostics ? StringUtils.join(getDiagnostics(), LINE_SEPARATOR) : "", - getAllCounters()); + constructFinalFullcounters()); } } @@ -2066,7 +2067,7 @@ void logJobHistoryVertexFailedEvent(VertexState state) throws IOException { || !recoveryData.isVertexFinished()) { TezCounters counters = null; try { - counters = getAllCounters(); + counters = constructFinalFullcounters(); } catch (LimitExceededException e) { // Ignore as failed vertex addDiagnostic("Counters limit exceeded: " + e.getMessage()); @@ -3325,7 +3326,7 @@ private void mayBeConstructFinalFullCounters() { // Already constructed. Just return. return; } - this.constructFinalFullcounters(); + this.fullCounters = this.constructFinalFullcounters(); } } @@ -3334,16 +3335,17 @@ private VertexStatisticsImpl constructStatistics() { } @Private - public void constructFinalFullcounters() { - this.fullCounters = new TezCounters(); - this.fullCounters.incrAllCounters(counters); + public TezCounters constructFinalFullcounters() { + AggregateTezCounters aggregateTezCounters = new AggregateTezCounters(); + aggregateTezCounters.aggrAllCounters(counters); this.vertexStats = new VertexStats(); for (Task t : this.tasks.values()) { vertexStats.updateStats(t.getReport()); TezCounters counters = t.getCounters(); - this.fullCounters.incrAllCounters(counters); + aggregateTezCounters.aggrAllCounters(counters); } + return aggregateTezCounters; } private static class RootInputInitFailedTransition implements diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/utils/DAGUtils.java b/tez-dag/src/main/java/org/apache/tez/dag/history/utils/DAGUtils.java index dce9e521cc..b2622adc74 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/utils/DAGUtils.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/utils/DAGUtils.java @@ -34,6 +34,8 @@ import org.apache.tez.common.ATSConstants; import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.VersionInfo; +import org.apache.tez.common.counters.AggregateTezCounter; +import org.apache.tez.common.counters.AggregateTezCounterDelegate; import org.apache.tez.common.counters.CounterGroup; import org.apache.tez.common.counters.TezCounter; import org.apache.tez.common.counters.TezCounters; @@ -153,6 +155,15 @@ public static Map convertCountersToATSMap(TezCounters counters) { counter.getDisplayName()); } counterMap.put(ATSConstants.COUNTER_VALUE, counter.getValue()); + if (counter instanceof AggregateTezCounter) { + counterMap.put(ATSConstants.COUNTER_INSTANCE_COUNT, + ((AggregateTezCounter)counter).getCount()); + counterMap.put(ATSConstants.COUNTER_MAX_VALUE, + ((AggregateTezCounter)counter).getMax()); + counterMap.put(ATSConstants.COUNTER_MIN_VALUE, + ((AggregateTezCounter)counter).getMin()); + + } counterList.add(counterMap); } } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java index 6268912b2f..859537bb68 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java @@ -45,6 +45,8 @@ import org.apache.hadoop.yarn.api.records.URL; import org.apache.log4j.Level; import org.apache.log4j.Logger; +import org.apache.tez.common.counters.AggregateFrameworkCounter; +import org.apache.tez.common.counters.AggregateTezCounterDelegate; import org.apache.tez.common.counters.CounterGroup; import org.apache.tez.common.counters.DAGCounter; import org.apache.tez.common.counters.TezCounters; @@ -457,6 +459,88 @@ public void launch(ContainerLaunchRequest event) { tezClient.stop(); } + @Test + public void testCountersAggregation() throws Exception { + TezConfiguration tezconf = new TezConfiguration(defaultConf); + MockTezClient tezClient = new MockTezClient("testMockAM", tezconf, true, null, null, null, + null, false, false); + tezClient.start(); + + final String vAName = "A"; + final String vBName = "B"; + final String procCounterName = "Proc"; + final String globalCounterName = "Global"; + DAG dag = DAG.create("testCountersAggregation"); + Vertex vA = Vertex.create(vAName, ProcessorDescriptor.create("Proc.class"), 10); + Vertex vB = Vertex.create(vBName, ProcessorDescriptor.create("Proc.class"), 1); + dag.addVertex(vA) + .addVertex(vB) + .addEdge( + Edge.create(vA, vB, EdgeProperty.create(DataMovementType.SCATTER_GATHER, + DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, + OutputDescriptor.create("Out"), InputDescriptor.create("In")))); + TezCounters temp = new TezCounters(); + temp.findCounter(new String(globalCounterName), new String(globalCounterName)).increment(1); + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + DataOutput out = new DataOutputStream(bos); + temp.write(out); + final byte[] payload = bos.toByteArray(); + + MockDAGAppMaster mockApp = tezClient.getLocalClient().getMockApp(); + MockContainerLauncher mockLauncher = mockApp.getContainerLauncher(); + mockLauncher.startScheduling(false); + mockApp.countersDelegate = new CountersDelegate() { + int counterValue = 0; + @Override + public TezCounters getCounters(TaskSpec taskSpec) { + String vName = taskSpec.getVertexName(); + TezCounters counters = new TezCounters(); + final DataInputByteBuffer in = new DataInputByteBuffer(); + in.reset(ByteBuffer.wrap(payload)); + try { + // this ensures that the serde code path is covered. + // the internal merges of counters covers the constructor code path. + counters.readFields(in); + } catch (IOException e) { + Assert.fail(e.getMessage()); + } + counters.findCounter(vName, procCounterName).setValue(++counterValue); + for (OutputSpec output : taskSpec.getOutputs()) { + counters.findCounter(vName, output.getDestinationVertexName()).setValue(++counterValue); + } + for (InputSpec input : taskSpec.getInputs()) { + counters.findCounter(vName, input.getSourceVertexName()).setValue(++counterValue); + } + return counters; + } + }; + mockApp.doSleep = false; + DAGClient dagClient = tezClient.submitDAG(dag); + mockLauncher.waitTillContainersLaunched(); + DAGImpl dagImpl = (DAGImpl) mockApp.getContext().getCurrentDAG(); + mockLauncher.startScheduling(true); + DAGStatus status = dagClient.waitForCompletion(); + Assert.assertEquals(DAGStatus.State.SUCCEEDED, status.getState()); + TezCounters counters = dagImpl.getAllCounters(); + + // verify processor counters + VertexImpl vAImpl = (VertexImpl) dagImpl.getVertex(vAName); + VertexImpl vBImpl = (VertexImpl) dagImpl.getVertex(vBName); + TezCounters vACounters = vAImpl.getAllCounters(); + TezCounters vBCounters = vBImpl.getAllCounters(); + + Assert.assertEquals(19, ((AggregateTezCounterDelegate)vACounters.findCounter(vAName, procCounterName)).getMax()); + Assert.assertEquals(1, ((AggregateTezCounterDelegate)vACounters.findCounter(vAName, procCounterName)).getMin()); + Assert.assertEquals(20, ((AggregateTezCounterDelegate)vACounters.findCounter(vAName, vBName)).getMax()); + Assert.assertEquals(2, ((AggregateTezCounterDelegate)vACounters.findCounter(vAName, vBName)).getMin()); + + Assert.assertEquals(21, ((AggregateTezCounterDelegate)vBCounters.findCounter(vBName, procCounterName)).getMin()); + Assert.assertEquals(21, ((AggregateTezCounterDelegate)vBCounters.findCounter(vBName, procCounterName)).getMax()); + Assert.assertEquals(22, ((AggregateTezCounterDelegate)vBCounters.findCounter(vBName, vAName)).getMin()); + Assert.assertEquals(22, ((AggregateTezCounterDelegate)vBCounters.findCounter(vBName, vAName)).getMax()); + + tezClient.stop(); + } @Test (timeout = 10000) public void testBasicCounters() throws Exception { From 60645a8259c1551891ec5f5789e20466bd2fc794 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Thu, 10 May 2018 16:38:16 -0500 Subject: [PATCH 079/512] TEZ-3933. Remove sleep from test TestExceptionPropagation (Jonathan Eagles via kshukla) --- .../java/org/apache/tez/test/TestExceptionPropagation.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestExceptionPropagation.java b/tez-tests/src/test/java/org/apache/tez/test/TestExceptionPropagation.java index 404e324dd6..f66bc93f7f 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestExceptionPropagation.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestExceptionPropagation.java @@ -116,7 +116,7 @@ private void startMiniTezCluster() { try { conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, TEST_ROOT_DIR); dfsCluster = - new MiniDFSCluster.Builder(conf).numDataNodes(3).format(true) + new MiniDFSCluster.Builder(conf).numDataNodes(1).storagesPerDatanode(1).format(true) .racks(null).build(); remoteFs = dfsCluster.getFileSystem(); } catch (IOException io) { @@ -295,7 +295,6 @@ public void testExceptionPropagationNonSession() throws Exception { appReport.getDiagnostics().trim()); } finally { stopNonSessionClient(); - Thread.sleep(10*1000); stopTezMiniCluster(); } } @@ -630,7 +629,6 @@ public void run(Map inputs, output.start(); output.getWriter(); - Thread.sleep(3*1000); if (this.exLocation == ExceptionLocation.PROCESSOR_RUN_ERROR) { throw new Error(this.exLocation.name()); } else if (this.exLocation == ExceptionLocation.PROCESSOR_RUN_EXCEPTION) { From 1ae62421aa6c3d9131e1673615b1c94c8e7011aa Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Mon, 14 May 2018 13:17:43 -0500 Subject: [PATCH 080/512] TEZ-3824. MRCombiner creates new JobConf copy per spill (Jonathan Eagles via jlowe) --- .../org/apache/tez/mapreduce/combine/MRCombiner.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java index 9514215e51..adfd24dda6 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java @@ -20,6 +20,7 @@ import java.io.IOException; +import org.apache.hadoop.mapred.JobConf; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience.Public; @@ -78,7 +79,13 @@ public class MRCombiner implements Combiner { private final TaskAttemptID mrTaskAttemptID; public MRCombiner(TaskContext taskContext) throws IOException { - this.conf = TezUtils.createConfFromUserPayload(taskContext.getUserPayload()); + final Configuration userConf = TezUtils.createConfFromUserPayload(taskContext.getUserPayload()); + useNewApi = ConfigUtils.useNewApi(userConf); + if (useNewApi) { + conf = new JobConf(userConf); + } else { + conf = userConf; + } assert(taskContext instanceof InputContext || taskContext instanceof OutputContext); if (taskContext instanceof OutputContext) { @@ -93,8 +100,6 @@ public MRCombiner(TaskContext taskContext) throws IOException { this.reporter = new MRTaskReporter((InputContext)taskContext); } - this.useNewApi = ConfigUtils.useNewApi(conf); - combineInputRecordsCounter = taskContext.getCounters().findCounter(TaskCounter.COMBINE_INPUT_RECORDS); combineOutputRecordsCounter = taskContext.getCounters().findCounter(TaskCounter.COMBINE_OUTPUT_RECORDS); From fd9c015750c320a9a70b917cf22f993cf144e89e Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Mon, 21 May 2018 09:32:08 -0500 Subject: [PATCH 081/512] TEZ-3937. Empty partition BitSet to byte[] conversion creates one extra byte in rounding error (Jonathan Eagles via jlowe) --- .../src/main/java/org/apache/tez/common/TezUtilsInternal.java | 2 +- .../src/test/java/org/apache/tez/common/TestTezUtils.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java index 5ba2972ea3..5d7aea3724 100644 --- a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java +++ b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java @@ -238,7 +238,7 @@ public static byte[] toByteArray(BitSet bits) { if (bits == null) { return null; } - byte[] bytes = new byte[bits.length() / 8 + 1]; + byte[] bytes = new byte[(bits.length() + 7) / 8]; for (int i = 0; i < bits.length(); i++) { if (bits.get(i)) { bytes[(bytes.length) - (i / 8) - 1] |= 1 << (i % 8); diff --git a/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java b/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java index 16efc8f52d..04eb2c0d9e 100644 --- a/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java +++ b/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java @@ -81,11 +81,11 @@ public void testCleanVertexName() { public void testBitSetToByteArray() { BitSet bitSet = createBitSet(0); byte[] bytes = TezUtilsInternal.toByteArray(bitSet); - Assert.assertTrue(bytes.length == ((bitSet.length() / 8) + 1)); + Assert.assertEquals(bytes.length, (bitSet.length() + 7) / 8); bitSet = createBitSet(1000); bytes = TezUtilsInternal.toByteArray(bitSet); - Assert.assertTrue(bytes.length == ((bitSet.length() / 8) + 1)); + Assert.assertEquals(bytes.length, (bitSet.length() + 7) / 8); } @Test (timeout=2000) From e72b0a23ab28ee4759aa5e328f5a19cecc020942 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Tue, 22 May 2018 13:28:55 -0500 Subject: [PATCH 082/512] TEZ-3935. DAG aware scheduler should release unassigned new containers rather than hold them (Jason Lowe via jeagles) --- .../apache/tez/dag/api/TezConfiguration.java | 14 ++++ .../dag/app/rm/DagAwareYarnTaskScheduler.java | 7 +- .../app/rm/TestDagAwareYarnTaskScheduler.java | 73 ++++++++++++++++++- 3 files changed, 92 insertions(+), 2 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 243f278127..50b17b9042 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -1020,6 +1020,20 @@ public TezConfiguration(boolean loadDefaults) { public static final boolean TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED_DEFAULT = false; + /** + * Boolean value. Whether to reuse new containers that could not be immediately assigned to + * pending requests. If enabled then newly assigned containers that cannot be immediately + * allocated will be held for potential reuse as if it were a container that had just completed + * a task. If disabled then newly assigned containers that cannot be immediately allocated will + * be released. Active only if container reuse is enabled. + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty(type="boolean") + public static final String TEZ_AM_CONTAINER_REUSE_NEW_CONTAINERS_ENABLED = + TEZ_AM_PREFIX + "container.reuse.new-containers.enabled"; + public static final boolean + TEZ_AM_CONTAINER_REUSE_NEW_CONTAINERS_ENABLED_DEFAULT = false; + /** * Int value. The amount of time to wait before assigning a container to the next level * of locality. NODE -> RACK -> NON_LOCAL. Delay scheduling parameter. Expert level setting. diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java index dab1cad8ba..167d879f8b 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java @@ -147,6 +147,7 @@ public class DagAwareYarnTaskScheduler extends TaskScheduler private boolean shouldReuseContainers; private boolean reuseRackLocal; private boolean reuseNonLocal; + private boolean reuseNewContainers; private long localitySchedulingDelay; private long idleContainerTimeoutMin; private long idleContainerTimeoutMax; @@ -192,6 +193,10 @@ void initialize(AMRMClientAsyncWrapper client) throws Exception { "Re-use Rack-Local cannot be disabled if Re-use Non-Local has been" + " enabled"); + reuseNewContainers = shouldReuseContainers && conf.getBoolean( + TezConfiguration.TEZ_AM_CONTAINER_REUSE_NEW_CONTAINERS_ENABLED, + TezConfiguration.TEZ_AM_CONTAINER_REUSE_NEW_CONTAINERS_ENABLED_DEFAULT); + localitySchedulingDelay = conf.getLong( TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS_DEFAULT); @@ -362,7 +367,7 @@ private synchronized List assignNewContainers(List newCon } for (HeldContainer hc : unassigned) { - if (shouldReuseContainers) { + if (reuseNewContainers) { idleTracker.add(hc); TaskRequest assigned = tryAssignReuseContainer(hc, appState, isSession); if (assigned != null) { diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java index 529f65c368..0910ed270b 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java @@ -1229,7 +1229,7 @@ public void testPreemptionNoHeadroom() throws Exception { } @Test(timeout=50000) - public void testIdleContainerAssignment() throws Exception { + public void testContainerAssignmentReleaseNewContainers() throws Exception { AMRMClientAsyncWrapperForTest mockRMClient = spy(new AMRMClientAsyncWrapperForTest()); String appHost = "host"; @@ -1241,6 +1241,77 @@ public void testIdleContainerAssignment() throws Exception { conf.setInt(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 100); conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, false); conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED, false); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_NEW_CONTAINERS_ENABLED, false); + conf.setInt(TezConfiguration.TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX, 100); + conf.setInt(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, 4000); + conf.setInt(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, 5000); + conf.setInt(TezConfiguration.TEZ_AM_SESSION_MIN_HELD_CONTAINERS, 5); + + DagInfo mockDagInfo = mock(DagInfo.class); + when(mockDagInfo.getTotalVertices()).thenReturn(10); + when(mockDagInfo.getVertexDescendants(anyInt())).thenReturn(new BitSet()); + TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf); + when(mockApp.getCurrentDagInfo()).thenReturn(mockDagInfo); + when(mockApp.isSession()).thenReturn(true); + TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp); + + MockClock clock = new MockClock(1000); + NewTaskSchedulerForTest scheduler = new NewTaskSchedulerForTest(drainableAppCallback, + mockRMClient, clock); + + scheduler.initialize(); + drainableAppCallback.drain(); + + scheduler.start(); + drainableAppCallback.drain(); + verify(mockRMClient).start(); + verify(mockRMClient).registerApplicationMaster(appHost, appPort, appUrl); + RegisterApplicationMasterResponse regResponse = mockRMClient.getRegistrationResponse(); + verify(mockApp).setApplicationRegistrationData(regResponse.getMaximumResourceCapability(), + regResponse.getApplicationACLs(), regResponse.getClientToAMTokenMasterKey(), + regResponse.getQueue()); + + assertEquals(scheduler.getClusterNodeCount(), mockRMClient.getClusterNodeCount()); + + final String rack1 = "/r1"; + final String rack2 = "/r2"; + final String node1Rack1 = "n1r1"; + final String node2Rack1 = "n2r1"; + final String node1Rack2 = "n1r2"; + MockDNSToSwitchMapping.addRackMapping(node1Rack1, rack1); + MockDNSToSwitchMapping.addRackMapping(node2Rack1, rack1); + MockDNSToSwitchMapping.addRackMapping(node1Rack2, rack2); + + Priority priorityv0 = Priority.newInstance(1); + MockTaskInfo taskv0t0 = new MockTaskInfo("taskv0t0", priorityv0, node1Rack1, rack1); + + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 1), 1); + ContainerId cid1 = ContainerId.newContainerId(attemptId, 1); + NodeId n2r1 = NodeId.newInstance(node2Rack1, 1); + Container container1 = Container.newInstance(cid1, n2r1, null, taskv0t0.capability, priorityv0, null); + + // verify new container is released is not immediately allocated + scheduler.onContainersAllocated(Collections.singletonList(container1)); + drainableAppCallback.drain(); + // app is not notified of the container being released since it never launched + verify(mockApp, never()).containerBeingReleased(cid1); + verify(mockRMClient).releaseAssignedContainer(eq(cid1)); + } + + @Test(timeout=50000) + public void testIdleContainerAssignmentReuseNewContainers() throws Exception { + AMRMClientAsyncWrapperForTest mockRMClient = spy(new AMRMClientAsyncWrapperForTest()); + + String appHost = "host"; + int appPort = 0; + String appUrl = "url"; + + Configuration conf = new Configuration(); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, true); + conf.setInt(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 100); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, false); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED, false); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_NEW_CONTAINERS_ENABLED, true); conf.setInt(TezConfiguration.TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX, 100); conf.setInt(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, 4000); conf.setInt(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, 5000); From bf87a0fd1fc5fc2de907fc81f99ea26f18881593 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Tue, 22 May 2018 17:03:55 -0500 Subject: [PATCH 083/512] TEZ-3902. Upgrade to netty-3.10.5.Final.jar (Jason Lowe via jeagles) --- pom.xml | 4 +-- .../src/main/assembly/tez-dist-minimal.xml | 1 + tez-ext-service-tests/pom.xml | 1 - .../tez/shufflehandler/ShuffleHandler.java | 26 +++++++++---------- .../tez/auxservices/ShuffleHandler.java | 26 +++++++++---------- .../tez/auxservices/TestShuffleHandler.java | 2 ++ .../http/async/netty/AsyncHttpConnection.java | 14 +++++----- 7 files changed, 37 insertions(+), 37 deletions(-) diff --git a/pom.xml b/pom.xml index 1c8caeb91d..917edb8df0 100644 --- a/pom.xml +++ b/pom.xml @@ -39,7 +39,7 @@ ${user.home}/clover.license 2.7.2 9.3.22.v20171030 - 3.6.2.Final + 3.10.5.Final 0.13.0 1.8 1.7.10 @@ -221,7 +221,7 @@ com.ning async-http-client - 1.8.16 + 1.9.40 org.slf4j diff --git a/tez-dist/src/main/assembly/tez-dist-minimal.xml b/tez-dist/src/main/assembly/tez-dist-minimal.xml index 80633ffd57..fbd1782273 100644 --- a/tez-dist/src/main/assembly/tez-dist-minimal.xml +++ b/tez-dist/src/main/assembly/tez-dist-minimal.xml @@ -24,6 +24,7 @@ true org.apache.tez:tez-aux-services + org.apache.tez:tez-ext-service-tests / diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml index d6d85732b6..e123a7a961 100644 --- a/tez-ext-service-tests/pom.xml +++ b/tez-ext-service-tests/pom.xml @@ -29,7 +29,6 @@ io.netty netty - 3.6.2.Final org.slf4j diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java index ebaf9fe601..47ac900ba5 100644 --- a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java +++ b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java @@ -367,9 +367,9 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) } // Check whether the shuffle version is compatible if (!ShuffleHeader.DEFAULT_HTTP_HEADER_NAME.equals( - request.getHeader(ShuffleHeader.HTTP_HEADER_NAME)) + request.headers().get(ShuffleHeader.HTTP_HEADER_NAME)) || !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION.equals( - request.getHeader(ShuffleHeader.HTTP_HEADER_VERSION))) { + request.headers().get(ShuffleHeader.HTTP_HEADER_VERSION))) { sendError(ctx, "Incompatible shuffle request version", BAD_REQUEST); } final Map> q = @@ -551,12 +551,12 @@ protected void setResponseHeaders(HttpResponse response, boolean keepAliveParam, long contentLength) { if (!connectionKeepAliveEnabled && !keepAliveParam) { LOG.info("Setting connection close header..."); - response.setHeader(HttpHeaders.Names.CONNECTION, CONNECTION_CLOSE); + response.headers().set(HttpHeaders.Names.CONNECTION, CONNECTION_CLOSE); } else { - response.setHeader(HttpHeaders.Names.CONTENT_LENGTH, + response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, String.valueOf(contentLength)); - response.setHeader(HttpHeaders.Names.CONNECTION, HttpHeaders.Values.KEEP_ALIVE); - response.setHeader(HttpHeaders.Values.KEEP_ALIVE, "timeout=" + response.headers().set(HttpHeaders.Names.CONNECTION, HttpHeaders.Values.KEEP_ALIVE); + response.headers().set(HttpHeaders.Values.KEEP_ALIVE, "timeout=" + connectionKeepAliveTimeOut); LOG.info("Content Length in shuffle : " + contentLength); } @@ -584,7 +584,7 @@ protected void verifyRequest(String appid, ChannelHandlerContext ctx, String enc_str = SecureShuffleUtils.buildMsgFrom(requestUri); // hash from the fetcher String urlHashStr = - request.getHeader(SecureShuffleUtils.HTTP_HEADER_URL_HASH); + request.headers().get(SecureShuffleUtils.HTTP_HEADER_URL_HASH); if (urlHashStr == null) { LOG.info("Missing header hash for " + appid); throw new IOException("fetcher cannot be authenticated"); @@ -600,11 +600,11 @@ protected void verifyRequest(String appid, ChannelHandlerContext ctx, String reply = SecureShuffleUtils.generateHash(urlHashStr.getBytes(Charsets.UTF_8), tokenSecret); - response.setHeader(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH, reply); + response.headers().set(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH, reply); // Put shuffle version into http header - response.setHeader(ShuffleHeader.HTTP_HEADER_NAME, + response.headers().set(ShuffleHeader.HTTP_HEADER_NAME, ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); - response.setHeader(ShuffleHeader.HTTP_HEADER_VERSION, + response.headers().set(ShuffleHeader.HTTP_HEADER_VERSION, ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); if (LOG.isDebugEnabled()) { int len = reply.length(); @@ -654,11 +654,11 @@ protected void sendError(ChannelHandlerContext ctx, protected void sendError(ChannelHandlerContext ctx, String message, HttpResponseStatus status) { HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status); - response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8"); + response.headers().set(CONTENT_TYPE, "text/plain; charset=UTF-8"); // Put shuffle version into http header - response.setHeader(ShuffleHeader.HTTP_HEADER_NAME, + response.headers().set(ShuffleHeader.HTTP_HEADER_NAME, ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); - response.setHeader(ShuffleHeader.HTTP_HEADER_VERSION, + response.headers().set(ShuffleHeader.HTTP_HEADER_VERSION, ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); response.setContent( ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8)); diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java index d48cc013fa..e22928edcb 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java @@ -992,9 +992,9 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) } // Check whether the shuffle version is compatible if (!ShuffleHeader.DEFAULT_HTTP_HEADER_NAME.equals( - request.getHeader(ShuffleHeader.HTTP_HEADER_NAME)) + request.headers().get(ShuffleHeader.HTTP_HEADER_NAME)) || !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION.equals( - request.getHeader(ShuffleHeader.HTTP_HEADER_VERSION))) { + request.headers().get(ShuffleHeader.HTTP_HEADER_VERSION))) { sendError(ctx, "Incompatible shuffle request version", BAD_REQUEST); } final Map> q = @@ -1280,9 +1280,9 @@ long getContentLength(List mapIds, String jobId, String dagId, String us protected void setResponseHeaders(HttpResponse response, boolean keepAliveParam, long contentLength) { if (connectionKeepAliveEnabled || keepAliveParam) { - response.setHeader(HttpHeaders.Names.CONTENT_LENGTH, String.valueOf(contentLength)); - response.setHeader(HttpHeaders.Names.CONNECTION, HttpHeaders.Values.KEEP_ALIVE); - response.setHeader(HttpHeaders.Values.KEEP_ALIVE, "timeout=" + connectionKeepAliveTimeOut); + response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, String.valueOf(contentLength)); + response.headers().set(HttpHeaders.Names.CONNECTION, HttpHeaders.Values.KEEP_ALIVE); + response.headers().set(HttpHeaders.Values.KEEP_ALIVE, "timeout=" + connectionKeepAliveTimeOut); if (LOG.isDebugEnabled()) { LOG.debug("Content Length in shuffle : " + contentLength); } @@ -1290,7 +1290,7 @@ protected void setResponseHeaders(HttpResponse response, boolean keepAliveParam, if (LOG.isDebugEnabled()) { LOG.debug("Setting connection close header..."); } - response.setHeader(HttpHeaders.Names.CONNECTION, CONNECTION_CLOSE); + response.headers().set(HttpHeaders.Names.CONNECTION, CONNECTION_CLOSE); } } @@ -1316,7 +1316,7 @@ protected void verifyRequest(String appid, ChannelHandlerContext ctx, String enc_str = SecureShuffleUtils.buildMsgFrom(requestUri); // hash from the fetcher String urlHashStr = - request.getHeader(SecureShuffleUtils.HTTP_HEADER_URL_HASH); + request.headers().get(SecureShuffleUtils.HTTP_HEADER_URL_HASH); if (urlHashStr == null) { LOG.info("Missing header hash for " + appid); throw new IOException("fetcher cannot be authenticated"); @@ -1332,11 +1332,11 @@ protected void verifyRequest(String appid, ChannelHandlerContext ctx, String reply = SecureShuffleUtils.generateHash(urlHashStr.getBytes(Charsets.UTF_8), tokenSecret); - response.setHeader(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH, reply); + response.headers().set(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH, reply); // Put shuffle version into http header - response.setHeader(ShuffleHeader.HTTP_HEADER_NAME, + response.headers().set(ShuffleHeader.HTTP_HEADER_NAME, ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); - response.setHeader(ShuffleHeader.HTTP_HEADER_VERSION, + response.headers().set(ShuffleHeader.HTTP_HEADER_VERSION, ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); if (LOG.isDebugEnabled()) { int len = reply.length(); @@ -1420,11 +1420,11 @@ protected void sendError(ChannelHandlerContext ctx, protected void sendError(ChannelHandlerContext ctx, String message, HttpResponseStatus status) { HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status); - response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8"); + response.headers().set(CONTENT_TYPE, "text/plain; charset=UTF-8"); // Put shuffle version into http header - response.setHeader(ShuffleHeader.HTTP_HEADER_NAME, + response.headers().set(ShuffleHeader.HTTP_HEADER_NAME, ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); - response.setHeader(ShuffleHeader.HTTP_HEADER_VERSION, + response.headers().set(ShuffleHeader.HTTP_HEADER_VERSION, ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); response.setContent( ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8)); diff --git a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java index b9fd0d2326..11c92fb50f 100644 --- a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java @@ -82,6 +82,7 @@ import org.jboss.netty.channel.socket.SocketChannel; import org.jboss.netty.channel.MessageEvent; import org.jboss.netty.channel.AbstractChannel; +import org.jboss.netty.handler.codec.http.DefaultHttpHeaders; import org.jboss.netty.handler.codec.http.DefaultHttpResponse; import org.jboss.netty.handler.codec.http.HttpHeaders; import org.jboss.netty.handler.codec.http.HttpRequest; @@ -1236,6 +1237,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { public HttpRequest createMockHttpRequest() { HttpRequest mockHttpRequest = mock(HttpRequest.class); Mockito.doReturn(HttpMethod.GET).when(mockHttpRequest).getMethod(); + Mockito.doReturn(new DefaultHttpHeaders()).when(mockHttpRequest).headers(); Mockito.doAnswer(new Answer() { @Override public Object answer(InvocationOnMock invocation) throws Throwable { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java b/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java index 735bb469c3..9243e97712 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java @@ -31,7 +31,6 @@ import org.apache.tez.http.HttpConnectionParams; import org.apache.tez.http.SSLFactory; import org.apache.tez.common.security.JobTokenSecretManager; -import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; import org.apache.tez.runtime.library.common.security.SecureShuffleUtils; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.ShuffleHeader; import org.apache.tez.util.StopWatch; @@ -92,15 +91,14 @@ private void initClient(HttpConnectionParams httpConnParams) throws IOException * setMaxConnections & addRequestFilter. */ builder - .setAllowPoolingConnection(httpConnParams.isKeepAlive()) - .setAllowSslConnectionPool(httpConnParams.isKeepAlive()) - .setCompressionEnabled(false) + .setAllowPoolingConnections(httpConnParams.isKeepAlive()) + .setAllowPoolingSslConnections(httpConnParams.isKeepAlive()) + .setCompressionEnforced(false) //.setExecutorService(applicationThreadPool) //.addRequestFilter(new ThrottleRequestFilter()) - .setMaximumConnectionsPerHost(1) - .setConnectionTimeoutInMs(httpConnParams.getConnectionTimeout()) - .setRequestTimeoutInMs(httpConnParams.getReadTimeout()) - .setUseRawUrl(true) + .setMaxConnectionsPerHost(1) + .setConnectTimeout(httpConnParams.getConnectionTimeout()) + .setDisableUrlEncodingForBoundedRequests(true) .build(); httpAsyncClient = new AsyncHttpClient(builder.build()); } From 7111078e98bddd26520a2f89cfc5c161f669ceea Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Thu, 24 May 2018 08:44:03 -0500 Subject: [PATCH 084/512] TEZ-3940. Reduce time to convert TaskFinishedEvent to string (Jonathan Eagles via jlowe) --- .../events/TaskAttemptFinishedEvent.java | 81 ++++++++++++++----- .../dag/history/events/TaskFinishedEvent.java | 48 ++++++++--- 2 files changed, 98 insertions(+), 31 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java index 96dc0997eb..18596495be 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java @@ -25,6 +25,8 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; import org.apache.tez.common.TezConverterUtils; +import org.apache.tez.common.counters.CounterGroup; +import org.apache.tez.common.counters.TezCounter; import org.apache.tez.runtime.api.TaskFailureType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -241,27 +243,68 @@ public void fromProtoStream(CodedInputStream inputStream) throws IOException { @Override public String toString() { - String counterStr = ""; + StringBuilder sb = new StringBuilder(); + sb.append("vertexName="); + sb.append(vertexName); + sb.append(", taskAttemptId="); + sb.append(taskAttemptId); + sb.append(", creationTime="); + sb.append(creationTime); + sb.append(", allocationTime="); + sb.append(allocationTime); + sb.append(", startTime="); + sb.append(startTime); + sb.append(", finishTime="); + sb.append(finishTime); + sb.append(", timeTaken="); + sb.append(finishTime - startTime); + sb.append(", status="); + sb.append(state.name()); + + if (taskFailureType != null) { + sb.append(", taskFailureType="); + sb.append(taskFailureType); + } + if (error != null) { + sb.append(", errorEnum="); + sb.append(error); + } + if (diagnostics != null) { + sb.append(", diagnostics="); + sb.append(diagnostics); + } + if (containerId != null) { + sb.append(", containerId="); + sb.append(containerId); + } + if (nodeId != null) { + sb.append(", nodeId="); + sb.append(nodeId); + } + if (nodeHttpAddress != null) { + sb.append(", nodeHttpAddress="); + sb.append(nodeHttpAddress); + } + if (state != TaskAttemptState.SUCCEEDED) { - counterStr = ", counters=" + ( tezCounters == null ? "null" : - tezCounters.toString() - .replaceAll("\\n", ", ").replaceAll("\\s+", " ")); + sb.append(", counters="); + if (tezCounters == null) { + sb.append("null"); + } else { + sb.append("Counters: "); + sb.append(tezCounters.countCounters()); + for (CounterGroup group : tezCounters) { + sb.append(", "); + sb.append(group.getDisplayName()); + for (TezCounter counter : group) { + sb.append(", "); + sb.append(counter.getDisplayName()).append("=") + .append(counter.getValue()); + } + } + } } - return "vertexName=" + vertexName - + ", taskAttemptId=" + taskAttemptId - + ", creationTime=" + creationTime - + ", allocationTime=" + allocationTime - + ", startTime=" + startTime - + ", finishTime=" + finishTime - + ", timeTaken=" + (finishTime - startTime) - + ", status=" + state.name() - + (taskFailureType != null ? ", taskFailureType=" + taskFailureType : "") - + (error != null ? ", errorEnum=" + error.name() : "") - + (diagnostics != null ? ", diagnostics=" + diagnostics : "") - + (containerId != null ? ", containerId=" + containerId.toString() : "") - + (nodeId != null ? ", nodeId=" + nodeId.toString() : "") - + (nodeHttpAddress != null ? ", nodeHttpAddress=" + nodeHttpAddress : "") - + counterStr; + return sb.toString(); } public TezTaskAttemptID getTaskAttemptID() { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java index 6befa1a399..dd7afdfc7c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java @@ -22,6 +22,8 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import org.apache.tez.common.counters.CounterGroup; +import org.apache.tez.common.counters.TezCounter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.tez.common.counters.TezCounters; @@ -122,18 +124,40 @@ public void fromProtoStream(CodedInputStream inputStream) throws IOException { @Override public String toString() { - return "vertexName=" + vertexName - + ", taskId=" + taskID - + ", startTime=" + startTime - + ", finishTime=" + finishTime - + ", timeTaken=" + (finishTime - startTime) - + ", status=" + state.name() - + ", successfulAttemptID=" + (successfulAttemptID == null ? "null" : - successfulAttemptID.toString()) - + ", diagnostics=" + diagnostics - + ", counters=" + ( tezCounters == null ? "null" : - tezCounters.toString() - .replaceAll("\\n", ", ").replaceAll("\\s+", " ")); + StringBuilder sb = new StringBuilder(); + sb.append("vertexName="); + sb.append(vertexName); + sb.append(", taskId="); + sb.append(taskID); + sb.append(", startTime="); + sb.append(startTime); + sb.append(", finishTime="); + sb.append(finishTime); + sb.append(", timeTaken="); + sb.append(finishTime - startTime); + sb.append(", status="); + sb.append(state.name()); + sb.append(", successfulAttemptID="); + sb.append(successfulAttemptID); + sb.append(", diagnostics="); + sb.append(diagnostics); + sb.append(", counters="); + if (tezCounters == null) { + sb.append("null"); + } else { + sb.append("Counters: "); + sb.append(tezCounters.countCounters()); + for (CounterGroup group : tezCounters) { + sb.append(", "); + sb.append(group.getDisplayName()); + for (TezCounter counter : group) { + sb.append(", "); + sb.append(counter.getDisplayName()).append("=") + .append(counter.getValue()); + } + } + } + return sb.toString(); } public TezTaskID getTaskID() { From f6624c1525c54f64bdf28996fc469fa5543df6af Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Thu, 24 May 2018 09:09:36 -0500 Subject: [PATCH 085/512] TEZ-3939. Remove performance hit of precondition check in AM for register running task attempt (Jonathan Eagles via jlowe) --- .../java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java index 9b700f83df..d52df4f44d 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java @@ -221,9 +221,8 @@ public void registerRunningTaskAttempt(ContainerId containerId, TaskSpec taskSpe int priority) { ContainerInfo containerInfo = registeredContainers.get(containerId); - Preconditions.checkNotNull(containerInfo, - "Cannot register task attempt: " + taskSpec.getTaskAttemptID() + " to unknown container: " + - containerId); + Preconditions.checkNotNull(containerInfo, "Cannot register task attempt %s to unknown container %s", + taskSpec.getTaskAttemptID(), containerId); synchronized (containerInfo) { if (containerInfo.taskSpec != null) { throw new TezUncheckedException( From cf0302c429b1b24b75371374e6676376decc34b0 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Tue, 29 May 2018 14:23:03 -0500 Subject: [PATCH 086/512] TEZ-3943. TezClient leaks DAGClient for prewarm (Sergey Shelukhin via jlowe) --- .../java/org/apache/tez/client/TezClient.java | 24 +++++++++++++++++-- .../org/apache/tez/client/TestTezClient.java | 21 +++++++++++++++- 2 files changed, 42 insertions(+), 3 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java index d2c1af4a07..9dd4a69351 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java @@ -142,7 +142,7 @@ public class TezClient { @VisibleForTesting final ServicePluginsDescriptor servicePluginsDescriptor; private JavaOptsChecker javaOptsChecker = null; - + private DAGClient prewarmDagClient = null; private int preWarmDAGCounter = 0; /* max submitDAG request size through IPC; beyond this we transfer them in the same way we transfer local resource */ @@ -591,6 +591,25 @@ public synchronized DAGClient submitDAG(DAG dag) throws TezException, IOExceptio } } + private void closePrewarmDagClient() { + if (prewarmDagClient == null) { + return; + } + try { + prewarmDagClient.tryKillDAG(); + LOG.info("Waiting for prewarm DAG to shut down"); + prewarmDagClient.waitForCompletion(); + } catch (Exception ex) { + LOG.warn("Failed to shut down the prewarm DAG " + prewarmDagClient, ex); + } + try { + prewarmDagClient.close(); + } catch (Exception e) { + LOG.warn("Failed to close prewarm DagClient " + prewarmDagClient, e); + } + prewarmDagClient = null; + } + private DAGClient submitDAGSession(DAG dag) throws TezException, IOException { Preconditions.checkState(isSession == true, "submitDAG with additional resources applies to only session mode. " + @@ -693,6 +712,7 @@ private DAGClient submitDAGSession(DAG dag) throws TezException, IOException { * @throws IOException */ public synchronized void stop() throws TezException, IOException { + closePrewarmDagClient(); try { if (amKeepAliveService != null) { amKeepAliveService.shutdownNow(); @@ -925,7 +945,7 @@ public synchronized void preWarm(PreWarmVertex preWarmVertex, "available", e); } if(isReady) { - submitDAG(dag); + prewarmDagClient = submitDAG(dag); } else { throw new SessionNotReady("Tez AM not ready, could not submit DAG"); } diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java index 2c0406153f..e959a556e4 100644 --- a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java +++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java @@ -38,6 +38,7 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; +import static org.mockito.Matchers.isNull; import static org.mockito.Mockito.RETURNS_DEEP_STUBS; import static org.mockito.Mockito.times; import static org.mockito.Mockito.atLeast; @@ -87,10 +88,15 @@ import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolBlockingPB; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetAMStatusRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetAMStatusResponseProto; +import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetDAGStatusRequestProto; +import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetDAGStatusResponseProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.ShutdownSessionRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.TezAppMasterStatusProto; import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto; +import org.apache.tez.dag.api.records.DAGProtos.DAGStatusProto; +import org.apache.tez.dag.api.records.DAGProtos.DAGStatusStateProto; +import org.apache.tez.dag.api.records.DAGProtos.ProgressProto; import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor; import org.hamcrest.CoreMatchers; import org.junit.Assert; @@ -405,7 +411,7 @@ public void testPreWarm() throws Exception { client.start(); when(client.mockYarnClient.getApplicationReport(client.mockAppId).getYarnApplicationState()) - .thenReturn(YarnApplicationState.RUNNING); + .thenReturn(YarnApplicationState.RUNNING); when( client.sessionAmProxy.getAMStatus((RpcController) any(), (GetAMStatusRequestProto) any())) @@ -419,9 +425,21 @@ public void testPreWarm() throws Exception { SubmitDAGRequestProto proto = captor1.getValue(); assertTrue(proto.getDAGPlan().getName().startsWith(TezConstants.TEZ_PREWARM_DAG_NAME_PREFIX)); + setClientToReportStoppedDags(client); client.stop(); } + private void setClientToReportStoppedDags(TezClientForTest client) throws Exception { + when(client.mockYarnClient.getApplicationReport(client.mockAppId).getYarnApplicationState()) + .thenReturn(YarnApplicationState.FINISHED); + when(client.sessionAmProxy.getDAGStatus(isNull(RpcController.class), any(GetDAGStatusRequestProto.class))) + .thenReturn(GetDAGStatusResponseProto.newBuilder().setDagStatus(DAGStatusProto.newBuilder() + .addDiagnostics("Diagnostics_0").setState(DAGStatusStateProto.DAG_SUCCEEDED) + .setDAGProgress(ProgressProto.newBuilder() + .setFailedTaskCount(0).setKilledTaskCount(0).setRunningTaskCount(0) + .setSucceededTaskCount(1).setTotalTaskCount(1).build()).build()).build()); + } + @Test (timeout=30000) public void testPreWarmWithTimeout() throws Exception { long startTime = 0 , endTime = 0; @@ -506,6 +524,7 @@ public void run() { assertTrue("Time taken is not as expected", (endTime - startTime) <= timeout); verify(spyClient, times(2)).submitDAG(any(DAG.class)); + setClientToReportStoppedDags(client); spyClient.stop(); client.stop(); } From 1e44ba768ddbef92a83aac64e80bd30fd1e2af89 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Fri, 1 Jun 2018 15:49:44 -0500 Subject: [PATCH 087/512] TEZ-3923. Move master to Hadoop 3+ and create separate 0.9.x line (Jonathan Eagles via kshukla) --- hadoop-shim-impls/pom.xml | 4 ++-- pom.xml | 2 +- tez-api/pom.xml | 5 ++++- tez-dist/pom.xml | 4 ++-- tez-plugins/pom.xml | 2 +- tez-tools/analyzers/pom.xml | 2 +- 6 files changed, 11 insertions(+), 8 deletions(-) diff --git a/hadoop-shim-impls/pom.xml b/hadoop-shim-impls/pom.xml index c0371f1dcf..dd52c28bd1 100644 --- a/hadoop-shim-impls/pom.xml +++ b/hadoop-shim-impls/pom.xml @@ -29,7 +29,7 @@ hadoop27 - true + false hadoop-shim-2.7 @@ -38,7 +38,7 @@ hadoop28 - false + true hadoop-shim-2.8 diff --git a/pom.xml b/pom.xml index 917edb8df0..582949dbbb 100644 --- a/pom.xml +++ b/pom.xml @@ -37,7 +37,7 @@ true ${user.home}/clover.license - 2.7.2 + 3.0.2 9.3.22.v20171030 3.10.5.Final 0.13.0 diff --git a/tez-api/pom.xml b/tez-api/pom.xml index d84ed64374..3d69f32421 100644 --- a/tez-api/pom.xml +++ b/tez-api/pom.xml @@ -207,7 +207,7 @@ hadoop27 - true + false @@ -219,6 +219,9 @@ hadoop28 + + true + org.apache.hadoop diff --git a/tez-dist/pom.xml b/tez-dist/pom.xml index 4c0f5b82fe..42edfd9d7b 100644 --- a/tez-dist/pom.xml +++ b/tez-dist/pom.xml @@ -59,7 +59,7 @@ hadoop27 - true + false @@ -82,7 +82,7 @@ hadoop28 - false + true diff --git a/tez-plugins/pom.xml b/tez-plugins/pom.xml index 31c7834860..ee1df771e6 100644 --- a/tez-plugins/pom.xml +++ b/tez-plugins/pom.xml @@ -45,7 +45,7 @@ hadoop28 - false + true tez-protobuf-history-plugin diff --git a/tez-tools/analyzers/pom.xml b/tez-tools/analyzers/pom.xml index a571a03ee1..4403b4ad88 100644 --- a/tez-tools/analyzers/pom.xml +++ b/tez-tools/analyzers/pom.xml @@ -40,7 +40,7 @@ hadoop28 - false + true job-analyzer From c6a09a1ff83b2dfd50893be2fe9b1db32cc928b1 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Fri, 1 Jun 2018 18:42:24 -0500 Subject: [PATCH 088/512] TEZ-3947. TestATSHistoryWithACLs fails with Hadoop3 and Jersey 1.19 (Eric Wohlstadter via jeagles) --- .../tez/dag/history/ats/acls/TestATSHistoryWithACLs.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java b/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java index 8e5c95c36f..4b5702ef7e 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java +++ b/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java @@ -147,7 +147,7 @@ private K getTimelineData(String url, Class clazz) { ClientResponse response = resource.accept(MediaType.APPLICATION_JSON) .get(ClientResponse.class); assertEquals(200, response.getStatus()); - assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType()); + assertTrue(MediaType.APPLICATION_JSON_TYPE.isCompatible(response.getType())); K entity = response.getEntity(clazz); assertNotNull(entity); @@ -458,7 +458,7 @@ public void testDagLoggingEnabled() throws Exception { ClientResponse response = resource.accept(MediaType.APPLICATION_JSON) .get(ClientResponse.class); assertEquals(200, response.getStatus()); - assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType()); + assertTrue(MediaType.APPLICATION_JSON_TYPE.isCompatible(response.getType())); TimelineEntity entity = response.getEntity(TimelineEntity.class); assertEquals(entity.getEntityType(), "TEZ_DAG_ID"); assertEquals(entity.getEvents().get(0).getEventType(), HistoryEventType.DAG_SUBMITTED.toString()); From 1a88da0221fb822e319ed81c8410143217336e4d Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Fri, 1 Jun 2018 18:54:10 -0500 Subject: [PATCH 089/512] TEZ-3929. Upgrade Jersey to 1.19 (Eric Wohlstadter via jeagles) --- pom.xml | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 582949dbbb..5b0ff14743 100644 --- a/pom.xml +++ b/pom.xml @@ -41,6 +41,7 @@ 9.3.22.v20171030 3.10.5.Final 0.13.0 + 1.19 1.8 1.7.10 [${javac.version},) @@ -724,12 +725,12 @@ com.sun.jersey jersey-client - 1.9 + ${jersey.version} com.sun.jersey jersey-json - 1.9 + ${jersey.version} org.bouncycastle From 69423096659621b0c2528ee92c3af7e7b48cfc96 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Mon, 4 Jun 2018 10:58:34 -0500 Subject: [PATCH 090/512] TEZ-3948. Tez distribution broken with hadoop3 (Jonathan Eagles via kshukla) --- tez-plugins/pom.xml | 4 +--- tez-tools/analyzers/pom.xml | 4 +--- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/tez-plugins/pom.xml b/tez-plugins/pom.xml index ee1df771e6..3430f8367f 100644 --- a/tez-plugins/pom.xml +++ b/tez-plugins/pom.xml @@ -30,9 +30,7 @@ hadoop27 - - !skipATS - + false tez-protobuf-history-plugin diff --git a/tez-tools/analyzers/pom.xml b/tez-tools/analyzers/pom.xml index 4403b4ad88..510ab20a4e 100644 --- a/tez-tools/analyzers/pom.xml +++ b/tez-tools/analyzers/pom.xml @@ -29,9 +29,7 @@ hadoop27 - - !skipATS - + false job-analyzer From b0eb9dc656ba42a2a9b67a043765ecec5bfd8cd5 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Mon, 4 Jun 2018 13:43:27 -0500 Subject: [PATCH 091/512] TEZ-3946. NoClassDefFoundError, org.apache.hadoop.mapred.ShuffleHandler. (Multiple tests with Hadoop3) (Eric Wohlstadter via jeagles) --- tez-plugins/tez-history-parser/pom.xml | 4 ++++ tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml | 4 ++++ tez-tools/analyzers/job-analyzer/pom.xml | 4 ++++ 3 files changed, 12 insertions(+) diff --git a/tez-plugins/tez-history-parser/pom.xml b/tez-plugins/tez-history-parser/pom.xml index 6f46047efe..bfeb88e56b 100644 --- a/tez-plugins/tez-history-parser/pom.xml +++ b/tez-plugins/tez-history-parser/pom.xml @@ -140,6 +140,10 @@ com.sun.jersey jersey-json + + org.apache.hadoop + hadoop-mapreduce-client-shuffle + diff --git a/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml b/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml index 3f5792fa29..dea92eabb8 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml @@ -139,6 +139,10 @@ jersey-json test + + org.apache.hadoop + hadoop-mapreduce-client-shuffle + diff --git a/tez-tools/analyzers/job-analyzer/pom.xml b/tez-tools/analyzers/job-analyzer/pom.xml index ec305e7296..8579897927 100644 --- a/tez-tools/analyzers/job-analyzer/pom.xml +++ b/tez-tools/analyzers/job-analyzer/pom.xml @@ -148,6 +148,10 @@ com.sun.jersey jersey-json + + org.apache.hadoop + hadoop-mapreduce-client-shuffle + From e0ca8c9a9313e44ff3fd151fe42ba1ff8fb6cec9 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Tue, 5 Jun 2018 15:29:48 -0500 Subject: [PATCH 092/512] TEZ-3949. TestATSHistoryV15 is failing with hadoop3+ (Jonathan Eagles via kshukla) --- tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml b/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml index d151a36d93..9f05b06755 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml @@ -112,6 +112,10 @@ test test-jar + + org.apache.hadoop + hadoop-mapreduce-client-shuffle + org.codehaus.jettison jettison From 09102e517b2afa20add2de4cc018b8abbb66feb3 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Tue, 5 Jun 2018 16:24:58 -0500 Subject: [PATCH 093/512] TEZ-3938. Task attempts failing due to not making progress (Kuhu Shukla via jeagles) --- .../tez/dag/app/dag/impl/TaskAttemptImpl.java | 5 +- .../tez/dag/app/dag/impl/TestTaskAttempt.java | 61 +++++++++++++++++++ 2 files changed, 63 insertions(+), 3 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java index c43bd984ad..6ad41f85d5 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java @@ -575,8 +575,6 @@ public TaskAttemptImpl(TezTaskAttemptID attemptId, EventHandler eventHandler, this.taskSpec = taskSpec; this.creationCausalTA = schedulingCausalTA; this.creationTime = clock.getTime(); - //set last notified progress time to current time - this.lastNotifyProgressTimestamp = clock.getTime(); this.reportedStatus = new TaskAttemptStatus(this.attemptId); initTaskAttemptStatus(reportedStatus); @@ -1434,6 +1432,7 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent origEvent) { ta.nodeHttpAddress = StringInterner.weakIntern(container.getNodeHttpAddress()); ta.nodeRackName = StringInterner.weakIntern(RackResolver.resolve(ta.containerNodeId.getHost()) .getNetworkLocation()); + ta.lastNotifyProgressTimestamp = ta.clock.getTime(); ta.launchTime = ta.clock.getTime(); @@ -1585,7 +1584,7 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) { ta.lastNotifyProgressTimestamp = ta.clock.getTime(); } else { long currTime = ta.clock.getTime(); - if (ta.hungIntervalMax > 0 && + if (ta.hungIntervalMax > 0 && ta.lastNotifyProgressTimestamp > 0 && currTime - ta.lastNotifyProgressTimestamp > ta.hungIntervalMax) { // task is hung String diagnostics = "Attempt failed because it appears to make no progress for " + diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java index 2bad2ef8d9..503e4185fc 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java @@ -18,6 +18,8 @@ package org.apache.tez.dag.app.dag.impl; +import org.apache.tez.dag.app.MockClock; +import org.apache.tez.dag.app.rm.AMSchedulerEventTAStateUpdated; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.fail; @@ -1138,6 +1140,65 @@ taListener, taskConf, new SystemClock(), assertEquals(2, taImpl.getCounters().findCounter("group", "counter").getValue()); } + @Test (timeout = 60000L) + public void testProgressAfterSubmit() throws Exception { + ApplicationId appId = ApplicationId.newInstance(1, 2); + ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance( + appId, 0); + TezDAGID dagID = TezDAGID.getInstance(appId, 1); + TezVertexID vertexID = TezVertexID.getInstance(dagID, 1); + TezTaskID taskID = TezTaskID.getInstance(vertexID, 1); + + MockEventHandler eventHandler = spy(new MockEventHandler()); + TaskCommunicatorManagerInterface taListener = createMockTaskAttemptListener(); + + Configuration taskConf = new Configuration(); + taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class); + taskConf.setBoolean("fs.file.impl.disable.cache", true); + taskConf.setLong(TezConfiguration.TEZ_TASK_PROGRESS_STUCK_INTERVAL_MS, 50); + + locationHint = TaskLocationHint.createTaskLocationHint( + new HashSet(Arrays.asList(new String[]{"127.0.0.1"})), null); + Resource resource = Resource.newInstance(1024, 1); + + NodeId nid = NodeId.newInstance("127.0.0.1", 0); + @SuppressWarnings("deprecation") + ContainerId contId = ContainerId.newInstance(appAttemptId, 3); + Container container = mock(Container.class); + when(container.getId()).thenReturn(contId); + when(container.getNodeId()).thenReturn(nid); + when(container.getNodeHttpAddress()).thenReturn("localhost:0"); + + AMContainerMap containers = new AMContainerMap( + mock(ContainerHeartbeatHandler.class), mock(TaskCommunicatorManagerInterface.class), + new ContainerContextMatcher(), appCtx); + containers.addContainerIfNew(container, 0, 0, 0); + + doReturn(new ClusterInfo()).when(appCtx).getClusterInfo(); + doReturn(containers).when(appCtx).getAllContainers(); + + TaskHeartbeatHandler mockHeartbeatHandler = mock(TaskHeartbeatHandler.class); + MockClock mockClock = new MockClock(); + TaskAttemptImpl taImpl = new MockTaskAttemptImpl(taskID, 1, eventHandler, + taListener, taskConf, mockClock, + mockHeartbeatHandler, appCtx, false, + resource, createFakeContainerContext(), false); + TezTaskAttemptID taskAttemptID = taImpl.getID(); + ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); + mockClock.incrementTime(20L); + taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); + mockClock.incrementTime(55L); + taImpl.handle(new TaskAttemptEventSubmitted(taskAttemptID, contId)); + taImpl.handle(new TaskAttemptEventStatusUpdate( + taskAttemptID, new TaskStatusUpdateEvent(null, 0.1f, null, false))); + verify(eventHandler, atLeast(1)).handle(arg.capture()); + if (arg.getValue() instanceof TaskAttemptEvent) { + taImpl.handle((TaskAttemptEvent) arg.getValue()); + } + Assert.assertEquals("Task Attempt's internal state should be SUBMITTED!", + taImpl.getInternalState(), TaskAttemptStateInternal.SUBMITTED); + } + @Test (timeout = 5000) public void testNoProgressFail() throws Exception { ApplicationId appId = ApplicationId.newInstance(1, 2); From 9058460f12043a96effb1bbc5fa019f9067b0c39 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Thu, 7 Jun 2018 11:42:49 -0500 Subject: [PATCH 094/512] TEZ-3944. TestTaskScheduler times-out on Hadoop3 (Jonathan Eagles viak kshukla) --- .../tez/dag/app/rm/TestTaskScheduler.java | 405 +++++++----------- 1 file changed, 157 insertions(+), 248 deletions(-) diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java index 1a647b1f2b..49f8fe3a5d 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java @@ -155,12 +155,12 @@ public void testTaskSchedulerNoReuse() throws Exception { Assert.assertEquals(scheduler.getClusterNodeCount(), mockRMClient.getClusterNodeCount()); - Object mockTask1 = mock(Object.class); - Object mockCookie1 = mock(Object.class); - Resource mockCapability = mock(Resource.class); + Object mockTask1 = new MockTask("task1"); + Object mockCookie1 = new Object(); + Resource mockCapability = Resource.newInstance(1024, 1); String[] hosts = {"host1", "host5"}; String[] racks = {"/default-rack", "/default-rack"}; - Priority mockPriority = mock(Priority.class); + Priority mockPriority = Priority.newInstance(1); ArgumentCaptor requestCaptor = ArgumentCaptor.forClass(CookieContainerRequest.class); // allocate task @@ -187,10 +187,10 @@ public void testTaskSchedulerNoReuse() throws Exception { releaseAssignedContainer((ContainerId) any()); // allocate tasks - Object mockTask2 = mock(Object.class); - Object mockCookie2 = mock(Object.class); - Object mockTask3 = mock(Object.class); - Object mockCookie3 = mock(Object.class); + Object mockTask2 = new MockTask("task2"); + Object mockCookie2 = new Object(); + Object mockTask3 = new MockTask("task3"); + Object mockCookie3 = new Object(); scheduler.allocateTask(mockTask1, mockCapability, hosts, racks, mockPriority, null, mockCookie1); drainableAppCallback.drain(); @@ -210,26 +210,23 @@ public void testTaskSchedulerNoReuse() throws Exception { addContainerRequest(requestCaptor.capture()); CookieContainerRequest request3 = requestCaptor.getValue(); - List containers = new ArrayList(); - Container mockContainer1 = mock(Container.class, RETURNS_DEEP_STUBS); - when(mockContainer1.getNodeId().getHost()).thenReturn("host1"); - ContainerId mockCId1 = mock(ContainerId.class); - when(mockContainer1.getId()).thenReturn(mockCId1); + NodeId host1 = NodeId.newInstance("host1", 1); + NodeId host2 = NodeId.newInstance("host2", 2); + NodeId host3 = NodeId.newInstance("host3", 3); + NodeId host4 = NodeId.newInstance("host4", 4); + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 1), 1); + ContainerId mockCId1 = ContainerId.newContainerId(attemptId, 1); + Container mockContainer1 = Container.newInstance(mockCId1, host1, null, mockCapability, mockPriority, null); + ContainerId mockCId2 = ContainerId.newContainerId(attemptId, 2); + Container mockContainer2 = Container.newInstance(mockCId2, host2, null, mockCapability, mockPriority, null); + ContainerId mockCId3 = ContainerId.newContainerId(attemptId, 3); + Container mockContainer3 = Container.newInstance(mockCId3, host3, null, mockCapability, mockPriority, null); + ContainerId mockCId4 = ContainerId.newContainerId(attemptId, 4); + Container mockContainer4 = Container.newInstance(mockCId4, host4, null, mockCapability, mockPriority, null); + List containers = new ArrayList<>(); containers.add(mockContainer1); - Container mockContainer2 = mock(Container.class, RETURNS_DEEP_STUBS); - when(mockContainer2.getNodeId().getHost()).thenReturn("host2"); - ContainerId mockCId2 = mock(ContainerId.class); - when(mockContainer2.getId()).thenReturn(mockCId2); containers.add(mockContainer2); - Container mockContainer3 = mock(Container.class, RETURNS_DEEP_STUBS); - when(mockContainer3.getNodeId().getHost()).thenReturn("host3"); - ContainerId mockCId3 = mock(ContainerId.class); - when(mockContainer3.getId()).thenReturn(mockCId3); containers.add(mockContainer3); - Container mockContainer4 = mock(Container.class, RETURNS_DEEP_STUBS); - when(mockContainer4.getNodeId().getHost()).thenReturn("host4"); - ContainerId mockCId4 = mock(ContainerId.class); - when(mockContainer4.getId()).thenReturn(mockCId4); containers.add(mockContainer4); scheduler.onContainersAllocated(containers); drainableAppCallback.drain(); @@ -284,21 +281,17 @@ public void testTaskSchedulerNoReuse() throws Exception { // verify blacklisting verify(mockRMClient, times(0)).addNodeToBlacklist((NodeId)any()); String badHost = "host6"; - NodeId badNodeId = mock(NodeId.class); - when(badNodeId.getHost()).thenReturn(badHost); + NodeId badNodeId = NodeId.newInstance(badHost, 1); scheduler.blacklistNode(badNodeId); verify(mockRMClient, times(1)).addNodeToBlacklist(badNodeId); - Object mockTask4 = mock(Object.class); - Object mockCookie4 = mock(Object.class); + Object mockTask4 = new MockTask("task4"); + Object mockCookie4 = new Object(); scheduler.allocateTask(mockTask4, mockCapability, null, null, mockPriority, null, mockCookie4); drainableAppCallback.drain(); verify(mockRMClient, times(5)).addContainerRequest(requestCaptor.capture()); - Container mockContainer5 = mock(Container.class, RETURNS_DEEP_STUBS); - when(mockContainer5.getNodeId().getHost()).thenReturn(badHost); - when(mockContainer5.getNodeId()).thenReturn(badNodeId); - ContainerId mockCId5 = mock(ContainerId.class); - when(mockContainer5.getId()).thenReturn(mockCId5); + ContainerId mockCId5 = ContainerId.newContainerId(attemptId, 5); + Container mockContainer5 = Container.newInstance(mockCId5, badNodeId, null, mockCapability, mockPriority, null); containers.clear(); containers.add(mockContainer5); scheduler.onContainersAllocated(containers); @@ -310,10 +303,9 @@ public void testTaskSchedulerNoReuse() throws Exception { verify(mockRMClient, times(4)).releaseAssignedContainer((ContainerId) any()); // verify request added back verify(mockRMClient, times(6)).addContainerRequest(requestCaptor.capture()); - Container mockContainer6 = mock(Container.class, RETURNS_DEEP_STUBS); - when(mockContainer6.getNodeId().getHost()).thenReturn("host7"); - ContainerId mockCId6 = mock(ContainerId.class); - when(mockContainer6.getId()).thenReturn(mockCId6); + ContainerId mockCId6 = ContainerId.newContainerId(attemptId, 6); + NodeId host7 = NodeId.newInstance("host7", 7); + Container mockContainer6 = Container.newInstance(mockCId6, host7, null, mockCapability, mockPriority, null); containers.clear(); containers.add(mockContainer6); scheduler.onContainersAllocated(containers); @@ -407,21 +399,19 @@ public void testTaskSchedulerInitiateStop() throws Exception { scheduler.start(); drainableAppCallback.drain(); - Object mockTask1 = mock(Object.class); - when(mockTask1.toString()).thenReturn("task1"); - Object mockCookie1 = mock(Object.class); - Resource mockCapability = mock(Resource.class); + Object mockTask1 = new MockTask("task1"); + Object mockCookie1 = new Object(); + Resource mockCapability = Resource.newInstance(1024, 1); String[] hosts = {"host1", "host5"}; String[] racks = {"/default-rack", "/default-rack"}; final Priority mockPriority1 = Priority.newInstance(1); final Priority mockPriority2 = Priority.newInstance(2); final Priority mockPriority3 = Priority.newInstance(3); - Object mockTask2 = mock(Object.class); - when(mockTask2.toString()).thenReturn("task2"); - Object mockCookie2 = mock(Object.class); - Object mockTask3 = mock(Object.class); - when(mockTask3.toString()).thenReturn("task3"); - Object mockCookie3 = mock(Object.class); + Priority mockPriority = Priority.newInstance(1); + Object mockTask2 = new MockTask("task2"); + Object mockCookie2 = new Object(); + Object mockTask3 = new MockTask("task3"); + Object mockCookie3 = new Object(); ArgumentCaptor requestCaptor = ArgumentCaptor.forClass(CookieContainerRequest.class); @@ -446,21 +436,14 @@ public void testTaskSchedulerInitiateStop() throws Exception { List containers = new ArrayList(); // sending lower priority container first to make sure its not matched - Container mockContainer1 = mock(Container.class, RETURNS_DEEP_STUBS); - when(mockContainer1.getNodeId().getHost()).thenReturn("host1"); - when(mockContainer1.getPriority()).thenReturn(mockPriority1); - when(mockContainer1.toString()).thenReturn("container1"); - ContainerId mockCId1 = mock(ContainerId.class); - when(mockContainer1.getId()).thenReturn(mockCId1); - when(mockCId1.toString()).thenReturn("container1"); + NodeId host1 = NodeId.newInstance("host1", 1); + NodeId host2 = NodeId.newInstance("host2", 2); + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 1), 1); + ContainerId mockCId1 = ContainerId.newContainerId(attemptId, 1); + Container mockContainer1 = Container.newInstance(mockCId1, host1, null, mockCapability, mockPriority, null); + ContainerId mockCId2 = ContainerId.newContainerId(attemptId, 2); + Container mockContainer2 = Container.newInstance(mockCId2, host2, null, mockCapability, mockPriority, null); containers.add(mockContainer1); - Container mockContainer2 = mock(Container.class, RETURNS_DEEP_STUBS); - when(mockContainer2.getNodeId().getHost()).thenReturn("host2"); - when(mockContainer2.getPriority()).thenReturn(mockPriority2); - when(mockContainer2.toString()).thenReturn("container2"); - ContainerId mockCId2 = mock(ContainerId.class); - when(mockContainer2.getId()).thenReturn(mockCId2); - when(mockCId2.toString()).thenReturn("container2"); containers.add(mockContainer2); ArrayList hostContainers = @@ -521,10 +504,9 @@ public void testTaskSchedulerWithReuse() throws Exception { scheduler.start(); drainableAppCallback.drain(); - Object mockTask1 = mock(Object.class); - when(mockTask1.toString()).thenReturn("task1"); - Object mockCookie1 = mock(Object.class); - Resource mockCapability = mock(Resource.class); + Object mockTask1 = new MockTask("task1"); + Object mockCookie1 = new Object(); + Resource mockCapability = Resource.newInstance(1024, 1); String[] hosts = {"host1", "host5"}; String[] racks = {"/default-rack", "/default-rack"}; final Priority mockPriority1 = Priority.newInstance(1); @@ -532,12 +514,10 @@ public void testTaskSchedulerWithReuse() throws Exception { final Priority mockPriority3 = Priority.newInstance(3); final Priority mockPriority4 = Priority.newInstance(4); final Priority mockPriority5 = Priority.newInstance(5); - Object mockTask2 = mock(Object.class); - when(mockTask2.toString()).thenReturn("task2"); - Object mockCookie2 = mock(Object.class); - Object mockTask3 = mock(Object.class); - when(mockTask3.toString()).thenReturn("task3"); - Object mockCookie3 = mock(Object.class); + Object mockTask2 = new MockTask("task2"); + Object mockCookie2 = new Object(); + Object mockTask3 = new MockTask("task3"); + Object mockCookie3 = new Object(); ArgumentCaptor requestCaptor = ArgumentCaptor.forClass(CookieContainerRequest.class); @@ -560,39 +540,24 @@ public void testTaskSchedulerWithReuse() throws Exception { addContainerRequest(requestCaptor.capture()); CookieContainerRequest request3 = requestCaptor.getValue(); - List containers = new ArrayList(); + NodeId host1 = NodeId.newInstance("host1", 1); + NodeId host2 = NodeId.newInstance("host2", 2); + NodeId host3 = NodeId.newInstance("host3", 3); + NodeId host4 = NodeId.newInstance("host4", 4); + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 1), 1); + ContainerId mockCId1 = ContainerId.newContainerId(attemptId, 1); + Container mockContainer1 = Container.newInstance(mockCId1, host1, null, mockCapability, mockPriority1, null); + ContainerId mockCId2 = ContainerId.newContainerId(attemptId, 2); + Container mockContainer2 = Container.newInstance(mockCId2, host2, null, mockCapability, mockPriority2, null); + ContainerId mockCId3 = ContainerId.newContainerId(attemptId, 3); + Container mockContainer3 = Container.newInstance(mockCId3, host3, null, mockCapability, mockPriority3, null); + ContainerId mockCId4 = ContainerId.newContainerId(attemptId, 4); + Container mockContainer4 = Container.newInstance(mockCId4, host4, null, mockCapability, mockPriority4, null); // sending lower priority container first to make sure its not matched - Container mockContainer4 = mock(Container.class, RETURNS_DEEP_STUBS); - when(mockContainer4.getNodeId().getHost()).thenReturn("host4"); - when(mockContainer4.toString()).thenReturn("container4"); - when(mockContainer4.getPriority()).thenReturn(mockPriority4); - ContainerId mockCId4 = mock(ContainerId.class); - when(mockContainer4.getId()).thenReturn(mockCId4); - when(mockCId4.toString()).thenReturn("container4"); + List containers = new ArrayList(); containers.add(mockContainer4); - Container mockContainer1 = mock(Container.class, RETURNS_DEEP_STUBS); - when(mockContainer1.getNodeId().getHost()).thenReturn("host1"); - when(mockContainer1.getPriority()).thenReturn(mockPriority1); - when(mockContainer1.toString()).thenReturn("container1"); - ContainerId mockCId1 = mock(ContainerId.class); - when(mockContainer1.getId()).thenReturn(mockCId1); - when(mockCId1.toString()).thenReturn("container1"); containers.add(mockContainer1); - Container mockContainer2 = mock(Container.class, RETURNS_DEEP_STUBS); - when(mockContainer2.getNodeId().getHost()).thenReturn("host2"); - when(mockContainer2.getPriority()).thenReturn(mockPriority2); - when(mockContainer2.toString()).thenReturn("container2"); - ContainerId mockCId2 = mock(ContainerId.class); - when(mockContainer2.getId()).thenReturn(mockCId2); - when(mockCId2.toString()).thenReturn("container2"); containers.add(mockContainer2); - Container mockContainer3 = mock(Container.class, RETURNS_DEEP_STUBS); - when(mockContainer3.getNodeId().getHost()).thenReturn("host3"); - when(mockContainer3.getPriority()).thenReturn(mockPriority3); - when(mockContainer3.toString()).thenReturn("container3"); - ContainerId mockCId3 = mock(ContainerId.class); - when(mockContainer3.getId()).thenReturn(mockCId3); - when(mockCId3.toString()).thenReturn("container3"); containers.add(mockContainer3); AtomicBoolean drainNotifier = new AtomicBoolean(false); @@ -652,25 +617,17 @@ public void testTaskSchedulerWithReuse() throws Exception { // verify blacklisting verify(mockRMClient, times(0)).addNodeToBlacklist((NodeId)any()); String badHost = "host6"; - NodeId badNodeId = mock(NodeId.class); - when(badNodeId.getHost()).thenReturn(badHost); + NodeId badNodeId = NodeId.newInstance(badHost, 1); scheduler.blacklistNode(badNodeId); verify(mockRMClient, times(1)).addNodeToBlacklist(badNodeId); - Object mockTask4 = mock(Object.class); - when(mockTask4.toString()).thenReturn("task4"); - Object mockCookie4 = mock(Object.class); + Object mockTask4 = new MockTask("task4"); + Object mockCookie4 = new Object(); scheduler.allocateTask(mockTask4, mockCapability, null, null, mockPriority4, null, mockCookie4); drainableAppCallback.drain(); verify(mockRMClient, times(4)).addContainerRequest(requestCaptor.capture()); - Container mockContainer5 = mock(Container.class, RETURNS_DEEP_STUBS); - when(mockContainer5.getNodeId().getHost()).thenReturn(badHost); - when(mockContainer5.getNodeId()).thenReturn(badNodeId); - ContainerId mockCId5 = mock(ContainerId.class); - when(mockContainer5.toString()).thenReturn("container5"); - when(mockCId5.toString()).thenReturn("container5"); - when(mockContainer5.getId()).thenReturn(mockCId5); - when(mockContainer5.getPriority()).thenReturn(mockPriority4); + ContainerId mockCId5 = ContainerId.newContainerId(attemptId, 5); + Container mockContainer5 = Container.newInstance(mockCId5, badNodeId, null, mockCapability, mockPriority4, null); containers.clear(); containers.add(mockContainer5); drainNotifier.set(false); @@ -684,12 +641,9 @@ public void testTaskSchedulerWithReuse() throws Exception { verify(mockRMClient, times(4)).releaseAssignedContainer((ContainerId) any()); // verify request added back verify(mockRMClient, times(5)).addContainerRequest(requestCaptor.capture()); - Container mockContainer6 = mock(Container.class, RETURNS_DEEP_STUBS); - when(mockContainer6.getNodeId().getHost()).thenReturn("host7"); - ContainerId mockCId6 = mock(ContainerId.class); - when(mockContainer6.getId()).thenReturn(mockCId6); - when(mockContainer6.toString()).thenReturn("container6"); - when(mockCId6.toString()).thenReturn("container6"); + NodeId host7 = NodeId.newInstance("host7", 7); + ContainerId mockCId6 = ContainerId.newContainerId(attemptId, 6); + Container mockContainer6 = Container.newInstance(mockCId6, host7, null, mockCapability, mockPriority4, null); containers.clear(); containers.add(mockContainer6); drainNotifier.set(false); @@ -712,9 +666,8 @@ public void testTaskSchedulerWithReuse() throws Exception { // verify container level matching // add a dummy task to prevent release of allocated containers - Object mockTask5 = mock(Object.class); - when(mockTask5.toString()).thenReturn("task5"); - Object mockCookie5 = mock(Object.class); + Object mockTask5 = new MockTask("task5"); + Object mockCookie5 = new Object(); scheduler.allocateTask(mockTask5, mockCapability, hosts, racks, mockPriority5, null, mockCookie5); verify(mockRMClient, times(6)).addContainerRequest(requestCaptor.capture()); @@ -722,29 +675,19 @@ public void testTaskSchedulerWithReuse() throws Exception { // add containers so that we can reference one of them for container specific // allocation containers.clear(); - Container mockContainer7 = mock(Container.class, RETURNS_DEEP_STUBS); - when(mockContainer7.getNodeId().getHost()).thenReturn("host5"); - ContainerId mockCId7 = mock(ContainerId.class); - when(mockContainer7.toString()).thenReturn("container7"); - when(mockCId7.toString()).thenReturn("container7"); - when(mockContainer7.getId()).thenReturn(mockCId7); - when(mockContainer7.getPriority()).thenReturn(mockPriority5); + NodeId host5 = NodeId.newInstance("host5", 5); + ContainerId mockCId7 = ContainerId.newContainerId(attemptId, 7); + Container mockContainer7 = Container.newInstance(mockCId7, host5, null, mockCapability, mockPriority5, null); containers.add(mockContainer7); - Container mockContainer8 = mock(Container.class, RETURNS_DEEP_STUBS); - when(mockContainer8.getNodeId().getHost()).thenReturn("host5"); - ContainerId mockCId8 = mock(ContainerId.class); - when(mockContainer8.toString()).thenReturn("container8"); - when(mockCId8.toString()).thenReturn("container8"); - when(mockContainer8.getId()).thenReturn(mockCId8); - when(mockContainer8.getPriority()).thenReturn(mockPriority5); + ContainerId mockCId8 = ContainerId.newContainerId(attemptId, 8); + Container mockContainer8 = Container.newInstance(mockCId8, host5, null, mockCapability, mockPriority5, null); containers.add(mockContainer8); drainNotifier.set(false); scheduler.onContainersAllocated(containers); drainableAppCallback.drain(); verify(mockRMClient, times(5)).releaseAssignedContainer((ContainerId) any()); - Object mockTask6 = mock(Object.class); - when(mockTask6.toString()).thenReturn("task6"); - Object mockCookie6 = mock(Object.class); + Object mockTask6 = new MockTask("task6"); + Object mockCookie6 = new Object(); // allocate request with container affinity scheduler.allocateTask(mockTask6, mockCapability, mockCId7, mockPriority5, null, mockCookie6); drainableAppCallback.drain(); @@ -823,69 +766,49 @@ public void testTaskSchedulerDetermineMinHeldContainers() throws Exception { String node1Rack3 = "n1r3"; ApplicationAttemptId appId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 0), 0); + NodeId emptyHost = NodeId.newInstance("", 1); Resource r = Resource.newInstance(0, 0); ContainerId mockCId1 = ContainerId.newInstance(appId, 0); - Container c1 = mock(Container.class, RETURNS_DEEP_STUBS); - when(c1.getNodeId().getHost()).thenReturn(""); // we are mocking directly + Container c1 = Container.newInstance(mockCId1, emptyHost, null, r, null, null); HeldContainer hc1 = Mockito.spy(new HeldContainer(c1, 0, 0, null, containerSignatureMatcher)); when(hc1.getNode()).thenReturn(node1Rack1); when(hc1.getRack()).thenReturn(rack1); - when(c1.getId()).thenReturn(mockCId1); - when(c1.getResource()).thenReturn(r); when(hc1.getContainer()).thenReturn(c1); ContainerId mockCId2 = ContainerId.newInstance(appId, 1); - Container c2 = mock(Container.class, RETURNS_DEEP_STUBS); - when(c2.getNodeId().getHost()).thenReturn(""); // we are mocking directly + Container c2 = Container.newInstance(mockCId2, emptyHost, null, r, null, null); HeldContainer hc2 = Mockito.spy(new HeldContainer(c2, 0, 0, null, containerSignatureMatcher)); when(hc2.getNode()).thenReturn(node2Rack1); when(hc2.getRack()).thenReturn(rack1); - when(c2.getId()).thenReturn(mockCId2); - when(c2.getResource()).thenReturn(r); when(hc2.getContainer()).thenReturn(c2); ContainerId mockCId3 = ContainerId.newInstance(appId, 2); - Container c3 = mock(Container.class, RETURNS_DEEP_STUBS); - when(c3.getNodeId().getHost()).thenReturn(""); // we are mocking directly + Container c3 = Container.newInstance(mockCId3, emptyHost, null, r, null, null); HeldContainer hc3 = Mockito.spy(new HeldContainer(c3, 0, 0, null, containerSignatureMatcher)); when(hc3.getNode()).thenReturn(node1Rack1); when(hc3.getRack()).thenReturn(rack1); - when(c3.getId()).thenReturn(mockCId3); - when(c3.getResource()).thenReturn(r); when(hc3.getContainer()).thenReturn(c3); ContainerId mockCId4 = ContainerId.newInstance(appId, 3); - Container c4 = mock(Container.class, RETURNS_DEEP_STUBS); - when(c4.getNodeId().getHost()).thenReturn(""); // we are mocking directly + Container c4 = Container.newInstance(mockCId4, emptyHost, null, r, null, null); HeldContainer hc4 = Mockito.spy(new HeldContainer(c4, 0, 0, null, containerSignatureMatcher)); when(hc4.getNode()).thenReturn(node2Rack1); when(hc4.getRack()).thenReturn(rack1); - when(c4.getId()).thenReturn(mockCId4); - when(c4.getResource()).thenReturn(r); when(hc4.getContainer()).thenReturn(c4); ContainerId mockCId5 = ContainerId.newInstance(appId, 4); - Container c5 = mock(Container.class, RETURNS_DEEP_STUBS); - when(c5.getNodeId().getHost()).thenReturn(""); // we are mocking directly + Container c5 = Container.newInstance(mockCId5, emptyHost, null, r, null, null); HeldContainer hc5 = Mockito.spy(new HeldContainer(c5, 0, 0, null, containerSignatureMatcher)); when(hc5.getNode()).thenReturn(node1Rack2); when(hc5.getRack()).thenReturn(rack2); - when(c5.getId()).thenReturn(mockCId5); - when(c5.getResource()).thenReturn(r); when(hc5.getContainer()).thenReturn(c5); ContainerId mockCId6 = ContainerId.newInstance(appId, 5); - Container c6 = mock(Container.class, RETURNS_DEEP_STUBS); - when(c6.getNodeId().getHost()).thenReturn(""); // we are mocking directly + Container c6 = Container.newInstance(mockCId6, emptyHost, null, r, null, null); HeldContainer hc6 = Mockito.spy(new HeldContainer(c6, 0, 0, null, containerSignatureMatcher)); when(hc6.getNode()).thenReturn(node2Rack2); when(hc6.getRack()).thenReturn(rack2); - when(c6.getId()).thenReturn(mockCId6); - when(c6.getResource()).thenReturn(r); when(hc6.getContainer()).thenReturn(c6); ContainerId mockCId7 = ContainerId.newInstance(appId, 6); - Container c7 = mock(Container.class, RETURNS_DEEP_STUBS); - when(c7.getNodeId().getHost()).thenReturn(""); // we are mocking directly + Container c7 = Container.newInstance(mockCId7, emptyHost, null, r, null, null); HeldContainer hc7 = Mockito.spy(new HeldContainer(c7, 0, 0, null, containerSignatureMatcher)); when(hc7.getNode()).thenReturn(node1Rack3); when(hc7.getRack()).thenReturn(rack3); - when(c7.getId()).thenReturn(mockCId7); - when(c7.getResource()).thenReturn(r); when(hc7.getContainer()).thenReturn(c7); scheduler.heldContainers.put(mockCId1, hc1); @@ -965,15 +888,13 @@ public void testTaskSchedulerHeldContainersReleaseAfterExpired() throws Exceptio scheduler.initialize(); scheduler.start(); + Resource mockCapability = Resource.newInstance(1024, 1); + NodeId emptyHost = NodeId.newInstance("", 1); ApplicationAttemptId appId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 0), 0); ContainerId containerId = ContainerId.newInstance(appId, 0); - Container c1 = mock(Container.class, RETURNS_DEEP_STUBS); - when(c1.getNodeId().getHost()).thenReturn(""); // we are mocking directly - - HeldContainer hc1 = mock(HeldContainer.class); - when(c1.getId()).thenReturn(containerId); - when(hc1.getContainer()).thenReturn(c1); - when(hc1.isNew()).thenReturn(false); + Container c1 = Container.newInstance(containerId, emptyHost, null, mockCapability, null, null); + + HeldContainer hc1 = new HeldContainer(c1, -1, -1, null, containerSignatureMatcher); // containerExpiryTime = 0 scheduler.heldContainers.put(containerId, hc1); @@ -1084,14 +1005,14 @@ public void testTaskSchedulerPreemption() throws Exception { verify(mockRMClient, times(0)).releaseAssignedContainer((ContainerId)any()); // allocate task - Object mockTask1 = mock(Object.class); - Object mockTask2 = mock(Object.class); - Object mockTask3 = mock(Object.class); - Object mockTask3Wait = mock(Object.class); - Object mockTask3Retry = mock(Object.class); - Object mockTask3KillA = mock(Object.class); - Object mockTask3KillB = mock(Object.class); - Object mockTaskPri8 = mock(Object.class); + Object mockTask1 = new MockTask("task1"); + Object mockTask2 = new MockTask("task2"); + Object mockTask3 = new MockTask("task3"); + Object mockTask3Wait = new MockTask("task3Wait"); + Object mockTask3Retry = new MockTask("task3Retry"); + Object mockTask3KillA = new MockTask("task3KillA"); + Object mockTask3KillB = new MockTask("task3KillB"); + Object mockTaskPri8 = new MockTask("taskPri8"); Object obj3 = new Object(); Priority pri2 = Priority.newInstance(2); Priority pri4 = Priority.newInstance(4); @@ -1146,35 +1067,21 @@ public void testTaskSchedulerPreemption() throws Exception { new LinkedList>(); anyList.add(anyContainers); + NodeId host1 = NodeId.newInstance("host1", 1); + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 1), 1); + ContainerId mockCId1 = ContainerId.newContainerId(attemptId, 1); + Container mockContainer1 = Container.newInstance(mockCId1, host1, null, taskAsk, pri2, null); + ContainerId mockCId2 = ContainerId.newContainerId(attemptId, 2); + Container mockContainer2 = Container.newInstance(mockCId2, host1, null, taskAsk, pri6, null); + ContainerId mockCId3 = ContainerId.newContainerId(attemptId, 3); + Container mockContainer3 = Container.newInstance(mockCId3, host1, null, taskAsk, pri6, null); + ContainerId mockCId4 = ContainerId.newContainerId(attemptId, 4); + Container mockContainer4 = Container.newInstance(mockCId4, host1, null, taskAsk, pri2, null); List containers = new ArrayList(); - Container mockContainer1 = mock(Container.class, RETURNS_DEEP_STUBS); - when(mockContainer1.getNodeId().getHost()).thenReturn("host1"); - when(mockContainer1.getResource()).thenReturn(taskAsk); - when(mockContainer1.getPriority()).thenReturn(pri2); - ContainerId mockCId1 = mock(ContainerId.class); - when(mockContainer1.getId()).thenReturn(mockCId1); containers.add(mockContainer1); - Container mockContainer2 = mock(Container.class, RETURNS_DEEP_STUBS); - when(mockContainer2.getNodeId().getHost()).thenReturn("host1"); - when(mockContainer2.getResource()).thenReturn(taskAsk); - when(mockContainer2.getPriority()).thenReturn(pri6); - ContainerId mockCId2 = mock(ContainerId.class); - when(mockContainer2.getId()).thenReturn(mockCId2); containers.add(mockContainer2); - Container mockContainer3A = mock(Container.class, RETURNS_DEEP_STUBS); - when(mockContainer3A.getNodeId().getHost()).thenReturn("host1"); - when(mockContainer3A.getResource()).thenReturn(taskAsk); - when(mockContainer3A.getPriority()).thenReturn(pri6); - ContainerId mockCId3A = mock(ContainerId.class); - when(mockContainer3A.getId()).thenReturn(mockCId3A); - containers.add(mockContainer3A); - Container mockContainer3B = mock(Container.class, RETURNS_DEEP_STUBS); - when(mockContainer3B.getNodeId().getHost()).thenReturn("host1"); - when(mockContainer3B.getResource()).thenReturn(taskAsk); - when(mockContainer3B.getPriority()).thenReturn(pri2); // high priority container - ContainerId mockCId3B = mock(ContainerId.class); - when(mockContainer3B.getId()).thenReturn(mockCId3B); - containers.add(mockContainer3B); + containers.add(mockContainer3); + containers.add(mockContainer4); when( mockRMClient.getMatchingRequests((Priority) any(), eq("host1"), (Resource) any())).thenAnswer( @@ -1232,11 +1139,11 @@ public Object answer(InvocationOnMock invocation) { scheduler.taskAllocations.get(mockTask1).getId()); Assert.assertEquals(mockCId2, scheduler.taskAllocations.get(mockTask3).getId()); - Assert.assertEquals(mockCId3A, + Assert.assertEquals(mockCId3, scheduler.taskAllocations.get(mockTask3KillA).getId()); // high priority container assigned to lower pri task. This task should still be preempted // because the task priority is relevant for preemption and not the container priority - Assert.assertEquals(mockCId3B, + Assert.assertEquals(mockCId4, scheduler.taskAllocations.get(mockTask3KillB).getId()); // no preemption @@ -1259,19 +1166,15 @@ public Object answer(InvocationOnMock invocation) { drainableAppCallback.drain(); verify(mockRMClient, times(6)).addContainerRequest(requestCaptor.capture()); verify(mockRMClient, times(0)).releaseAssignedContainer((ContainerId)any()); - - Container mockContainer4 = mock(Container.class, RETURNS_DEEP_STUBS); - when(mockContainer4.getNodeId().getHost()).thenReturn("host1"); - when(mockContainer4.getResource()).thenReturn(taskAsk); - when(mockContainer4.getPriority()).thenReturn(pri8); - ContainerId mockCId4 = mock(ContainerId.class); - when(mockContainer4.getId()).thenReturn(mockCId4); + + ContainerId mockCId5 = ContainerId.newContainerId(attemptId, 5); + Container mockContainer5 = Container.newInstance(mockCId5, host1, null, taskAsk, pri8, null); containers.clear(); - containers.add(mockContainer4); + containers.add(mockContainer5); // new lower pri container added that wont be matched and eventually preempted // Fudge new container being present in delayed allocation list due to race - HeldContainer heldContainer = new HeldContainer(mockContainer4, -1, -1, null, + HeldContainer heldContainer = new HeldContainer(mockContainer5, -1, -1, null, containerSignatureMatcher); scheduler.delayedContainerManager.delayedContainers.add(heldContainer); // no preemption - container assignment attempts < 3 @@ -1292,7 +1195,7 @@ public Object answer(InvocationOnMock invocation) { scheduler.getProgress(); drainableAppCallback.drain(); verify(mockRMClient, times(1)).releaseAssignedContainer((ContainerId)any()); - verify(mockRMClient, times(1)).releaseAssignedContainer(mockCId4); + verify(mockRMClient, times(1)).releaseAssignedContainer(mockCId5); // internally re-request pri8 task request because we release pri8 new container verify(mockRMClient, times(7)).addContainerRequest(requestCaptor.capture()); CookieContainerRequest reAdded = requestCaptor.getValue(); @@ -1335,7 +1238,7 @@ public Object answer(InvocationOnMock invocation) { scheduler.getProgress(); // third heartbeat drainableAppCallback.drain(); verify(mockRMClient, times(2)).releaseAssignedContainer((ContainerId)any()); - verify(mockRMClient, times(1)).releaseAssignedContainer(mockCId3B); + verify(mockRMClient, times(1)).releaseAssignedContainer(mockCId4); Assert.assertEquals(scheduler.numHeartbeats, scheduler.heartbeatAtLastPreemption); // there are pending preemptions. scheduler.getProgress(); // first heartbeat @@ -1345,7 +1248,7 @@ public Object answer(InvocationOnMock invocation) { drainableAppCallback.drain(); // Next oldest mockTaskPri3KillA gets preempted to clear 10% of outstanding running preemptable tasks verify(mockRMClient, times(3)).releaseAssignedContainer((ContainerId)any()); - verify(mockRMClient, times(1)).releaseAssignedContainer(mockCId3A); + verify(mockRMClient, times(1)).releaseAssignedContainer(mockCId3); AppFinalStatus finalStatus = new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, "", DEFAULT_APP_URL); @@ -1384,9 +1287,9 @@ public void testTaskSchedulerPreemption2() throws Exception { verify(mockRMClient, times(0)).releaseAssignedContainer((ContainerId)any()); // allocate task - Object mockTask1 = mock(Object.class); - Object mockTask2 = mock(Object.class); - Object mockTask3 = mock(Object.class); + Object mockTask1 = new MockTask("task1"); + Object mockTask2 = new MockTask("task2"); + Object mockTask3 = new MockTask("task3"); Object obj3 = new Object(); Priority pri2 = Priority.newInstance(2); Priority pri4 = Priority.newInstance(4); @@ -1411,13 +1314,11 @@ public void testTaskSchedulerPreemption2() throws Exception { Assert.assertEquals(totalResource, scheduler.getTotalResources()); verify(mockRMClient, times(0)).releaseAssignedContainer((ContainerId)any()); + NodeId host1 = NodeId.newInstance("host1", 1); + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 1), 1); + ContainerId mockCId1 = ContainerId.newContainerId(attemptId, 1); + Container mockContainer1 = Container.newInstance(mockCId1, host1, null, taskAsk, pri4, null); List containers = new ArrayList(); - Container mockContainer1 = mock(Container.class, RETURNS_DEEP_STUBS); - when(mockContainer1.getNodeId().getHost()).thenReturn("host1"); - when(mockContainer1.getResource()).thenReturn(taskAsk); - when(mockContainer1.getPriority()).thenReturn(pri4); - ContainerId mockCId1 = mock(ContainerId.class); - when(mockContainer1.getId()).thenReturn(mockCId1); containers.add(mockContainer1); Mockito.doAnswer(new Answer() { @@ -1536,12 +1437,12 @@ public void testLocalityMatching() throws Exception { String defaultRack[] = { "/default-rack" }; String otherRack[] = { "/other-rack" }; - Object mockTask1 = mock(Object.class); + Object mockTask1 = new MockTask("task1"); CookieContainerRequest mockCookie1 = mock(CookieContainerRequest.class, RETURNS_DEEP_STUBS); when(mockCookie1.getCookie().getTask()).thenReturn(mockTask1); - Object mockTask2 = mock(Object.class); + Object mockTask2 = new MockTask("task2"); CookieContainerRequest mockCookie2 = mock(CookieContainerRequest.class, RETURNS_DEEP_STUBS); when(mockCookie2.getCookie().getTask()).thenReturn(mockTask2); @@ -1640,17 +1541,15 @@ public void testContainerExpired() throws Exception { scheduler.start(); drainableAppCallback.drain(); - Object mockTask1 = mock(Object.class); - when(mockTask1.toString()).thenReturn("task1"); - Object mockCookie1 = mock(Object.class); - Resource mockCapability = mock(Resource.class); + Object mockTask1 = new MockTask("task1"); + Object mockCookie1 = new Object(); + Resource mockCapability = Resource.newInstance(1024, 1); String[] hosts = {"host1", "host5"}; String[] racks = {"/default-rack", "/default-rack"}; final Priority mockPriority1 = Priority.newInstance(1); final Priority mockPriority2 = Priority.newInstance(2); - Object mockTask2 = mock(Object.class); - when(mockTask2.toString()).thenReturn("task2"); - Object mockCookie2 = mock(Object.class); + Object mockTask2 = new MockTask("task2"); + Object mockCookie2 = new Object(); ArgumentCaptor requestCaptor = ArgumentCaptor.forClass(CookieContainerRequest.class); @@ -1669,13 +1568,10 @@ public void testContainerExpired() throws Exception { List containers = new ArrayList(); // sending only lower priority container to make sure its not matched - Container mockContainer2 = mock(Container.class, RETURNS_DEEP_STUBS); - when(mockContainer2.getNodeId().getHost()).thenReturn("host2"); - when(mockContainer2.getPriority()).thenReturn(mockPriority2); - when(mockContainer2.toString()).thenReturn("container2"); - ContainerId mockCId2 = mock(ContainerId.class); - when(mockContainer2.getId()).thenReturn(mockCId2); - when(mockCId2.toString()).thenReturn("container2"); + NodeId host2 = NodeId.newInstance("host2", 2); + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 1), 1); + ContainerId mockCId2 = ContainerId.newContainerId(attemptId, 2); + Container mockContainer2 = Container.newInstance(mockCId2, host2, null, mockCapability, mockPriority2, null); containers.add(mockContainer2); scheduler.onContainersAllocated(containers); @@ -1713,4 +1609,17 @@ private Container createContainer(int id, String host, Resource resource, + ":0", resource, priority, null); return container; } + + static class MockTask { + final String name; + + MockTask(String name) { + this.name = name; + } + + @Override + public String toString() { + return name; + } + } } From a0ea6e12225c04e02db7d2ba2e683de09e5fa9a5 Mon Sep 17 00:00:00 2001 From: Harish JP Date: Mon, 11 Jun 2018 18:57:22 +0530 Subject: [PATCH 095/512] TEZ-3951. TezClient wait too long for the DAGClient for prewarm; tries to shut down the wrong DAG (Sergey Shelukhin via Harish Jaiprakash) --- .../java/org/apache/tez/client/TezClient.java | 38 +++++++++++++------ .../apache/tez/dag/api/client/DAGClient.java | 14 ++++++- .../tez/dag/api/client/DAGClientImpl.java | 33 +++++++++++++--- .../org/apache/tez/client/TestTezClient.java | 29 ++++++++++++++ .../tez/dag/api/client/MRDAGClient.java | 5 +++ 5 files changed, 101 insertions(+), 18 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java index 9dd4a69351..ad00592076 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java @@ -112,6 +112,7 @@ public class TezClient { private static final String appIdStrPrefix = "application"; private static final String APPLICATION_ID_PREFIX = appIdStrPrefix + '_'; + private static final long PREWARM_WAIT_MS = 500; @VisibleForTesting static final String NO_CLUSTER_DIAGNOSTICS_MSG = "No cluster diagnostics found."; @@ -584,23 +585,33 @@ public DAGClientAMProtocolBlockingPB sendAMHeartbeat(DAGClientAMProtocolBlocking * if submission timed out */ public synchronized DAGClient submitDAG(DAG dag) throws TezException, IOException { - if (isSession) { - return submitDAGSession(dag); - } else { - return submitDAGApplication(dag); + DAGClient result = isSession ? submitDAGSession(dag) : submitDAGApplication(dag); + if (result != null) { + closePrewarmDagClient(); // Assume the current DAG replaced the prewarm one; no need to kill. } + return result; } - private void closePrewarmDagClient() { + private void killAndClosePrewarmDagClient(long waitTimeMs) { if (prewarmDagClient == null) { return; } try { - prewarmDagClient.tryKillDAG(); - LOG.info("Waiting for prewarm DAG to shut down"); - prewarmDagClient.waitForCompletion(); - } catch (Exception ex) { - LOG.warn("Failed to shut down the prewarm DAG " + prewarmDagClient, ex); + prewarmDagClient.tryKillDAG(); + if (waitTimeMs > 0) { + LOG.info("Waiting for prewarm DAG to shut down"); + prewarmDagClient.waitForCompletion(waitTimeMs); + } + } + catch (Exception ex) { + LOG.warn("Failed to shut down the prewarm DAG " + prewarmDagClient, ex); + } + closePrewarmDagClient(); + } + + private void closePrewarmDagClient() { + if (prewarmDagClient == null) { + return; } try { prewarmDagClient.close(); @@ -705,6 +716,11 @@ private DAGClient submitDAGSession(DAG dag) throws TezException, IOException { frameworkClient); } + @VisibleForTesting + protected long getPrewarmWaitTimeMs() { + return PREWARM_WAIT_MS; + } + /** * Stop the client. This terminates the connection to the YARN cluster. * In session mode, this shuts down the session DAG App Master @@ -712,7 +728,7 @@ private DAGClient submitDAGSession(DAG dag) throws TezException, IOException { * @throws IOException */ public synchronized void stop() throws TezException, IOException { - closePrewarmDagClient(); + killAndClosePrewarmDagClient(getPrewarmWaitTimeMs()); try { if (amKeepAliveService != null) { amKeepAliveService.shutdownNow(); diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java index c70da75c9e..6c0ebbda1b 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java @@ -103,7 +103,7 @@ public abstract VertexStatus getVertexStatus(String vertexName, public abstract void tryKillDAG() throws IOException, TezException; /** - * Wait for DAG to complete without printing any vertex statuses + * Wait forever for DAG to complete without printing any vertex statuses * * @return Final DAG Status * @throws IOException @@ -112,6 +112,17 @@ public abstract VertexStatus getVertexStatus(String vertexName, */ public abstract DAGStatus waitForCompletion() throws IOException, TezException, InterruptedException; + /** + * Wait for DAG to complete without printing any vertex statuses + * + * @param timeMs Maximum wait duration + * @return Final DAG Status, or null on timeout or if DAG is no longer running + * @throws IOException + * @throws TezException + * @throws InterruptedException + */ + public abstract DAGStatus waitForCompletion(long timeMs) throws IOException, TezException, InterruptedException; + /** * Wait for DAG to complete and periodically print *all* vertices' status. * @@ -125,4 +136,5 @@ public abstract VertexStatus getVertexStatus(String vertexName, */ public abstract DAGStatus waitForCompletionWithStatusUpdates(@Nullable Set statusGetOpts) throws IOException, TezException, InterruptedException; + } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java index 1cf0bfcd82..9e17b9b1e2 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java @@ -19,6 +19,7 @@ package org.apache.tez.dag.api.client; import javax.annotation.Nullable; + import java.io.IOException; import java.text.DecimalFormat; import java.util.Collections; @@ -28,7 +29,6 @@ import java.util.Set; import com.google.common.annotations.VisibleForTesting; - import com.google.common.base.Preconditions; import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException; @@ -337,16 +337,21 @@ public void tryKillDAG() throws IOException, TezException { } } + @Override + public DAGStatus waitForCompletion(long timeMs) throws IOException, TezException, InterruptedException { + return _waitForCompletionWithStatusUpdates(timeMs, false, EnumSet.noneOf(StatusGetOpts.class)); + } + @Override public DAGStatus waitForCompletion() throws IOException, TezException, InterruptedException { - return _waitForCompletionWithStatusUpdates(false, EnumSet.noneOf(StatusGetOpts.class)); + return _waitForCompletionWithStatusUpdates(-1, false, EnumSet.noneOf(StatusGetOpts.class)); } @Override public DAGStatus waitForCompletionWithStatusUpdates( @Nullable Set statusGetOpts) throws IOException, TezException, InterruptedException { - return _waitForCompletionWithStatusUpdates(true, statusGetOpts); + return _waitForCompletionWithStatusUpdates(-1, true, statusGetOpts); } @Override @@ -504,15 +509,21 @@ protected DAGStatus getDAGStatusViaRM() throws TezException, IOException { return dagStatus; } - private DAGStatus _waitForCompletionWithStatusUpdates(boolean vertexUpdates, + private DAGStatus _waitForCompletionWithStatusUpdates(long timeMs, + boolean vertexUpdates, @Nullable Set statusGetOpts) throws IOException, TezException, InterruptedException { DAGStatus dagStatus; boolean initPrinted = false; boolean runningPrinted = false; double dagProgress = -1.0; // Print the first one // monitoring + Long maxNs = timeMs >= 0 ? (System.nanoTime() + (timeMs * 1000000L)) : null; while (true) { - dagStatus = getDAGStatus(statusGetOpts, SLEEP_FOR_COMPLETION); + try { + dagStatus = getDAGStatus(statusGetOpts, SLEEP_FOR_COMPLETION); + } catch (DAGNotRunningException ex) { + return null; + } if (!initPrinted && (dagStatus.getState() == DAGStatus.State.INITING || dagStatus.getState() == DAGStatus.State.SUBMITTED)) { initPrinted = true; // Print once @@ -525,6 +536,9 @@ private DAGStatus _waitForCompletionWithStatusUpdates(boolean vertexUpdates, || dagStatus.getState() == DAGStatus.State.ERROR) { break; } + if (maxNs != null && System.nanoTime() > maxNs) { + return null; + } }// End of while(true) Set vertexNames = Collections.emptySet(); @@ -537,7 +551,14 @@ private DAGStatus _waitForCompletionWithStatusUpdates(boolean vertexUpdates, vertexNames = getDAGStatus(statusGetOpts).getVertexProgress().keySet(); } dagProgress = monitorProgress(vertexNames, dagProgress, null, dagStatus); - dagStatus = getDAGStatus(statusGetOpts, SLEEP_FOR_COMPLETION); + try { + dagStatus = getDAGStatus(statusGetOpts, SLEEP_FOR_COMPLETION); + } catch (DAGNotRunningException ex) { + return null; + } + if (maxNs != null && System.nanoTime() > maxNs) { + return null; + } }// end of while // Always print the last status irrespective of progress change monitorProgress(vertexNames, -1.0, statusGetOpts, dagStatus); diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java index e959a556e4..0c297d3fae 100644 --- a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java +++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java @@ -115,6 +115,7 @@ class TezClientForTest extends TezClient { YarnClient mockYarnClient; ApplicationId mockAppId; boolean callRealGetSessionAMProxy; + Long prewarmTimeoutMs; public TezClientForTest(String name, TezConfiguration tezConf, @Nullable Map localResources, @@ -135,6 +136,15 @@ protected DAGClientAMProtocolBlockingPB getAMProxy(ApplicationId appId) } return super.getAMProxy(appId); } + + public void setPrewarmTimeoutMs(Long prewarmTimeoutMs) { + this.prewarmTimeoutMs = prewarmTimeoutMs; + } + + @Override + protected long getPrewarmWaitTimeMs() { + return prewarmTimeoutMs == null ? super.getPrewarmWaitTimeMs() : prewarmTimeoutMs; + } } TezClientForTest configureAndCreateTezClient() throws YarnException, IOException, ServiceException { @@ -429,6 +439,25 @@ public void testPreWarm() throws Exception { client.stop(); } + + @Test (timeout=5000) + public void testPreWarmCloseStuck() throws Exception { + TezClientForTest client = configureAndCreateTezClient(); + client.setPrewarmTimeoutMs(10L); // Don't wait too long. + client.start(); + + when(client.mockYarnClient.getApplicationReport(client.mockAppId).getYarnApplicationState()) + .thenReturn(YarnApplicationState.RUNNING); + when(client.sessionAmProxy.getAMStatus((RpcController) any(), (GetAMStatusRequestProto) any())) + .thenReturn(GetAMStatusResponseProto.newBuilder().setStatus(TezAppMasterStatusProto.READY).build()); + + PreWarmVertex vertex = PreWarmVertex.create("PreWarm", 1, Resource.newInstance(1, 1)); + client.preWarm(vertex); + // Keep prewarm in "running" state. Client should give up waiting; if it doesn't, the test will time out. + client.stop(); + } + + private void setClientToReportStoppedDags(TezClientForTest client) throws Exception { when(client.mockYarnClient.getApplicationReport(client.mockAppId).getYarnApplicationState()) .thenReturn(YarnApplicationState.FINISHED); diff --git a/tez-mapreduce/src/main/java/org/apache/tez/dag/api/client/MRDAGClient.java b/tez-mapreduce/src/main/java/org/apache/tez/dag/api/client/MRDAGClient.java index 42b52e03ec..16dc2f8c01 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/dag/api/client/MRDAGClient.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/dag/api/client/MRDAGClient.java @@ -85,6 +85,11 @@ public DAGStatus waitForCompletion() throws IOException, TezException, Interrupt return realClient.waitForCompletion(); } + @Override + public DAGStatus waitForCompletion(long timeMs) throws IOException, TezException, InterruptedException { + return realClient.waitForCompletion(timeMs); + } + @Override public DAGStatus waitForCompletionWithStatusUpdates( @Nullable Set statusGetOpts) throws IOException, TezException, InterruptedException { From 3f2373e2b2ab3825ef50e9f19b8704265542a8b2 Mon Sep 17 00:00:00 2001 From: Sergey Shelukhin Date: Mon, 11 Jun 2018 16:17:06 -0700 Subject: [PATCH 096/512] TEZ-3953: Restore ABI-compat for DAGClient for TEZ-3951 (Sergey Shelukhin via Gopal V) Signed-off-by: Gopal V --- .../main/java/org/apache/tez/dag/api/client/DAGClient.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java index 6c0ebbda1b..ec20ef1de4 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java @@ -121,7 +121,10 @@ public abstract VertexStatus getVertexStatus(String vertexName, * @throws TezException * @throws InterruptedException */ - public abstract DAGStatus waitForCompletion(long timeMs) throws IOException, TezException, InterruptedException; + public DAGStatus waitForCompletion(long timeMs) throws IOException, TezException, InterruptedException { + // Make non-abstract to avoid compat issues in Hive. + throw new UnsupportedOperationException(); + } /** * Wait for DAG to complete and periodically print *all* vertices' status. From a39f02e4073af4ee8d6b6a7bcb06198a10aa5aac Mon Sep 17 00:00:00 2001 From: Sreenath Date: Wed, 27 Jun 2018 15:32:03 +0530 Subject: [PATCH 097/512] TEZ-3959. HTTP 502 for bower install (Harish Jaiprakash via Sree) --- tez-ui/src/main/webapp/package.json | 2 +- tez-ui/src/main/webapp/yarn.lock | 51 +++++++++++------------------ 2 files changed, 20 insertions(+), 33 deletions(-) diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json index 0d2a46ffed..a14d21583a 100644 --- a/tez-ui/src/main/webapp/package.json +++ b/tez-ui/src/main/webapp/package.json @@ -25,7 +25,7 @@ "node": ">= 0.10.0" }, "devDependencies": { - "bower": "1.7.7", + "bower": "1.8.4", "broccoli-asset-rev": "2.4.2", "broccoli-funnel": "1.0.1", "broccoli-merge-trees": "1.1.1", diff --git a/tez-ui/src/main/webapp/yarn.lock b/tez-ui/src/main/webapp/yarn.lock index 866677b01f..fbcdd21369 100644 --- a/tez-ui/src/main/webapp/yarn.lock +++ b/tez-ui/src/main/webapp/yarn.lock @@ -47,10 +47,6 @@ amdefine@>=0.0.4: version "1.0.1" resolved "https://registry.yarnpkg.com/amdefine/-/amdefine-1.0.1.tgz#4a5282ac164729e93619bcfd3ad151f817ce91f5" -ansi-regex@*, ansi-regex@^2.0.0: - version "2.1.1" - resolved "https://registry.yarnpkg.com/ansi-regex/-/ansi-regex-2.1.1.tgz#c3b33ab5ee360d86e0e628f0468ae7ef27d654df" - ansi-regex@^0.2.0, ansi-regex@^0.2.1: version "0.2.1" resolved "https://registry.yarnpkg.com/ansi-regex/-/ansi-regex-0.2.1.tgz#0d8e946967a3d8143f93e24e298525fc1b2235f9" @@ -59,6 +55,10 @@ ansi-regex@^1.0.0: version "1.1.1" resolved "https://registry.yarnpkg.com/ansi-regex/-/ansi-regex-1.1.1.tgz#41c847194646375e6a1a5d10c3ca054ef9fc980d" +ansi-regex@^2.0.0: + version "2.1.1" + resolved "https://registry.yarnpkg.com/ansi-regex/-/ansi-regex-2.1.1.tgz#c3b33ab5ee360d86e0e628f0468ae7ef27d654df" + ansi-styles@^1.1.0: version "1.1.0" resolved "https://registry.yarnpkg.com/ansi-styles/-/ansi-styles-1.1.0.tgz#eaecbf66cd706882760b2f4691582b8f55d7a7de" @@ -187,10 +187,6 @@ ast-types@0.8.12: version "0.8.12" resolved "https://registry.yarnpkg.com/ast-types/-/ast-types-0.8.12.tgz#a0d90e4351bb887716c83fd637ebf818af4adfcc" -ast-types@0.8.15: - version "0.8.15" - resolved "https://registry.yarnpkg.com/ast-types/-/ast-types-0.8.15.tgz#8eef0827f04dff0ec8857ba925abe3fea6194e52" - ast-types@0.9.6: version "0.9.6" resolved "https://registry.yarnpkg.com/ast-types/-/ast-types-0.9.6.tgz#102c9e9e9005d3e7e3829bf0c4fa24ee862ee9b9" @@ -500,7 +496,11 @@ bower-shrinkwrap-resolver-ext@^0.1.0: semver "^5.3.0" string.prototype.endswith "^0.2.0" -bower@1.7.7, bower@^1.3.12: +bower@1.8.4: + version "1.8.4" + resolved "https://registry.yarnpkg.com/bower/-/bower-1.8.4.tgz#e7876a076deb8137f7d06525dc5e8c66db82f28a" + +bower@^1.3.12: version "1.7.7" resolved "https://registry.yarnpkg.com/bower/-/bower-1.7.7.tgz#2fd7ff3ebdcba5a8ffcd84c397c8fdfe9f825f92" @@ -1759,7 +1759,7 @@ ember-truth-helpers@1.3.0: version "1.3.0" resolved "https://registry.yarnpkg.com/ember-truth-helpers/-/ember-truth-helpers-1.3.0.tgz#6ed9f83ce9a49f52bb416d55e227426339a64c60" dependencies: - ember-cli-babel "^5.1.5" + ember-cli-babel "^5.1.6" ember-wormhole@^0.3.4: version "0.3.6" @@ -2329,7 +2329,7 @@ glob-parent@^2.0.0: minimatch "^2.0.1" once "^1.3.0" -glob@5.0.13, glob@^5.0.10, glob@~5.0.0: +glob@5.0.13, glob@^5.0.10: version "5.0.13" resolved "https://registry.yarnpkg.com/glob/-/glob-5.0.13.tgz#0b6ffc3ac64eb90669f723a00a0ebb7281b33f8f" dependencies: @@ -2339,7 +2339,7 @@ glob@5.0.13, glob@^5.0.10, glob@~5.0.0: once "^1.3.0" path-is-absolute "^1.0.0" -glob@5.x, glob@^5.0.15, glob@~5.0.15: +glob@5.x, glob@^5.0.15, glob@~5.0.0, glob@~5.0.15: version "5.0.15" resolved "https://registry.yarnpkg.com/glob/-/glob-5.0.15.tgz#1bc936b9e02f4a603fcc222ecf7633d30b8b93b1" dependencies: @@ -4020,7 +4020,7 @@ realize-package-specifier@~3.0.1: dezalgo "^1.0.1" npm-package-arg "^4.1.1" -recast@0.10.33: +recast@0.10.33, recast@^0.10.10: version "0.10.33" resolved "https://registry.yarnpkg.com/recast/-/recast-0.10.33.tgz#942808f7aa016f1fa7142c461d7e5704aaa8d697" dependencies: @@ -4029,15 +4029,6 @@ recast@0.10.33: private "~0.1.5" source-map "~0.5.0" -recast@^0.10.10: - version "0.10.43" - resolved "https://registry.yarnpkg.com/recast/-/recast-0.10.43.tgz#b95d50f6d60761a5f6252e15d80678168491ce7f" - dependencies: - ast-types "0.8.15" - esprima-fb "~15001.1001.0-dev-harmony-fb" - private "~0.1.5" - source-map "~0.5.0" - recast@^0.11.17, recast@^0.11.3: version "0.11.23" resolved "https://registry.yarnpkg.com/recast/-/recast-0.11.23.tgz#451fd3004ab1e4df9b4e4b66376b2a21912462d3" @@ -4424,10 +4415,6 @@ spdx-expression-parse@~1.0.0: version "1.0.4" resolved "https://registry.yarnpkg.com/spdx-expression-parse/-/spdx-expression-parse-1.0.4.tgz#9bdf2f20e1f40ed447fbe273266191fced51626c" -spdx-license-ids@*: - version "2.0.1" - resolved "https://registry.yarnpkg.com/spdx-license-ids/-/spdx-license-ids-2.0.1.tgz#02017bcc3534ee4ffef6d58d20e7d3e9a1c3c8ec" - spdx-license-ids@^1.0.0, spdx-license-ids@^1.0.2: version "1.2.2" resolved "https://registry.yarnpkg.com/spdx-license-ids/-/spdx-license-ids-1.2.2.tgz#c9df7a3424594ade6bd11900d596696dc06bac57" @@ -4489,12 +4476,6 @@ stringstream@~0.0.4: version "0.0.5" resolved "https://registry.yarnpkg.com/stringstream/-/stringstream-0.0.5.tgz#4e484cd4de5a0bbbee18e46307710a8a81621878" -strip-ansi@*, strip-ansi@^3.0.0: - version "3.0.1" - resolved "https://registry.yarnpkg.com/strip-ansi/-/strip-ansi-3.0.1.tgz#6a385fb8853d952d5ff05d0e8aaf94278dc63dcf" - dependencies: - ansi-regex "^2.0.0" - strip-ansi@^0.3.0: version "0.3.0" resolved "https://registry.yarnpkg.com/strip-ansi/-/strip-ansi-0.3.0.tgz#25f48ea22ca79187f3174a4db8759347bb126220" @@ -4507,6 +4488,12 @@ strip-ansi@^2.0.1: dependencies: ansi-regex "^1.0.0" +strip-ansi@^3.0.0: + version "3.0.1" + resolved "https://registry.yarnpkg.com/strip-ansi/-/strip-ansi-3.0.1.tgz#6a385fb8853d952d5ff05d0e8aaf94278dc63dcf" + dependencies: + ansi-regex "^2.0.0" + strip-ansi@~0.1.0: version "0.1.1" resolved "https://registry.yarnpkg.com/strip-ansi/-/strip-ansi-0.1.1.tgz#39e8a98d044d150660abe4a6808acf70bb7bc991" From 314dfc79b4b3f528b680b4fee73ad0dca3a3a19b Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Thu, 28 Jun 2018 16:13:02 -0500 Subject: [PATCH 098/512] TEZ-3962. Configuration decode leaks an Inflater object (Eric Wohlstadter via jlowe) --- .../src/main/java/org/apache/tez/common/TezUtils.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java index efd450290f..aed9e0f332 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java @@ -112,11 +112,12 @@ public static Configuration createConfFromByteString(ByteString byteString) thro Preconditions.checkNotNull(byteString, "ByteString must be specified"); // SnappyInputStream uncompressIs = new // SnappyInputStream(byteString.newInput()); - InflaterInputStream uncompressIs = new InflaterInputStream(byteString.newInput()); - DAGProtos.ConfigurationProto confProto = DAGProtos.ConfigurationProto.parseFrom(uncompressIs); - Configuration conf = new Configuration(false); - readConfFromPB(confProto, conf); - return conf; + try(InflaterInputStream uncompressIs = new InflaterInputStream(byteString.newInput())) { + DAGProtos.ConfigurationProto confProto = DAGProtos.ConfigurationProto.parseFrom(uncompressIs); + Configuration conf = new Configuration(false); + readConfFromPB(confProto, conf); + return conf; + } } /** From 06757e9d03c613ef5fd1dfcb1d86615ba2222818 Mon Sep 17 00:00:00 2001 From: Harish JP Date: Fri, 29 Jun 2018 22:31:15 +0530 Subject: [PATCH 099/512] TEZ-3960. Better error handling in proto history logger and add doAs support. (harishjp) --- .../apache/tez/dag/api/TezConfiguration.java | 11 ++ .../logging/proto/DagManifesFileScanner.java | 105 +++++++++++++++--- .../logging/proto/DatePartitionedLogger.java | 46 +++++--- .../proto/ProtoHistoryLoggingService.java | 26 +++-- .../logging/proto/ProtoMessageReader.java | 9 +- .../logging/proto/ProtoMessageWriter.java | 12 +- .../proto/TestDagManifestFileScanner.java | 65 ++++++++++- .../proto/TestProtoHistoryLoggingService.java | 9 +- 8 files changed, 234 insertions(+), 49 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 50b17b9042..43014a4dfc 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -1473,6 +1473,17 @@ public TezConfiguration(boolean loadDefaults) { TEZ_PREFIX + "history.logging.proto-sync-window-secs"; public static final long TEZ_HISTORY_LOGGING_PROTO_SYNC_WINDOWN_SECS_DEFAULT = 60L; + /** + * Long value. The amount of time in seconds to wait to ensure all events for a day is synced + * to disk. This should be maximum time variation b/w machines + maximum time to sync file + * content and metadata. + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty(type="boolean") + public static final String TEZ_HISTORY_LOGGING_PROTO_DOAS = + TEZ_PREFIX + "history.logging.proto-doas"; + public static final boolean TEZ_HISTORY_LOGGING_PROTO_DOAS_DEFAULT = false; + /** * Int value. Time, in milliseconds, to wait while flushing YARN ATS data during shutdown. * Expert level setting. diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DagManifesFileScanner.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DagManifesFileScanner.java index c8ea02f188..697083c30c 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DagManifesFileScanner.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DagManifesFileScanner.java @@ -19,58 +19,75 @@ import java.io.Closeable; import java.io.IOException; +import java.security.PrivilegedAction; import java.time.LocalDate; import java.time.LocalDateTime; import java.util.ArrayList; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; import org.apache.commons.io.IOUtils; -import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.history.logging.proto.HistoryLoggerProtos.ManifestEntryProto; import org.codehaus.jackson.map.ObjectMapper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** - * Helper class to scan all the dag manifest files to get manifest entries. + * Helper class to scan all the dag manifest files to get manifest entries. This class is + * not thread safe. */ public class DagManifesFileScanner implements Closeable { - private static final int OFFSET_VERSION = 1; + private static final Logger LOG = LoggerFactory.getLogger(DagManifesFileScanner.class); + private static final int SCANNER_OFFSET_VERSION = 2; + private static final int MAX_RETRY = 3; private final ObjectMapper mapper = new ObjectMapper(); private final DatePartitionedLogger manifestLogger; private final long syncTime; + private final boolean withDoas; private String scanDir; private Map offsets; - private List newFiles; + private Map retryCount; + private List newFiles; private ProtoMessageReader reader; + private String currentFilePath; public DagManifesFileScanner(DatePartitionedLogger manifestLogger) { this.manifestLogger = manifestLogger; this.syncTime = manifestLogger.getConfig().getLong( TezConfiguration.TEZ_HISTORY_LOGGING_PROTO_SYNC_WINDOWN_SECS, TezConfiguration.TEZ_HISTORY_LOGGING_PROTO_SYNC_WINDOWN_SECS_DEFAULT); + this.withDoas = manifestLogger.getConfig().getBoolean( + TezConfiguration.TEZ_HISTORY_LOGGING_PROTO_DOAS, + TezConfiguration.TEZ_HISTORY_LOGGING_PROTO_DOAS_DEFAULT); this.setOffset(LocalDate.ofEpochDay(0)); } + // Update the offset version and checks below to ensure correct versions are supported. // All public to simplify json conversion. public static class DagManifestOffset { public int version; public String scanDir; public Map offsets; + public Map retryCount; } public void setOffset(String offset) { try { DagManifestOffset dagOffset = mapper.readValue(offset, DagManifestOffset.class); - if (dagOffset.version != OFFSET_VERSION) { + if (dagOffset.version > SCANNER_OFFSET_VERSION) { throw new IllegalArgumentException("Version mismatch: " + dagOffset.version); } this.scanDir = dagOffset.scanDir; - this.offsets = dagOffset.offsets; + this.offsets = dagOffset.offsets == null ? new HashMap<>() : dagOffset.offsets; + this.retryCount = dagOffset.retryCount == null ? new HashMap<>() : dagOffset.retryCount; this.newFiles = new ArrayList<>(); } catch (IOException e) { throw new IllegalArgumentException("Invalid offset", e); @@ -80,15 +97,17 @@ public void setOffset(String offset) { public void setOffset(LocalDate date) { this.scanDir = manifestLogger.getDirForDate(date); this.offsets = new HashMap<>(); + this.retryCount = new HashMap<>(); this.newFiles = new ArrayList<>(); } public String getOffset() { try { DagManifestOffset offset = new DagManifestOffset(); - offset.version = OFFSET_VERSION; + offset.version = SCANNER_OFFSET_VERSION; offset.scanDir = scanDir; offset.offsets = offsets; + offset.retryCount = retryCount; return mapper.writeValueAsString(offset); } catch (IOException e) { throw new RuntimeException("Unexpected exception while converting to json.", e); @@ -98,17 +117,26 @@ public String getOffset() { public ManifestEntryProto getNext() throws IOException { while (true) { if (reader != null) { - ManifestEntryProto evt = reader.readEvent(); + ManifestEntryProto evt = null; + try { + evt = reader.readEvent(); + retryCount.remove(currentFilePath); + } catch (IOException e) { + LOG.error("Error trying to read event from file: {}", currentFilePath, e); + incrementError(currentFilePath); + } if (evt != null) { offsets.put(reader.getFilePath().getName(), reader.getOffset()); return evt; } else { IOUtils.closeQuietly(reader); reader = null; + currentFilePath = null; } } if (!newFiles.isEmpty()) { - this.reader = manifestLogger.getReader(newFiles.remove(0)); + this.reader = getNextReader(); + this.currentFilePath = reader != null ? reader.getFilePath().toString() : null; } else { if (!loadMore()) { return null; @@ -117,6 +145,32 @@ public ManifestEntryProto getNext() throws IOException { } } + private void incrementError(String path) { + int count = retryCount.getOrDefault(path, 0); + retryCount.put(path, count + 1); + } + + private ProtoMessageReader getNextReader() throws IOException { + FileStatus status = newFiles.remove(0); + PrivilegedAction> action = () -> { + try { + return manifestLogger.getReader(status.getPath()); + } catch (IOException e) { + String path = status.getPath().toString(); + LOG.error("Error trying to open file: {}", path, e); + incrementError(path); + return null; + } + }; + if (withDoas) { + UserGroupInformation proxyUser = UserGroupInformation.createProxyUser( + status.getOwner(), UserGroupInformation.getCurrentUser()); + return proxyUser.doAs(action); + } else { + return action.run(); + } + } + @Override public void close() throws IOException { if (reader != null) { @@ -125,15 +179,35 @@ public void close() throws IOException { } } - private boolean loadMore() throws IOException { + private void filterErrors(List files) { + Iterator iter = files.iterator(); + while (iter.hasNext()) { + FileStatus status = iter.next(); + String path = status.getPath().toString(); + if (retryCount.getOrDefault(path, 0) > MAX_RETRY) { + LOG.warn("Removing file {}, too many errors", path); + iter.remove(); + } + } + } + + private void loadNewFiles(String todayDir) throws IOException { newFiles = manifestLogger.scanForChangedFiles(scanDir, offsets); + if (!scanDir.equals(todayDir)) { + filterErrors(newFiles); + } + } + + private boolean loadMore() throws IOException { + LocalDateTime now = manifestLogger.getNow(); + LocalDate today = now.toLocalDate(); + String todayDir = manifestLogger.getDirForDate(today); + loadNewFiles(todayDir); while (newFiles.isEmpty()) { - LocalDateTime utcNow = manifestLogger.getNow(); - if (utcNow.getHour() * 3600 + utcNow.getMinute() * 60 + utcNow.getSecond() < syncTime) { + if (now.getHour() * 3600 + now.getMinute() * 60 + now.getSecond() < syncTime) { // We are in the delay window for today, do not advance date if we are moving from // yesterday. - String yesterDir = manifestLogger.getDirForDate(utcNow.toLocalDate().minusDays(1)); - if (yesterDir.equals(scanDir)) { + if (scanDir.equals(manifestLogger.getDirForDate(today.minusDays(1)))) { return false; } } @@ -143,7 +217,8 @@ private boolean loadMore() throws IOException { } scanDir = nextDir; offsets = new HashMap<>(); - newFiles = manifestLogger.scanForChangedFiles(scanDir, offsets); + retryCount = new HashMap<>(); + loadNewFiles(todayDir); } return true; } diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DatePartitionedLogger.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DatePartitionedLogger.java index 8f89b2e37b..4ac64c6955 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DatePartitionedLogger.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DatePartitionedLogger.java @@ -33,6 +33,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.yarn.util.Clock; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import com.google.protobuf.MessageLite; import com.google.protobuf.Parser; @@ -43,27 +45,40 @@ * @param The proto message type. */ public class DatePartitionedLogger { + private static final Logger LOG = LoggerFactory.getLogger(DatePartitionedLogger.class); // Everyone has permission to write, but with sticky set so that delete is restricted. // This is required, since the path is same for all users and everyone writes into it. private static final FsPermission DIR_PERMISSION = FsPermission.createImmutable((short)01777); + // Since the directories have broad permissions restrict the file read access. + private static final FsPermission FILE_UMASK = FsPermission.createImmutable((short)0066); + private final Parser parser; private final Path basePath; private final Configuration conf; private final Clock clock; - private final FileSystem fileSystem; public DatePartitionedLogger(Parser parser, Path baseDir, Configuration conf, Clock clock) throws IOException { - this.conf = conf; + this.conf = new Configuration(conf); this.clock = clock; this.parser = parser; - this.fileSystem = baseDir.getFileSystem(conf); - if (!fileSystem.exists(baseDir)) { - fileSystem.mkdirs(baseDir); - fileSystem.setPermission(baseDir, DIR_PERMISSION); + createDirIfNotExists(baseDir); + this.basePath = baseDir.getFileSystem(conf).resolvePath(baseDir); + FsPermission.setUMask(this.conf, FILE_UMASK); + } + + private void createDirIfNotExists(Path path) throws IOException { + FileSystem fileSystem = path.getFileSystem(conf); + try { + if (!fileSystem.exists(path)) { + fileSystem.mkdirs(path); + fileSystem.setPermission(path, DIR_PERMISSION); + } + } catch (IOException e) { + // Ignore this exception, if there is a problem it'll fail when trying to read or write. + LOG.warn("Error while trying to set permission: ", e); } - this.basePath = fileSystem.resolvePath(baseDir); } /** @@ -86,13 +101,14 @@ public ProtoMessageReader getReader(Path filePath) throws IOException { */ public Path getPathForDate(LocalDate date, String fileName) throws IOException { Path path = new Path(basePath, getDirForDate(date)); - if (!fileSystem.exists(path)) { - fileSystem.mkdirs(path); - fileSystem.setPermission(path, DIR_PERMISSION); - } + createDirIfNotExists(path); return new Path(path, fileName); } + public Path getPathForSubdir(String dirName, String fileName) { + return new Path(new Path(basePath, dirName), fileName); + } + /** * Extract the date from the directory name, this should be a directory created by this class. */ @@ -116,6 +132,7 @@ public String getDirForDate(LocalDate date) { public String getNextDirectory(String currentDir) throws IOException { // Fast check, if the next day directory exists return it. String nextDate = getDirForDate(getDateFromDir(currentDir).plusDays(1)); + FileSystem fileSystem = basePath.getFileSystem(conf); if (fileSystem.exists(new Path(basePath, nextDate))) { return nextDate; } @@ -135,10 +152,11 @@ public String getNextDirectory(String currentDir) throws IOException { * Returns new or changed files in the given directory. The offsets are used to find * changed files. */ - public List scanForChangedFiles(String subDir, Map currentOffsets) + public List scanForChangedFiles(String subDir, Map currentOffsets) throws IOException { Path dirPath = new Path(basePath, subDir); - List newFiles = new ArrayList<>(); + FileSystem fileSystem = basePath.getFileSystem(conf); + List newFiles = new ArrayList<>(); if (!fileSystem.exists(dirPath)) { return newFiles; } @@ -147,7 +165,7 @@ public List scanForChangedFiles(String subDir, Map currentOf Long offset = currentOffsets.get(fileName); // If the offset was never added or offset < fileSize. if (offset == null || offset < status.getLen()) { - newFiles.add(new Path(dirPath, fileName)); + newFiles.add(status); } } return newFiles; diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java index 60cbda5edd..206b1c1030 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java @@ -19,6 +19,7 @@ package org.apache.tez.dag.history.logging.proto; import java.io.IOException; +import java.time.LocalDate; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -49,13 +50,15 @@ public class ProtoHistoryLoggingService extends HistoryLoggingService { private boolean loggingDisabled = false; private final LinkedBlockingQueue eventQueue = - new LinkedBlockingQueue(10000); + new LinkedBlockingQueue<>(10000); private Thread eventHandlingThread; private final AtomicBoolean stopped = new AtomicBoolean(false); private TezProtoLoggers loggers; private ProtoMessageWriter appEventsWriter; private ProtoMessageWriter dagEventsWriter; + private ProtoMessageWriter manifestEventsWriter; + private LocalDate manifestDate; private TezDAGID currentDagId; private long dagSubmittedEventOffset = -1; @@ -101,6 +104,7 @@ protected void serviceStop() throws Exception { eventHandlingThread.join(); IOUtils.closeQuietly(appEventsWriter); IOUtils.closeQuietly(dagEventsWriter); + IOUtils.closeQuietly(manifestEventsWriter); LOG.info("Stopped ProtoHistoryLoggingService"); } @@ -161,7 +165,8 @@ private void handleEvent(DAGHistoryEvent event) throws IOException { } else if (type == HistoryEventType.DAG_SUBMITTED) { finishCurrentDag(null); currentDagId = dagId; - dagEventsWriter = loggers.getDagEventsLogger().getWriter(dagId.toString()); + dagEventsWriter = loggers.getDagEventsLogger().getWriter(dagId.toString() + + "_" + appContext.getApplicationAttemptId().getAttemptId()); dagSubmittedEventOffset = dagEventsWriter.getOffset(); dagEventsWriter.writeProto(converter.convert(historyEvent)); } else if (dagEventsWriter != null) { @@ -174,16 +179,21 @@ private void finishCurrentDag(DAGFinishedEvent event) throws IOException { if (dagEventsWriter == null) { return; } - ProtoMessageWriter writer = null; try { long finishEventOffset = -1; if (event != null) { finishEventOffset = dagEventsWriter.getOffset(); dagEventsWriter.writeProto(converter.convert(event)); } - // Do not cache this writer, it should be created at the time of writing - writer = loggers.getManifestEventsLogger() - .getWriter(appContext.getApplicationAttemptId().toString()); + DatePartitionedLogger manifestLogger = loggers.getManifestEventsLogger(); + if (manifestDate == null || !manifestDate.equals(manifestLogger.getNow().toLocalDate())) { + // The day has changed write to a new file. + IOUtils.closeQuietly(manifestEventsWriter); + manifestEventsWriter = manifestLogger.getWriter( + appContext.getApplicationAttemptId().toString()); + manifestDate = manifestLogger.getDateFromDir( + manifestEventsWriter.getPath().getParent().getName()); + } ManifestEntryProto.Builder entry = ManifestEntryProto.newBuilder() .setDagId(currentDagId.toString()) .setAppId(currentDagId.getApplicationId().toString()) @@ -196,13 +206,13 @@ private void finishCurrentDag(DAGFinishedEvent event) throws IOException { if (event != null) { entry.setDagId(event.getDagID().toString()); } - writer.writeProto(entry.build()); + manifestEventsWriter.writeProto(entry.build()); + manifestEventsWriter.hflush(); appEventsWriter.hflush(); } finally { // On an error, cleanup everything this will ensure, we do not use one dag's writer // into another dag. IOUtils.closeQuietly(dagEventsWriter); - IOUtils.closeQuietly(writer); dagEventsWriter = null; currentDagId = null; dagSubmittedEventOffset = -1; diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageReader.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageReader.java index e5f5e6befa..d736feaa77 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageReader.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageReader.java @@ -24,19 +24,22 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.SequenceFile.Reader; import com.google.protobuf.MessageLite; import com.google.protobuf.Parser; public class ProtoMessageReader implements Closeable { private final Path filePath; - private final SequenceFile.Reader reader; + private final Reader reader; private final ProtoMessageWritable writable; ProtoMessageReader(Configuration conf, Path filePath, Parser parser) throws IOException { this.filePath = filePath; - this.reader = new SequenceFile.Reader(conf, SequenceFile.Reader.file(filePath)); + // The writer does not flush the length during hflush. Using length options lets us read + // past length in the FileStatus but it will throw EOFException during a read instead + // of returning null. + this.reader = new Reader(conf, Reader.file(filePath), Reader.length(Long.MAX_VALUE)); this.writable = new ProtoMessageWritable<>(parser); } diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWriter.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWriter.java index ca9ba61e02..869b603c3e 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWriter.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWriter.java @@ -26,24 +26,24 @@ import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.SequenceFile.CompressionType; +import org.apache.hadoop.io.SequenceFile.Writer; import com.google.protobuf.MessageLite; import com.google.protobuf.Parser; public class ProtoMessageWriter implements Closeable { private final Path filePath; - private final SequenceFile.Writer writer; + private final Writer writer; private final ProtoMessageWritable writable; ProtoMessageWriter(Configuration conf, Path filePath, Parser parser) throws IOException { this.filePath = filePath; this.writer = SequenceFile.createWriter( conf, - SequenceFile.Writer.file(filePath), - SequenceFile.Writer.keyClass(NullWritable.class), - SequenceFile.Writer.valueClass(ProtoMessageWritable.class), - SequenceFile.Writer.appendIfExists(true), - SequenceFile.Writer.compression(CompressionType.RECORD)); + Writer.file(filePath), + Writer.keyClass(NullWritable.class), + Writer.valueClass(ProtoMessageWritable.class), + Writer.compression(CompressionType.RECORD)); this.writable = new ProtoMessageWritable<>(parser); } diff --git a/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestDagManifestFileScanner.java b/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestDagManifestFileScanner.java index fcaa315f34..4950522429 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestDagManifestFileScanner.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestDagManifestFileScanner.java @@ -20,6 +20,9 @@ import java.io.IOException; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.util.Clock; import org.apache.tez.dag.api.TezConfiguration; @@ -43,12 +46,14 @@ public void setupTest() throws Exception { clock = new MockClock(); Configuration conf = new Configuration(false); conf.set(TezConfiguration.TEZ_HISTORY_LOGGING_PROTO_BASE_DIR, basePath); + // LocalFileSystem does not implement truncate. + conf.set("fs.file.impl", "org.apache.hadoop.fs.RawLocalFileSystem"); TezProtoLoggers loggers = new TezProtoLoggers(); loggers.setup(conf, clock); manifestLogger = loggers.getManifestEventsLogger(); } - @Test + @Test(timeout=5000) public void testNormal() throws Exception { clock.setTime(0); // 0th day. createManifestEvents(0, 8); @@ -85,6 +90,37 @@ public void testNormal() throws Exception { // Not able to test append since the LocalFileSystem does not implement append. } + private Path deleteFilePath = null; + @Test(timeout=5000) + public void testError() throws Exception { + clock.setTime(0); // 0th day. + createManifestEvents(0, 4); + corruptFiles(); + clock.setTime((24 * 60 * 60 + 1) * 1000); // 1 day 1 sec. + createManifestEvents(24 * 3600, 1); + + DagManifesFileScanner scanner = new DagManifesFileScanner(manifestLogger); + Assert.assertNotNull(scanner.getNext()); + deleteFilePath.getFileSystem(manifestLogger.getConfig()).delete(deleteFilePath, false); + // 4 files - 1 file deleted - 1 truncated - 1 corrupted => 1 remains. + Assert.assertNull(scanner.getNext()); + + // Save offset for later use. + String offset = scanner.getOffset(); + + // Move time outside the window, it should skip files with error and give more data for + // next day. + clock.setTime((24 * 60 * 60 + 61) * 1000); // 1 day 61 sec. + Assert.assertNotNull(scanner.getNext()); + Assert.assertNull(scanner.getNext()); + + // Reset the offset + scanner.setOffset(offset); + Assert.assertNotNull(scanner.getNext()); + Assert.assertNull(scanner.getNext()); + scanner.close(); + } + private void createManifestEvents(long time, int numEvents) throws IOException { for (int i = 0; i < numEvents; ++i) { ApplicationId appId = ApplicationId.newInstance(1000l, i); @@ -103,6 +139,33 @@ private void createManifestEvents(long time, int numEvents) throws IOException { } } + private void corruptFiles() throws IOException { + int op = 0; + Configuration conf = manifestLogger.getConfig(); + Path base = new Path( + conf.get(TezConfiguration.TEZ_HISTORY_LOGGING_PROTO_BASE_DIR) + "/dag_meta"); + FileSystem fs = base.getFileSystem(conf); + for (FileStatus status : fs.listStatus(base)) { + if (status.isDirectory()) { + for (FileStatus file : fs.listStatus(status.getPath())) { + if (!file.getPath().getName().startsWith("application_")) { + continue; + } + switch (op) { + case 0: + case 1: + fs.truncate(file.getPath(), op == 1 ? 0 : file.getLen() - 20); + break; + case 3: + deleteFilePath = file.getPath(); + break; + } + op++; + } + } + } + } + private static class MockClock implements Clock { private long time = 0; diff --git a/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestProtoHistoryLoggingService.java b/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestProtoHistoryLoggingService.java index 4bd5d4e401..bc79b0746d 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestProtoHistoryLoggingService.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestProtoHistoryLoggingService.java @@ -21,6 +21,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import java.io.EOFException; import java.io.IOException; import java.time.LocalDate; import java.util.ArrayList; @@ -86,14 +87,18 @@ public void testService() throws Exception { // Verify dag events are logged. DatePartitionedLogger dagLogger = loggers.getDagEventsLogger(); - Path dagFilePath = dagLogger.getPathForDate(LocalDate.ofEpochDay(0), dagId.toString()); + Path dagFilePath = dagLogger.getPathForDate(LocalDate.ofEpochDay(0), dagId.toString() + "_" + 1); ProtoMessageReader reader = dagLogger.getReader(dagFilePath); HistoryEventProto evt = reader.readEvent(); int ind = 1; while (evt != null) { Assert.assertEquals(protos.get(ind), evt); ind++; - evt = reader.readEvent(); + try { + evt = reader.readEvent(); + } catch (EOFException e) { + evt = null; + } } reader.close(); From 3baab55810ce5477c4048125e2b192bde9ec134d Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Fri, 6 Jul 2018 09:16:44 -0500 Subject: [PATCH 100/512] TEZ-3954. Reduce Tez Shuffle Handler Memory needs for holding TezIndexRecords (Jonathan Eagles via kshukla) --- .../tez/auxservices/ShuffleHandler.java | 50 +++++++++++++++---- .../tez/auxservices/TestShuffleHandler.java | 8 +-- 2 files changed, 45 insertions(+), 13 deletions(-) diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java index e22928edcb..24a821fe96 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java @@ -1146,7 +1146,7 @@ public ChannelFuture sendMap(ReduceContext reduceContext) try { MapOutputInfo info = reduceContext.getInfoMap().get(mapId); if (info == null) { - info = getMapOutputInfo(reduceContext.dagId, mapId, + info = getMapOutputInfo(reduceContext.dagId, mapId, reduceContext.getReduceRange(), reduceContext.getJobId(), reduceContext.getUser()); } @@ -1204,7 +1204,7 @@ private String getDagLocation(String jobId, String dagId, String user) { } protected MapOutputInfo getMapOutputInfo(String dagId, String mapId, - String jobId, + Range reduceRange, String jobId, String user) throws IOException { AttemptPathInfo pathInfo; try { @@ -1233,8 +1233,13 @@ protected MapOutputInfo getMapOutputInfo(String dagId, String mapId, pathInfo.indexPath); } + MapOutputInfo outputInfo; + if (reduceRange.first == reduceRange.last) { + outputInfo = new MapOutputInfo(pathInfo.dataPath, spillRecord.getIndex(reduceRange.first), reduceRange); + } else { - MapOutputInfo outputInfo = new MapOutputInfo(pathInfo.dataPath, spillRecord); + outputInfo = new MapOutputInfo(pathInfo.dataPath, spillRecord, reduceRange); + } return outputInfo; } @@ -1262,12 +1267,12 @@ long getContentLength(List mapIds, String jobId, String dagId, String us int reduceCountVSize = WritableUtils.getVIntSize(reduceRange.getLast() - reduceRange.getFirst() + 1); for (String mapId : mapIds) { contentLength += reduceCountVSize; - MapOutputInfo outputInfo = getMapOutputInfo(dagId, mapId, jobId, user); + MapOutputInfo outputInfo = getMapOutputInfo(dagId, mapId, reduceRange, jobId, user); if (mapOutputInfoMap.size() < mapOutputMetaInfoCacheSize) { mapOutputInfoMap.put(mapId, outputInfo); } for (int reduce = reduceRange.getFirst(); reduce <= reduceRange.getLast(); reduce++) { - TezIndexRecord indexRecord = outputInfo.spillRecord.getIndex(reduce); + TezIndexRecord indexRecord = outputInfo.getIndex(reduce); ShuffleHeader header = new ShuffleHeader(mapId, indexRecord.getPartLength(), indexRecord.getRawLength(), reduce); @@ -1295,12 +1300,37 @@ protected void setResponseHeaders(HttpResponse response, boolean keepAliveParam, } class MapOutputInfo { - final Path mapOutputFileName; - final TezSpillRecord spillRecord; + private final Path mapOutputFileName; + private TezSpillRecord spillRecord; + private TezIndexRecord indexRecord; + private final Range reduceRange; + + MapOutputInfo(Path mapOutputFileName, TezIndexRecord indexRecord, Range reduceRange) { + this.mapOutputFileName = mapOutputFileName; + this.indexRecord = indexRecord; + this.reduceRange = reduceRange; + } - MapOutputInfo(Path mapOutputFileName, TezSpillRecord spillRecord) { + MapOutputInfo(Path mapOutputFileName, TezSpillRecord spillRecord, Range reduceRange) { this.mapOutputFileName = mapOutputFileName; this.spillRecord = spillRecord; + this.reduceRange = reduceRange; + } + + TezIndexRecord getIndex(int index) { + if (index < reduceRange.first || index > reduceRange.last) { + throw new IllegalArgumentException("Reduce Index: " + index + " out of range for " + mapOutputFileName); + } + if (spillRecord != null) { + return spillRecord.getIndex(index); + } else { + return indexRecord; + } + } + + public void finish() { + spillRecord = null; + indexRecord = null; } } @@ -1356,7 +1386,7 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch, WritableUtils.writeVInt(dobRange, reduceRange.getLast() - reduceRange.getFirst() + 1); ch.write(wrappedBuffer(dobRange.getData(), 0, dobRange.getLength())); for (int reduce = reduceRange.getFirst(); reduce <= reduceRange.getLast(); reduce++) { - TezIndexRecord index = outputInfo.spillRecord.getIndex(reduce); + TezIndexRecord index = outputInfo.getIndex(reduce); // Records are only valid if they have a non-zero part length if (index.getPartLength() != 0) { if (firstIndex == null) { @@ -1368,6 +1398,8 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch, ShuffleHeader header = new ShuffleHeader(mapId, index.getPartLength(), index.getRawLength(), reduce); DataOutputBuffer dob = new DataOutputBuffer(); header.write(dob); + // Free the memory needed to store the spill and index records + outputInfo.finish(); ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength())); } diff --git a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java index 11c92fb50f..7d53abceb1 100644 --- a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java @@ -111,7 +111,7 @@ protected void verifyRequest(String appid, ChannelHandlerContext ctx, } @Override protected MapOutputInfo getMapOutputInfo(String dagId, String mapId, - String jobId, + Range reduceRange, String jobId, String user) throws IOException { // Do nothing. @@ -236,7 +236,7 @@ protected Shuffle getShuffle(Configuration conf) { return new Shuffle(conf) { @Override protected MapOutputInfo getMapOutputInfo(String dagId, String mapId, - String jobId, + Range reduceRange, String jobId, String user) throws IOException { return null; @@ -346,7 +346,7 @@ protected Shuffle getShuffle(final Configuration conf) { return new Shuffle(conf) { @Override protected MapOutputInfo getMapOutputInfo(String dagId, String mapId, - String jobId, String user) + Range reduceRange, String jobId, String user) throws IOException { return null; } @@ -568,7 +568,7 @@ protected Shuffle getShuffle(Configuration conf) { return new Shuffle(conf) { @Override protected MapOutputInfo getMapOutputInfo(String dagId, String mapId, - String jobId, + Range reduceRange, String jobId, String user) throws IOException { // Do nothing. From 5b75ef118b5e7702485c29b91890c0639ac0e0b5 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Fri, 6 Jul 2018 16:53:57 -0500 Subject: [PATCH 101/512] TEZ-3963. Possible InflaterInputStream leaked in TezCommonUtils and related classes (Jaume M via jlowe) --- .../src/main/java/org/apache/tez/common/TezCommonUtils.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java index 9cb76d9007..c8e8e0292c 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java @@ -394,8 +394,9 @@ public static byte[] decompressByteStringToByteArray(ByteString byteString) thro @Private public static byte[] decompressByteStringToByteArray(ByteString byteString, Inflater inflater) throws IOException { inflater.reset(); - return IOUtils.toByteArray(new InflaterInputStream(byteString.newInput(), inflater)); - + try (InflaterInputStream inflaterInputStream = new InflaterInputStream(byteString.newInput(), inflater)) { + return IOUtils.toByteArray(inflaterInputStream); + } } public static String getCredentialsInfo(Credentials credentials, String identifier) { From 12ff93779f5c8ef5708bf3f14fb49a628bc41218 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Mon, 9 Jul 2018 10:44:06 -0500 Subject: [PATCH 102/512] TEZ-3912. Fetchers should be more robust to corrupted inputs (Kuhu Shukla via jeagles) --- .../library/common/shuffle/ShuffleUtils.java | 9 ++-- .../common/shuffle/TestShuffleUtils.java | 53 +++++++++++++++++++ 2 files changed, 59 insertions(+), 3 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java index bf58172ef1..df4281a94e 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java @@ -125,7 +125,7 @@ public static void shuffleToMemory(byte[] shuffleData, LOG.debug("Read " + shuffleData.length + " bytes from input for " + identifier); } - } catch (InternalError | IOException e) { + } catch (InternalError | Exception e) { // Close the streams LOG.info("Failed to read data to memory for " + identifier + ". len=" + compressedLength + ", decomp=" + decompressedLength + ". ExceptionMessage=" + e.getMessage()); @@ -135,9 +135,12 @@ public static void shuffleToMemory(byte[] shuffleData, // on decompression failures. Catching and re-throwing as IOException // to allow fetch failure logic to be processed. throw new IOException(e); + } else if (e instanceof IOException) { + throw e; + } else { + // Re-throw as an IOException + throw new IOException(e); } - // Re-throw - throw e; } } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java index 1d2d4280b5..f61c7e5f66 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java @@ -41,6 +41,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.net.SocketTimeoutException; import java.nio.ByteBuffer; import java.util.Arrays; import java.util.BitSet; @@ -300,6 +301,58 @@ public void testInternalErrorTranslation() throws Exception { } } + @Test + public void testExceptionTranslation() throws Exception { + String codecErrorMsg = "codec failure"; + CompressionInputStream mockCodecStream = mock(CompressionInputStream.class); + when(mockCodecStream.read(any(byte[].class), anyInt(), anyInt())) + .thenThrow(new IllegalArgumentException(codecErrorMsg)); + Decompressor mockDecoder = mock(Decompressor.class); + CompressionCodec mockCodec = mock(CompressionCodec.class); + when(mockCodec.createDecompressor()).thenReturn(mockDecoder); + when(mockCodec.createInputStream(any(InputStream.class), any(Decompressor.class))) + .thenReturn(mockCodecStream); + byte[] header = new byte[] { (byte) 'T', (byte) 'I', (byte) 'F', (byte) 1}; + try { + ShuffleUtils.shuffleToMemory(new byte[1024], new ByteArrayInputStream(header), + 1024, 128, mockCodec, false, 0, mock(Logger.class), null); + Assert.fail("shuffle was supposed to throw!"); + } catch (IOException e) { + Assert.assertTrue(e.getCause() instanceof IllegalArgumentException); + Assert.assertTrue(e.getMessage().contains(codecErrorMsg)); + } + CompressionInputStream mockCodecStream1 = mock(CompressionInputStream.class); + when(mockCodecStream1.read(any(byte[].class), anyInt(), anyInt())) + .thenThrow(new SocketTimeoutException(codecErrorMsg)); + CompressionCodec mockCodec1 = mock(CompressionCodec.class); + when(mockCodec1.createDecompressor()).thenReturn(mockDecoder); + when(mockCodec1.createInputStream(any(InputStream.class), any(Decompressor.class))) + .thenReturn(mockCodecStream1); + try { + ShuffleUtils.shuffleToMemory(new byte[1024], new ByteArrayInputStream(header), + 1024, 128, mockCodec1, false, 0, mock(Logger.class), null); + Assert.fail("shuffle was supposed to throw!"); + } catch (IOException e) { + Assert.assertTrue(e instanceof SocketTimeoutException); + Assert.assertTrue(e.getMessage().contains(codecErrorMsg)); + } + CompressionInputStream mockCodecStream2 = mock(CompressionInputStream.class); + when(mockCodecStream2.read(any(byte[].class), anyInt(), anyInt())) + .thenThrow(new InternalError(codecErrorMsg)); + CompressionCodec mockCodec2 = mock(CompressionCodec.class); + when(mockCodec2.createDecompressor()).thenReturn(mockDecoder); + when(mockCodec2.createInputStream(any(InputStream.class), any(Decompressor.class))) + .thenReturn(mockCodecStream2); + try { + ShuffleUtils.shuffleToMemory(new byte[1024], new ByteArrayInputStream(header), + 1024, 128, mockCodec2, false, 0, mock(Logger.class), null); + Assert.fail("shuffle was supposed to throw!"); + } catch (IOException e) { + Assert.assertTrue(e.getCause() instanceof InternalError); + Assert.assertTrue(e.getMessage().contains(codecErrorMsg)); + } + } + @Test public void testShuffleToDiskChecksum() throws Exception { // verify sending a stream of zeroes without checksum validation From 919d328afd1e580a5d72eb64a7ffd5995a93cf4e Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Wed, 11 Jul 2018 11:04:50 -0500 Subject: [PATCH 103/512] TEZ-3955. Upgrade hadoop dependency to 3.0.3 (Jonathan Eagles via jlowe) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 5b0ff14743..20feb4c417 100644 --- a/pom.xml +++ b/pom.xml @@ -37,7 +37,7 @@ true ${user.home}/clover.license - 3.0.2 + 3.0.3 9.3.22.v20171030 3.10.5.Final 0.13.0 From 4496f66673de97bbe92e9b143696635038c2b799 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Thu, 12 Jul 2018 10:48:07 -0500 Subject: [PATCH 104/512] TEZ-3964. Inflater not closed in some places (Jaume M via jlowe) --- .../org/apache/tez/common/TezCommonUtils.java | 7 ++++++- .../tez/dag/history/utils/DAGUtils.java | 20 +++++++++++++++++-- 2 files changed, 24 insertions(+), 3 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java index c8e8e0292c..16165e2377 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java @@ -388,7 +388,12 @@ public static ByteString compressByteArrayToByteString(byte[] inBytes, Deflater @Private public static byte[] decompressByteStringToByteArray(ByteString byteString) throws IOException { - return decompressByteStringToByteArray(byteString, newInflater()); + Inflater inflater = newInflater(); + try { + return decompressByteStringToByteArray(byteString, inflater); + } finally { + inflater.end(); + } } @Private diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/utils/DAGUtils.java b/tez-dag/src/main/java/org/apache/tez/dag/history/utils/DAGUtils.java index b2622adc74..77d3447875 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/utils/DAGUtils.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/utils/DAGUtils.java @@ -139,7 +139,7 @@ public static JSONObject convertCountersToJSON(TezCounters counters) } public static Map convertCountersToATSMap(TezCounters counters) { - Map object = new LinkedHashMap(); + Map object = new LinkedHashMap(); if (counters == null) { return object; } @@ -198,8 +198,24 @@ static Map createDagInfoMap(DAGPlan dagPlan) { return dagInfo; } - public static Map convertDAGPlanToATSMap(DAGPlan dagPlan) throws IOException { + public static Map convertDAGPlanToATSMap(final DAGPlan + dagPlan) throws IOException { final Inflater inflater = TezCommonUtils.newInflater(); + try { + return convertDAGPlanToATSMap(dagPlan, inflater); + } finally { + inflater.end(); + } + } + + /** + * Auxiliary method to convert dagPlan to ATS Map. + * @param dagPlan dag plan. + * @param inflater inflater. This method shouldn't end it. + * @return ATS MAP + */ + private static Map convertDAGPlanToATSMap(DAGPlan dagPlan, + final Inflater inflater) { final String VERSION_KEY = "version"; final int version = 2; Map dagMap = new LinkedHashMap(); From 633fde2cd41ae0a2f51b5c34190318bae54a2ee6 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Thu, 12 Jul 2018 17:44:19 -0500 Subject: [PATCH 105/512] TEZ-3970. NullPointerException in Tez ShuffleHandler Ranged Fetch (Jonathan Eagles via kshukla) --- .../tez/auxservices/ShuffleHandler.java | 2 +- .../tez/auxservices/TestShuffleHandler.java | 114 ++++++++++++++++-- 2 files changed, 104 insertions(+), 12 deletions(-) diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java index 24a821fe96..f294edced9 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java @@ -1399,9 +1399,9 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch, DataOutputBuffer dob = new DataOutputBuffer(); header.write(dob); // Free the memory needed to store the spill and index records - outputInfo.finish(); ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength())); } + outputInfo.finish(); final long rangeOffset = firstIndex.getStartOffset(); final long rangePartLength = lastIndex.getStartOffset() + lastIndex.getPartLength() - firstIndex.getStartOffset(); diff --git a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java index 7d53abceb1..7c421a9c17 100644 --- a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java @@ -55,6 +55,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.io.nativeio.NativeIO; import org.apache.hadoop.mapred.JobID; import org.apache.hadoop.mapred.MapTask; @@ -75,6 +76,8 @@ import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext; import org.apache.hadoop.yarn.server.api.ApplicationTerminationContext; import org.apache.hadoop.yarn.server.records.Version; +import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord; +import org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord; import org.jboss.netty.channel.Channel; import org.jboss.netty.channel.ChannelFuture; import org.jboss.netty.channel.ChannelHandlerContext; @@ -657,6 +660,102 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, shuffleHandler.stop(); } + /** + * Validate the ranged fetch works as expected + */ + @Test(timeout = 10000) + public void testRangedFetch() throws IOException { + Configuration conf = new Configuration(); + conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); + conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, 3); + conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, + "simple"); + UserGroupInformation.setConfiguration(conf); + File absLogDir = new File("target", + TestShuffleHandler.class.getSimpleName() + "LocDir").getAbsoluteFile(); + conf.set(YarnConfiguration.NM_LOCAL_DIRS, absLogDir.getAbsolutePath()); + ApplicationId appId = ApplicationId.newInstance(12345, 1); + LOG.info(appId.toString()); + String appAttemptId = "attempt_12345_1_m_1_0"; + String user = "randomUser"; + String reducerIdStart = "0"; + String reducerIdEnd = "1"; + List fileMap = new ArrayList<>(); + createShuffleHandlerFiles(absLogDir, user, appId.toString(), appAttemptId, + conf, fileMap); + ShuffleHandler shuffleHandler = new ShuffleHandler() { + + @Override + protected Shuffle getShuffle(Configuration conf) { + // replace the shuffle handler with one stubbed for testing + return new Shuffle(conf) { + + @Override + protected void verifyRequest(String appid, ChannelHandlerContext ctx, + HttpRequest request, HttpResponse response, URL requestUri) + throws IOException { + // Do nothing. + } + + }; + } + }; + shuffleHandler.init(conf); + try { + shuffleHandler.start(); + DataOutputBuffer outputBuffer = new DataOutputBuffer(); + outputBuffer.reset(); + Token jt = + new Token("identifier".getBytes(), + "password".getBytes(), new Text(user), new Text("shuffleService")); + jt.write(outputBuffer); + shuffleHandler + .initializeApplication(new ApplicationInitializationContext(user, + appId, ByteBuffer.wrap(outputBuffer.getData(), 0, + outputBuffer.getLength()))); + URL url = + new URL( + "http://127.0.0.1:" + + shuffleHandler.getConfig().get( + ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY) + + "/mapOutput?job=job_12345_0001&dag=1&reduce=" + reducerIdStart + "-" + reducerIdEnd + + "&map=attempt_12345_1_m_1_0"); + HttpURLConnection conn = (HttpURLConnection) url.openConnection(); + conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME, + ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); + conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION, + ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); + conn.connect(); + boolean succeeded = false; + try { + DataInputStream is = new DataInputStream(conn.getInputStream()); + int partitionCount = WritableUtils.readVInt(is); + List headers = new ArrayList<>(2); + for (int i = 0; i < partitionCount; i++) { + ShuffleHeader header = new ShuffleHeader(); + header.readFields(is); + Assert.assertEquals("Incorrect map id", "attempt_12345_1_m_1_0", header.getMapId()); + Assert.assertEquals("Incorrect reduce id", i, header.getPartition()); + headers.add(header); + } + for (ShuffleHeader header: headers) { + byte[] bytes = new byte[(int)header.getCompressedLength()]; + is.read(bytes); + } + succeeded = true; + // Read one more byte to force EOF + is.readByte(); + Assert.fail("More fetch bytes that expected in stream"); + } catch (EOFException e) { + Assert.assertTrue("Failed to copy ranged fetch", succeeded); + } + + } finally { + shuffleHandler.stop(); + FileUtil.fullyDelete(absLogDir); + } + } + /** * Validate the ownership of the map-output files being pulled in. The * local-file-system owner of the file should match the user component in the @@ -785,18 +884,11 @@ private static void createIndexFile(File indexFile, Configuration conf) System.out.println("Deleting existing file"); indexFile.delete(); } - indexFile.createNewFile(); - FSDataOutputStream output = FileSystem.getLocal(conf).getRaw().append( - new Path(indexFile.getAbsolutePath())); Checksum crc = new PureJavaCrc32(); - crc.reset(); - CheckedOutputStream chk = new CheckedOutputStream(output, crc); - String msg = "Writing new index file. This file will be used only " + - "for the testing."; - chk.write(Arrays.copyOf(msg.getBytes(), - MapTask.MAP_OUTPUT_INDEX_RECORD_LENGTH)); - output.writeLong(chk.getChecksum().getValue()); - output.close(); + TezSpillRecord tezSpillRecord = new TezSpillRecord(2); + tezSpillRecord.putIndex(new TezIndexRecord(0, 10, 10), 0); + tezSpillRecord.putIndex(new TezIndexRecord(10, 10, 10), 1); + tezSpillRecord.writeToFile(new Path(indexFile.getAbsolutePath()), conf, crc); } @Test From 957e34f81fd5cdf3afd035a4a5e2259743eba562 Mon Sep 17 00:00:00 2001 From: Eric Wohlstadter Date: Thu, 12 Jul 2018 15:03:16 -0700 Subject: [PATCH 106/512] TEZ-3916: Add hadoop-azure-datalake jar to azure profile (Eric Wohlstadter via Gopal V) Signed-off-by: Gopal V --- pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pom.xml b/pom.xml index 20feb4c417..9653c23eaa 100644 --- a/pom.xml +++ b/pom.xml @@ -1296,6 +1296,12 @@ runtime ${hadoop.version} + + org.apache.hadoop + hadoop-azure-datalake + runtime + ${hadoop.version} + From 7e397b4f8ebc5f8faec3e6fc4fd634c9d02cb3e4 Mon Sep 17 00:00:00 2001 From: Jaume Marhuenda Date: Thu, 12 Jul 2018 22:39:09 -0700 Subject: [PATCH 107/512] TEZ-3965: TestMROutput: Fix the hard-coded /tmp/output paths (Jaume Marhuenda, reviewed by Gopal V) Signed-off-by: Gopal V --- .../tez/mapreduce/output/TestMROutput.java | 36 ++++++++++++------- 1 file changed, 24 insertions(+), 12 deletions(-) diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/output/TestMROutput.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/output/TestMROutput.java index f3403e655c..c60ca228b3 100644 --- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/output/TestMROutput.java +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/output/TestMROutput.java @@ -28,6 +28,7 @@ import java.util.HashMap; import java.util.List; +import com.google.common.io.Files; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -66,6 +67,7 @@ import org.apache.tez.runtime.api.impl.TezUmbilical; import org.apache.tez.runtime.library.api.KeyValueWriter; import org.apache.tez.runtime.library.processor.SimpleProcessor; +import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Test; @@ -75,13 +77,21 @@ public class TestMROutput { + static File tmpDir; + + @BeforeClass + public static void setupClass () { + tmpDir = Files.createTempDir(); + tmpDir.deleteOnExit(); + } + @Test(timeout = 5000) public void testNewAPI_TextOutputFormat() throws Exception { - String outputPath = "/tmp/output"; Configuration conf = new Configuration(); conf.setBoolean(MRConfig.IS_MAP_PROCESSOR, true); DataSinkDescriptor dataSink = MROutput - .createConfigBuilder(conf, TextOutputFormat.class, outputPath) + .createConfigBuilder(conf, TextOutputFormat.class, + tmpDir.getPath()) .build(); OutputContext outputContext = createMockOutputContext(dataSink.getOutputDescriptor().getUserPayload()); @@ -101,11 +111,12 @@ public void testNewAPI_TextOutputFormat() throws Exception { @Test(timeout = 5000) public void testOldAPI_TextOutputFormat() throws Exception { - String outputPath = "/tmp/output"; Configuration conf = new Configuration(); conf.setBoolean(MRConfig.IS_MAP_PROCESSOR, false); DataSinkDescriptor dataSink = MROutput - .createConfigBuilder(conf, org.apache.hadoop.mapred.TextOutputFormat.class, outputPath) + .createConfigBuilder(conf, + org.apache.hadoop.mapred.TextOutputFormat.class, + tmpDir.getPath()) .build(); OutputContext outputContext = createMockOutputContext(dataSink.getOutputDescriptor().getUserPayload()); @@ -125,12 +136,12 @@ public void testOldAPI_TextOutputFormat() throws Exception { @Test(timeout = 5000) public void testNewAPI_SequenceFileOutputFormat() throws Exception { - String outputPath = "/tmp/output"; JobConf conf = new JobConf(); conf.setOutputKeyClass(NullWritable.class); conf.setOutputValueClass(Text.class); DataSinkDescriptor dataSink = MROutput - .createConfigBuilder(conf, SequenceFileOutputFormat.class, outputPath) + .createConfigBuilder(conf, SequenceFileOutputFormat.class, + tmpDir.getPath()) .build(); OutputContext outputContext = createMockOutputContext(dataSink.getOutputDescriptor().getUserPayload()); @@ -149,12 +160,13 @@ public void testNewAPI_SequenceFileOutputFormat() throws Exception { @Test(timeout = 5000) public void testOldAPI_SequenceFileOutputFormat() throws Exception { - String outputPath = "/tmp/output"; JobConf conf = new JobConf(); conf.setOutputKeyClass(NullWritable.class); conf.setOutputValueClass(Text.class); DataSinkDescriptor dataSink = MROutput - .createConfigBuilder(conf, org.apache.hadoop.mapred.SequenceFileOutputFormat.class, outputPath) + .createConfigBuilder(conf, + org.apache.hadoop.mapred.SequenceFileOutputFormat.class, + tmpDir.getPath()) .build(); OutputContext outputContext = createMockOutputContext(dataSink.getOutputDescriptor().getUserPayload()); @@ -175,11 +187,11 @@ public void testOldAPI_SequenceFileOutputFormat() throws Exception { // set while creating recordWriters @Test(timeout = 5000) public void testNewAPI_WorkOutputPathOutputFormat() throws Exception { - String outputPath = "/tmp/output"; Configuration conf = new Configuration(); conf.setBoolean(MRConfig.IS_MAP_PROCESSOR, true); DataSinkDescriptor dataSink = MROutput - .createConfigBuilder(conf, NewAPI_WorkOutputPathReadingOutputFormat.class, outputPath) + .createConfigBuilder(conf, NewAPI_WorkOutputPathReadingOutputFormat.class, + tmpDir.getPath()) .build(); OutputContext outputContext = createMockOutputContext(dataSink.getOutputDescriptor().getUserPayload()); @@ -201,11 +213,11 @@ public void testNewAPI_WorkOutputPathOutputFormat() throws Exception { // set while creating recordWriters @Test(timeout = 5000) public void testOldAPI_WorkOutputPathOutputFormat() throws Exception { - String outputPath = "/tmp/output"; Configuration conf = new Configuration(); conf.setBoolean(MRConfig.IS_MAP_PROCESSOR, false); DataSinkDescriptor dataSink = MROutput - .createConfigBuilder(conf, OldAPI_WorkOutputPathReadingOutputFormat.class, outputPath) + .createConfigBuilder(conf, OldAPI_WorkOutputPathReadingOutputFormat.class, + tmpDir.getPath()) .build(); OutputContext outputContext = createMockOutputContext(dataSink.getOutputDescriptor().getUserPayload()); From dd6a09dc4522ffe3b4df8d0ef4904573211263c0 Mon Sep 17 00:00:00 2001 From: Nishant Dash Date: Tue, 24 Jul 2018 10:14:53 -0500 Subject: [PATCH 108/512] TEZ-3942. RPC getTask writable optimization invalid in hadoop 2.8+ Signed-off-by: Jason Lowe --- .../java/org/apache/tez/common/TezUtils.java | 3 +- .../apache/tez/dag/api/DagTypeConverters.java | 2 +- .../apache/tez/dag/api/EntityDescriptor.java | 121 ++++-------------- .../org/apache/tez/dag/api/UserPayload.java | 11 ++ .../tez/dag/api/TestEntityDescriptor.java | 75 ++++++++--- 5 files changed, 98 insertions(+), 114 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java index aed9e0f332..072c02f97b 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.io.OutputStream; +import java.nio.ByteBuffer; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -97,7 +98,7 @@ public static ByteString createByteStringFromConf(Configuration conf) throws IOE * @throws java.io.IOException */ public static UserPayload createUserPayloadFromConf(Configuration conf) throws IOException { - return UserPayload.create(createByteStringFromConf(conf).asReadOnlyByteBuffer()); + return UserPayload.create(ByteBuffer.wrap(createByteStringFromConf(conf).toByteArray())); } /** diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java index c5d9c0b4f7..acc5f12933 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java @@ -735,7 +735,7 @@ public static ByteBuffer convertFromTezUserPayload(@Nullable UserPayload payload if (payload == null) { return null; } - return payload.getPayload(); + return payload.getRawPayload(); } public static VertexExecutionContextProto convertToProto( diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/EntityDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/EntityDescriptor.java index dcc4ebf779..13d4a93f0a 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/EntityDescriptor.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/EntityDescriptor.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.nio.ByteBuffer; +import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.io.DataOutputBuffer; @@ -94,36 +95,40 @@ public String getClassName() { return this.className; } + void writeSingular(DataOutput out, ByteBuffer bb) throws IOException { + out.write(bb.array(), 0, bb.array().length); + } + + void writeSegmented(DataOutput out, ByteBuffer bb) throws IOException { + // This code is just for fallback in case serialization is changed to + // use something other than DataOutputBuffer. + int len; + byte[] buf = new byte[SERIALIZE_BUFFER_SIZE]; + do { + len = Math.min(bb.remaining(), SERIALIZE_BUFFER_SIZE); + bb.get(buf, 0, len); + out.write(buf, 0, len); + } while (bb.remaining() > 0); + } + @Override public void write(DataOutput out) throws IOException { Text.writeString(out, className); // TODO: TEZ-305 - using protobuf serde instead of Writable serde. ByteBuffer bb = DagTypeConverters.convertFromTezUserPayload(userPayload); - if (bb == null) { + if (bb == null || bb.remaining() == 0) { out.writeInt(-1); + return; + } + + // write size + out.writeInt(bb.remaining()); + if (bb.hasArray()) { + writeSingular(out, bb); } else { - int size = bb.remaining(); - if (size == 0) { - out.writeInt(-1); - } else { - out.writeInt(size); - if (out instanceof DataOutputBuffer) { - DataOutputBuffer buf = (DataOutputBuffer) out; - buf.write(new ByteBufferDataInput(bb), size); - } else { - // This code is just for fallback in case serialization is changed to - // use something other than DataOutputBuffer. - int len; - byte[] buf = new byte[SERIALIZE_BUFFER_SIZE]; - do { - len = Math.min(bb.remaining(), SERIALIZE_BUFFER_SIZE); - bb.get(buf, 0, len); - out.write(buf, 0, len); - } while (bb.remaining() > 0); - } - } - out.writeInt(userPayload.getVersion()); + writeSegmented(out, bb); } + out.writeInt(userPayload.getVersion()); } @Override @@ -144,76 +149,4 @@ public String toString() { userPayload == null ? false : userPayload.getPayload() == null ? false : true; return "ClassName=" + className + ", hasPayload=" + hasPayload; } - - private static class ByteBufferDataInput implements DataInput { - - private final ByteBuffer bb; - - public ByteBufferDataInput(ByteBuffer bb) { - this.bb = bb; - } - - @Override - public void readFully(byte[] b) throws IOException { - bb.get(b, 0, bb.remaining()); - } - - @Override - public void readFully(byte[] b, int off, int len) throws IOException { - bb.get(b, off, len); - } - - @Override - public int skipBytes(int n) throws IOException { - throw new UnsupportedOperationException(); - } - @Override - public boolean readBoolean() throws IOException { - throw new UnsupportedOperationException(); - } - @Override - public byte readByte() throws IOException { - return bb.get(); - } - @Override - public int readUnsignedByte() throws IOException { - throw new UnsupportedOperationException(); - } - @Override - public short readShort() throws IOException { - throw new UnsupportedOperationException(); - } - @Override - public int readUnsignedShort() throws IOException { - throw new UnsupportedOperationException(); - } - @Override - public char readChar() throws IOException { - throw new UnsupportedOperationException(); - } - @Override - public int readInt() throws IOException { - throw new UnsupportedOperationException(); - } - @Override - public long readLong() throws IOException { - throw new UnsupportedOperationException(); - } - @Override - public float readFloat() throws IOException { - throw new UnsupportedOperationException(); - } - @Override - public double readDouble() throws IOException { - throw new UnsupportedOperationException(); - } - @Override - public String readLine() throws IOException { - throw new UnsupportedOperationException(); - } - @Override - public String readUTF() throws IOException { - throw new UnsupportedOperationException(); - } - } } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/UserPayload.java b/tez-api/src/main/java/org/apache/tez/dag/api/UserPayload.java index fa617b5317..087b17ab9d 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/UserPayload.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/UserPayload.java @@ -62,6 +62,17 @@ public static UserPayload create(@Nullable ByteBuffer payload, int version) { return new UserPayload(payload, version); } + /** + * Return the payload as a ByteBuffer. + * @return ByteBuffer. + */ + @Nullable + public ByteBuffer getRawPayload() { + // Note: Several bits of serialization, including deepCopyAsArray depend on a new instance of the + // ByteBuffer being returned, since they modify it. If changing this code to return the same + // ByteBuffer - deepCopyAsArray and TezEntityDescriptor need to be looked at. + return payload == EMPTY_BYTE ? null : payload.duplicate(); + } /** * Return the payload as a read-only ByteBuffer. * @return read-only ByteBuffer. diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/TestEntityDescriptor.java b/tez-api/src/test/java/org/apache/tez/dag/api/TestEntityDescriptor.java index 1e8a99db99..606bf42717 100644 --- a/tez-api/src/test/java/org/apache/tez/dag/api/TestEntityDescriptor.java +++ b/tez-api/src/test/java/org/apache/tez/dag/api/TestEntityDescriptor.java @@ -23,41 +23,80 @@ import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; +import java.nio.ByteBuffer; import org.apache.commons.lang.RandomStringUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.DataOutputBuffer; import org.apache.tez.common.TezUtils; import org.junit.Assert; import org.junit.Test; +import org.mockito.Mockito; + +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.spy; public class TestEntityDescriptor { - @Test + public void verifyResults(InputDescriptor entityDescriptor, InputDescriptor deserialized, UserPayload payload, + String confVal) throws IOException { + Assert.assertEquals(entityDescriptor.getClassName(), deserialized.getClassName()); + // History text is not serialized when sending to tasks + Assert.assertNull(deserialized.getHistoryText()); + Assert.assertArrayEquals(payload.deepCopyAsArray(), deserialized.getUserPayload().deepCopyAsArray()); + Configuration deserializedConf = TezUtils.createConfFromUserPayload(deserialized.getUserPayload()); + Assert.assertEquals(confVal, deserializedConf.get("testKey")); + } + + public void testSingularWrite(InputDescriptor entityDescriptor, InputDescriptor deserialized, UserPayload payload, + String confVal) throws IOException { + DataOutputBuffer out = new DataOutputBuffer(); + entityDescriptor.write(out); + out.close(); + ByteArrayOutputStream bos = new ByteArrayOutputStream(out.getData().length); + bos.write(out.getData()); + + Mockito.verify(entityDescriptor).writeSingular(eq(out), any(ByteBuffer.class)); + deserialized.readFields(new DataInputStream(new ByteArrayInputStream(bos.toByteArray()))); + verifyResults(entityDescriptor, deserialized, payload, confVal); + } + + public void testSegmentedWrite(InputDescriptor entityDescriptor, InputDescriptor deserialized, UserPayload payload, + String confVal) throws IOException { + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(bos); + entityDescriptor.write(out); + out.close(); + + Mockito.verify(entityDescriptor).writeSegmented(eq(out), any(ByteBuffer.class)); + deserialized.readFields(new DataInputStream(new ByteArrayInputStream(bos.toByteArray()))); + verifyResults(entityDescriptor, deserialized, payload, confVal); + } + + @Test (timeout=1000) public void testEntityDescriptorHadoopSerialization() throws IOException { - // This tests the alternate serialization code path - // if the DataOutput is not DataOutputBuffer + /* This tests the alternate serialization code path + * if the DataOutput is not DataOutputBuffer + * AND, if it indeed is, with a read/write payload */ Configuration conf = new Configuration(true); String confVal = RandomStringUtils.random(10000, true, true); conf.set("testKey", confVal); UserPayload payload = TezUtils.createUserPayloadFromConf(conf); + + InputDescriptor deserialized = InputDescriptor.create("dummy"); InputDescriptor entityDescriptor = InputDescriptor.create("inputClazz").setUserPayload(payload) - .setHistoryText("Bar123"); + .setHistoryText("Bar123"); + InputDescriptor entityDescriptorLivingInFear = spy(entityDescriptor); - ByteArrayOutputStream bos = new ByteArrayOutputStream(); - DataOutputStream out = new DataOutputStream(bos); - entityDescriptor.write(out); - out.close(); + testSingularWrite(entityDescriptorLivingInFear, deserialized, payload, confVal); - InputDescriptor deserialized = InputDescriptor.create("dummy"); - deserialized.readFields(new DataInputStream(new ByteArrayInputStream(bos.toByteArray()))); - - Assert.assertEquals(entityDescriptor.getClassName(), deserialized.getClassName()); - // History text is not serialized when sending to tasks - Assert.assertNull(deserialized.getHistoryText()); - Assert.assertArrayEquals(payload.deepCopyAsArray(), deserialized.getUserPayload().deepCopyAsArray()); - Configuration deserializedConf = TezUtils.createConfFromUserPayload(deserialized.getUserPayload()); - Assert.assertEquals(confVal, deserializedConf.get("testKey")); + /* make read-only payload */ + payload = UserPayload.create(payload.getPayload()); + entityDescriptor = InputDescriptor.create("inputClazz").setUserPayload(payload) + .setHistoryText("Bar123"); + entityDescriptorLivingInFear = spy(entityDescriptor); + testSegmentedWrite(entityDescriptorLivingInFear, deserialized, payload, confVal); } - } From fe22f3276d6d97f6b5dfab24490ee2ca32bf73c3 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Fri, 27 Jul 2018 09:56:10 -0500 Subject: [PATCH 109/512] TEZ-3934. LegacySpeculator sometime issues wrong number of speculative attempts (Nishant Dash via jeagles) --- .../speculation/legacy/LegacySpeculator.java | 54 +++++++++++-------- 1 file changed, 33 insertions(+), 21 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java index 9fbea19f03..c132fb16b8 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java @@ -85,8 +85,7 @@ public class LegacySpeculator { // in progress. private static final long MAX_WAITTING_TIME_FOR_HEARTBEAT = 9 * 1000; - - private final Set mayHaveSpeculated = new HashSet(); + private final Set waitingToSpeculate = new HashSet(); private Vertex vertex; private TaskRuntimeEstimator estimator; @@ -229,24 +228,44 @@ private long speculationValue(Task task, long now, boolean shouldUseTimeout) { if (task.getState() == TaskState.SUCCEEDED) { return NOT_RUNNING; } - - if (!mayHaveSpeculated.contains(taskID) && !shouldUseTimeout) { - acceptableRuntime = estimator.thresholdRuntime(taskID); - if (acceptableRuntime == Long.MAX_VALUE) { - return ON_SCHEDULE; - } - } - - TezTaskAttemptID runningTaskAttemptID = null; int numberRunningAttempts = 0; for (TaskAttempt taskAttempt : attempts.values()) { - if (taskAttempt.getState() == TaskAttemptState.RUNNING - || taskAttempt.getState() == TaskAttemptState.STARTING) { + TaskAttemptState taskAttemptState = taskAttempt.getState(); + if (taskAttemptState == TaskAttemptState.RUNNING + || taskAttemptState == TaskAttemptState.STARTING) { if (++numberRunningAttempts > 1) { + waitingToSpeculate.remove(taskID); return ALREADY_SPECULATING; } + } + } + + // If we are here, there's at most one task attempt. + if (numberRunningAttempts == 0) { + return NOT_RUNNING; + } + + if ((numberRunningAttempts == 1) && waitingToSpeculate.contains(taskID)) { + return ALREADY_SPECULATING; + } + else { + if (!shouldUseTimeout) { + acceptableRuntime = estimator.thresholdRuntime(taskID); + if (acceptableRuntime == Long.MAX_VALUE) { + return ON_SCHEDULE; + } + } + } + + TezTaskAttemptID runningTaskAttemptID = null; + + for (TaskAttempt taskAttempt : attempts.values()) { + TaskAttemptState taskAttemptState = taskAttempt.getState(); + if (taskAttemptState == TaskAttemptState.RUNNING + || taskAttemptState == TaskAttemptState.STARTING) { + runningTaskAttemptID = taskAttempt.getID(); long taskAttemptStartTime @@ -311,13 +330,6 @@ private long speculationValue(Task task, long now, boolean shouldUseTimeout) { } } - // If we are here, there's at most one task attempt. - if (numberRunningAttempts == 0) { - return NOT_RUNNING; - } - - - if ((acceptableRuntime == Long.MIN_VALUE) && !shouldUseTimeout) { acceptableRuntime = estimator.thresholdRuntime(taskID); if (acceptableRuntime == Long.MAX_VALUE) { @@ -332,7 +344,7 @@ private long speculationValue(Task task, long now, boolean shouldUseTimeout) { protected void addSpeculativeAttempt(TezTaskID taskID) { LOG.info("DefaultSpeculator.addSpeculativeAttempt -- we are speculating " + taskID); vertex.scheduleSpeculativeTask(taskID); - mayHaveSpeculated.add(taskID); + waitingToSpeculate.add(taskID); } private int maybeScheduleASpeculation() { From 90c8195de0ddee9da3d4cd07519c8e287c85ad50 Mon Sep 17 00:00:00 2001 From: Jaume Marhuenda Date: Thu, 9 Aug 2018 14:39:56 -0700 Subject: [PATCH 110/512] TEZ-3974: Correctness regression of TEZ-955 in TEZ-2937 (Jaume Marhuenda, reviewed by Gopal V) --- .../LogicalIOProcessorRuntimeTask.java | 25 +++- .../TestLogicalIOProcessorRuntimeTask.java | 114 +++++++++++++++--- 2 files changed, 115 insertions(+), 24 deletions(-) diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java index 5c2ab77e2a..0ac916f10b 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java @@ -380,30 +380,43 @@ public void close() throws Exception { "Can only run while in RUNNING state. Current: " + this.state); this.state.set(State.CLOSED); + + List> allCloseInputEvents = Lists.newArrayList(); // Close the Inputs. for (InputSpec inputSpec : inputSpecs) { String srcVertexName = inputSpec.getSourceVertexName(); initializedInputs.remove(srcVertexName); List closeInputEvents = ((InputFrameworkInterface)inputsMap.get(srcVertexName)).close(); - sendTaskGeneratedEvents(closeInputEvents, - EventProducerConsumerType.INPUT, taskSpec.getVertexName(), - srcVertexName, taskSpec.getTaskAttemptID()); + allCloseInputEvents.add(closeInputEvents); } + List> allCloseOutputEvents = Lists.newArrayList(); // Close the Outputs. for (OutputSpec outputSpec : outputSpecs) { String destVertexName = outputSpec.getDestinationVertexName(); initializedOutputs.remove(destVertexName); List closeOutputEvents = ((LogicalOutputFrameworkInterface)outputsMap.get(destVertexName)).close(); - sendTaskGeneratedEvents(closeOutputEvents, - EventProducerConsumerType.OUTPUT, taskSpec.getVertexName(), - destVertexName, taskSpec.getTaskAttemptID()); + allCloseOutputEvents.add(closeOutputEvents); } // Close the Processor. processorClosed = true; processor.close(); + for (int i = 0; i < allCloseInputEvents.size(); i++) { + String srcVertexName = inputSpecs.get(i).getSourceVertexName(); + sendTaskGeneratedEvents(allCloseInputEvents.get(i), + EventProducerConsumerType.INPUT, taskSpec.getVertexName(), + srcVertexName, taskSpec.getTaskAttemptID()); + } + + for (int i = 0; i < allCloseOutputEvents.size(); i++) { + String destVertexName = outputSpecs.get(i).getDestinationVertexName(); + sendTaskGeneratedEvents(allCloseOutputEvents.get(i), + EventProducerConsumerType.OUTPUT, taskSpec.getVertexName(), + destVertexName, taskSpec.getTaskAttemptID()); + } + } finally { setTaskDone(); // Clear the interrupt status since the task execution is done. diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/TestLogicalIOProcessorRuntimeTask.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/TestLogicalIOProcessorRuntimeTask.java index c1bb3a13c7..599f98f666 100644 --- a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/TestLogicalIOProcessorRuntimeTask.java +++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/TestLogicalIOProcessorRuntimeTask.java @@ -25,6 +25,7 @@ import static org.mockito.Mockito.mock; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.HashMap; import java.util.LinkedList; import java.util.List; @@ -51,6 +52,7 @@ import org.apache.tez.runtime.api.OutputContext; import org.apache.tez.runtime.api.ProcessorContext; import org.apache.tez.runtime.api.Writer; +import org.apache.tez.runtime.api.events.CompositeDataMovementEvent; import org.apache.tez.runtime.api.impl.ExecutionContextImpl; import org.apache.tez.runtime.api.impl.InputSpec; import org.apache.tez.runtime.api.impl.OutputSpec; @@ -62,6 +64,7 @@ import com.google.common.collect.HashMultimap; import com.google.common.collect.Lists; import com.google.common.collect.Multimap; +import org.mockito.Mockito; public class TestLogicalIOProcessorRuntimeTask { @@ -77,10 +80,14 @@ public void testAutoStart() throws Exception { ScalingAllocator.class.getName()); TezTaskAttemptID taId1 = createTaskAttemptID(vertexId, 1); - TaskSpec task1 = createTaskSpec(taId1, "dag1", "vertex1", 30); + TaskSpec task1 = createTaskSpec(taId1, "dag1", + "vertex1", 30, TestProcessor.class.getName(), + TestOutput.class.getName()); TezTaskAttemptID taId2 = createTaskAttemptID(vertexId, 2); - TaskSpec task2 = createTaskSpec(taId2, "dag2", "vertex1", 10); + TaskSpec task2 = createTaskSpec(taId2, "dag2", + "vertex1", 10, TestProcessor.class.getName(), + TestOutput.class.getName()); TezSharedExecutor sharedExecutor = new TezSharedExecutor(tezConf); LogicalIOProcessorRuntimeTask lio1 = new LogicalIOProcessorRuntimeTask(task1, 0, tezConf, null, @@ -142,6 +149,50 @@ public void testAutoStart() throws Exception { } + /** + * We should expect no events being sent to the AM if an + * exception happens in the close method of the processor + */ + @Test + @SuppressWarnings("unchecked") + public void testExceptionHappensInClose() throws Exception { + TezDAGID dagId = createTezDagId(); + TezVertexID vertexId = createTezVertexId(dagId); + Map serviceConsumerMetadata = new HashMap<>(); + Multimap startedInputsMap = HashMultimap.create(); + TezUmbilical umbilical = mock(TezUmbilical.class); + TezConfiguration tezConf = new TezConfiguration(); + tezConf.set(TezConfiguration.TEZ_TASK_SCALE_MEMORY_ALLOCATOR_CLASS, + ScalingAllocator.class.getName()); + + TezTaskAttemptID taId1 = createTaskAttemptID(vertexId, 1); + TaskSpec task1 = createTaskSpec(taId1, "dag1", "vertex1", 30, + FaultyTestProcessor.class.getName(), + TestOutputWithEvents.class.getName()); + + TezSharedExecutor sharedExecutor = new TezSharedExecutor(tezConf); + LogicalIOProcessorRuntimeTask lio1 = new LogicalIOProcessorRuntimeTask(task1, 0, tezConf, null, + umbilical, serviceConsumerMetadata, new HashMap(), startedInputsMap, null, + "", new ExecutionContextImpl("localhost"), Runtime.getRuntime().maxMemory(), true, + new DefaultHadoopShim(), sharedExecutor); + + try { + lio1.initialize(); + lio1.run(); + + try { + lio1.close(); + fail("RuntimeException should have been thrown"); + } catch (RuntimeException e) { + // No events should be sent thorught the umbilical protocol + Mockito.verify(umbilical, Mockito.never()).addEvents(Mockito.anyList()); + } + } finally { + sharedExecutor.shutdownNow(); + cleanupAndTest(lio1); + } + } + private void cleanupAndTest(LogicalIOProcessorRuntimeTask lio) throws InterruptedException { ProcessorContext procContext = lio.getProcessorContext(); @@ -175,7 +226,7 @@ private void cleanupAndTest(LogicalIOProcessorRuntimeTask lio) throws Interrupte assertEquals(0, lio.outputSpecs.size()); assertTrue(lio.groupInputSpecs == null || lio.groupInputSpecs.size() == 0); } - + assertEquals(0, lio.inputsMap.size()); assertEquals(0, lio.inputContextMap.size()); assertEquals(0, lio.outputsMap.size()); @@ -190,11 +241,12 @@ private void cleanupAndTest(LogicalIOProcessorRuntimeTask lio) throws Interrupte } private TaskSpec createTaskSpec(TezTaskAttemptID taskAttemptID, - String dagName, String vertexName, int parallelism) { - ProcessorDescriptor processorDesc = createProcessorDescriptor(); + String dagName, String vertexName, int parallelism, + String processorClassname, String outputClassName) { + ProcessorDescriptor processorDesc = createProcessorDescriptor(processorClassname); TaskSpec taskSpec = new TaskSpec(taskAttemptID, dagName, vertexName, parallelism, processorDesc, - createInputSpecList(), createOutputSpecList(), null, null); + createInputSpecList(), createOutputSpecList(outputClassName), null, null); return taskSpec; } @@ -204,14 +256,14 @@ private List createInputSpecList() { return Lists.newArrayList(inputSpec); } - private List createOutputSpecList() { - OutputDescriptor outputtDesc = OutputDescriptor.create(TestOutput.class.getName()); + private List createOutputSpecList(String outputClassName) { + OutputDescriptor outputtDesc = OutputDescriptor.create(outputClassName); OutputSpec outputSpec = new OutputSpec("outedge", outputtDesc, 1); return Lists.newArrayList(outputSpec); } - private ProcessorDescriptor createProcessorDescriptor() { - ProcessorDescriptor desc = ProcessorDescriptor.create(TestProcessor.class.getName()); + private ProcessorDescriptor createProcessorDescriptor(String className) { + ProcessorDescriptor desc = ProcessorDescriptor.create(className); return desc; } @@ -248,15 +300,25 @@ public void run(Map inputs, Map out getContext().notifyProgress(); } - @Override - public void handleEvents(List processorEvents) { - - } + @Override + public void handleEvents(List processorEvents) { + } - @Override - public void close() throws Exception { - - } + @Override + public void close() throws Exception { + } + + } + + public static class FaultyTestProcessor extends TestProcessor { + public FaultyTestProcessor(ProcessorContext context) { + super(context); + } + + @Override + public void close() throws Exception { + throw new RuntimeException(); + } } @@ -336,6 +398,22 @@ public void handleEvents(List outputEvents) { public List close() throws Exception { return null; } + } + + public static class TestOutputWithEvents extends TestOutput { + + public static volatile int startCount = 0; + public static volatile int vertexParallelism; + + public TestOutputWithEvents(OutputContext outputContext, int numPhysicalOutputs) { + super(outputContext, numPhysicalOutputs); + } + @Override + public List close() throws Exception { + return Arrays.asList( + CompositeDataMovementEvent.create(0, + 0, null)); + } } } From 4b9a7be1b98cff00c44e7d3ffb2486bb59ca6804 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Thu, 16 Aug 2018 13:07:36 -0500 Subject: [PATCH 111/512] TEZ-3978. DAGClientServer Socket exception when localhost name lookup failures (Jonathan Eagles via jlowe) Signed-off-by: Jason Lowe --- .../java/org/apache/tez/dag/api/client/DAGClientServer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientServer.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientServer.java index 14de870744..204024489f 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientServer.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientServer.java @@ -128,7 +128,7 @@ private Server createServer(Class pbProtocol, InetSocketAddress addr, Configu BlockingService blockingService, String portRangeConfig) throws IOException { RPC.setProtocolEngine(conf, pbProtocol, ProtobufRpcEngine.class); RPC.Server server = new RPC.Builder(conf).setProtocol(pbProtocol) - .setInstance(blockingService).setBindAddress(addr.getHostName()) + .setInstance(blockingService).setBindAddress(addr.getHostString()) .setPort(addr.getPort()).setNumHandlers(numHandlers).setVerbose(false) .setPortRangeConfig(portRangeConfig).setSecretManager(secretManager) .build(); From 261bbdd5929d562758deb31085b565db8e92d6a2 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Thu, 23 Aug 2018 15:33:06 -0500 Subject: [PATCH 112/512] TEZ-3977. Add Eric Wohlstadter's public key to KEYS (Eric Wohlstadter via jeagles) --- KEYS | 59 +++++++++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 59 insertions(+) diff --git a/KEYS b/KEYS index 6485099611..507acc62f4 100644 --- a/KEYS +++ b/KEYS @@ -415,3 +415,62 @@ Qq/U6wJoMEY/jqYNH38nKVv2WCKUGqRC6k4gQYA3DhFr2vvJJzEwku3SdBB/Q/0m IRkj5+CIG0OiKX+FQ4e1Tl2RaA== =jxBF -----END PGP PUBLIC KEY BLOCK----- +pub rsa4096 2018-08-07 [SC] + 41592509F87E9A1B1F47A6BADEF0A5E5D41087FD +uid [ultimate] Eric Wohlstadter (CODE SIGNING KEY) +sig 3 DEF0A5E5D41087FD 2018-08-07 Eric Wohlstadter (CODE SIGNING KEY) +sub rsa4096 2018-08-07 [E] +sig DEF0A5E5D41087FD 2018-08-07 Eric Wohlstadter (CODE SIGNING KEY) + +-----BEGIN PGP PUBLIC KEY BLOCK----- + +mQINBFtp7ZYBEACn9MbrBq95cimBZSOuJMPKyjbzH3V668uTLz22Cx4K7HBzLBNT +OJtYdHPL9etvK61DpdfkjSET0jSuNHSd/eSy6IvKCgZ1ngR6mgIX+eC3MabbJ3tc +0sv1qF18loqcB86LhQYXFQEYfX3tiSTp3XThOIqg4i8qBK2V9qq8+5Xa5/vBkm31 +rRB9bvBsDBM/OlXCnfHgFMWjsLZHPw2+H+TlERBusjdXZCipWAfKrHKiJ8MvNrDs +mf4GrdYe0Rmaij2NlYEs/pxdAPaVlZOyil87xvDjVsCyfYqNncDlGH/i63sSN3CZ +cttIdZ7U0xb4+1LN8HLPjGRNIVUXahnILm35XJK4r46YVaADbnYHHGeFiOdJlxIM +IP4/eeSctWPoU0fNVeQCFVtilrNusYphUf3lLFFpxWrEUAty4hhqY7YEnlLnAGcV +xMfItvY7iB6+IfDgELHtKg6o8ikgp9YYPkrDFtMWFq9hVlH6HcTXl2cVL2FKBKUj +Z6UmR3q3duO74X6FAgekhTvtc5uTVBWGVVkgAK56EPGbiQL66GBwyCDg1Jx+D2Is +E/iLC9qvQZcdOnn0njk0eCus8Op67yN0AMh2imDrkTY82wxmVQIxVGWK2QkiII5P +6UhmoWr93BVhoOakernyt2C1YvFWY3NAOHUfg2CElR2UsNDIdSDcm2d98wARAQAB +tDhFcmljIFdvaGxzdGFkdGVyIChDT0RFIFNJR05JTkcgS0VZKSA8d29obHN0YWRA +Z21haWwuY29tPokCTgQTAQgAOBYhBEFZJQn4fpobH0emut7wpeXUEIf9BQJbae2W +AhsDBQsJCAcCBhUKCQgLAgQWAgMBAh4BAheAAAoJEN7wpeXUEIf9yngP/Rnd25uv +X4UWvokid2gpPxUX6LGqSPoSeQGMupm+epGIS08+nLYZ+7AZRL0hUm8aVx9AYiX3 +oEZ7G5rR7aKg6SlyZkUcDUH525pcRBS/R9F9PuSe2lo2nK1jwfKK8QtDqP4mJGWb +InyrBaG/KfW5PazrGHU8Li086J62Xa7hJ0xUlcNl95LDgiBCFjMGZpVjAG+8G3tw +DxK3Aowljpkl4v149N3ZF6nXFgXmzoGpskZ7H2asAWeNPmNMWiYhKL1Mu7DzjRJj +u1mxtMobrB8jwACOJy7KJvB5RVzGjgzRwEjpX+WR4NqfclpwqlHUMG/JkDKOKntM +BSxZX7JuTUEPdaxuuAW3XMuIKzToDIS17LyqZhR5hcFUp8ee2Z3ESVfN2BTbzNtn +O2OT7MP9Cy2DUqNwqWyBKKOH2cFKN8z/xO978ysepc5PbrOWGlHIrtfKxku6Sq3n +tGM1znIxXTJz8hKQ9dBXGMj4IMnGKMBR2y1BUtdlxUM6J4Co6BMlIk95puDs29hn +6geD1mbEcV3WrpBMbrdHMWqElSXZDwfiZ4WeaEgLxh3gzAWUUMbs14PhB/Fyj+Wg +OMa5tIAuz82PpL2m8pDDTg3hP/Klj2Zh5aiZIJHuBexvC2OBcveoPkbDCgt0eFaD +M5CZzrsQSHnsWoBOCTKBxrs3B8tq79nf+UORuQINBFtp7ZYBEAC28qcYvnInrjkx +na3vSQGDA0Z0gp9g7KtU0TBiEWapvfQw0byD/aTAeuTEo8uZprI1nlfgIQrYlZ9o +FKVljiP7w4m+SDeO9NV9o1WAqEgQVHy9645eaEnCE5tBkwUaiVct2yMnqEXCrgRD +6m2MVrMJ8t7Ch5hRDCdAJ4jrsXJnxUvalWNrWTJxmRKxYJJt4wFFWBdgUrRlsTdt +RlfgQyg3N/RDHTpU5FrVwuQ8/goSnDmPuAVSO6PnbKMNiGo0rmhDRG+pzQIV+APY +lmH/xdB/+MaE8I2XeG9+qKP8vZAOeiBV3csjuDcrTp1xy1dqbYYMMEIvwADtRiD7 +6EflhXXhsV5voUkipOTLO3vUaIwctWFiLQadbHb+lkqQwY038OEgFwjOQqwC5LVe +nTtORgxMJT4RBxLdjqZsRl2PO6T/13x8zX+gvFdKPs6hVMAsM5Y8hJBAHcsSxn6y +ICS+ZFg9EFZN+H/QbI0m9EO56LxBVMDTL8DtoZBs/sDLZghb2vztVoWXeKcIPogN +VoopGJaFWFOS9OROh3bqHic4dWzzBbmcxZLYCZvF4AP+q+9fBFF0EftSWMXViw8U +AXfzOKPPiOj1J/UHQ4mdyCfW/3c/kAgrrJ4JzH3Q7ADZQcFE4SGMThC1+5VsBkF8 +YGrgN+40zRQw/KSnGtlrlemLJsYfCQARAQABiQI2BBgBCAAgFiEEQVklCfh+mhsf +R6a63vCl5dQQh/0FAltp7ZYCGwwACgkQ3vCl5dQQh/2GUA/+LSdK+L9BHaKUWEnv +jKWtXagtS8BH5yPZuyYsR83Vwa0vSHyyQcK2hW7moJ1EL8/ftvsc47U60D9zDHdw +oW6ppNnRLWWtb7WJihKHnGcgdgIGdkdvq1QIhpA5QR2P5CLtqvsuIX6an6+wUXtL +WbBCh59ygjCgttczGt81IJEoRorktPweqHn3NqvsYpASMYy/V5mg+fllPH1Q14QX +2wRyKewP2UPr5E67CREjTxL/yh0CqDNtlxgAxXxTfcqtUB6ws7Y289dEosW7zkB/ +rCDII6HcLMoFFP+lFJiNskU/Z6LrzlBYuqUG+C51NpgXJ38F9j1zXOovagzXL86v +8TP3Fvy4TYG4rpYZA7OTy32IUpk278QadQsjrBDpk0R5RqtawrzeF1GOm9gX34ch +zpWneXz0jyjyBO3rXwwzTHZD0ugI4fjpPSyySkjDRd0HoV4N8eplY/uYb1OmnmCm +UlOBx+HjdmbNNW8v8bOAk97tAhOHYQBSIgJoZwKp0+2m0yW3KHgw3IEnMwuzwd20 +dk5somRZEdbsc88tOifNcZ7GJgHxenWlL/FF6jx2woXpyeV5ywd2cNhBmxWIYfNd +hRwuLzTx3guPM5hbjwtn8tEQQxsQ+0xgqHWn9pqGcVA8fjdLVbLFpHed7Pq+m7ig +O7s517hm1WD7LEkj6DyoRpxfNWE= +=KvCb +-----END PGP PUBLIC KEY BLOCK----- From 5949b0cef6e9b23c1ac5c75461c027b40cafc91d Mon Sep 17 00:00:00 2001 From: Jaume Marhuenda Date: Mon, 27 Aug 2018 20:34:41 -0700 Subject: [PATCH 113/512] TEZ-3958: Add internal vertex priority information into the tez dag.dot debug information (Jaume Marhuenda via Gopal V) --- .../src/main/java/org/apache/tez/Utils.java | 167 ++++++++++++++++++ .../org/apache/tez/dag/app/DAGAppMaster.java | 80 +-------- .../java/org/apache/tez/dag/app/dag/DAG.java | 9 + .../apache/tez/dag/app/dag/DAGScheduler.java | 22 +++ .../apache/tez/dag/app/dag/impl/DAGImpl.java | 9 + .../dag/impl/DAGSchedulerNaturalOrder.java | 5 +- .../DAGSchedulerNaturalOrderControlled.java | 5 +- .../dag/app/dag/impl/TestDAGScheduler.java | 13 +- 8 files changed, 222 insertions(+), 88 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/Utils.java b/tez-dag/src/main/java/org/apache/tez/Utils.java index 6f03a673de..dbde327107 100644 --- a/tez-dag/src/main/java/org/apache/tez/Utils.java +++ b/tez-dag/src/main/java/org/apache/tez/Utils.java @@ -14,18 +14,30 @@ package org.apache.tez; +import javax.annotation.Nullable; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.event.Event; +import org.apache.tez.common.TezCommonUtils; +import org.apache.tez.dag.api.records.DAGProtos; import org.apache.tez.dag.app.AppContext; import org.apache.tez.dag.app.dag.DAG; +import org.apache.tez.dag.app.dag.DAGScheduler; import org.apache.tez.dag.app.dag.DAGTerminationCause; +import org.apache.tez.dag.app.dag.Vertex; import org.apache.tez.dag.app.dag.event.DAGEventTerminateDag; import org.apache.tez.dag.records.TezDAGID; +import org.apache.tez.dag.utils.Graph; import org.apache.tez.serviceplugins.api.DagInfo; import org.apache.tez.serviceplugins.api.ServicePluginError; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.File; +import java.util.HashMap; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + @InterfaceAudience.Private /** * Utility class within the tez-dag module @@ -34,6 +46,11 @@ public class Utils { private static final Logger LOG = LoggerFactory.getLogger(Utils.class); + /** + * Pattern to clean the labels in the .dot generation. + */ + private static Pattern sanitizeLabelPattern = Pattern.compile("[:\\-\\W]+"); + public static String getContainerLauncherIdentifierString(int launcherIndex, AppContext appContext) { String name; try { @@ -92,6 +109,156 @@ public static void processNonFatalServiceErrorReport(String entityString, } } + /** + * Generate a visualization file. + * @param dag DAG. + * @param dagPB DAG plan. + * @param scheduler scheduler that provide the priorities of the vertexes. + */ + public static void generateDAGVizFile(final DAG dag, + final DAGProtos.DAGPlan dagPB, @Nullable final DAGScheduler scheduler) { + generateDAGVizFile(dag, dagPB, TezCommonUtils.getTrimmedStrings( + System.getenv(ApplicationConstants.Environment.LOG_DIRS.name())), + scheduler); + } + + /** + * Generate a visualization file. + * @param dag DAG. + * @param dagPB DAG plan. + * @param logDirs directories where the file will be written. + * @param scheduler scheduler that will provide the priorities + * of the vertexes. + */ + public static void generateDAGVizFile(final DAG dag, + final DAGProtos.DAGPlan dagPB, + final String[] logDirs, final @Nullable DAGScheduler scheduler) { + TezDAGID dagId = dag.getID(); + + HashMap nameToVertex = null; + if (scheduler != null) { + nameToVertex = new HashMap<>(dag.getVertices().size()); + for (Vertex v: dag.getVertices().values()) { + nameToVertex.put(v.getName(), v); + } + } + + Graph graph = new Graph(sanitizeLabelForViz(dagPB.getName())); + for (DAGProtos.VertexPlan vertexPlan : dagPB.getVertexList()) { + StringBuilder nodeLabel = new StringBuilder( + sanitizeLabelForViz(vertexPlan.getName()) + + "[" + getShortClassName( + vertexPlan.getProcessorDescriptor().getClassName())); + + if (scheduler != null) { + Vertex vertex = nameToVertex.get(vertexPlan.getName()); + if (vertex != null) { + try { + int priority = (scheduler.getPriorityLowLimit(dag, vertex) + + scheduler.getPriorityHighLimit(dag,vertex)) / 2; + nodeLabel.append(", priority=").append(priority).append("]"); + } catch (UnsupportedOperationException e) { + LOG.info("The DAG graphviz file with priorities will not" + + " be generate since the scheduler " + + scheduler.getClass().getSimpleName() + " doesn't" + + " override the methods to get the priorities"); + return; + } + } + } + Graph.Node n = graph.newNode(sanitizeLabelForViz(vertexPlan.getName()), + nodeLabel.toString()); + for (DAGProtos.RootInputLeafOutputProto input + : vertexPlan.getInputsList()) { + Graph.Node inputNode = graph.getNode( + sanitizeLabelForViz(vertexPlan.getName()) + + "_" + sanitizeLabelForViz(input.getName())); + inputNode.setLabel(sanitizeLabelForViz(vertexPlan.getName()) + + "[" + sanitizeLabelForViz(input.getName()) + "]"); + inputNode.setShape("box"); + inputNode.addEdge(n, "Input" + + " [inputClass=" + getShortClassName( + input.getIODescriptor().getClassName()) + + ", initializer=" + getShortClassName( + input.getControllerDescriptor().getClassName()) + "]"); + } + for (DAGProtos.RootInputLeafOutputProto output + : vertexPlan.getOutputsList()) { + Graph.Node outputNode = graph.getNode(sanitizeLabelForViz( + vertexPlan.getName()) + + "_" + sanitizeLabelForViz(output.getName())); + outputNode.setLabel(sanitizeLabelForViz(vertexPlan.getName()) + + "[" + sanitizeLabelForViz(output.getName()) + "]"); + outputNode.setShape("box"); + n.addEdge(outputNode, "Output" + + " [outputClass=" + getShortClassName( + output.getIODescriptor().getClassName()) + + ", committer=" + getShortClassName( + output.getControllerDescriptor().getClassName()) + "]"); + } + } + + for (DAGProtos.EdgePlan e : dagPB.getEdgeList()) { + + Graph.Node n = graph.getNode(sanitizeLabelForViz( + e.getInputVertexName())); + n.addEdge(graph.getNode(sanitizeLabelForViz( + e.getOutputVertexName())), + "[" + + "input=" + getShortClassName(e.getEdgeSource().getClassName()) + + ", output=" + getShortClassName( + e.getEdgeDestination().getClassName()) + + ", dataMovement=" + e.getDataMovementType().name().trim() + + ", schedulingType=" + + e.getSchedulingType().name().trim() + "]"); + } + + String outputFile = ""; + if (logDirs != null && logDirs.length != 0) { + outputFile += logDirs[0]; + outputFile += File.separator; + } + outputFile += dagId.toString(); + // Means we have set the priorities + if (scheduler != null) { + outputFile += "_priority"; + } + outputFile += ".dot"; + + try { + LOG.info("Generating DAG graphviz file" + + ", dagId=" + dagId.toString() + + ", filePath=" + outputFile); + graph.save(outputFile); + } catch (Exception e) { + LOG.warn("Error occurred when trying to save graph structure" + + " for dag " + dagId.toString(), e); + } + } + + /** + * Get the short name of the class. + * @param className long name + * @return short name + */ + private static String getShortClassName(final String className) { + int pos = className.lastIndexOf("."); + if (pos != -1 && pos < className.length() - 1) { + return className.substring(pos + 1); + } + return className; + } + + /** + * Replace some characters with underscores. + * @param label label to sanitize + * @return the label with the replaced characters + */ + private static String sanitizeLabelForViz(final String label) { + Matcher m = sanitizeLabelPattern.matcher(label); + return m.replaceAll("_"); + } + @SuppressWarnings("unchecked") private static void sendEvent(AppContext appContext, Event event) { appContext.getEventHandler().handle(event); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index c4b8df0fd8..42a9d57cf3 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -55,8 +55,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantReadWriteLock; -import java.util.regex.Matcher; -import java.util.regex.Pattern; import com.google.common.collect.BiMap; import com.google.common.collect.HashBiMap; @@ -66,6 +64,7 @@ import org.apache.commons.cli.GnuParser; import org.apache.commons.cli.Options; import org.apache.commons.lang.exception.ExceptionUtils; +import org.apache.tez.Utils; import org.apache.tez.client.CallerContext; import org.apache.tez.client.TezClientUtils; import org.apache.tez.common.TezUtils; @@ -131,7 +130,6 @@ import org.apache.tez.dag.api.TezUncheckedException; import org.apache.tez.dag.api.client.DAGClientHandler; import org.apache.tez.dag.api.client.DAGClientServer; -import org.apache.tez.dag.api.records.DAGProtos; import org.apache.tez.dag.api.records.DAGProtos.DAGPlan; import org.apache.tez.dag.api.records.DAGProtos.PlanLocalResourcesProto; import org.apache.tez.dag.api.records.DAGProtos.VertexPlan; @@ -179,7 +177,6 @@ import org.apache.tez.dag.history.utils.DAGUtils; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezVertexID; -import org.apache.tez.dag.utils.Graph; import org.apache.tez.dag.utils.RelocalizationUtils; import org.apache.tez.dag.utils.Simple2LevelVersionComparator; import org.apache.tez.hadoop.shim.HadoopShim; @@ -227,8 +224,6 @@ public class DAGAppMaster extends AbstractService { public static final int SHUTDOWN_HOOK_PRIORITY = 30; private static final Joiner PATH_JOINER = Joiner.on('/'); - private static Pattern sanitizeLabelPattern = Pattern.compile("[:\\-\\W]+"); - @VisibleForTesting static final String INVALID_SESSION_ERR_MSG = "Initial application attempt in session mode failed. " + "Application cannot recover and continue properly as DAG recovery has been disabled"; @@ -1038,82 +1033,11 @@ DAGImpl createDAG(DAGPlan dagPB, TezDAGID dagId) { LOG.warn("Failed to generate json for DAG", e); } - generateDAGVizFile(dagId, dagPB, logDirs); + Utils.generateDAGVizFile(newDag, dagPB, logDirs, newDag.getDAGScheduler()); writePBTextFile(newDag); return newDag; } // end createDag() - String getShortClassName(String className) { - int pos = className.lastIndexOf("."); - if (pos != -1 && pos < className.length()-1) { - return className.substring(pos+1); - } - return className; - } - - - private String sanitizeLabelForViz(String label) { - Matcher m = sanitizeLabelPattern.matcher(label); - return m.replaceAll("_"); - } - - private void generateDAGVizFile(TezDAGID dagId, DAGPlan dagPB, String[] logDirs) { - Graph graph = new Graph(sanitizeLabelForViz(dagPB.getName())); - - for (VertexPlan v : dagPB.getVertexList()) { - String nodeLabel = sanitizeLabelForViz(v.getName()) - + "[" + getShortClassName(v.getProcessorDescriptor().getClassName() + "]"); - Graph.Node n = graph.newNode(sanitizeLabelForViz(v.getName()), nodeLabel); - for (DAGProtos.RootInputLeafOutputProto input : v.getInputsList()) { - Graph.Node inputNode = graph.getNode(sanitizeLabelForViz(v.getName()) - + "_" + sanitizeLabelForViz(input.getName())); - inputNode.setLabel(sanitizeLabelForViz(v.getName()) - + "[" + sanitizeLabelForViz(input.getName()) + "]"); - inputNode.setShape("box"); - inputNode.addEdge(n, "Input" - + " [inputClass=" + getShortClassName(input.getIODescriptor().getClassName()) - + ", initializer=" + getShortClassName(input.getControllerDescriptor().getClassName()) + "]"); - } - for (DAGProtos.RootInputLeafOutputProto output : v.getOutputsList()) { - Graph.Node outputNode = graph.getNode(sanitizeLabelForViz(v.getName()) - + "_" + sanitizeLabelForViz(output.getName())); - outputNode.setLabel(sanitizeLabelForViz(v.getName()) - + "[" + sanitizeLabelForViz(output.getName()) + "]"); - outputNode.setShape("box"); - n.addEdge(outputNode, "Output" - + " [outputClass=" + getShortClassName(output.getIODescriptor().getClassName()) - + ", committer=" + getShortClassName(output.getControllerDescriptor().getClassName()) + "]"); - } - } - - for (DAGProtos.EdgePlan e : dagPB.getEdgeList()) { - - Graph.Node n = graph.getNode(sanitizeLabelForViz(e.getInputVertexName())); - n.addEdge(graph.getNode(sanitizeLabelForViz(e.getOutputVertexName())), - "[" - + "input=" + getShortClassName(e.getEdgeSource().getClassName()) - + ", output=" + getShortClassName(e.getEdgeDestination().getClassName()) - + ", dataMovement=" + e.getDataMovementType().name().trim() - + ", schedulingType=" + e.getSchedulingType().name().trim() + "]"); - } - - String outputFile = ""; - if (logDirs != null && logDirs.length != 0) { - outputFile += logDirs[0]; - outputFile += File.separator; - } - outputFile += dagId.toString() + ".dot"; - - try { - LOG.info("Generating DAG graphviz file" - + ", dagId=" + dagId.toString() - + ", filePath=" + outputFile); - graph.save(outputFile); - } catch (Exception e) { - LOG.warn("Error occurred when trying to save graph structure" - + " for dag " + dagId.toString(), e); - } - } private void writePBTextFile(DAG dag) { if (dag.getConf().getBoolean(TezConfiguration.TEZ_GENERATE_DEBUG_ARTIFACTS, diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java index 10c4257c5b..5c2eba122e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java @@ -38,6 +38,8 @@ import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.serviceplugins.api.DagInfo; +import javax.annotation.Nullable; + /** * Main interface to interact with the job. */ @@ -97,4 +99,11 @@ VertexStatusBuilder getVertexStatus(String vertexName, org.apache.tez.dag.api.Vertex.VertexExecutionContext getDefaultExecutionContext(); + /** + * + * @return the DAGScheduler that will schedule + * this DAG, null if it doesn't exist + */ + @Nullable DAGScheduler getDAGScheduler(); + } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAGScheduler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAGScheduler.java index 3055cd30da..2fa735ef8f 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAGScheduler.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAGScheduler.java @@ -89,4 +89,26 @@ public void taskCompleted(DAGEventSchedulerUpdate event) { public abstract void scheduleTaskEx(DAGEventSchedulerUpdate event); public abstract void taskCompletedEx(DAGEventSchedulerUpdate event); + + /** + * Get the low limit priority for a particular vertex. + * @param vertex to get the priority of + * @return the priority + */ + public int getPriorityLowLimit(final DAG dag, final Vertex vertex) { + final int vertexDistanceFromRoot = vertex.getDistanceFromRoot(); + return ((vertexDistanceFromRoot + 1) * dag.getTotalVertices() * 3) + + (vertex.getVertexId().getId() * 3); + } + + /** + * Get the low hight priority for a particular vertex. Default + * to the low limit priority minus two. + * @param vertex to get the priority of + * @return the priority + */ + public int getPriorityHighLimit(final DAG dag, final Vertex vertex) { + return getPriorityLowLimit(dag, vertex) - 2; + } + } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java index bd5e0ff019..6dcc7f0ad8 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java @@ -42,6 +42,7 @@ import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.exception.ExceptionUtils; +import org.apache.tez.Utils; import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.common.counters.LimitExceededException; import org.apache.tez.dag.app.dag.event.DAGEventTerminateDag; @@ -1620,6 +1621,9 @@ DAGState initializeDAG() { } } + // This is going to override the previously generated file + // which didn't have the priorities + Utils.generateDAGVizFile(this, jobPlan, dagScheduler); return DAGState.INITED; } @@ -2382,6 +2386,11 @@ public boolean isComplete() { } } + @Override + public DAGScheduler getDAGScheduler() { + return dagScheduler; + } + // output of either vertex or vertex group public static class OutputKey { String outputName; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrder.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrder.java index 3a16f46f97..2383db83b1 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrder.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrder.java @@ -46,11 +46,10 @@ public DAGSchedulerNaturalOrder(DAG dag, EventHandler dispatcher) { public void scheduleTaskEx(DAGEventSchedulerUpdate event) { TaskAttempt attempt = event.getAttempt(); Vertex vertex = dag.getVertex(attempt.getVertexID()); - int vertexDistanceFromRoot = vertex.getDistanceFromRoot(); // natural priority. Handles failures and retries. - int priorityLowLimit = ((vertexDistanceFromRoot + 1) * dag.getTotalVertices() * 3) + (vertex.getVertexId().getId() * 3); - int priorityHighLimit = priorityLowLimit - 2; + int priorityLowLimit = getPriorityLowLimit(dag, vertex); + int priorityHighLimit = getPriorityHighLimit(dag, vertex); if (LOG.isDebugEnabled()) { LOG.debug("Scheduling " + attempt.getID() + " between priorityLow: " + priorityLowLimit diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrderControlled.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrderControlled.java index 34cc92ff9b..c51783bc2d 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrderControlled.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrderControlled.java @@ -76,11 +76,10 @@ public DAGSchedulerNaturalOrderControlled(DAG dag, EventHandler dispatcher) { public void scheduleTaskEx(DAGEventSchedulerUpdate event) { TaskAttempt attempt = event.getAttempt(); Vertex vertex = dag.getVertex(attempt.getVertexID()); - int vertexDistanceFromRoot = vertex.getDistanceFromRoot(); // natural priority. Handles failures and retries. - int priorityLowLimit = ((vertexDistanceFromRoot + 1) * dag.getTotalVertices() * 3) + (vertex.getVertexId().getId() * 3); - int priorityHighLimit = priorityLowLimit - 2; + int priorityLowLimit = getPriorityLowLimit(dag, vertex); + int priorityHighLimit = getPriorityHighLimit(dag, vertex); TaskAttemptEventSchedule attemptEvent = new TaskAttemptEventSchedule( attempt.getID(), priorityLowLimit, priorityHighLimit); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGScheduler.java index f38f6890fc..07c361aded 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGScheduler.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGScheduler.java @@ -58,14 +58,19 @@ public void testDAGSchedulerNaturalOrder() { TaskAttempt mockAttempt = mock(TaskAttempt.class); when(mockDag.getVertex((TezVertexID) any())).thenReturn(mockVertex); when(mockDag.getTotalVertices()).thenReturn(4); - when(mockVertex.getDistanceFromRoot()).thenReturn(0).thenReturn(1) - .thenReturn(2); + when(mockVertex.getDistanceFromRoot()) + .thenReturn(0).thenReturn(0) + .thenReturn(1).thenReturn(1) + .thenReturn(2).thenReturn(2); TezVertexID vId0 = TezVertexID.fromString("vertex_1436907267600_195589_1_00"); TezVertexID vId1 = TezVertexID.fromString("vertex_1436907267600_195589_1_01"); TezVertexID vId2 = TezVertexID.fromString("vertex_1436907267600_195589_1_02"); TezVertexID vId3 = TezVertexID.fromString("vertex_1436907267600_195589_1_03"); - when(mockVertex.getVertexId()).thenReturn(vId0).thenReturn(vId1) - .thenReturn(vId2).thenReturn(vId3); + when(mockVertex.getVertexId()) + .thenReturn(vId0).thenReturn(vId0) + .thenReturn(vId1).thenReturn(vId1) + .thenReturn(vId2).thenReturn(vId2) + .thenReturn(vId3).thenReturn(vId3); DAGEventSchedulerUpdate event = new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt); From a37a367b400dd5b1cab22b35a47fb664055f3df6 Mon Sep 17 00:00:00 2001 From: Gopal V Date: Mon, 27 Aug 2018 20:37:27 -0700 Subject: [PATCH 114/512] TEZ-3980: ShuffleRunner: the wake loop needs to check for shutdown (Gopal V, reviewed by Gunther Hagleitner) --- .../runtime/library/common/shuffle/impl/ShuffleManager.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java index 0a0286e1e7..5f3693fd54 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java @@ -325,6 +325,9 @@ protected Void callInternal() throws Exception { && numCompletedInputs.get() < numInputs) { inputContext.notifyProgress(); boolean ret = wakeLoop.await(1000, TimeUnit.MILLISECONDS); + if (isShutdown.get()) { + break; + } } } finally { lock.unlock(); From 088c80ce231bff5e3c8c77b8b26a164204f9dae8 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Thu, 6 Sep 2018 11:24:12 -0500 Subject: [PATCH 115/512] TEZ-3973. Add Kuhu Shukla's (kshukla) public key to KEYS (Kuhu Shukla via jeagles) --- KEYS | 59 +++++++++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 59 insertions(+) diff --git a/KEYS b/KEYS index 507acc62f4..8cc3868ade 100644 --- a/KEYS +++ b/KEYS @@ -474,3 +474,62 @@ hRwuLzTx3guPM5hbjwtn8tEQQxsQ+0xgqHWn9pqGcVA8fjdLVbLFpHed7Pq+m7ig O7s517hm1WD7LEkj6DyoRpxfNWE= =KvCb -----END PGP PUBLIC KEY BLOCK----- +pub 4096R/AAFFE291 2018-07-27 + Key fingerprint = 857B 5F7D BE3B CD7B 12EA 512E 4405 B74B AAFF E291 +uid Kuhu Shukla (CODE SIGNING KEY) +sig 3 AAFFE291 2018-07-27 Kuhu Shukla (CODE SIGNING KEY) +sub 4096R/411BE2EF 2018-07-27 +sig AAFFE291 2018-07-27 Kuhu Shukla (CODE SIGNING KEY) + +-----BEGIN PGP PUBLIC KEY BLOCK----- +Version: GnuPG v1 + +mQINBFtbbsEBEADrAFoEVSfSNJjEfplb8zUC6RSRRyeCSDcClixOM1xPQCR0TUX9 +tQZTlAMIa6jjXxELMQjCJWKYIgCiTYU/eM3BSjYuS6vaKuno+9HdNhvjH8o/P2Sm +Ucgj8cD7nlYvVFJGdL5u5AvM3SCw1u8KaCAIB5uq4bYnWmKCGqF8IStUHNcyOFdi +ew2AFxFQL3AitNwdJoyQZ7o3TO8RFY+NgWXIu9A0OOuxK/evyIqo4/n/9yegv8U2 +Bn7Alw2Q1YxfnOwIUoJ9Ug8qg1BZ0UShVpSoYRj6O1dzpGGsbVf5oBUTd4K8nR4R +toKHdcQgdxeUYN/7OSzeKc91LWwkzQJu9RGEYuluobdi862ocUqLCxa3JOK9l6VR +Od11haQdcbudc8hKX6iI+m6XzvRGwrRF9C+lJRku9u2UX11qHD5y8LKoDDbodAfB +FCz2AyYaZEgFmkD8DmXsnUIzimYO6lDr+ikF+xdEjMdyT45we5vTxNRPnMzW4we9 +z9y6Gtu7KX5/ast76gMSRkiOa0A0l2xc5WVxnn2XBhgGd9sFoXMd9WN2ByGNzJ1e +ruXFZuSqDL2k4cmt3hXt6sZTtexKcl0ErxlmIuNNM062ZkVw5/bdLuInodmWlTBB +XTVZRmoXLRCGA6IN+C+qkZU/RNxJZLRySIDlJd9BbZeK3R00bulJwpUx9QARAQAB +tDNLdWh1IFNodWtsYSAoQ09ERSBTSUdOSU5HIEtFWSkgPGtzaHVrbGFAYXBhY2hl +Lm9yZz6JAjcEEwEKACEFAltbbsECGwMFCwkIBwMFFQoJCAsFFgIDAQACHgECF4AA +CgkQRAW3S6r/4pFjSRAAqC0oLoSNS4+1VhiKMzCUp6RR6q1i+CsVNvDeXpS+wl0v +4QSZaH4Z+0HVLV+EU0acjAu9hUL/i2oQm3wmBMjZFFdR+kUKXBOiYyYLwvG7PBKa +ONO9SvF3x36dTkHFQ17wCOkmd690Eui/q26nqRwFwWEx8aqKhNLGckrTxeylYWIB +elwTSXHByOlVo1toxw+vzBMXrJdS5FCTaQzDvFhQN2N4yqOBExuxDC31/Llpm4qp +efE6PjgaUYjdiFLci9hIASCN8uo944tmAbA1KAc23QzM5D0SbjXlNU+zar3uCwdi +DJn0yBEGRMsUbLWwnAfbqawaZOn1zw5vY6xJkCourq47v+PGz5PiXM3rPE9QtpoA +MDBukygbR6ItdM+MP7zXXNNuxnerjXcIvkfihIBBDM9qkiI3Fb9NXRMgLcVbVZ5P ++65EWUcBqV5HU1720B5s5EaE1nEZDtHBRefSlz2NKLa0r/dDYwxoCbD82iKaJ+sI +okysnJ3334kgqXrJ1TA7tzZ+USapSO5Of5WAqbvURSwjcIWwNsM1N6tuwDqt5PGu +DJzvkWWqJ1jUadZp0wUB9hBlO+7+Mf6yxt20L+m/UBU8iiVi0jT2/W4sFs/KR2IN +KSkCGwjM19KzQ1nf/AmhJsDbl1Oa92gPGd/WpMVbdkRrSrNqzGzBWmynH4SVh/K5 +Ag0EW1tuwQEQAMjS8mtg9Z/q6dLMOC4cLExlfOTJv4qJS/jVlBpqIEhZEtHJ2b8u +IqqQTd1TX/LOEFGrjHEcrkORyTh2F5giMpnM46L9pEARn4fh4xSdkEwF07IJm+R7 +Al+lV/YqFWH3ar7bAbY6CbeaxcuSxsmw7qB23qDYhF+/SjnJZZsBOGEBkmsQ8fHE +sf5Nc0W7c+RYC2UZc8t1iZeKIqy+dmAuy4ew/aVsb9qpN2yQGhNOPvzxfXLrk4V7 +npfhidYUHKCGC4oozgyTkCuRkdU7DPsOo9YkYrh4Nh41QrE11omA/VPkrcwn9Jm5 +hMUbsTTfUX4nfRiCnWhMDNg3XxCaxWJz4c3nlXQahVsP53DvgxSH7rpNFZPiWH42 +AMQV9RXPCd2ot0cV2eGtkwaJJiUNzBJqWT8Pvrh8QzHKr93adiYXI+5q3/UffFtJ +AolKAoiCuv2x1b/73747UMUHxkpDEM89w8INnd/UOZiqpmxNwRxuG2EnKFL79EB/ +lJOPnS97XBn+G3nT7OMsbQfFPeN1LXsFPboPb6lhUwaHvuS28bJQbm9xZQvPPZFr +2XzrK+W1ncBCcdwsufIf94SbPwQwya5aqB6VDTtlYzymzDrWLtyDonObYiSO+Ovu +KkVcZrZYatQlQA4R94WGtEBeuDvI+PYcVJJ7ndhe0EZDNxIVgW2SEjKJABEBAAGJ +Ah8EGAEKAAkFAltbbsECGwwACgkQRAW3S6r/4pEYKxAA6k9cC9EJIetbrM3xCrJA +Iif+6NPnc+KnrbZWsgoe3Ytpe9r9fo9vhvBDw1E6f5n6gL9y/nyqoE/c3nRfYOyo +DJW/AyZ8Bybb/OFVi8fHwy1cWYWpY5LIArmyAPuAiq1RcLkn07xtswdePYEuDnLh +NmhLAJqxAmJGASS1TGuTj/JS4IBTgNNiH4lSSQuwgRgIupMTCspswSeJK40HC+yg +YWIeG4JdzTrLcwmXvglTu3NmLkPIsIeoUlxjiL1op2LaNB8ca1O5RdnYaHjqE+V+ +kwhfu1KK6YwGz5e4Z8Iri0VNHZ04saSJQMU62NBx2NP+F6JIwepsoNAkDoToOFMu +l11Zs19tJGwy4zL1xihjCOVkOm1FH7Fv6eHgo/lV2mYjwmgaQcFEPWdxuBU82E7d +yzb2mInLAS1yAa03LPU1Im5W8eVep8HXRMe5NksYD0b5EbpAZVI5NSI/XMKyLoXc +BI7cMzwI4xmDpfmdqgNf/tSsUO7+D2F8dcRJJRIW1V4j/dEUQK2eCG071iNU02qx +20RP89v7X23we9PxzWhXK/IijxDgMENEoxWQ3s2z/uHCHHHtB2E/JY72ODP5gBJx +rLd0KJ/s8l76FzLMY7kKEuzMJ/pbzLrch57sQ5DU7qt6n82lfQPJRs9Rpu57tSOe +KVgQFocqA0Ql+jgz3TyGNxQ= +=/McP +-----END PGP PUBLIC KEY BLOCK----- From 567dc28dd8b2daca6a7f38c4768574bb67df780c Mon Sep 17 00:00:00 2001 From: Jaume Marhuenda Date: Mon, 10 Sep 2018 12:50:00 -0700 Subject: [PATCH 116/512] TEZ-3984: Shuffle: Out of Band DME event sending causes errors (Jaume Marhuenda, reviewed by Gopal V) Signed-off-by: Gopal V --- .../common/sort/impl/ExternalSorter.java | 6 +++++- .../common/sort/impl/PipelinedSorter.java | 21 +++++++++++++++---- .../common/sort/impl/dflt/DefaultSorter.java | 4 ++-- .../output/OrderedPartitionedKVOutput.java | 6 +++--- .../common/sort/impl/TestPipelinedSorter.java | 9 ++++++-- 5 files changed, 34 insertions(+), 12 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java index b6fe4575de..9e65862e0c 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java @@ -21,11 +21,14 @@ import java.io.File; import java.io.IOException; import java.io.InputStream; +import java.util.Collections; import java.util.Iterator; +import java.util.List; import java.util.Map; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Maps; +import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.OutputStatisticsReporter; import org.apache.tez.runtime.library.api.IOInterruptedException; import org.slf4j.Logger; @@ -68,11 +71,12 @@ public abstract class ExternalSorter { private static final Logger LOG = LoggerFactory.getLogger(ExternalSorter.class); - public void close() throws IOException { + public List close() throws IOException { spillFileIndexPaths.clear(); spillFilePaths.clear(); reportStatistics(); outputContext.notifyProgress(); + return Collections.emptyList(); } public abstract void flush() throws IOException; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java index 7915662bbe..028dd2f4e0 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java @@ -123,6 +123,11 @@ public class PipelinedSorter extends ExternalSorter { private final Deflater deflater; private final String auxiliaryService; + /** + * Store the events to be send in close. + */ + private final List finalEvents; + // TODO Set additional countesr - total bytes written, spills etc. public PipelinedSorter(OutputContext outputContext, Configuration conf, int numOutputs, @@ -236,6 +241,7 @@ public PipelinedSorter(OutputContext outputContext, Configuration conf, int numO keySerializer.open(span.out); minSpillsForCombine = this.conf.getInt(TezRuntimeConfiguration.TEZ_RUNTIME_COMBINE_MIN_SPILLS, 3); deflater = TezCommonUtils.newBestCompressionDeflater(); + finalEvents = Lists.newLinkedList(); } ByteBuffer allocateSpace() { @@ -695,8 +701,6 @@ public void flush() throws IOException { } if (!isFinalMergeEnabled()) { - //Generate events for all spills - List events = Lists.newLinkedList(); //For pipelined shuffle, previous events are already sent. Just generate the last event alone int startIndex = (pipelinedShuffle) ? (numSpills - 1) : 0; @@ -705,13 +709,12 @@ public void flush() throws IOException { for (int i = startIndex; i < endIndex; i++) { boolean isLastEvent = (i == numSpills - 1); String pathComponent = (outputContext.getUniqueIdentifier() + "_" + i); - ShuffleUtils.generateEventOnSpill(events, isFinalMergeEnabled(), isLastEvent, + ShuffleUtils.generateEventOnSpill(finalEvents, isFinalMergeEnabled(), isLastEvent, outputContext, i, indexCacheList.get(i), partitions, sendEmptyPartitionDetails, pathComponent, partitionStats, reportDetailedPartitionStats(), auxiliaryService, deflater); LOG.info(outputContext.getDestinationVertexName() + ": Adding spill event for spill (final update=" + isLastEvent + "), spillId=" + i); } - outputContext.sendEvents(events); return; } @@ -850,6 +853,16 @@ public void flush() throws IOException { } } + /** + * Close and send events. + * @return events to be returned by the edge. + * @throws IOException parent can throw this. + */ + public final List close() throws IOException { + super.close(); + return finalEvents; + } + private interface PartitionedRawKeyValueIterator extends TezRawKeyValueIterator { int getPartition(); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java index 557a53855f..9b5a43c363 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java @@ -753,10 +753,10 @@ public void flush() throws IOException { } @Override - public void close() throws IOException { - super.close(); + public List close() throws IOException { kvbuffer = null; kvmeta = null; + return super.close(); } boolean isClosed() { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java index 7d3e0b4d51..32a4f4de03 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java @@ -181,12 +181,12 @@ public synchronized void handleEvents(List outputEvents) { @Override public synchronized List close() throws IOException { - List returnEvents = null; + List returnEvents = Lists.newLinkedList(); if (sorter != null) { sorter.flush(); - sorter.close(); + returnEvents.addAll(sorter.close()); this.endTime = System.nanoTime(); - returnEvents = generateEvents(); + returnEvents.addAll(generateEvents()); sorter = null; } else { LOG.warn(getContext().getDestinationVertexName() + diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java index 727f8acab1..bd7f585adb 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java @@ -59,12 +59,14 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.List; import java.util.Map; import java.util.TreeMap; import java.util.UUID; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyListOf; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.doReturn; @@ -402,12 +404,15 @@ public void testWithPipelinedShuffle() throws IOException { initialAvailableMem); //Write 100 keys each of size 10 - writeData(sorter, 10000, 100); + writeData(sorter, 10000, 100, false); + sorter.flush(); + List events = sorter.close(); //final merge is disabled. Final output file would not be populated in this case. assertTrue(sorter.finalOutputFile == null); conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT, true); - verify(outputContext, times(1)).sendEvents(anyListOf(Event.class)); + verify(outputContext, times(0)).sendEvents(any()); + assertTrue(events.size() > 0); } @Test From 082b8e46a4d64913c0e57943e61ade96a93a5c6e Mon Sep 17 00:00:00 2001 From: Hitesh Shah Date: Tue, 11 Sep 2018 09:34:19 -0700 Subject: [PATCH 117/512] TEZ-3989. Fix by-laws related to emeritus clause. Signed-off-by: Jason Lowe --- docs/src/site/markdown/by-laws.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/src/site/markdown/by-laws.md b/docs/src/site/markdown/by-laws.md index 28ecabb852..8037c349b8 100644 --- a/docs/src/site/markdown/by-laws.md +++ b/docs/src/site/markdown/by-laws.md @@ -42,7 +42,8 @@ All of the volunteers who are contributing time, code, documentation, or resourc The project's Committers are responsible for the project's technical management. All committers have write access to the project's source repositories. Committers may cast binding votes on any technical discussion regarding the project. -Committer access is by invitation only and must be approved by lazy consensus of the active PMC members. A Committer may request removal of their commit privileges by their own declaration. A committer will be considered "emeritus/inactive" by not contributing in any form to the project for over 1 year. An emeritus committer may request reinstatement of commit access from the PMC. Such reinstatement is subject to lazy consensus of active PMC members. + +Committer access is by invitation only and must be approved by lazy consensus of the active PMC members. A Committer may request removal of their commit privileges by their own declaration. Commit access can be revoked by a unanimous vote of all the active PMC members (except the committer in question if they are also a PMC member). @@ -61,7 +62,7 @@ The Project Management Committee (PMC) for Apache Tez was created by a resolutio - Nominating new PMC members and committers - Maintaining these bylaws and other guidelines of the project -Membership of the PMC is by invitation only and must be approved by a lazy consensus of active PMC members. A PMC member is considered "emeritus/inactive" by not contributing in any form to the project for over one year. An emeritus PMC member may request reinstatement to the PMC. Such reinstatement is subject to lazy consensus of active PMC members. A PMC member may resign their membership from the PMC by their own declaration. Membership of the PMC can be revoked by an unanimous vote of all the active PMC members other than the member in question. +Membership of the PMC is by invitation only and must be approved by a lazy consensus of active PMC members. A PMC member may resign their membership from the PMC by their own declaration. Membership of the PMC can also be revoked via a Board resolution. The chair of the PMC is appointed by the ASF board. The chair is an office holder of the Apache Software Foundation (Vice President, Apache Tez) and has primary responsibility to the board for the management of the projects within the scope of the Tez PMC. The chair reports to the board quarterly on developments within the Tez project. The PMC may consider the position of PMC chair annually, and if supported by a successful vote to change the PMC chair, may recommend a new chair to the board. Ultimately, however, it is the board's responsibility who it chooses to appoint as the PMC chair. From 68f4cf93a3c7924c38594f2a1db0c0039986464f Mon Sep 17 00:00:00 2001 From: Eric Wohlstadter Date: Fri, 14 Sep 2018 10:51:58 -0500 Subject: [PATCH 118/512] TEZ-3988. Update snapshot version in master to 0.10.1-SNAPSHOT Signed-off-by: Jason Lowe --- docs/pom.xml | 2 +- hadoop-shim-impls/hadoop-shim-2.7/pom.xml | 2 +- hadoop-shim-impls/hadoop-shim-2.8/pom.xml | 2 +- hadoop-shim-impls/pom.xml | 2 +- hadoop-shim/pom.xml | 2 +- pom.xml | 2 +- tez-api/pom.xml | 2 +- tez-common/pom.xml | 2 +- tez-dag/pom.xml | 2 +- tez-dist/pom.xml | 2 +- tez-examples/pom.xml | 2 +- tez-ext-service-tests/pom.xml | 2 +- tez-mapreduce/pom.xml | 2 +- tez-plugins/pom.xml | 2 +- tez-plugins/tez-aux-services/pom.xml | 2 +- tez-plugins/tez-history-parser/pom.xml | 2 +- tez-plugins/tez-protobuf-history-plugin/pom.xml | 2 +- tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml | 2 +- tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml | 2 +- tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml | 2 +- tez-plugins/tez-yarn-timeline-history/pom.xml | 2 +- tez-runtime-internals/pom.xml | 2 +- tez-runtime-library/pom.xml | 2 +- tez-tests/pom.xml | 2 +- tez-tools/analyzers/job-analyzer/pom.xml | 2 +- tez-tools/analyzers/pom.xml | 2 +- tez-tools/pom.xml | 2 +- tez-tools/tez-javadoc-tools/pom.xml | 2 +- tez-tools/tez-tfile-parser/pom.xml | 2 +- tez-ui/pom.xml | 2 +- 30 files changed, 30 insertions(+), 30 deletions(-) diff --git a/docs/pom.xml b/docs/pom.xml index 045222057c..67a913adf7 100644 --- a/docs/pom.xml +++ b/docs/pom.xml @@ -27,7 +27,7 @@ org.apache.tez tez - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-docs pom diff --git a/hadoop-shim-impls/hadoop-shim-2.7/pom.xml b/hadoop-shim-impls/hadoop-shim-2.7/pom.xml index 08c2255905..d75ed460c1 100644 --- a/hadoop-shim-impls/hadoop-shim-2.7/pom.xml +++ b/hadoop-shim-impls/hadoop-shim-2.7/pom.xml @@ -19,7 +19,7 @@ hadoop-shim-impls org.apache.tez - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT 4.0.0 hadoop-shim-2.7 diff --git a/hadoop-shim-impls/hadoop-shim-2.8/pom.xml b/hadoop-shim-impls/hadoop-shim-2.8/pom.xml index bf4865c597..9450c75c60 100644 --- a/hadoop-shim-impls/hadoop-shim-2.8/pom.xml +++ b/hadoop-shim-impls/hadoop-shim-2.8/pom.xml @@ -19,7 +19,7 @@ hadoop-shim-impls org.apache.tez - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT 4.0.0 hadoop-shim-2.8 diff --git a/hadoop-shim-impls/pom.xml b/hadoop-shim-impls/pom.xml index dd52c28bd1..77192a016d 100644 --- a/hadoop-shim-impls/pom.xml +++ b/hadoop-shim-impls/pom.xml @@ -20,7 +20,7 @@ tez org.apache.tez - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT hadoop-shim-impls pom diff --git a/hadoop-shim/pom.xml b/hadoop-shim/pom.xml index df565c387a..383c5cf2b6 100644 --- a/hadoop-shim/pom.xml +++ b/hadoop-shim/pom.xml @@ -20,7 +20,7 @@ tez org.apache.tez - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT hadoop-shim diff --git a/pom.xml b/pom.xml index 9653c23eaa..8add2e0926 100644 --- a/pom.xml +++ b/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez pom - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez diff --git a/tez-api/pom.xml b/tez-api/pom.xml index 3d69f32421..6659f4f9e4 100644 --- a/tez-api/pom.xml +++ b/tez-api/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-api diff --git a/tez-common/pom.xml b/tez-common/pom.xml index b3c2892943..6b82cdc7f8 100644 --- a/tez-common/pom.xml +++ b/tez-common/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-common diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml index ee48b23050..7476bbc214 100644 --- a/tez-dag/pom.xml +++ b/tez-dag/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-dag diff --git a/tez-dist/pom.xml b/tez-dist/pom.xml index 42edfd9d7b..ea6041d76a 100644 --- a/tez-dist/pom.xml +++ b/tez-dist/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-dist diff --git a/tez-examples/pom.xml b/tez-examples/pom.xml index 2586869e14..3ac8534b6d 100644 --- a/tez-examples/pom.xml +++ b/tez-examples/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-examples diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml index e123a7a961..f27e35c81a 100644 --- a/tez-ext-service-tests/pom.xml +++ b/tez-ext-service-tests/pom.xml @@ -20,7 +20,7 @@ tez org.apache.tez - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-ext-service-tests diff --git a/tez-mapreduce/pom.xml b/tez-mapreduce/pom.xml index 7e50fcc54c..a263d83b54 100644 --- a/tez-mapreduce/pom.xml +++ b/tez-mapreduce/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-mapreduce diff --git a/tez-plugins/pom.xml b/tez-plugins/pom.xml index 3430f8367f..6ce710ea27 100644 --- a/tez-plugins/pom.xml +++ b/tez-plugins/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-plugins pom diff --git a/tez-plugins/tez-aux-services/pom.xml b/tez-plugins/tez-aux-services/pom.xml index 38f05f9631..0a80788d2a 100644 --- a/tez-plugins/tez-aux-services/pom.xml +++ b/tez-plugins/tez-aux-services/pom.xml @@ -20,7 +20,7 @@ tez-plugins org.apache.tez - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-aux-services diff --git a/tez-plugins/tez-history-parser/pom.xml b/tez-plugins/tez-history-parser/pom.xml index bfeb88e56b..3804253857 100644 --- a/tez-plugins/tez-history-parser/pom.xml +++ b/tez-plugins/tez-history-parser/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez-plugins - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-history-parser diff --git a/tez-plugins/tez-protobuf-history-plugin/pom.xml b/tez-plugins/tez-protobuf-history-plugin/pom.xml index 5ed61974eb..f84bb23dad 100644 --- a/tez-plugins/tez-protobuf-history-plugin/pom.xml +++ b/tez-plugins/tez-protobuf-history-plugin/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-protobuf-history-plugin diff --git a/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml b/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml index 9488fe319c..c817dcfbd1 100644 --- a/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml +++ b/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-yarn-timeline-cache-plugin diff --git a/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml b/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml index dea92eabb8..27288953fa 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-yarn-timeline-history-with-acls diff --git a/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml b/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml index 9f05b06755..8939263799 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-yarn-timeline-history-with-fs diff --git a/tez-plugins/tez-yarn-timeline-history/pom.xml b/tez-plugins/tez-yarn-timeline-history/pom.xml index 6137e986e3..99406650e6 100644 --- a/tez-plugins/tez-yarn-timeline-history/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-yarn-timeline-history diff --git a/tez-runtime-internals/pom.xml b/tez-runtime-internals/pom.xml index 851923eb1c..8fe7d79e40 100644 --- a/tez-runtime-internals/pom.xml +++ b/tez-runtime-internals/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-runtime-internals diff --git a/tez-runtime-library/pom.xml b/tez-runtime-library/pom.xml index 4ba6ad7eb9..b59d2fb774 100644 --- a/tez-runtime-library/pom.xml +++ b/tez-runtime-library/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-runtime-library diff --git a/tez-tests/pom.xml b/tez-tests/pom.xml index fd9b7bd277..c0bc884435 100644 --- a/tez-tests/pom.xml +++ b/tez-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-tests diff --git a/tez-tools/analyzers/job-analyzer/pom.xml b/tez-tools/analyzers/job-analyzer/pom.xml index 8579897927..a857bb3a53 100644 --- a/tez-tools/analyzers/job-analyzer/pom.xml +++ b/tez-tools/analyzers/job-analyzer/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-perf-analyzer - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-job-analyzer diff --git a/tez-tools/analyzers/pom.xml b/tez-tools/analyzers/pom.xml index 510ab20a4e..64484d3915 100644 --- a/tez-tools/analyzers/pom.xml +++ b/tez-tools/analyzers/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-tools - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-perf-analyzer pom diff --git a/tez-tools/pom.xml b/tez-tools/pom.xml index d6b67c8163..34f9f992e0 100644 --- a/tez-tools/pom.xml +++ b/tez-tools/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-tools pom diff --git a/tez-tools/tez-javadoc-tools/pom.xml b/tez-tools/tez-javadoc-tools/pom.xml index b7cdbd4eea..b17604f207 100644 --- a/tez-tools/tez-javadoc-tools/pom.xml +++ b/tez-tools/tez-javadoc-tools/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-tools - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-javadoc-tools diff --git a/tez-tools/tez-tfile-parser/pom.xml b/tez-tools/tez-tfile-parser/pom.xml index 74a8c558ac..c2f40201a3 100644 --- a/tez-tools/tez-tfile-parser/pom.xml +++ b/tez-tools/tez-tfile-parser/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-tools - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-tfile-parser diff --git a/tez-ui/pom.xml b/tez-ui/pom.xml index 59040a5dda..78d59fd7d6 100644 --- a/tez-ui/pom.xml +++ b/tez-ui/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.10.0-SNAPSHOT + 0.10.1-SNAPSHOT tez-ui war From c852dbecf5690dbf922d427701b0a3e8e7283f69 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Tue, 18 Sep 2018 17:14:44 -0500 Subject: [PATCH 119/512] TEZ-3972. Tez DAG can hang when a single task fails to fetch (Kuhu Shukla via jeagles) --- .../tez/dag/app/dag/impl/TaskAttemptImpl.java | 6 +- .../tez/dag/app/dag/impl/TestTaskAttempt.java | 108 +++++++++++++++++- 2 files changed, 110 insertions(+), 4 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java index 6ad41f85d5..bbec9ea6a1 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java @@ -1797,9 +1797,9 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl attempt, int readErrorTimespanSec = (int)((time - firstErrReportTime)/1000); boolean crossTimeDeadline = readErrorTimespanSec >= MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC; - float failureFraction = ((float) attempt.uniquefailedOutputReports.size()) - / outputFailedEvent.getConsumerTaskNumber(); - + int runningTasks = attempt.appContext.getCurrentDAG().getVertex( + failedDestTaId.getTaskID().getVertexID()).getRunningTasks(); + float failureFraction = runningTasks > 0 ? ((float) attempt.uniquefailedOutputReports.size()) / runningTasks : 0; boolean withinFailureFractionLimits = (failureFraction <= MAX_ALLOWED_OUTPUT_FAILURES_FRACTION); boolean withinOutputFailureLimits = diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java index 503e4185fc..5ab68f7d77 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java @@ -1820,6 +1820,7 @@ public void testMultipleOutputFailed() throws Exception { doReturn(containers).when(appCtx).getAllContainers(); HistoryEventHandler mockHistHandler = mock(HistoryEventHandler.class); doReturn(mockHistHandler).when(appCtx).getHistoryHandler(); + DAGImpl mockDAG = mock(DAGImpl.class); TaskHeartbeatHandler mockHeartbeatHandler = mock(TaskHeartbeatHandler.class); MockTaskAttemptImpl taImpl = new MockTaskAttemptImpl(taskID, 1, eventHandler, @@ -1852,6 +1853,14 @@ taListener, taskConf, new SystemClock(), EventMetaData mockMeta = mock(EventMetaData.class); TezTaskAttemptID mockDestId1 = mock(TezTaskAttemptID.class); when(mockMeta.getTaskAttemptID()).thenReturn(mockDestId1); + TezTaskID destTaskID = mock(TezTaskID.class); + TezVertexID destVertexID = mock(TezVertexID.class); + when(mockDestId1.getTaskID()).thenReturn(destTaskID); + when(destTaskID.getVertexID()).thenReturn(destVertexID); + Vertex destVertex = mock(VertexImpl.class); + when(destVertex.getRunningTasks()).thenReturn(11); + when(mockDAG.getVertex(destVertexID)).thenReturn(destVertex); + when(appCtx.getCurrentDAG()).thenReturn(mockDAG); TezEvent tzEvent = new TezEvent(mockReEvent, mockMeta); taImpl.handle(new TaskAttemptEventOutputFailed(taskAttemptID, tzEvent, 11)); @@ -1868,7 +1877,14 @@ taListener, taskConf, new SystemClock(), // different destination attempt reports error. now threshold crossed TezTaskAttemptID mockDestId2 = mock(TezTaskAttemptID.class); - when(mockMeta.getTaskAttemptID()).thenReturn(mockDestId2); + when(mockMeta.getTaskAttemptID()).thenReturn(mockDestId2); + destTaskID = mock(TezTaskID.class); + destVertexID = mock(TezVertexID.class); + when(mockDestId2.getTaskID()).thenReturn(destTaskID); + when(destTaskID.getVertexID()).thenReturn(destVertexID); + destVertex = mock(VertexImpl.class); + when(destVertex.getRunningTasks()).thenReturn(11); + when(mockDAG.getVertex(destVertexID)).thenReturn(destVertex); taImpl.handle(new TaskAttemptEventOutputFailed(taskAttemptID, tzEvent, 11)); assertEquals("Task attempt is not in FAILED state", taImpl.getState(), @@ -1923,6 +1939,7 @@ taListener, taskConf, new SystemClock(), mockReEvent = InputReadErrorEvent.create("", 1, 1); mockMeta = mock(EventMetaData.class); mockDestId1 = mock(TezTaskAttemptID.class); + when(mockDestId1.getTaskID()).thenReturn(destTaskID); when(mockMeta.getTaskAttemptID()).thenReturn(mockDestId1); tzEvent = new TezEvent(mockReEvent, mockMeta); //This should fail even when MAX_ALLOWED_OUTPUT_FAILURES_FRACTION is within limits, as @@ -1957,9 +1974,11 @@ taListener, taskConf, new SystemClock(), mockReEvent = InputReadErrorEvent.create("", 1, 1); mockMeta = mock(EventMetaData.class); mockDestId1 = mock(TezTaskAttemptID.class); + when(mockDestId1.getTaskID()).thenReturn(destTaskID); when(mockMeta.getTaskAttemptID()).thenReturn(mockDestId1); tzEvent = new TezEvent(mockReEvent, mockMeta); when(mockClock.getTime()).thenReturn(1000L); + when(destVertex.getRunningTasks()).thenReturn(1000); // time deadline not exceeded for a couple of read error events taImpl3.handle(new TaskAttemptEventOutputFailed(taskAttemptID3, tzEvent, 1000)); assertEquals("Task attempt is not in succeeded state", taImpl3.getState(), @@ -1978,6 +1997,93 @@ taListener, taskConf, new SystemClock(), verify(mockHeartbeatHandler, times(1)).unregister(taskAttemptID3); } + @Test(timeout = 60000) + public void testTAFailureBasedOnRunningTasks() throws Exception { + ApplicationId appId = ApplicationId.newInstance(1, 2); + ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance( + appId, 0); + TezDAGID dagID = TezDAGID.getInstance(appId, 1); + TezVertexID vertexID = TezVertexID.getInstance(dagID, 1); + TezTaskID taskID = TezTaskID.getInstance(vertexID, 1); + + MockEventHandler mockEh = new MockEventHandler(); + MockEventHandler eventHandler = spy(mockEh); + TaskCommunicatorManagerInterface taListener = createMockTaskAttemptListener(); + + Configuration taskConf = new Configuration(); + taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class); + taskConf.setBoolean("fs.file.impl.disable.cache", true); + + locationHint = TaskLocationHint.createTaskLocationHint( + new HashSet(Arrays.asList(new String[]{"127.0.0.1"})), null); + Resource resource = Resource.newInstance(1024, 1); + + NodeId nid = NodeId.newInstance("127.0.0.1", 0); + @SuppressWarnings("deprecation") + ContainerId contId = ContainerId.newInstance(appAttemptId, 3); + Container container = mock(Container.class); + when(container.getId()).thenReturn(contId); + when(container.getNodeId()).thenReturn(nid); + when(container.getNodeHttpAddress()).thenReturn("localhost:0"); + + AMContainerMap containers = new AMContainerMap( + mock(ContainerHeartbeatHandler.class), mock(TaskCommunicatorManagerInterface.class), + new ContainerContextMatcher(), appCtx); + containers.addContainerIfNew(container, 0, 0, 0); + + doReturn(new ClusterInfo()).when(appCtx).getClusterInfo(); + doReturn(containers).when(appCtx).getAllContainers(); + HistoryEventHandler mockHistHandler = mock(HistoryEventHandler.class); + doReturn(mockHistHandler).when(appCtx).getHistoryHandler(); + DAGImpl mockDAG = mock(DAGImpl.class); + + TaskHeartbeatHandler mockHeartbeatHandler = mock(TaskHeartbeatHandler.class); + MockTaskAttemptImpl taImpl = new MockTaskAttemptImpl(taskID, 1, eventHandler, + taListener, taskConf, new SystemClock(), + mockHeartbeatHandler, appCtx, false, + resource, createFakeContainerContext(), false); + TezTaskAttemptID taskAttemptID = taImpl.getID(); + + taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); + taImpl.handle(new TaskAttemptEventSubmitted(taskAttemptID, contId)); + taImpl.handle(new TaskAttemptEventStartedRemotely(taskAttemptID)); + verify(mockHeartbeatHandler).register(taskAttemptID); + taImpl.handle(new TaskAttemptEvent(taskAttemptID, + TaskAttemptEventType.TA_DONE)); + assertEquals("Task attempt is not in succeeded state", taImpl.getState(), + TaskAttemptState.SUCCEEDED); + verify(mockHeartbeatHandler).unregister(taskAttemptID); + + int expectedEventsTillSucceeded = 8; + ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); + ArgumentCaptor histArg = ArgumentCaptor.forClass(DAGHistoryEvent.class); + verify(eventHandler, times(expectedEventsTillSucceeded)).handle(arg.capture()); + verify(mockHistHandler, times(2)).handle(histArg.capture()); // start and finish + DAGHistoryEvent histEvent = histArg.getValue(); + TaskAttemptFinishedEvent finishEvent = (TaskAttemptFinishedEvent)histEvent.getHistoryEvent(); + long finishTime = finishEvent.getFinishTime(); + verifyEventType(arg.getAllValues(), TaskEventTAUpdate.class, 2); + + InputReadErrorEvent mockReEvent = InputReadErrorEvent.create("", 0, 1); + EventMetaData mockMeta = mock(EventMetaData.class); + TezTaskAttemptID mockDestId1 = mock(TezTaskAttemptID.class); + when(mockMeta.getTaskAttemptID()).thenReturn(mockDestId1); + TezTaskID destTaskID = mock(TezTaskID.class); + TezVertexID destVertexID = mock(TezVertexID.class); + when(mockDestId1.getTaskID()).thenReturn(destTaskID); + when(destTaskID.getVertexID()).thenReturn(destVertexID); + Vertex destVertex = mock(VertexImpl.class); + when(destVertex.getRunningTasks()).thenReturn(5); + when(mockDAG.getVertex(destVertexID)).thenReturn(destVertex); + when(appCtx.getCurrentDAG()).thenReturn(mockDAG); + TezEvent tzEvent = new TezEvent(mockReEvent, mockMeta); + taImpl.handle(new TaskAttemptEventOutputFailed(taskAttemptID, tzEvent, 11)); + + // failure threshold is met due to running tasks. state is FAILED + assertEquals("Task attempt is not in FAILED state", taImpl.getState(), + TaskAttemptState.FAILED); + } + @SuppressWarnings("deprecation") @Test(timeout = 5000) public void testKilledInNew() throws ServicePluginException { From 381dac0ca3cb14fe2f71c54f31cbdabf44ab4b36 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Thu, 20 Sep 2018 10:26:42 -0500 Subject: [PATCH 120/512] TEZ-3982. DAGAppMaster and tasks should not report negative or invalid progress (Kuhu Shukla via jeagles) --- .../org/apache/tez/common/ProgressHelper.java | 5 +- .../org/apache/tez/dag/app/DAGAppMaster.java | 2 +- .../apache/tez/dag/app/dag/impl/DAGImpl.java | 12 ++- .../apache/tez/dag/app/TestDAGAppMaster.java | 83 +++++++++++++++++++ 4 files changed, 98 insertions(+), 4 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/ProgressHelper.java b/tez-api/src/main/java/org/apache/tez/common/ProgressHelper.java index 407a20e299..07b066c2ea 100644 --- a/tez-api/src/main/java/org/apache/tez/common/ProgressHelper.java +++ b/tez-api/src/main/java/org/apache/tez/common/ProgressHelper.java @@ -47,7 +47,10 @@ public void run() { if (inputs != null && inputs.size() != 0) { for (LogicalInput input : inputs.values()) { if (input instanceof AbstractLogicalInput) { - progSum += ((AbstractLogicalInput) input).getProgress(); + float inputProgress = ((AbstractLogicalInput) input).getProgress(); + if (inputProgress >= 0.0f && inputProgress <= 1.0f) { + progSum += inputProgress; + } } } progress = (1.0f) * progSum / inputs.size(); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index 42a9d57cf3..177ba56014 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -1195,7 +1195,7 @@ public List getDiagnostics() { } public float getProgress() { - if (isSession && state.equals(DAGAppMasterState.IDLE)) { + if (isSession && getState().equals(DAGAppMasterState.IDLE)) { return 0.0f; } if(currentDAG != null) { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java index 6dcc7f0ad8..db51ceea95 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java @@ -806,9 +806,17 @@ public float getProgress() { try { float progress = 0.0f; for (Vertex v : getVertices().values()) { - progress += v.getProgress(); + float vertexProgress = v.getProgress(); + if (vertexProgress >= 0.0f && vertexProgress <= 1.0f) { + progress += vertexProgress; + } + } + float dagProgress = progress / getTotalVertices(); + if (dagProgress >= 0.0f && progress <= 1.0f) { + return dagProgress; + } else { + return 0.0f; } - return progress / getTotalVertices(); } finally { this.readLock.unlock(); } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java index 570c6dcd95..7a7dfe2b04 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java @@ -14,12 +14,18 @@ package org.apache.tez.dag.app; +import org.apache.hadoop.yarn.util.MonotonicClock; +import org.apache.tez.dag.app.dag.DAGState; +import org.apache.tez.dag.app.dag.Vertex; +import org.apache.tez.dag.records.TezVertexID; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import java.io.ByteArrayInputStream; import java.io.DataInput; @@ -29,8 +35,10 @@ import java.io.FileOutputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.HashMap; import java.util.LinkedList; import java.util.List; +import java.util.Map; import com.google.common.base.Preconditions; import com.google.common.collect.BiMap; @@ -387,6 +395,81 @@ public void testDagCredentialsWithMerge() throws Exception { testDagCredentials(true); } + @Test + public void testBadProgress() throws Exception { + TezConfiguration conf = new TezConfiguration(); + conf.setBoolean(TezConfiguration.TEZ_AM_CREDENTIALS_MERGE, true); + conf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); + conf.set(TezConfiguration.TEZ_AM_STAGING_DIR, TEST_DIR.toString()); + ApplicationId appId = ApplicationId.newInstance(1, 1); + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1); + + // create some sample AM credentials + Credentials amCreds = new Credentials(); + JobTokenSecretManager jtsm = new JobTokenSecretManager(); + JobTokenIdentifier identifier = new JobTokenIdentifier( + new Text(appId.toString())); + Token sessionToken = + new Token(identifier, jtsm); + sessionToken.setService(identifier.getJobId()); + TokenCache.setSessionToken(sessionToken, amCreds); + TestTokenSecretManager ttsm = new TestTokenSecretManager(); + Text tokenAlias1 = new Text("alias1"); + Token amToken1 = new Token( + new TestTokenIdentifier(new Text("amtoken1")), ttsm); + amCreds.addToken(tokenAlias1, amToken1); + + FileSystem fs = FileSystem.getLocal(conf); + FSDataOutputStream sessionJarsPBOutStream = + TezCommonUtils.createFileForAM(fs, new Path(TEST_DIR.toString(), + TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME)); + DAGProtos.PlanLocalResourcesProto.getDefaultInstance() + .writeDelimitedTo(sessionJarsPBOutStream); + sessionJarsPBOutStream.close(); + DAGAppMaster am = spy(new DAGAppMaster(attemptId, + ContainerId.newContainerId(attemptId, 1), + "127.0.0.1", 0, 0, new MonotonicClock(), 1, true, + TEST_DIR.toString(), new String[] {TEST_DIR.toString()}, + new String[] {TEST_DIR.toString()}, + new TezApiVersionInfo().getVersion(), amCreds, + "someuser", null)); + when(am.getState()).thenReturn(DAGAppMasterState.RUNNING); + am.init(conf); + am.start(); + Credentials dagCreds = new Credentials(); + Token dagToken1 = new Token( + new TestTokenIdentifier(new Text("dagtoken1")), ttsm); + dagCreds.addToken(tokenAlias1, dagToken1); + Text tokenAlias3 = new Text("alias3"); + Token dagToken2 = new Token( + new TestTokenIdentifier(new Text("dagtoken2")), ttsm); + dagCreds.addToken(tokenAlias3, dagToken2); + TezDAGID dagId = TezDAGID.getInstance(appId, 1); + DAGPlan dagPlan = DAGPlan.newBuilder() + .setName("somedag") + .setCredentialsBinary( + DagTypeConverters.convertCredentialsToProto(dagCreds)) + .build(); + DAGImpl dag = spy(am.createDAG(dagPlan, dagId)); + am.setCurrentDAG(dag); + when(dag.getState()).thenReturn(DAGState.RUNNING); + Map map = new HashMap(); + TezVertexID mockVertexID = mock(TezVertexID.class); + Vertex mockVertex = mock(Vertex.class); + when(mockVertex.getProgress()).thenReturn(Float.NaN); + map.put(mockVertexID, mockVertex); + when(dag.getVertices()).thenReturn(map); + when(dag.getTotalVertices()).thenReturn(1); + Assert.assertEquals("Progress was NaN and should be reported as 0", + 0, am.getProgress(), 0); + when(mockVertex.getProgress()).thenReturn(-10f); + Assert.assertEquals("Progress was negative and should be reported as 0", + 0, am.getProgress(), 0); + when(mockVertex.getProgress()).thenReturn(10f); + Assert.assertEquals("Progress was greater than 1 and should be reported as 0", + 0, am.getProgress(), 0); + } + @SuppressWarnings("deprecation") private void testDagCredentials(boolean doMerge) throws IOException { TezConfiguration conf = new TezConfiguration(); From 22e2a21791295716e8891134f52b076bfbff1f8f Mon Sep 17 00:00:00 2001 From: Jaume M Date: Mon, 24 Sep 2018 16:13:24 -0700 Subject: [PATCH 121/512] TEZ-3981. UnorderedPartitionedKVWriter.getInitialMemoryRequirement may return negative memory (Jaume M via jeagles) --- .../library/common/writers/UnorderedPartitionedKVWriter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java index 948417d917..0486ddc664 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java @@ -670,7 +670,7 @@ private long writePartition(int pos, WrappedBuffer wrappedBuffer, Writer writer, } public static long getInitialMemoryRequirement(Configuration conf, long maxAvailableTaskMemory) { - int initialMemRequestMb = conf.getInt( + long initialMemRequestMb = conf.getInt( TezRuntimeConfiguration.TEZ_RUNTIME_UNORDERED_OUTPUT_BUFFER_SIZE_MB, TezRuntimeConfiguration.TEZ_RUNTIME_UNORDERED_OUTPUT_BUFFER_SIZE_MB_DEFAULT); Preconditions.checkArgument(initialMemRequestMb != 0, From 64c04f1121ef1d04118e36b0e4fc3808205a50a8 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Wed, 26 Sep 2018 13:48:48 -0700 Subject: [PATCH 122/512] TEZ-3975. Add OWASP Dependency Check to the build Signed-off-by: Jason Lowe --- pom.xml | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/pom.xml b/pom.xml index 8add2e0926..8bce5ddce8 100644 --- a/pom.xml +++ b/pom.xml @@ -61,6 +61,7 @@ 3.0.1 2.10.4 2.4.3 + 1.3.6 ${scm.url} @@ -905,6 +906,11 @@ + + org.owasp + dependency-check-maven + ${dependency-check-maven.version} + org.codehaus.mojo findbugs-maven-plugin @@ -1065,6 +1071,17 @@ ${basedir}/findbugs-exclude.xml + + + org.owasp + dependency-check-maven + ${dependency-check-maven.version} + From 7d73bb2dc6163b04ec34ddf7b7a0300a78464f61 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Fri, 28 Sep 2018 10:15:24 -0500 Subject: [PATCH 123/512] TEZ-3994. Upgrade maven-surefire-plugin to 0.21.0 to support yetus Signed-off-by: Jason Lowe --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 8bce5ddce8..86402ff90b 100644 --- a/pom.xml +++ b/pom.xml @@ -890,7 +890,7 @@ org.apache.maven.plugins maven-surefire-plugin - 2.14.1 + 2.21.0 1 false From 39d76a656216d4843908279ef8eaa29a4cc83104 Mon Sep 17 00:00:00 2001 From: Jaume Marhuenda Date: Tue, 9 Oct 2018 13:12:17 -0500 Subject: [PATCH 124/512] TEZ-3969. TaskAttemptImpl: static fields initialized in instance ctor (Jaume Marhuenda via jegales) --- tez-dag/findbugs-exclude.xml | 11 ----- .../org/apache/tez/dag/app/dag/Vertex.java | 13 ++++++ .../tez/dag/app/dag/impl/TaskAttemptImpl.java | 38 +++++++-------- .../tez/dag/app/dag/impl/VertexImpl.java | 46 +++++++++++++++++++ .../tez/dag/app/dag/impl/TestTaskAttempt.java | 28 ++++++++--- 5 files changed, 97 insertions(+), 39 deletions(-) diff --git a/tez-dag/findbugs-exclude.xml b/tez-dag/findbugs-exclude.xml index 1150ccb3c7..a6ce38053b 100644 --- a/tez-dag/findbugs-exclude.xml +++ b/tez-dag/findbugs-exclude.xml @@ -252,15 +252,4 @@ - - - - - - - - - - - diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java index 0e54e9fc0b..0b2406fad6 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java @@ -213,6 +213,19 @@ interface VertexConfig { int getMaxFailedTaskAttempts(); boolean getTaskRescheduleHigherPriority(); boolean getTaskRescheduleRelaxedLocality(); + + /** + * @return tez.task.max.allowed.output.failures. + */ + int getMaxAllowedOutputFailures(); + /** + * @return tez.task.max.allowed.output.failures.fraction. + */ + double getMaxAllowedOutputFailuresFraction(); + /** + * @return tez.am.max.allowed.time-sec.for-read-error. + */ + int getMaxAllowedTimeForTaskReadErrorSec(); } void incrementRejectedTaskAttemptCount(); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java index bbec9ea6a1..7399979e66 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java @@ -214,9 +214,6 @@ public static DataEventDependencyInfo fromProto(DataEventDependencyInfoProto pro Set taskRacks = new HashSet(); private Map uniquefailedOutputReports = Maps.newHashMap(); - private static double MAX_ALLOWED_OUTPUT_FAILURES_FRACTION; - private static int MAX_ALLOWED_OUTPUT_FAILURES; - private static int MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC; protected final boolean isRescheduled; private final Resource taskResource; @@ -548,18 +545,6 @@ public TaskAttemptImpl(TezTaskAttemptID attemptId, EventHandler eventHandler, Vertex vertex, TaskLocationHint locationHint, TaskSpec taskSpec, TezTaskAttemptID schedulingCausalTA) { - // TODO: Move these configs over to Vertex.VertexConfig - MAX_ALLOWED_OUTPUT_FAILURES = conf.getInt(TezConfiguration - .TEZ_TASK_MAX_ALLOWED_OUTPUT_FAILURES, TezConfiguration - .TEZ_TASK_MAX_ALLOWED_OUTPUT_FAILURES_DEFAULT); - - MAX_ALLOWED_OUTPUT_FAILURES_FRACTION = conf.getDouble(TezConfiguration - .TEZ_TASK_MAX_ALLOWED_OUTPUT_FAILURES_FRACTION, TezConfiguration - .TEZ_TASK_MAX_ALLOWED_OUTPUT_FAILURES_FRACTION_DEFAULT); - - MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC = conf.getInt( - TezConfiguration.TEZ_AM_MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC, - TezConfiguration.TEZ_AM_MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC_DEFAULT); ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock(); this.readLock = rwLock.readLock(); this.writeLock = rwLock.writeLock(); @@ -1793,17 +1778,24 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl attempt, attempt.uniquefailedOutputReports.put(failedDestTaId, time); firstErrReportTime = time; } - + + int maxAllowedOutputFailures = attempt.getVertex().getVertexConfig() + .getMaxAllowedOutputFailures(); + int maxAllowedTimeForTaskReadErrorSec = attempt.getVertex() + .getVertexConfig().getMaxAllowedTimeForTaskReadErrorSec(); + double maxAllowedOutputFailuresFraction = attempt.getVertex() + .getVertexConfig().getMaxAllowedOutputFailuresFraction(); + int readErrorTimespanSec = (int)((time - firstErrReportTime)/1000); - boolean crossTimeDeadline = readErrorTimespanSec >= MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC; + boolean crossTimeDeadline = readErrorTimespanSec >= maxAllowedTimeForTaskReadErrorSec; int runningTasks = attempt.appContext.getCurrentDAG().getVertex( failedDestTaId.getTaskID().getVertexID()).getRunningTasks(); float failureFraction = runningTasks > 0 ? ((float) attempt.uniquefailedOutputReports.size()) / runningTasks : 0; boolean withinFailureFractionLimits = - (failureFraction <= MAX_ALLOWED_OUTPUT_FAILURES_FRACTION); + (failureFraction <= maxAllowedOutputFailuresFraction); boolean withinOutputFailureLimits = - (attempt.uniquefailedOutputReports.size() < MAX_ALLOWED_OUTPUT_FAILURES); + (attempt.uniquefailedOutputReports.size() < maxAllowedOutputFailures); // If needed we can launch a background task without failing this task // to generate a copy of the output just in case. @@ -1813,10 +1805,12 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl attempt, } String message = attempt.getID() + " being failed for too many output errors. " + "failureFraction=" + failureFraction - + ", MAX_ALLOWED_OUTPUT_FAILURES_FRACTION=" + MAX_ALLOWED_OUTPUT_FAILURES_FRACTION + + ", MAX_ALLOWED_OUTPUT_FAILURES_FRACTION=" + + maxAllowedOutputFailuresFraction + ", uniquefailedOutputReports=" + attempt.uniquefailedOutputReports.size() - + ", MAX_ALLOWED_OUTPUT_FAILURES=" + MAX_ALLOWED_OUTPUT_FAILURES - + ", MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC=" + MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC + + ", MAX_ALLOWED_OUTPUT_FAILURES=" + maxAllowedOutputFailures + + ", MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC=" + + maxAllowedTimeForTaskReadErrorSec + ", readErrorTimespan=" + readErrorTimespanSec; LOG.info(message); attempt.addDiagnosticInfo(message); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index 0184657c2a..a4d2de183a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -4690,6 +4690,19 @@ static class VertexConfigImpl implements VertexConfig { private final boolean taskRescheduleHigherPriority; private final boolean taskRescheduleRelaxedLocality; + /** + * See tez.task.max.allowed.output.failures.fraction. + */ + private final double maxAllowedOutputFailuresFraction; + /** + * See tez.task.max.allowed.output.failures. + */ + private final int maxAllowedOutputFailures; + /** + * See tez.am.max.allowed.time-sec.for-read-error. + */ + private final int maxAllowedTimeForTaskReadErrorSec; + public VertexConfigImpl(Configuration conf) { this.maxFailedTaskAttempts = conf.getInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS, TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS_DEFAULT); @@ -4699,6 +4712,18 @@ public VertexConfigImpl(Configuration conf) { this.taskRescheduleRelaxedLocality = conf.getBoolean(TezConfiguration.TEZ_AM_TASK_RESCHEDULE_RELAXED_LOCALITY, TezConfiguration.TEZ_AM_TASK_RESCHEDULE_RELAXED_LOCALITY_DEFAULT); + + this.maxAllowedOutputFailures = conf.getInt(TezConfiguration + .TEZ_TASK_MAX_ALLOWED_OUTPUT_FAILURES, TezConfiguration + .TEZ_TASK_MAX_ALLOWED_OUTPUT_FAILURES_DEFAULT); + + this.maxAllowedOutputFailuresFraction = conf.getDouble(TezConfiguration + .TEZ_TASK_MAX_ALLOWED_OUTPUT_FAILURES_FRACTION, TezConfiguration + .TEZ_TASK_MAX_ALLOWED_OUTPUT_FAILURES_FRACTION_DEFAULT); + + this.maxAllowedTimeForTaskReadErrorSec = conf.getInt( + TezConfiguration.TEZ_AM_MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC, + TezConfiguration.TEZ_AM_MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC_DEFAULT); } @Override @@ -4715,5 +4740,26 @@ public boolean getTaskRescheduleHigherPriority() { public boolean getTaskRescheduleRelaxedLocality() { return taskRescheduleRelaxedLocality; } + + /** + * @return maxAllowedOutputFailures. + */ + @Override public int getMaxAllowedOutputFailures() { + return maxAllowedOutputFailures; + } + + /** + * @return maxAllowedOutputFailuresFraction. + */ + @Override public double getMaxAllowedOutputFailuresFraction() { + return maxAllowedOutputFailuresFraction; + } + + /** + * @return maxAllowedTimeForTaskReadErrorSec. + */ + @Override public int getMaxAllowedTimeForTaskReadErrorSec() { + return maxAllowedTimeForTaskReadErrorSec; + } } } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java index 5ab68f7d77..503881039a 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java @@ -160,15 +160,20 @@ public void setupTest() { when(appCtx.getContainerLauncherName(anyInt())).thenReturn( TezConstants.getTezYarnServicePluginName()); - mockVertex = mock(Vertex.class); - when(mockVertex.getServicePluginInfo()).thenReturn(servicePluginInfo); - when(mockVertex.getVertexConfig()).thenReturn(new VertexImpl.VertexConfigImpl(vertexConf)); + createMockVertex(vertexConf); HistoryEventHandler mockHistHandler = mock(HistoryEventHandler.class); doReturn(mockHistHandler).when(appCtx).getHistoryHandler(); LogManager.getRootLogger().setLevel(Level.DEBUG); } + private void createMockVertex(Configuration conf) { + mockVertex = mock(Vertex.class); + when(mockVertex.getServicePluginInfo()).thenReturn(servicePluginInfo); + when(mockVertex.getVertexConfig()).thenReturn( + new VertexImpl.VertexConfigImpl(conf)); + } + @Test(timeout = 5000) public void testLocalityRequest() { TaskAttemptImpl.ScheduleTaskattemptTransition sta = @@ -1919,7 +1924,11 @@ taListener, taskConf, new SystemClock(), verify(eventHandler, times(expectedEventsAfterFetchFailure)).handle( arg.capture()); - taskConf.setInt(TezConfiguration.TEZ_TASK_MAX_ALLOWED_OUTPUT_FAILURES, 1); + Configuration newVertexConf = new Configuration(vertexConf); + newVertexConf.setInt(TezConfiguration.TEZ_TASK_MAX_ALLOWED_OUTPUT_FAILURES, + 1); + createMockVertex(newVertexConf); + TezTaskID taskID2 = TezTaskID.getInstance(vertexID, 2); MockTaskAttemptImpl taImpl2 = new MockTaskAttemptImpl(taskID2, 1, eventHandler, taListener, taskConf, new SystemClock(), @@ -1953,8 +1962,15 @@ taListener, taskConf, new SystemClock(), Clock mockClock = mock(Clock.class); int readErrorTimespanSec = 1; - taskConf.setInt(TezConfiguration.TEZ_TASK_MAX_ALLOWED_OUTPUT_FAILURES, 10); - taskConf.setInt(TezConfiguration.TEZ_AM_MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC, readErrorTimespanSec); + + newVertexConf = new Configuration(vertexConf); + newVertexConf.setInt(TezConfiguration.TEZ_TASK_MAX_ALLOWED_OUTPUT_FAILURES, + 10); + newVertexConf.setInt( + TezConfiguration.TEZ_AM_MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC, + readErrorTimespanSec); + createMockVertex(newVertexConf); + TezTaskID taskID3 = TezTaskID.getInstance(vertexID, 3); MockTaskAttemptImpl taImpl3 = new MockTaskAttemptImpl(taskID3, 1, eventHandler, taListener, taskConf, mockClock, From 7d5c66ac0ccf222b0611de3984abb5b8e97cf349 Mon Sep 17 00:00:00 2001 From: Jaume Marhuenda Date: Tue, 9 Oct 2018 13:31:58 -0500 Subject: [PATCH 125/512] TEZ-3995. Fix dot files produced by tests to prevent ASF license warnings in yetus (Jaume Marhuenda via jegales) --- tez-dag/pom.xml | 28 ++++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml index 7476bbc214..5f63079e8b 100644 --- a/tez-dag/pom.xml +++ b/tez-dag/pom.xml @@ -31,6 +31,7 @@ org.apache.tez.dag.app.rm.container.AMContainerImpl Tez Tez.gv + ${project.build.directory}/dagviz tez-dag @@ -186,6 +187,33 @@ org.apache.rat apache-rat-plugin + + maven-antrun-plugin + + + generate-sources + generate-sources + + + + + + + + run + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + ${test.tmp.dir} + + + org.apache.hadoop hadoop-maven-plugins From 2f1738888d3a536b4cdc026c4697ae3d7d3a956e Mon Sep 17 00:00:00 2001 From: Gopal V Date: Tue, 9 Oct 2018 17:02:35 -0500 Subject: [PATCH 126/512] TEZ-4003. Add gopalv@apache.org to KEYS file (Gopal V via jeagles) --- KEYS | 59 +++++++++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 59 insertions(+) diff --git a/KEYS b/KEYS index 8cc3868ade..37818138ab 100644 --- a/KEYS +++ b/KEYS @@ -533,3 +533,62 @@ rLd0KJ/s8l76FzLMY7kKEuzMJ/pbzLrch57sQ5DU7qt6n82lfQPJRs9Rpu57tSOe KVgQFocqA0Ql+jgz3TyGNxQ= =/McP -----END PGP PUBLIC KEY BLOCK----- +pub rsa4096 2018-09-20 [SC] + 6CFAA64865AD19C55C5662680C5267F97FBEC4F9 +uid [ultimate] Gopal Vijayaraghavan (CODE SIGNING KEY) +sig 3 0C5267F97FBEC4F9 2018-09-20 Gopal Vijayaraghavan (CODE SIGNING KEY) +sub rsa4096 2018-09-20 [E] +sig 0C5267F97FBEC4F9 2018-09-20 Gopal Vijayaraghavan (CODE SIGNING KEY) + +-----BEGIN PGP PUBLIC KEY BLOCK----- + +mQINBFuj10QBEACqjKNBEOd8TsAAsOlVxkWOYIh0A+iXhNrvDexpsNnTAGVrT008 +1qn0A+qM95qBqgmcgEgHiw8m2WhmByRzMQrsh6QeHq5piaiTP4jWE5B8sltUXjPS +BdOx377Sw/IIrbdzE8vBjwdKPcm5vP/lT8MCsue9wBWcgTBTiOeOqPkTByAofgKD +gOKbkS+tKuX3I7+L88eGs+ABIOYZg+p1rNN13KySp24zr9FNJzj5/z7T/So8gjJN +CG8qPeOtAaNgahCjOPWKCGIDQFSzlT86Grm6+vvSorFW2wcPo2A1k7PZ1eR35KWU +XGRLJPYy/0UMfZHehsSsypoAsPP0ccXqvWlJ7pbdpB9MQB3WgDFkuZrsr5Ifr1Ok +/YZEB6h8WHDgdY80V+cAadjfgVksKpwTD4oyaOxj6Jhds7P7OnKeEXDhb3JqzLiO +ueXkTXA0t30AEH4ybGyUpsJJ3512cgZymp7X51Lf+g5QujzCgpeQv/Ay7tQvVT28 +DKXSmZgblVwehcEpCUvDVgVB4+9d74abxZKhKH+g9JgR/Ll6NX50alO1DLbeK8pw +Sao96CpYD0f6AHqtncIDSi8LKIK8g65xVb/PxiQNekafIC3mia+Gw9bVxAmTpmHA +lQF3LKpl7eGVr7oTR1ek67I+Uw3EqcfHubfqfe98u0BiQ4KtNgMxa0eWOQARAQAB +tDtHb3BhbCBWaWpheWFyYWdoYXZhbiAoQ09ERSBTSUdOSU5HIEtFWSkgPGdvcGFs +dkBhcGFjaGUub3JnPokCTgQTAQgAOBYhBGz6pkhlrRnFXFZiaAxSZ/l/vsT5BQJb +o9dEAhsDBQsJCAcCBhUKCQgLAgQWAgMBAh4BAheAAAoJEAxSZ/l/vsT5dE8P/jgH +lWyGSbdndQQFbk35D4IyNAi2dYjLtmsDcvfytQMKolN3MB3X3vExhxNv2nR8+Lte +PN6+u85hA1qYtfj483f+9f4jAvI3TSttrrgfU7OXUV4vs+D7IFOZ1mhpykoeINaQ +udXitwaQuaW4irBsJwBHc386N6i5jGxySb/RGsiZneVue0CCnw+eveXkKc1+VMz3 +YDpCpz/fcsPS1RXM2cKFnb36tI8cZF3QbM7/R0iStGw8+ETNclLkHQ5TrjBBcu/u +o1PCy5dTSXe6unN1+rePTJZsrhc4QHldcfNoRY28LRzmgq+s4tsI3yfx4fMnxT9u +sD+zuhpiIc2yUylW9ccAPolc6PPuBQjiPCPfiAvRRsOD0HTKUvLi0ZluCn+BZUjB +xCYuYHB6OvfHDKgqBDi9oeueSL7wykup2DfTtlYCfAKwFPduNqJIvmDkjiASAdUO +hpCFnvepsEtjlsorrokj/8HEv04eatDdXDKgJeMivPtaTSsLVfANv+o1yC6T38Am +sTxpw6P5wsnxf4uaEda3AeD3DxYzXwmQGX6HnO7vDYKkMWbSKNuUb4PlHGeQ8POk +r4Df+ae9uD74hz9jFOeXwJDV23kehnsmZvj2L7uqPUnHYfOeCa61GjfNPwgyZeZw +HNKwQMI4Oyic1l5958gMyJYgc9YctXkpI5evBEU6uQINBFuj10QBEADA42yYBkJ8 +NS6NTDDxy6As3KIWQFqq3wkGxlG1hcwSGuQFgtD2g8Iid1uBW1CrItZ8PU0SPL6C +pFoeeWYOC45nXymzTSQENSYNlUYgwHnosCACQMK22UsRRJXNGsJDAUNmV9P5kJLd ++l+TtVry081zGULkhV2yXnatG3HIXpTKAKPlZZIfypzYtta9wob6pFosJ7Kg3dJx +SlveTzj672awtWeK6oKIw4HFrSSTE6bbUJ/3PBgPZ0rA4R7TJTpX+u+nOGL2k7Fq +kbtUF+nJsfJ8lJIL/vo26t2f/Bzy/4DYZyUi0NfSvzYdD0MzMPg7Af5pa9YwXsgN +Qe7zsTIvvXaj+SMmvAnNq9YWgAPV0ADxwgytMmhNwkW6CoNfSlgBXFK9Et7Ymdm0 +sHE6OGeM09FPQLL3OhcYH5UcKFUoLNO6wM+6El0UVXOY5M9EZZkvSzUh8WrOo3X2 +hjhaG3KXYB56y6B020VYiZ6wInxb1SRpCJdJ2vpe2atI3cmzhBKCdxuOQQHAqtxW +a91bDLT8o8XxePV8DCXsz9RbOAYRBTpA4cNAQGeJCBre9iaBHHspJXTyMBuX0fDJ ++ykXtin5OsqN7SmYKUiNbHVfzvH+yYXgkhxsnAPT9J2uuxXWnhZyV1sDgJEz16rP +kG4fZRMfayeDwlFtHxOfGkD9votfjo0FmQARAQABiQI2BBgBCAAgFiEEbPqmSGWt +GcVcVmJoDFJn+X++xPkFAluj10QCGwwACgkQDFJn+X++xPkO3RAAiWxkaVdFs1IB +4xe/UUIRkdBHjo90IUN73NK6kkK7Gd15XLn7endx+vD8WXhEswxnf5y8irDW/3Pz +KUUPAxMOcxnwviEVUdg5A3qKEGZ8EyOkpwqF1QCVeq6M1hYwCo1k9Xwn0NI4TYoy +04USJu9HgcZ0Mhoy1nBz6FfAwP2Z8Lf/aYPg5DV1OWJsmd87+PiM6SV+Wb/VPl6g +BlEyJ+dIZUQ0l2zQeonO6nR3kP1Xx5O0KeUQF2HziDXNTylFinmGDjURseC0MPFh +PJ/69q5yA2M7bICokfegEdUZpJqDh7K3JPhXKBAow/PuqGcwfZsmbjXBMC0jvsNB +91iXmLdeJXblWt5F0Tr7ss0ZZK3t2uQwNJ7+0DCxIqni0pO8PI9TQKa9slN+kSNG +H0u/IINsU7PjRK82cCWnu6CR1dnjuqERGK/xFMdE2vn+cIJsXMOysdP889xSCg6t +0vA+xXWcYoEhCoi/pTsF5WAdCPX+O4mNWMGZpWZcuCArMvShsaqqsT44uc7/Zam3 +FFEqFoa87Q918U67M8KN9qvNN5Tuy4vRjUtdwoYbPtVvAVommVu8vKmFEu+5jqC+ +aU33ZET6JYY37Zvn0KiOWLP/M08a8g/JOdTP1308wOoL4t+L9tgfy/Pwcv9EtQrj +RH25lXA/2HxJ4hqptH/ClwVJJeFs65M= +=p0Tr +-----END PGP PUBLIC KEY BLOCK----- From a83b1e9d467f5a74ed72da76b1de6e725cc33ebe Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Wed, 10 Oct 2018 16:21:21 -0500 Subject: [PATCH 127/512] TEZ-3990. The number of shuffle penalties for a host/inputAttemptIdentifier should be capped (Kuhu Shukla via jeagles) --- .../library/api/TezRuntimeConfiguration.java | 10 ++++ .../orderedgrouped/ShuffleScheduler.java | 16 ++++-- .../orderedgrouped/TestShuffleScheduler.java | 50 +++++++++++++++++++ 3 files changed, 72 insertions(+), 4 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java index 23f1f9bac7..85c53a5098 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java @@ -238,6 +238,15 @@ public class TezRuntimeConfiguration { "shuffle.fetch.failures.limit"; public static final int TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES_LIMIT_DEFAULT = 5; + /** + * Specifies in milliseconds the maximum delay a penalized host can have before being retried, + * defaults to 10 minutes. + */ + @ConfigurationProperty(type = "integer") + public static final String TEZ_RUNTIME_SHUFFLE_HOST_PENALTY_TIME_LIMIT_MS = TEZ_RUNTIME_PREFIX + + "shuffle.host.penalty.time.limit"; + public static final int TEZ_RUNTIME_SHUFFLE_HOST_PENALTY_TIME_LIMIT_MS_DEFAULT = 600000; + @Private @Unstable @ConfigurationProperty(type = "integer") @@ -609,6 +618,7 @@ public class TezRuntimeConfiguration { tezRuntimeKeys.add(TEZ_RUNTIME_CLEANUP_FILES_ON_INTERRUPT); tezRuntimeKeys.add(TEZ_RUNTIME_UNORDERED_PARTITIONED_KVWRITER_BUFFER_MERGE_PERCENT); tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_FETCHER_USE_SHARED_POOL); + tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_HOST_PENALTY_TIME_LIMIT_MS); defaultConf.addResource("core-default.xml"); defaultConf.addResource("core-site.xml"); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java index 981e2246f7..d84793211e 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java @@ -60,7 +60,6 @@ import org.apache.tez.http.HttpConnectionParams; import org.apache.tez.common.CallableWithNdc; import org.apache.tez.common.security.JobTokenSecretManager; -import org.apache.tez.dag.api.TezConstants; import org.apache.tez.runtime.library.common.CompositeInputAttemptIdentifier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -245,6 +244,7 @@ enum ShuffleErrors { private final boolean compositeFetch; private volatile Thread shuffleSchedulerThread = null; + private final int maxPenaltyTime; private long totalBytesShuffledTillNow = 0; private final DecimalFormat mbpsFormat = new DecimalFormat("0.00"); @@ -417,6 +417,8 @@ public ShuffleScheduler(InputContext inputContext, this.firstEventReceived = inputContext.getCounters().findCounter(TaskCounter.FIRST_EVENT_RECEIVED); this.lastEventReceived = inputContext.getCounters().findCounter(TaskCounter.LAST_EVENT_RECEIVED); this.compositeFetch = ShuffleUtils.isTezShuffleHandler(conf); + this.maxPenaltyTime = conf.getInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_HOST_PENALTY_TIME_LIMIT_MS, + TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_HOST_PENALTY_TIME_LIMIT_MS_DEFAULT); pipelinedShuffleInfoEventsMap = Maps.newConcurrentMap(); LOG.info("ShuffleScheduler running for sourceVertex: " @@ -831,7 +833,8 @@ private void penalizeHost(MapHost host, int failures) { long delay = (long) (INITIAL_PENALTY * Math.pow(PENALTY_GROWTH_RATE, failures)); - penalties.add(new Penalty(host, delay)); + long penaltyDelay = Math.min(delay, maxPenaltyTime); + penalties.add(new Penalty(host, penaltyDelay)); } private int getFailureCount(InputAttemptIdentifier srcAttempt) { @@ -1149,7 +1152,12 @@ public InputAttemptIdentifier getIdentifierForFetchedOutput( String path, int reduceId) { return pathToIdentifierMap.get(new PathPartition(path, reduceId)); } - + + @VisibleForTesting + DelayQueue getPenalties() { + return penalties; + } + private synchronized boolean inputShouldBeConsumed(InputAttemptIdentifier id) { boolean isInputFinished = false; if (id instanceof CompositeInputAttemptIdentifier) { @@ -1281,7 +1289,7 @@ public synchronized boolean isDone() { /** * A structure that records the penalty for a host. */ - private static class Penalty implements Delayed { + static class Penalty implements Delayed { MapHost host; private long endTime; diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java index 381ad85abe..7a7b1ee369 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java @@ -34,6 +34,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.conf.Configuration; @@ -55,6 +56,7 @@ import org.apache.tez.runtime.library.common.CompositeInputAttemptIdentifier; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.mockito.invocation.InvocationOnMock; @@ -953,6 +955,54 @@ public Void call() throws Exception { } } + @Test (timeout = 120000) + public void testPenalties() throws Exception { + InputContext inputContext = createTezInputContext(); + Configuration conf = new TezConfiguration(); + conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_HOST_PENALTY_TIME_LIMIT_MS, 20000); + int numInputs = 10; + Shuffle shuffle = mock(Shuffle.class); + MergeManager mergeManager = mock(MergeManager.class); + + final ShuffleSchedulerForTest scheduler = + new ShuffleSchedulerForTest(inputContext, conf, numInputs, shuffle, mergeManager, + mergeManager, + System.currentTimeMillis(), null, false, 0, "srcName"); + + ExecutorService executor = Executors.newFixedThreadPool(1); + + Future executorFuture = executor.submit(new Callable() { + @Override + public Void call() throws Exception { + scheduler.start(); + return null; + } + }); + + InputAttemptIdentifier[] identifiers = new InputAttemptIdentifier[numInputs]; + + for (int i = 0; i < numInputs; i++) { + CompositeInputAttemptIdentifier inputAttemptIdentifier = + new CompositeInputAttemptIdentifier(i, 0, "attempt_", 1); + scheduler.addKnownMapOutput("host" + i, 10000, 1, inputAttemptIdentifier); + identifiers[i] = inputAttemptIdentifier; + } + + MapHost[] mapHosts = new MapHost[numInputs]; + int count = 0; + for (MapHost mh : scheduler.mapLocations.values()) { + mapHosts[count++] = mh; + } + + for (int i = 0; i < 10; i++) { + scheduler.copyFailed(identifiers[0], mapHosts[0], false, false, false); + } + ShuffleScheduler.Penalty[] penaltyArray = new ShuffleScheduler.Penalty[scheduler.getPenalties().size()]; + scheduler.getPenalties().toArray(penaltyArray); + for (int i = 0; i < penaltyArray.length; i++) { + Assert.assertTrue(penaltyArray[i].getDelay(TimeUnit.MILLISECONDS) <= 20000); + } + } private InputContext createTezInputContext() throws IOException { ApplicationId applicationId = ApplicationId.newInstance(1, 1); From 927781566bccd0900357691fc4de76e7469492de Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Wed, 10 Oct 2018 16:59:04 -0500 Subject: [PATCH 128/512] TEZ-3961. Tez UI web.xml tries to reach out to java.sun.com for validation after moving to jetty-9 (Kuhu Shukla via jeagles) --- tez-ui/src/main/webapp/WEB-INF/web.xml | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/tez-ui/src/main/webapp/WEB-INF/web.xml b/tez-ui/src/main/webapp/WEB-INF/web.xml index 62bfe31e90..49a34d8a87 100644 --- a/tez-ui/src/main/webapp/WEB-INF/web.xml +++ b/tez-ui/src/main/webapp/WEB-INF/web.xml @@ -16,10 +16,6 @@ * limitations under the License. --> - - TEZ UI - \ No newline at end of file + From 79af4e8d06417829986dfc34b3627ead15d563ee Mon Sep 17 00:00:00 2001 From: Jaume Marhuenda Date: Tue, 23 Oct 2018 15:30:13 -0700 Subject: [PATCH 129/512] TEZ-3976: Batch ShuffleManager error report events (Jaume Marhuenda, reviewed by Gopal V) Signed-off-by: Gopal V --- .../api/events/InputReadErrorEvent.java | 45 +++++- .../library/api/TezRuntimeConfiguration.java | 10 ++ .../common/shuffle/impl/ShuffleManager.java | 129 ++++++++++++++++-- .../shuffle/impl/TestShuffleManager.java | 66 +++++++++ 4 files changed, 232 insertions(+), 18 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java index 7d2e0d25a8..cabc39fc8e 100644 --- a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java +++ b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java @@ -21,6 +21,8 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.tez.runtime.api.Event; +import java.util.Objects; + /** * Event generated by an Input to indicate error when trying to retrieve data. * This is not necessarily a fatal event - it's an indication to the AM to retry @@ -44,17 +46,31 @@ public final class InputReadErrorEvent extends Event { */ private final int version; - private InputReadErrorEvent(String diagnostics, int index, - int version) { + /** + * Number of failures. + */ + private final int numFailures; + + private InputReadErrorEvent(final String diagnostics, final int index, + final int version, final int numFailures) { super(); this.diagnostics = diagnostics; this.index = index; this.version = version; + this.numFailures = numFailures; } public static InputReadErrorEvent create(String diagnostics, int index, int version) { - return new InputReadErrorEvent(diagnostics, index, version); + return create(diagnostics, index, version, 1); + } + + /** + * Create an InputReadErrorEvent. + */ + public static InputReadErrorEvent create(final String diagnostics, final int index, + final int version, final int numFailures) { + return new InputReadErrorEvent(diagnostics, index, version, numFailures); } public String getDiagnostics() { @@ -69,4 +85,27 @@ public int getVersion() { return version; } + /** + * @return number of failures + */ + public int getNumFailures() { + return numFailures; + } + + @Override + public int hashCode() { + return Objects.hash(index, version); + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + InputReadErrorEvent that = (InputReadErrorEvent) o; + return index == that.index && version == that.version; + } } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java index 85c53a5098..86792e2fc3 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java @@ -512,6 +512,15 @@ public class TezRuntimeConfiguration { TEZ_RUNTIME_PREFIX + "enable.final-merge.in.output"; public static final boolean TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT_DEFAULT = true; + /** + * Expert level setting. How long should @link{ShuffleManager} wait for batching + * before sending the events in milliseconds. Set to -1 to not wait. + */ + @ConfigurationProperty(type = "integer") + public static final String TEZ_RUNTIME_SHUFFLE_BATCH_WAIT = + TEZ_RUNTIME_PREFIX + "shuffle.batch.wait"; + public static final int TEZ_RUNTIME_SHUFFLE_BATCH_WAIT_DEFAULT = -1; + /** * Share data fetched between tasks running on the same host if applicable @@ -619,6 +628,7 @@ public class TezRuntimeConfiguration { tezRuntimeKeys.add(TEZ_RUNTIME_UNORDERED_PARTITIONED_KVWRITER_BUFFER_MERGE_PERCENT); tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_FETCHER_USE_SHARED_POOL); tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_HOST_PENALTY_TIME_LIMIT_MS); + tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_BATCH_WAIT); defaultConf.addResource("core-default.xml"); defaultConf.addResource("core-site.xml"); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java index 5f3693fd54..ba8592f8ae 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java @@ -26,6 +26,7 @@ import java.util.Arrays; import java.util.BitSet; import java.util.Collections; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -35,6 +36,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; @@ -46,6 +48,8 @@ import javax.crypto.SecretKey; import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.yarn.util.Clock; +import org.apache.hadoop.yarn.util.MonotonicClock; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.http.HttpConnectionParams; import org.apache.tez.runtime.api.TaskFailureType; @@ -114,9 +118,30 @@ public class ShuffleManager implements FetcherCallback { @VisibleForTesting final ListeningExecutorService fetcherExecutor; + /** + * Executor for ReportCallable. + */ + private ExecutorService reporterExecutor; + + /** + * Lock to sync failedEvents. + */ + private final ReentrantLock reportLock = new ReentrantLock(); + + /** + * Condition to wake up the thread notifying when events fail. + */ + private final Condition reportCondition = reportLock.newCondition(); + + /** + * Events reporting fetcher failed. + */ + private final HashMap failedEvents + = new HashMap<>(); + private final ListeningExecutorService schedulerExecutor; private final RunShuffleCallable schedulerCallable; - + private final BlockingQueue completedInputs; private final AtomicBoolean inputReadyNotificationSent = new AtomicBoolean(false); @VisibleForTesting @@ -151,6 +176,11 @@ public class ShuffleManager implements FetcherCallback { private final int ifileBufferSize; private final boolean ifileReadAhead; private final int ifileReadAheadLength; + + /** + * Holds the time to wait for failures to batch them and send less events. + */ + private final int maxTimeToWaitForReportMillis; private final String srcNameTrimmed; @@ -199,7 +229,8 @@ public ShuffleManager(InputContext inputContext, Configuration conf, int numInpu this.bytesShuffledToDiskCounter = inputContext.getCounters().findCounter(TaskCounter.SHUFFLE_BYTES_TO_DISK); this.bytesShuffledToMemCounter = inputContext.getCounters().findCounter(TaskCounter.SHUFFLE_BYTES_TO_MEM); this.bytesShuffledDirectDiskCounter = inputContext.getCounters().findCounter(TaskCounter.SHUFFLE_BYTES_DISK_DIRECT); - + + this.ifileBufferSize = bufferSize; this.ifileReadAhead = ifileReadAheadEnabled; this.ifileReadAheadLength = ifileReadAheadLength; @@ -212,6 +243,10 @@ public ShuffleManager(InputContext inputContext, Configuration conf, int numInpu this.verifyDiskChecksum = conf.getBoolean( TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_VERIFY_DISK_CHECKSUM, TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_VERIFY_DISK_CHECKSUM_DEFAULT); + this.maxTimeToWaitForReportMillis = conf.getInt( + TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_BATCH_WAIT, + TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_BATCH_WAIT_DEFAULT); + this.shufflePhaseTime = inputContext.getCounters().findCounter(TaskCounter.SHUFFLE_PHASE_TIME); this.firstEventReceived = inputContext.getCounters().findCounter(TaskCounter.FIRST_EVENT_RECEIVED); @@ -302,12 +337,63 @@ public ShuffleManager(InputContext inputContext, Configuration conf, int numInpu public void run() throws IOException { Preconditions.checkState(inputManager != null, "InputManager must be configured"); + if (maxTimeToWaitForReportMillis > 0) { + reporterExecutor = Executors.newSingleThreadExecutor( + new ThreadFactoryBuilder().setDaemon(true) + .setNameFormat("ShuffleRunner {" + srcNameTrimmed + "}") + .build()); + Future reporterFuture = reporterExecutor.submit(new ReporterCallable()); + } + ListenableFuture runShuffleFuture = schedulerExecutor.submit(schedulerCallable); Futures.addCallback(runShuffleFuture, new SchedulerFutureCallback()); // Shutdown this executor once this task, and the callback complete. schedulerExecutor.shutdown(); } - + + private class ReporterCallable extends CallableWithNdc { + /** + * Measures if the batching interval has ended. + */ + private final Clock clock; + ReporterCallable() { + clock = new MonotonicClock(); + } + + @Override + protected Void callInternal() throws Exception { + long nextReport = 0; + while (!isShutdown.get()) { + try { + reportLock.lock(); + while (failedEvents.isEmpty()) { + boolean signaled = reportCondition.await(maxTimeToWaitForReportMillis, + TimeUnit.MILLISECONDS); + } + + long currentTime = clock.getTime(); + if (currentTime > nextReport) { + if (failedEvents.size() > 0) { + List failedEventsToSend = Lists.newArrayListWithCapacity( + failedEvents.size()); + for (InputReadErrorEvent key : failedEvents.keySet()) { + failedEventsToSend.add(InputReadErrorEvent + .create(key.getDiagnostics(), key.getIndex(), + key.getVersion(), failedEvents.get(key))); + } + inputContext.sendEvents(failedEventsToSend); + failedEvents.clear(); + nextReport = currentTime + maxTimeToWaitForReportMillis; + } + } + } finally { + reportLock.unlock(); + } + } + return null; + } + } + private class RunShuffleCallable extends CallableWithNdc { private final Configuration conf; @@ -804,18 +890,27 @@ public void fetchFailed(String host, if (srcAttemptIdentifier == null) { reportFatalError(null, "Received fetchFailure for an unknown src (null)"); } else { - InputReadErrorEvent readError = InputReadErrorEvent.create( - "Fetch failure while fetching from " - + TezRuntimeUtils.getTaskAttemptIdentifier( - inputContext.getSourceVertexName(), - srcAttemptIdentifier.getInputIdentifier(), - srcAttemptIdentifier.getAttemptNumber()), - srcAttemptIdentifier.getInputIdentifier(), - srcAttemptIdentifier.getAttemptNumber()); - - List failedEvents = Lists.newArrayListWithCapacity(1); - failedEvents.add(readError); - inputContext.sendEvents(failedEvents); + InputReadErrorEvent readError = InputReadErrorEvent.create( + "Fetch failure while fetching from " + + TezRuntimeUtils.getTaskAttemptIdentifier( + inputContext.getSourceVertexName(), + srcAttemptIdentifier.getInputIdentifier(), + srcAttemptIdentifier.getAttemptNumber()), + srcAttemptIdentifier.getInputIdentifier(), + srcAttemptIdentifier.getAttemptNumber()); + if (maxTimeToWaitForReportMillis > 0) { + try { + reportLock.lock(); + failedEvents.merge(readError, 1, (a, b) -> a + b); + reportCondition.signal(); + } finally { + reportLock.unlock(); + } + } else { + List events = Lists.newArrayListWithCapacity(1); + events.add(readError); + inputContext.sendEvents(events); + } } } /////////////////// End of Methods from FetcherCallbackHandler @@ -849,6 +944,10 @@ public void shutdown() throws InterruptedException { if (this.schedulerExecutor != null && !this.schedulerExecutor.isShutdown()) { this.schedulerExecutor.shutdownNow(); } + if (this.reporterExecutor != null + && !this.reporterExecutor.isShutdown()) { + this.reporterExecutor.shutdownNow(); + } if (this.fetcherExecutor != null && !this.fetcherExecutor.isShutdown()) { this.fetcherExecutor.shutdownNow(); // Interrupts all running fetchers. } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java index 103f83d3cc..94f7f5a487 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java @@ -20,7 +20,9 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyInt; +import static org.mockito.Matchers.anyList; import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doAnswer; @@ -35,6 +37,7 @@ import java.io.InputStream; import java.io.OutputStream; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.LinkedList; import java.util.List; import java.util.concurrent.ExecutorService; @@ -57,6 +60,7 @@ import org.apache.tez.runtime.api.ExecutionContext; import org.apache.tez.runtime.api.InputContext; import org.apache.tez.runtime.api.events.DataMovementEvent; +import org.apache.tez.runtime.api.events.InputReadErrorEvent; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; import org.apache.tez.runtime.library.common.shuffle.FetchedInput; @@ -67,8 +71,10 @@ import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.mockito.ArgumentCaptor; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -214,6 +220,64 @@ public void run() { verify(inputContext, atLeast(3)).notifyProgress(); } + @Test (timeout = 200000) + public void testFetchFailed() throws Exception { + InputContext inputContext = createInputContext(); + final ShuffleManager shuffleManager = spy(createShuffleManager(inputContext, 1)); + Thread schedulerGetHostThread = new Thread(new Runnable() { + @Override + public void run() { + try { + shuffleManager.run(); + } catch (Exception e) { + e.printStackTrace(); + } + } + }); + InputAttemptIdentifier inputAttemptIdentifier + = new InputAttemptIdentifier(1, 1); + + schedulerGetHostThread.start(); + Thread.sleep(1000); + shuffleManager.fetchFailed("host1", inputAttemptIdentifier, false); + Thread.sleep(1000); + + ArgumentCaptor captor = ArgumentCaptor.forClass(List.class); + verify(inputContext, times(1)) + .sendEvents(captor.capture()); + Assert.assertEquals("Size was: " + captor.getAllValues().size(), + captor.getAllValues().size(), 1); + List capturedList = captor.getAllValues().get(0); + Assert.assertEquals("Size was: " + capturedList.size(), + capturedList.size(), 1); + InputReadErrorEvent inputEvent = (InputReadErrorEvent)capturedList.get(0); + Assert.assertEquals("Number of failures was: " + inputEvent.getNumFailures(), + inputEvent.getNumFailures(), 1); + + shuffleManager.fetchFailed("host1", inputAttemptIdentifier, false); + shuffleManager.fetchFailed("host1", inputAttemptIdentifier, false); + + Thread.sleep(1000); + verify(inputContext, times(1)).sendEvents(any()); + + // Wait more than five seconds for the batch to go out + Thread.sleep(5000); + captor = ArgumentCaptor.forClass(List.class); + verify(inputContext, times(2)) + .sendEvents(captor.capture()); + Assert.assertEquals("Size was: " + captor.getAllValues().size(), + captor.getAllValues().size(), 2); + capturedList = captor.getAllValues().get(1); + Assert.assertEquals("Size was: " + capturedList.size(), + capturedList.size(), 1); + inputEvent = (InputReadErrorEvent)capturedList.get(0); + Assert.assertEquals("Number of failures was: " + inputEvent.getNumFailures(), + inputEvent.getNumFailures(), 2); + + + schedulerGetHostThread.interrupt(); + } + private ShuffleManagerForTest createShuffleManager( InputContext inputContext, int expectedNumOfPhysicalInputs) throws IOException { @@ -222,6 +286,8 @@ private ShuffleManagerForTest createShuffleManager( doReturn(outDirs).when(inputContext).getWorkDirs(); conf.setStrings(TezRuntimeFrameworkConfigs.LOCAL_DIRS, inputContext.getWorkDirs()); + // 5 seconds + conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_BATCH_WAIT, 5000); DataOutputBuffer out = new DataOutputBuffer(); Token token = new Token(new JobTokenIdentifier(), From d4a62deee9e258ba1a3883b02c4c4446629933ae Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Fri, 26 Oct 2018 14:42:06 -0500 Subject: [PATCH 130/512] TEZ-4012. Add docker support for Tez. (Jonathan Eagles via kshukla) --- build-tools/docker/Dockerfile | 235 +++++++++++++++++++++++++++ build-tools/docker/tez_env_checks.sh | 117 +++++++++++++ 2 files changed, 352 insertions(+) create mode 100644 build-tools/docker/Dockerfile create mode 100644 build-tools/docker/tez_env_checks.sh diff --git a/build-tools/docker/Dockerfile b/build-tools/docker/Dockerfile new file mode 100644 index 0000000000..8d76861b5a --- /dev/null +++ b/build-tools/docker/Dockerfile @@ -0,0 +1,235 @@ + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Dockerfile for installing the necessary dependencies for building Hadoop. +# See BUILDING.txt. + +FROM ubuntu:xenial + +WORKDIR /root + +SHELL ["/bin/bash", "-o", "pipefail", "-c"] + +##### +# Disable suggests/recommends +##### +RUN echo APT::Install-Recommends "0"\; > /etc/apt/apt.conf.d/10disableextras +RUN echo APT::Install-Suggests "0"\; >> /etc/apt/apt.conf.d/10disableextras + +ENV DEBIAN_FRONTEND noninteractive +ENV DEBCONF_TERSE true + +###### +# Install common dependencies from packages. Versions here are either +# sufficient or irrelevant. +# +# WARNING: DO NOT PUT JAVA APPS HERE! Otherwise they will install default +# Ubuntu Java. See Java section below! +###### +# hadolint ignore=DL3008 +RUN apt-get -q update \ + && apt-get -q install -y --no-install-recommends \ + apt-utils \ + build-essential \ + bzip2 \ + clang \ + curl \ + doxygen \ + fuse \ + g++ \ + gcc \ + git \ + gnupg-agent \ + libbz2-dev \ + libcurl4-openssl-dev \ + libfuse-dev \ + libprotobuf-dev \ + libprotoc-dev \ + libsasl2-dev \ + libsnappy-dev \ + libssl-dev \ + libtool \ + locales \ + make \ + pinentry-curses \ + pkg-config \ + python \ + python2.7 \ + python-pip \ + python-pkg-resources \ + python-setuptools \ + python-wheel \ + rsync \ + software-properties-common \ + snappy \ + sudo \ + valgrind \ + zlib1g-dev \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* + + +####### +# OpenJDK 8 +####### +# hadolint ignore=DL3008 +RUN apt-get -q update \ + && apt-get -q install -y --no-install-recommends openjdk-8-jdk libbcprov-java \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* + + +###### +# Install cmake 3.1.0 (3.5.1 ships with Xenial) +###### +RUN mkdir -p /opt/cmake \ + && curl -L -s -S \ + https://cmake.org/files/v3.1/cmake-3.1.0-Linux-x86_64.tar.gz \ + -o /opt/cmake.tar.gz \ + && tar xzf /opt/cmake.tar.gz --strip-components 1 -C /opt/cmake +ENV CMAKE_HOME /opt/cmake +ENV PATH "${PATH}:/opt/cmake/bin" + +###### +# Install Google Protobuf 2.5.0 (2.6.0 ships with Xenial) +###### +# hadolint ignore=DL3003 +RUN mkdir -p /opt/protobuf-src \ + && curl -L -s -S \ + https://github.com/google/protobuf/releases/download/v2.5.0/protobuf-2.5.0.tar.gz \ + -o /opt/protobuf.tar.gz \ + && tar xzf /opt/protobuf.tar.gz --strip-components 1 -C /opt/protobuf-src \ + && cd /opt/protobuf-src \ + && ./configure --prefix=/opt/protobuf \ + && make install \ + && cd /root \ + && rm -rf /opt/protobuf-src +ENV PROTOBUF_HOME /opt/protobuf +ENV PATH "${PATH}:/opt/protobuf/bin" + +###### +# Install Apache Maven 3.3.9 (3.3.9 ships with Xenial) +###### +# hadolint ignore=DL3008 +RUN apt-get -q update \ + && apt-get -q install -y --no-install-recommends maven \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* +ENV MAVEN_HOME /usr + +###### +# Install findbugs 3.0.1 (3.0.1 ships with Xenial) +# Ant is needed for findbugs +###### +# hadolint ignore=DL3008 +RUN apt-get -q update \ + && apt-get -q install -y --no-install-recommends findbugs ant \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* +ENV FINDBUGS_HOME /usr + +#### +# Install shellcheck (0.4.6, the latest as of 2017-09-26) +#### +# hadolint ignore=DL3008 +RUN add-apt-repository -y ppa:jonathonf/ghc-8.0.2 \ + && apt-get -q update \ + && apt-get -q install -y --no-install-recommends shellcheck \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* + +#### +# Install bats (0.4.0, the latest as of 2017-09-26, ships with Xenial) +#### +# hadolint ignore=DL3008 +RUN apt-get -q update \ + && apt-get -q install -y --no-install-recommends bats \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* + +#### +# Install pylint at fixed version (2.0.0 removed python2 support) +# https://github.com/PyCQA/pylint/issues/2294 +#### +RUN pip2 install pylint==1.9.2 + +#### +# Install dateutil.parser +#### +RUN pip2 install python-dateutil==2.7.3 + +### +# Install node.js for web UI framework (4.2.6 ships with Xenial) +### +# hadolint ignore=DL3008, DL3016 +RUN apt-get -q update \ + && apt-get install -y --no-install-recommends nodejs npm \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* \ + && ln -s /usr/bin/nodejs /usr/bin/node \ + && npm install npm@latest -g \ + && npm install -g jshint + +### +# Install hadolint +#### +RUN curl -L -s -S \ + https://github.com/hadolint/hadolint/releases/download/v1.11.1/hadolint-Linux-x86_64 \ + -o /bin/hadolint \ + && chmod a+rx /bin/hadolint \ + && shasum -a 512 /bin/hadolint | \ + awk '$1!="734e37c1f6619cbbd86b9b249e69c9af8ee1ea87a2b1ff71dccda412e9dac35e63425225a95d71572091a3f0a11e9a04c2fc25d9e91b840530c26af32b9891ca" {exit(1)}' + +### +# Avoid out of memory errors in builds +### +ENV MAVEN_OPTS -Xms256m -Xmx1536m + + +### +# Everything past this point is either not needed for testing or breaks Yetus. +# So tell Yetus not to read the rest of the file: +# YETUS CUT HERE +### + +#### +# Install svn & Forrest (for Apache Hadoop website) +### +# hadolint ignore=DL3008 +RUN apt-get -q update \ + && apt-get -q install -y --no-install-recommends subversion \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* + +RUN mkdir -p /opt/apache-forrest \ + && curl -L -s -S \ + https://archive.apache.org/dist/forrest/0.8/apache-forrest-0.8.tar.gz \ + -o /opt/forrest.tar.gz \ + && tar xzf /opt/forrest.tar.gz --strip-components 1 -C /opt/apache-forrest +RUN echo 'forrest.home=/opt/apache-forrest' > build.properties +ENV FORREST_HOME=/opt/apache-forrest + +# Hugo static website generator (for new tez site and Ozone docs) +RUN curl -L -o hugo.deb https://github.com/gohugoio/hugo/releases/download/v0.30.2/hugo_0.30.2_Linux-64bit.deb \ + && dpkg --install hugo.deb \ + && rm hugo.deb + +# Add a welcome message and environment checks. +COPY tez_env_checks.sh /root/tez_env_checks.sh +RUN chmod 755 /root/tez_env_checks.sh +# hadolint ignore=SC2016 +RUN echo '${HOME}/tez_env_checks.sh' >> /root/.bashrc diff --git a/build-tools/docker/tez_env_checks.sh b/build-tools/docker/tez_env_checks.sh new file mode 100644 index 0000000000..c9420e79e9 --- /dev/null +++ b/build-tools/docker/tez_env_checks.sh @@ -0,0 +1,117 @@ +#!/usr/bin/env bash + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# SHELLDOC-IGNORE + +# ------------------------------------------------------- +function showWelcome { +cat < Date: Tue, 30 Oct 2018 08:51:02 -0500 Subject: [PATCH 131/512] TEZ-4004. Update jetty9 to align with Hadoop and Hive (Jonathan Eagles via kshukla) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 86402ff90b..9d8797c256 100644 --- a/pom.xml +++ b/pom.xml @@ -38,7 +38,7 @@ true ${user.home}/clover.license 3.0.3 - 9.3.22.v20171030 + 9.3.24.v20180605 3.10.5.Final 0.13.0 1.19 From e6722a96b4827d19850a5a73bc7024941a9deb54 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Tue, 20 Nov 2018 16:48:59 -0600 Subject: [PATCH 132/512] TEZ-3998. Allow CONCURRENT edge property in DAG construction and introduce ConcurrentSchedulingType (Yingda Chen via jeagles) --- .../main/java/org/apache/tez/dag/api/DAG.java | 88 +++---- .../org/apache/tez/dag/api/EdgeProperty.java | 64 +++-- .../apache/tez/dag/api/TezConfiguration.java | 128 +++++---- .../org/apache/tez/dag/api/TestDAGVerify.java | 92 +++---- .../org/apache/tez/dag/app/DAGAppMaster.java | 70 +++-- .../edgemanager/SilentEdgeManager.java | 89 +++++++ .../VertexManagerWithConcurrentInput.java | 245 ++++++++++++++++++ .../TestVertexManagerWithConcurrentInput.java | 114 ++++++++ 8 files changed, 694 insertions(+), 196 deletions(-) create mode 100644 tez-runtime-library/src/main/java/org/apache/tez/dag/library/edgemanager/SilentEdgeManager.java create mode 100644 tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/VertexManagerWithConcurrentInput.java create mode 100644 tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestVertexManagerWithConcurrentInput.java diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java index 735c749ee5..f8a2ddc5dc 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java @@ -79,16 +79,16 @@ import com.google.common.collect.Sets; /** - * Top level entity that defines the DAG (Directed Acyclic Graph) representing - * the data flow graph. Consists of a set of Vertices and Edges connecting the - * vertices. Vertices represent transformations of data and edges represent + * Top level entity that defines the DAG (Directed Acyclic Graph) representing + * the data flow graph. Consists of a set of Vertices and Edges connecting the + * vertices. Vertices represent transformations of data and edges represent * movement of data between vertices. */ @Public public class DAG { - + private static final Logger LOG = LoggerFactory.getLogger(DAG.class); - + final BidiMap vertices = new DualLinkedHashBidiMap(); final Set edges = Sets.newHashSet(); @@ -132,7 +132,7 @@ public synchronized DAG addTaskLocalFiles(Map localFiles) TezCommonUtils.addAdditionalLocalResources(localFiles, commonTaskLocalFiles, "DAG " + getName()); return this; } - + public synchronized DAG addVertex(Vertex vertex) { if (vertices.containsKey(vertex.getName())) { throw new IllegalStateException( @@ -145,18 +145,18 @@ public synchronized DAG addVertex(Vertex vertex) { public synchronized Vertex getVertex(String vertexName) { return vertices.get(vertexName); } - + /** * One of the methods that can be used to provide information about required * Credentials when running on a secure cluster. A combination of this and * addURIsForCredentials should be used to specify information about all * credentials required by a DAG. AM specific credentials are not used when * executing a DAG. - * + * * Set credentials which will be required to run this dag. This method can be * used if the client has already obtained some or all of the required * credentials. - * + * * @param credentials Credentials for the DAG * @return {@link DAG} */ @@ -196,7 +196,7 @@ public synchronized DAG setCallerContext(CallerContext callerContext) { } /** - * Create a group of vertices that share a common output. This can be used to implement + * Create a group of vertices that share a common output. This can be used to implement * unions efficiently. * @param name Name of the group. * @param members {@link Vertex} members of the group @@ -243,15 +243,15 @@ public synchronized DAGAccessControls getDagAccessControls() { * setCredentials should be used to specify information about all credentials * required by a DAG. AM specific credentials are not used when executing a * DAG. - * + * * This method can be used to specify a list of URIs for which Credentials * need to be obtained so that the job can run. An incremental list of URIs * can be provided by making multiple calls to the method. - * + * * Currently, @{link credentials} can only be fetched for HDFS and other * {@link org.apache.hadoop.fs.FileSystem} implementations that support * credentials. - * + * * @param uris * a list of {@link URI}s * @return {@link DAG} @@ -263,7 +263,7 @@ public synchronized DAG addURIsForCredentials(Collection uris) { } /** - * + * * @return an unmodifiable list representing the URIs for which credentials * are required. */ @@ -271,7 +271,7 @@ public synchronized DAG addURIsForCredentials(Collection uris) { public synchronized Collection getURIsForCredentials() { return Collections.unmodifiableCollection(urisForCredentials); } - + @Private public synchronized Set getVertices() { return Collections.unmodifiableSet(this.vertices.values()); @@ -304,7 +304,7 @@ public synchronized DAG addEdge(Edge edge) { edges.add(edge); return this; } - + /** * Add a {@link GroupInputEdge} to the DAG. * @param edge {@link GroupInputEdge} @@ -328,7 +328,7 @@ public synchronized DAG addEdge(GroupInputEdge edge) { VertexGroup av = edge.getInputVertexGroup(); av.addOutputVertex(edge.getOutputVertex(), edge); groupInputEdges.add(edge); - + // add new edge between members of VertexGroup and destVertex of the GroupInputEdge List newEdges = Lists.newLinkedList(); Vertex dstVertex = edge.getOutputVertex(); @@ -337,14 +337,14 @@ public synchronized DAG addEdge(GroupInputEdge edge) { newEdges.add(Edge.create(member, dstVertex, edge.getEdgeProperty())); } dstVertex.addGroupInput(uv.getGroupName(), uv.getGroupInfo()); - + for (Edge e : newEdges) { addEdge(e); } - + return this; } - + /** * Get the DAG name * @return DAG name @@ -433,7 +433,7 @@ void checkAndInferOneToOneParallelism() { newKnownTasksVertices.add(vertex); } } - + // walk through all known source 1-1 edges and infer parallelism // add newly inferred vertices for consideration as known sources // the outer loop will run for every new level of inferring the parallelism @@ -456,19 +456,19 @@ void checkAndInferOneToOneParallelism() { } } } - + // check for inconsistency and errors for (Edge e : edges) { Vertex inputVertex = e.getInputVertex(); Vertex outputVertex = e.getOutputVertex(); - + if (e.getEdgeProperty().getDataMovementType() == DataMovementType.ONE_TO_ONE) { if (inputVertex.getParallelism() != outputVertex.getParallelism()) { // both should be equal or equal to -1. if (outputVertex.getParallelism() != -1) { throw new TezUncheckedException( "1-1 Edge. Destination vertex parallelism must match source vertex. " - + "Vertex: " + inputVertex.getName() + " does not match vertex: " + + "Vertex: " + inputVertex.getName() + " does not match vertex: " + outputVertex.getName()); } } @@ -527,7 +527,7 @@ void checkAndInferOneToOneParallelism() { } } } - + // AnnotatedVertex is used by verify() private static class AnnotatedVertex { Vertex v; @@ -573,7 +573,7 @@ Deque verify(boolean restricted) throws IllegalStateException { if (vertices.isEmpty()) { throw new IllegalStateException("Invalid dag containing 0 vertices"); } - + // check for valid vertices, duplicate vertex names, // and prepare for cycle detection Map vertexMap = new HashMap(); @@ -591,14 +591,14 @@ Deque verify(boolean restricted) throws IllegalStateException { for (Edge e : edges) { // Construct structure for cycle detection Vertex inputVertex = e.getInputVertex(); - Vertex outputVertex = e.getOutputVertex(); + Vertex outputVertex = e.getOutputVertex(); List edgeList = edgeMap.get(inputVertex); if (edgeList == null) { edgeList = new ArrayList(); edgeMap.put(inputVertex, edgeList); } edgeList.add(e); - + // Construct map for Input name verification Set inboundSet = inboundVertexMap.get(outputVertex); if (inboundSet == null) { @@ -606,7 +606,7 @@ Deque verify(boolean restricted) throws IllegalStateException { inboundVertexMap.put(outputVertex, inboundSet); } inboundSet.add(inputVertex.getName()); - + // Construct map for Output name verification Set outboundSet = outboundVertexMap.get(inputVertex); if (outboundSet == null) { @@ -618,7 +618,7 @@ Deque verify(boolean restricted) throws IllegalStateException { // check input and output names don't collide with vertex names for (Vertex vertex : vertices.values()) { - for (RootInputLeafOutput + for (RootInputLeafOutput input : vertex.getInputs()) { if (vertexMap.containsKey(input.getName())) { throw new IllegalStateException("Vertex: " @@ -627,7 +627,7 @@ Deque verify(boolean restricted) throws IllegalStateException { + input.getName()); } } - for (RootInputLeafOutput + for (RootInputLeafOutput output : vertex.getOutputs()) { if (vertexMap.containsKey(output.getName())) { throw new IllegalStateException("Vertex: " @@ -641,7 +641,7 @@ Deque verify(boolean restricted) throws IllegalStateException { // Check for valid InputNames for (Entry> entry : inboundVertexMap.entrySet()) { Vertex vertex = entry.getKey(); - for (RootInputLeafOutput + for (RootInputLeafOutput input : vertex.getInputs()) { if (entry.getValue().contains(input.getName())) { throw new IllegalStateException("Vertex: " @@ -655,7 +655,7 @@ Deque verify(boolean restricted) throws IllegalStateException { // Check for valid OutputNames for (Entry> entry : outboundVertexMap.entrySet()) { Vertex vertex = entry.getKey(); - for (RootInputLeafOutput + for (RootInputLeafOutput output : vertex.getOutputs()) { if (entry.getValue().contains(output.getName())) { throw new IllegalStateException("Vertex: " @@ -665,8 +665,8 @@ Deque verify(boolean restricted) throws IllegalStateException { } } } - - + + // Not checking for repeated input names / output names vertex names on the same vertex, // since we only allow 1 at the moment. // When additional inputs are supported, this can be chceked easily (and early) @@ -678,16 +678,12 @@ Deque verify(boolean restricted) throws IllegalStateException { if (restricted) { for (Edge e : edges) { - if (e.getEdgeProperty().getDataSourceType() != - DataSourceType.PERSISTED) { + DataSourceType dataSourceType = e.getEdgeProperty().getDataSourceType(); + if (dataSourceType != DataSourceType.PERSISTED && + dataSourceType != DataSourceType.EPHEMERAL) { throw new IllegalStateException( "Unsupported source type on edge. " + e); } - if (e.getEdgeProperty().getSchedulingType() != - SchedulingType.SEQUENTIAL) { - throw new IllegalStateException( - "Unsupported scheduling type on edge. " + e); - } } } @@ -878,13 +874,13 @@ public synchronized DAGPlan createDag(Configuration tezConf, Credentials extraCr groupBuilder.addGroupMembers(v.getName()); } groupBuilder.addAllOutputs(groupInfo.outputs); - for (Map.Entry entry : + for (Map.Entry entry : groupInfo.edgeMergedInputs.entrySet()) { groupBuilder.addEdgeMergedInputs( PlanGroupInputEdgeInfo.newBuilder().setDestVertexName(entry.getKey()). setMergedInput(DagTypeConverters.convertToDAGPlan(entry.getValue()))); } - dagBuilder.addVertexGroups(groupBuilder); + dagBuilder.addVertexGroups(groupBuilder); } } @@ -956,7 +952,7 @@ public synchronized DAGPlan createDag(Configuration tezConf, Credentials extraCr dagCredentials.addAll(dataSink.getCredentials()); } } - + VertexPlan.Builder vertexBuilder = VertexPlan.newBuilder(); vertexBuilder.setName(vertex.getName()); vertexBuilder.setType(PlanVertexType.NORMAL); // vertex type is implicitly NORMAL until TEZ-46. @@ -1045,7 +1041,7 @@ public synchronized DAGPlan createDag(Configuration tezConf, Credentials extraCr } } } - + if (vertex.getVertexManagerPlugin() != null) { vertexBuilder.setVertexManagerPlugin(DagTypeConverters .convertToDAGPlan(vertex.getVertexManagerPlugin())); diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java b/tez-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java index 07fb2c140f..c203f8c6b6 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java @@ -43,7 +43,7 @@ public class EdgeProperty { */ public enum DataMovementType { /** - * Output on this edge produced by the i-th source task is available to the + * Output on this edge produced by the i-th source task is available to the * i-th destination task. */ ONE_TO_ONE, @@ -58,20 +58,20 @@ public enum DataMovementType { * are gathered by designated destination tasks. */ SCATTER_GATHER, - + /** * Custom routing defined by the user. */ CUSTOM } - + /** * Determines the lifetime of the data produced on this edge by a source task. */ public enum DataSourceType { /** * Data produced by the source is persisted and available even when the - * task is not running. The data may become unavailable and may cause the + * task is not running. The data may become unavailable and may cause the * source task to be re-executed. */ PERSISTED, @@ -82,31 +82,51 @@ public enum DataSourceType { PERSISTED_RELIABLE, /** * Data produced by the source task is available only while the source task - * is running. This requires the destination task to run concurrently with - * the source task. This is not supported yet. + * is running. This requires the destination task to run concurrently with + * the source task. Development in progress. */ @Unstable EPHEMERAL } - + /** - * Determines when the destination task is eligible to run, once the source + * Determines when the destination task is eligible to run, once the source * task is eligible to run. */ public enum SchedulingType { /** - * Destination task is eligible to run after one or more of its source tasks + * Destination task is eligible to run after one or more of its source tasks * have started or completed. */ SEQUENTIAL, /** * Destination task must run concurrently with the source task. - * This is not supported yet. + * Development in progress. */ @Unstable CONCURRENT } - + + /** + * Determines the relevant event(s) that will assist in scheduling downstream vertex + * connected via a edge with CONCURRENT {@link SchedulingType}. + */ + public enum ConcurrentEdgeTriggerType { + /** + * trigger tasks scheduling for downstream vertex(es) upon upstream being configured + * this effectively simultaneously schedules downstream and upstream vertices + * connected on both ends of a concurrent edge. + */ + SOURCE_VERTEX_CONFIGURED, + + /** + * trigger tasks scheduling for downstream vertex(es) by "running" event(s) of upstream tasks + * this will be fully supported with TEZ-3999 + */ + SOURCE_TASK_STARTED + } + + final DataMovementType dataMovementType; final DataSourceType dataSourceType; final SchedulingType schedulingType; @@ -172,7 +192,7 @@ private EdgeProperty(DataMovementType dataMovementType, Preconditions.checkArgument(dataMovementType != DataMovementType.CUSTOM, DataMovementType.CUSTOM + " cannot be used with this constructor"); } - + private EdgeProperty(EdgeManagerPluginDescriptor edgeManagerDescriptor, DataSourceType dataSourceType, @@ -182,7 +202,7 @@ private EdgeProperty(EdgeManagerPluginDescriptor edgeManagerDescriptor, this(edgeManagerDescriptor, DataMovementType.CUSTOM, dataSourceType, schedulingType, edgeSource, edgeDestination); } - + private EdgeProperty(EdgeManagerPluginDescriptor edgeManagerDescriptor, DataMovementType dataMovementType, DataSourceType dataSourceType, SchedulingType schedulingType, OutputDescriptor edgeSource, InputDescriptor edgeDestination) { @@ -193,7 +213,7 @@ private EdgeProperty(EdgeManagerPluginDescriptor edgeManagerDescriptor, this.inputDescriptor = edgeDestination; this.outputDescriptor = edgeSource; } - + /** * Get the {@link DataMovementType} * @return {@link DataMovementType} @@ -201,7 +221,7 @@ private EdgeProperty(EdgeManagerPluginDescriptor edgeManagerDescriptor, public DataMovementType getDataMovementType() { return dataMovementType; } - + /** * Get the {@link DataSourceType} * @return {@link DataSourceType} @@ -209,7 +229,7 @@ public DataMovementType getDataMovementType() { public DataSourceType getDataSourceType() { return dataSourceType; } - + /** * Get the {@link SchedulingType} * @return {@link SchedulingType} @@ -217,30 +237,30 @@ public DataSourceType getDataSourceType() { public SchedulingType getSchedulingType() { return schedulingType; } - + /** * @return the {@link InputDescriptor} which will consume data from the edge. */ public InputDescriptor getEdgeDestination() { return inputDescriptor; } - + /** * @return the {@link OutputDescriptor} which produces data on the edge. */ public OutputDescriptor getEdgeSource() { return outputDescriptor; } - + /** - * Returns the Edge Manager specifications for this edge. + * Returns the Edge Manager specifications for this edge. * @return @link {@link EdgeManagerPluginDescriptor} if a custom edge was setup, null otherwise. */ @Private public EdgeManagerPluginDescriptor getEdgeManagerDescriptor() { return edgeManagerDescriptor; } - + @Override public String toString() { return "{ " + dataMovementType + " : " + inputDescriptor.getClassName() @@ -248,5 +268,5 @@ public String toString() { + " >> " + (edgeManagerDescriptor == null ? "NullEdgeManager" : edgeManagerDescriptor.getClassName()) + " }"; } - + } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 43014a4dfc..791e634c90 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -28,6 +28,7 @@ import org.apache.tez.common.annotation.ConfigurationClass; import org.apache.tez.common.annotation.ConfigurationProperty; +import org.apache.tez.dag.api.EdgeProperty.ConcurrentEdgeTriggerType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience.Private; @@ -41,7 +42,7 @@ /** - * Defines the configurations for Tez. These configurations are typically specified in + * Defines the configurations for Tez. These configurations are typically specified in * tez-site.xml on the client machine where TezClient is used to launch the Tez application. * tez-site.xml is expected to be picked up from the classpath of the client process. * @see Detailed Configuration Information @@ -131,7 +132,7 @@ public TezConfiguration(boolean loadDefaults) { public static final String TEZ_TASK_PREFIX = TEZ_PREFIX + "task."; /** - * Boolean value. If true then Tez will try to automatically delete temporary job + * Boolean value. If true then Tez will try to automatically delete temporary job * artifacts that it creates within the specified staging dir. Does not affect any user data. */ @ConfigurationScope(Scope.AM) @@ -183,7 +184,7 @@ public TezConfiguration(boolean loadDefaults) { + "use.concurrent-dispatcher"; @Private public static boolean TEZ_AM_USE_CONCURRENT_DISPATCHER_DEFAULT = false; - + @Private @ConfigurationScope(Scope.AM) public static final String TEZ_AM_CONCURRENT_DISPATCHER_CONCURRENCY = TEZ_AM_PREFIX @@ -196,7 +197,7 @@ public TezConfiguration(boolean loadDefaults) { * code is written according to best practices then the same code can execute in either mode based * on this configuration. Session mode is more aggressive in reserving execution resources and is * typically used for interactive applications where multiple DAGs are submitted in quick succession - * by the same user. For long running applications, one-off executions, batch jobs etc non-session + * by the same user. For long running applications, one-off executions, batch jobs etc non-session * mode is recommended. If session mode is enabled then container reuse is recommended. */ @ConfigurationScope(Scope.AM) @@ -271,12 +272,12 @@ public TezConfiguration(boolean loadDefaults) { public static final int TEZ_TASK_MAX_ALLOWED_OUTPUT_FAILURES_DEFAULT = 10; /** - * int value. Represents the maximum time in seconds for which a consumer attempt can report - * a read error against its producer attempt, after which the producer attempt will be re-run - * to re-generate the output. There are other heuristics which determine the retry and mainly - * try to guard against a flurry of re-runs due to intermittent read errors + * int value. Represents the maximum time in seconds for which a consumer attempt can report + * a read error against its producer attempt, after which the producer attempt will be re-run + * to re-generate the output. There are other heuristics which determine the retry and mainly + * try to guard against a flurry of re-runs due to intermittent read errors * (due to network issues). This configuration puts a time limit on those heuristics to ensure - * jobs dont hang indefinitely due to lack of closure in those heuristics + * jobs dont hang indefinitely due to lack of closure in those heuristics * * Expert level setting. */ @@ -288,9 +289,9 @@ public TezConfiguration(boolean loadDefaults) { /** * Boolean value. Determines when the final outputs to data sinks are committed. Commit is an - * output specific operation and typically involves making the output visible for consumption. - * If the config is true, then the outputs are committed at the end of DAG completion after all - * constituent vertices have completed. If false, outputs for each vertex are committed after that + * output specific operation and typically involves making the output visible for consumption. + * If the config is true, then the outputs are committed at the end of DAG completion after all + * constituent vertices have completed. If false, outputs for each vertex are committed after that * vertex succeeds. Depending on the desired output visibility and downstream consumer dependencies * this value must be appropriately chosen. Defaults to the safe choice of true. */ @@ -330,7 +331,7 @@ public TezConfiguration(boolean loadDefaults) { @ConfigurationScope(Scope.AM) @ConfigurationProperty public static final String TEZ_AM_LAUNCH_CMD_OPTS = TEZ_AM_PREFIX + "launch.cmd-opts"; - public static final String TEZ_AM_LAUNCH_CMD_OPTS_DEFAULT = + public static final String TEZ_AM_LAUNCH_CMD_OPTS_DEFAULT = "-XX:+PrintGCDetails -verbose:gc -XX:+PrintGCTimeStamps -XX:+UseNUMA -XX:+UseParallelGC"; /** @@ -408,6 +409,19 @@ public TezConfiguration(boolean loadDefaults) { + "launch.env"; public static final String TEZ_AM_LAUNCH_ENV_DEFAULT = ""; + /** + * String value. In the presence of concurrent input edge to a vertex, this describes + * the timing of scheduling downstream vertex tasks. It may be closely related to the + * type of event that will contribute to a scheduling decision. + */ + @ConfigurationScope(Scope.VERTEX) + @ConfigurationProperty + public static final String TEZ_CONCURRENT_EDGE_TRIGGER_TYPE = + TEZ_TASK_PREFIX + "concurrent.edge.trigger.type"; + public static final String TEZ_CONCURRENT_EDGE_TRIGGER_TYPE_DEFAULT = + ConcurrentEdgeTriggerType.SOURCE_VERTEX_CONFIGURED.name(); + + /** * String value. Env settings will be merged with {@link #TEZ_TASK_LAUNCH_ENV} * during the launch of the task process. This property will typically be configured to @@ -508,16 +522,16 @@ public TezConfiguration(boolean loadDefaults) { @Unstable /** - * Boolean value. Enable speculative execution of slower tasks. This can help reduce job latency + * Boolean value. Enable speculative execution of slower tasks. This can help reduce job latency * when some tasks are running slower due bad/slow machines */ @ConfigurationScope(Scope.VERTEX) // TODO Verify the vertex speculation, TEZ-1788 @ConfigurationProperty(type="boolean") public static final String TEZ_AM_SPECULATION_ENABLED = TEZ_AM_PREFIX + "speculation.enabled"; public static final boolean TEZ_AM_SPECULATION_ENABLED_DEFAULT = false; - + /** - * Float value. Specifies how many standard deviations away from the mean task execution time + * Float value. Specifies how many standard deviations away from the mean task execution time * should be considered as an outlier/slow task. */ @Unstable @@ -539,14 +553,14 @@ public TezConfiguration(boolean loadDefaults) { /** * Int value. Upper limit on the number of threads user to launch containers in the app - * master. Expert level setting. + * master. Expert level setting. */ @ConfigurationScope(Scope.AM) @ConfigurationProperty(type="integer") public static final String TEZ_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT = TEZ_AM_PREFIX + "containerlauncher.thread-count-limit"; - public static final int TEZ_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT_DEFAULT = + public static final int TEZ_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT_DEFAULT = 500; @@ -560,8 +574,8 @@ public TezConfiguration(boolean loadDefaults) { public static final int TEZ_AM_MAX_TASK_FAILURES_PER_NODE_DEFAULT = 10; /** - * Int value. Specifies the number of times the app master can be launched in order to recover - * from app master failure. Typically app master failures are non-recoverable. This parameter + * Int value. Specifies the number of times the app master can be launched in order to recover + * from app master failure. Typically app master failures are non-recoverable. This parameter * is for cases where the app master is not at fault but is lost due to system errors. * Expert level setting. */ @@ -582,7 +596,7 @@ public TezConfiguration(boolean loadDefaults) { public static final int TEZ_AM_VERTEX_MAX_TASK_CONCURRENCY_DEFAULT = -1; /** - * Int value. The maximum number of attempts that can fail for a particular task before the task is failed. + * Int value. The maximum number of attempts that can fail for a particular task before the task is failed. * This does not count killed attempts. Task failure results in DAG failure. */ @ConfigurationScope(Scope.VERTEX) @@ -612,7 +626,7 @@ public TezConfiguration(boolean loadDefaults) { public static final boolean TEZ_AM_TASK_RESCHEDULE_RELAXED_LOCALITY_DEFAULT=true; /** - * Boolean value. Enabled blacklisting of nodes of nodes that are considered faulty. These nodes + * Boolean value. Enabled blacklisting of nodes of nodes that are considered faulty. These nodes * will not be used to execute tasks. */ @ConfigurationScope(Scope.AM) @@ -620,11 +634,11 @@ public TezConfiguration(boolean loadDefaults) { public static final String TEZ_AM_NODE_BLACKLISTING_ENABLED = TEZ_AM_PREFIX + "node-blacklisting.enabled"; public static final boolean TEZ_AM_NODE_BLACKLISTING_ENABLED_DEFAULT = true; - + /** * Int value. Specifies the percentage of nodes in the cluster that may be considered faulty. - * This limits the number of nodes that are blacklisted in an effort to minimize the effects of - * temporary surges in failures (e.g. due to network outages). + * This limits the number of nodes that are blacklisted in an effort to minimize the effects of + * temporary surges in failures (e.g. due to network outages). */ @ConfigurationScope(Scope.AM) @ConfigurationProperty(type="integer") @@ -651,7 +665,7 @@ public TezConfiguration(boolean loadDefaults) { public static final String TEZ_AM_CLIENT_THREAD_COUNT = TEZ_AM_PREFIX + "client.am.thread-count"; public static final int TEZ_AM_CLIENT_THREAD_COUNT_DEFAULT = 2; - + /** * String value. Range of ports that the AM can use when binding for client connections. Leave blank * to use all possible ports. Expert level setting. It's hadoop standard range configuration. @@ -721,7 +735,7 @@ public TezConfiguration(boolean loadDefaults) { public static final int TEZ_AM_DAG_CLEANUP_THREAD_COUNT_LIMIT_DEFAULT = 10; /** Int value. The amount of memory in MB to be used by tasks. This applies to all tasks across - * all vertices. Setting it to the same value for all tasks is helpful for container reuse and + * all vertices. Setting it to the same value for all tasks is helpful for container reuse and * thus good for performance typically. */ @ConfigurationScope(Scope.DAG) // TODO vertex level @ConfigurationProperty(type="integer") @@ -736,7 +750,7 @@ public TezConfiguration(boolean loadDefaults) { @ConfigurationProperty(type="integer") public static final String TEZ_TASK_RESOURCE_CPU_VCORES = TEZ_TASK_PREFIX + "resource.cpu.vcores"; - public static final int TEZ_TASK_RESOURCE_CPU_VCORES_DEFAULT = 1; + public static final int TEZ_TASK_RESOURCE_CPU_VCORES_DEFAULT = 1; /** * Int value. The maximum heartbeat interval between the AM and RM in milliseconds @@ -751,7 +765,7 @@ public TezConfiguration(boolean loadDefaults) { /** * Int value. The maximum amount of time, in milliseconds, to wait before a task asks an - * AM for another task. Increasing this can help improve app master scalability for a large + * AM for another task. Increasing this can help improve app master scalability for a large * number of concurrent tasks. Expert level setting. */ @ConfigurationScope(Scope.AM) @@ -761,7 +775,7 @@ public TezConfiguration(boolean loadDefaults) { public static final int TEZ_TASK_GET_TASK_SLEEP_INTERVAL_MS_MAX_DEFAULT = 200; /** - * Int value. The maximum heartbeat interval, in milliseconds, between the app master and tasks. + * Int value. The maximum heartbeat interval, in milliseconds, between the app master and tasks. * Increasing this can help improve app master scalability for a large number of concurrent tasks. * Expert level setting. */ @@ -772,8 +786,8 @@ public TezConfiguration(boolean loadDefaults) { public static final int TEZ_TASK_AM_HEARTBEAT_INTERVAL_MS_DEFAULT = 100; /** - * Int value. Interval, in milliseconds, after which counters are sent to AM in heartbeat from - * tasks. This reduces the amount of network traffice between AM and tasks to send high-volume + * Int value. Interval, in milliseconds, after which counters are sent to AM in heartbeat from + * tasks. This reduces the amount of network traffice between AM and tasks to send high-volume * counters. Improves AM scalability. Expert level setting. */ @ConfigurationScope(Scope.AM) @@ -792,7 +806,7 @@ public TezConfiguration(boolean loadDefaults) { public static final String TEZ_TASK_MAX_EVENTS_PER_HEARTBEAT = TEZ_TASK_PREFIX + "max-events-per-heartbeat"; public static final int TEZ_TASK_MAX_EVENTS_PER_HEARTBEAT_DEFAULT = 500; - + /** * Int value. Maximum number of pending task events before a task will stop * asking for more events in the task heartbeat. @@ -827,16 +841,16 @@ public TezConfiguration(boolean loadDefaults) { public static final boolean TEZ_TASK_INITIALIZE_PROCESSOR_IO_SERIALLY_DEFAULT = false; /** - * Long value. Interval, in milliseconds, within which any of the tasks Input/Processor/Output - * components need to make successive progress notifications. If the progress is not notified + * Long value. Interval, in milliseconds, within which any of the tasks Input/Processor/Output + * components need to make successive progress notifications. If the progress is not notified * for this interval then the task will be considered hung and terminated. - * The value for this config should be larger than {@link TezConfiguration#TASK_HEARTBEAT_TIMEOUT_MS} + * The value for this config should be larger than {@link TezConfiguration#TASK_HEARTBEAT_TIMEOUT_MS} * and larger than 2 times the value of {@link TezConfiguration#TEZ_TASK_AM_HEARTBEAT_INTERVAL_MS}. * A config value <=0 disables this. */ @ConfigurationScope(Scope.VERTEX) @ConfigurationProperty - public static final String TEZ_TASK_PROGRESS_STUCK_INTERVAL_MS = TEZ_TASK_PREFIX + + public static final String TEZ_TASK_PROGRESS_STUCK_INTERVAL_MS = TEZ_TASK_PREFIX + "progress.stuck.interval-ms"; public static final long TEZ_TASK_PROGRESS_STUCK_INTERVAL_MS_DEFAULT = -1; @@ -1010,7 +1024,7 @@ public TezConfiguration(boolean loadDefaults) { /** * Boolean value. Whether to reuse containers for non-local tasks. Active only if reuse is - * enabled. Turning this on can severely affect locality and can be bad for jobs with high data + * enabled. Turning this on can severely affect locality and can be bad for jobs with high data * volume being read from the primary data sources. */ @ConfigurationScope(Scope.AM) @@ -1047,15 +1061,15 @@ public TezConfiguration(boolean loadDefaults) { TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS_DEFAULT = 250l; /** - * Int value. The minimum amount of time to hold on to a container that is idle. Only active when - * reuse is enabled. Set to -1 to never release idle containers (not recommended). + * Int value. The minimum amount of time to hold on to a container that is idle. Only active when + * reuse is enabled. Set to -1 to never release idle containers (not recommended). */ @ConfigurationScope(Scope.AM) @ConfigurationProperty(type="integer") public static final String TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS = TEZ_AM_PREFIX + "container.idle.release-timeout-min.millis"; public static final long - TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS_DEFAULT = 5000l; + TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS_DEFAULT = 5000l; /** * Int value. The maximum amount of time to hold on to a container if no task can be @@ -1064,7 +1078,7 @@ public TezConfiguration(boolean loadDefaults) { * TezConfiguration#TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS. * Containers will have an expire time set to a random value between * TezConfiguration#TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS && - * TezConfiguration#TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS. This + * TezConfiguration#TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS. This * creates a graceful reduction in the amount of idle resources held */ @ConfigurationScope(Scope.AM) @@ -1073,9 +1087,9 @@ public TezConfiguration(boolean loadDefaults) { TEZ_AM_PREFIX + "container.idle.release-timeout-max.millis"; public static final long TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS_DEFAULT = 10000l; - + /** - * Int value. The minimum number of containers that will be held in session mode. Not active in + * Int value. The minimum number of containers that will be held in session mode. Not active in * non-session mode. Enables an idle session (not running any DAG) to hold on to a minimum number * of containers to provide fast response times for the next DAG. */ @@ -1086,7 +1100,7 @@ public TezConfiguration(boolean loadDefaults) { public static final int TEZ_AM_SESSION_MIN_HELD_CONTAINERS_DEFAULT = 0; /** - * Boolean value. Allow/disable logging for all dags in a session + * Boolean value. Allow/disable logging for all dags in a session */ @Private @ConfigurationScope(Scope.AM) @@ -1119,7 +1133,7 @@ public TezConfiguration(boolean loadDefaults) { public static final float TEZ_VERTEX_FAILURES_MAXPERCENT_DEFAULT = 0.0f; /** * Int value. The number of RM heartbeats to wait after preempting running tasks before preempting - * more running tasks. After preempting a task, we need to wait at least 1 heartbeat so that the + * more running tasks. After preempting a task, we need to wait at least 1 heartbeat so that the * RM can act on the released resources and assign new ones to us. Expert level setting. */ @ConfigurationScope(Scope.AM) @@ -1130,8 +1144,8 @@ public TezConfiguration(boolean loadDefaults) { /** * Int value. Time (in millisecs) that an unsatisfied request will wait before preempting other - * resources. In rare cases, the cluster says there are enough free resources but does not end - * up getting enough on a node to actually assign it to the job. This configuration tries to put + * resources. In rare cases, the cluster says there are enough free resources but does not end + * up getting enough on a node to actually assign it to the job. This configuration tries to put * a deadline on such wait to prevent indefinite job hangs. */ @ConfigurationScope(Scope.AM) @@ -1169,7 +1183,7 @@ public TezConfiguration(boolean loadDefaults) { * * Specify additional user classpath information to be used for Tez AM and all containers. * This will be appended to the classpath after PWD - * + * * 'tez.lib.uris.classpath' defines the relative classpath into the archives * that are set in 'tez.lib.uris' * @@ -1195,7 +1209,7 @@ public TezConfiguration(boolean loadDefaults) { public static final String TEZ_AUX_URIS = TEZ_PREFIX + "aux.uris"; /** - * Boolean value. Allows to ignore 'tez.lib.uris'. Useful during development as well as + * Boolean value. Allows to ignore 'tez.lib.uris'. Useful during development as well as * raw Tez application where classpath is propagated with application * via {@link LocalResource}s. This is mainly useful for developer/debugger scenarios. */ @@ -1261,8 +1275,8 @@ public TezConfiguration(boolean loadDefaults) { /** * Int value. Time (in seconds) to wait for AM to come up when trying to submit a DAG - * from the client. Only relevant in session mode. If the cluster is busy and cannot launch the - * AM then this timeout may be hit. In those case, using non-session mode is recommended if + * from the client. Only relevant in session mode. If the cluster is busy and cannot launch the + * AM then this timeout may be hit. In those case, using non-session mode is recommended if * applicable. Otherwise increase the timeout (set to -1 for infinity. Not recommended) */ @ConfigurationScope(Scope.AM) @@ -1433,7 +1447,7 @@ public TezConfiguration(boolean loadDefaults) { public static final int TEZ_HISTORY_LOGGING_TIMELINE_NUM_DAGS_PER_GROUP_DEFAULT = 1; /** - * String value. The directory into which history data will be written. This defaults to the + * String value. The directory into which history data will be written. This defaults to the * container logging directory. This is relevant only when SimpleHistoryLoggingService is being * used for {@link TezConfiguration#TEZ_HISTORY_LOGGING_SERVICE_CLASS} */ @@ -1542,7 +1556,7 @@ public TezConfiguration(boolean loadDefaults) { + "yarn.ats.acl.dag.domain.id"; /** - * Boolean value. Enable recovery of DAGs. This allows a restarted app master to recover the + * Boolean value. Enable recovery of DAGs. This allows a restarted app master to recover the * incomplete DAGs from the previous instance of the app master. */ @ConfigurationScope(Scope.AM) @@ -1641,10 +1655,10 @@ public TezConfiguration(boolean loadDefaults) { public static final boolean TEZ_AM_ACLS_ENABLED_DEFAULT = true; /** - * String value. + * String value. * AM view ACLs. This allows the specified users/groups to view the status of the AM and all DAGs * that run within this AM. - * Comma separated list of users, followed by whitespace, followed by a comma separated list of + * Comma separated list of users, followed by whitespace, followed by a comma separated list of * groups */ @ConfigurationScope(Scope.AM) @@ -1655,7 +1669,7 @@ public TezConfiguration(boolean loadDefaults) { * String value. * AM modify ACLs. This allows the specified users/groups to run modify operations on the AM * such as submitting DAGs, pre-warming the session, killing DAGs or shutting down the session. - * Comma separated list of users, followed by whitespace, followed by a comma separated list of + * Comma separated list of users, followed by whitespace, followed by a comma separated list of * groups */ @ConfigurationScope(Scope.AM) diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java index e3c40aaef0..bde4622d9b 100644 --- a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java +++ b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java @@ -120,7 +120,7 @@ public void testVerifyOneToOne() { dag.addEdge(e1); dag.verify(); } - + @Test(timeout = 5000) // v1 (known) -> v2 (-1) -> v3 (-1) public void testVerifyOneToOneInferParallelism() { @@ -153,7 +153,7 @@ public void testVerifyOneToOneInferParallelism() { Assert.assertEquals(dummyTaskCount, v2.getParallelism()); Assert.assertEquals(dummyTaskCount, v3.getParallelism()); } - + @Test(timeout = 5000) // v1 (known) -> v2 (-1) -> v3 (-1) // The test checks resiliency to ordering of the vertices/edges @@ -187,7 +187,7 @@ public void testVerifyOneToOneInferParallelismReverseOrder() { Assert.assertEquals(dummyTaskCount, v2.getParallelism()); Assert.assertEquals(dummyTaskCount, v3.getParallelism()); } - + @Test(timeout = 5000) public void testVerifyOneToOneNoInferParallelism() { Vertex v1 = Vertex.create("v1", @@ -211,7 +211,7 @@ public void testVerifyOneToOneNoInferParallelism() { dag.verify(); Assert.assertEquals(-1, v2.getParallelism()); } - + @Test(timeout = 5000) // v1 (-1) -> v2 (known) -> v3 (-1) public void testVerifyOneToOneIncorrectParallelism1() { @@ -296,7 +296,7 @@ public void testVerifyOneToOneIncorrectParallelism2() { "1-1 Edge. Destination vertex parallelism must match source vertex")); } } - + @Test(timeout = 5000) public void testVerifyBroadcast() { Vertex v1 = Vertex.create("v1", @@ -317,7 +317,7 @@ public void testVerifyBroadcast() { dag.verify(); } - @Test(expected = IllegalStateException.class, timeout = 5000) + @Test(timeout = 5000) public void testVerify3() { Vertex v1 = Vertex.create("v1", ProcessorDescriptor.create(dummyProcessorClassName), @@ -337,7 +337,7 @@ public void testVerify3() { dag.verify(); } - @Test(expected = IllegalStateException.class, timeout = 5000) + @Test(timeout = 5000) public void testVerify4() { Vertex v1 = Vertex.create("v1", ProcessorDescriptor.create(dummyProcessorClassName), @@ -525,7 +525,7 @@ public void repeatedVertexName() { System.out.println(ex.getMessage()); Assert.assertTrue(ex.getMessage().startsWith("Vertex v1 already defined")); } - + @Test(expected = IllegalStateException.class, timeout = 5000) public void testInputAndInputVertexNameCollision() { Vertex v1 = Vertex.create("v1", @@ -534,22 +534,22 @@ public void testInputAndInputVertexNameCollision() { Vertex v2 = Vertex.create("v2", ProcessorDescriptor.create("MapProcessor"), dummyTaskCount, dummyTaskResource); - + v2.addDataSource("v1", DataSourceDescriptor.create(null, null, null)); - + Edge e1 = Edge.create(v1, v2, EdgeProperty.create(DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, OutputDescriptor.create("dummy output class"), InputDescriptor.create("dummy input class"))); - + DAG dag = DAG.create("testDag"); dag.addVertex(v1); dag.addVertex(v2); dag.addEdge(e1); dag.verify(); } - + @Test(expected = IllegalStateException.class, timeout = 5000) public void testOutputAndOutputVertexNameCollision() { Vertex v1 = Vertex.create("v1", @@ -558,22 +558,22 @@ public void testOutputAndOutputVertexNameCollision() { Vertex v2 = Vertex.create("v2", ProcessorDescriptor.create("MapProcessor"), dummyTaskCount, dummyTaskResource); - + v1.addDataSink("v2", DataSinkDescriptor.create(null, null, null)); - + Edge e1 = Edge.create(v1, v2, EdgeProperty.create(DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, OutputDescriptor.create("dummy output class"), InputDescriptor.create("dummy input class"))); - + DAG dag = DAG.create("testDag"); dag.addVertex(v1); dag.addVertex(v2); dag.addEdge(e1); dag.verify(); } - + @Test(expected = IllegalStateException.class, timeout = 5000) public void testOutputAndVertexNameCollision() { Vertex v1 = Vertex.create("v1", @@ -582,15 +582,15 @@ public void testOutputAndVertexNameCollision() { Vertex v2 = Vertex.create("v2", ProcessorDescriptor.create("MapProcessor"), dummyTaskCount, dummyTaskResource); - + v1.addDataSink("v2", DataSinkDescriptor.create(null, null, null)); - + DAG dag = DAG.create("testDag"); dag.addVertex(v1); dag.addVertex(v2); dag.verify(); } - + @Test(expected = IllegalStateException.class, timeout = 5000) public void testInputAndVertexNameCollision() { Vertex v1 = Vertex.create("v1", @@ -599,9 +599,9 @@ public void testInputAndVertexNameCollision() { Vertex v2 = Vertex.create("v2", ProcessorDescriptor.create("MapProcessor"), dummyTaskCount, dummyTaskResource); - + v1.addDataSource("v2", DataSourceDescriptor.create(null, null, null)); - + DAG dag = DAG.create("testDag"); dag.addVertex(v1); dag.addVertex(v2); @@ -640,7 +640,7 @@ public void BinaryInputAllowed() { dag.addEdge(e2); dag.verify(); } - + @Test(timeout = 5000) public void testVertexGroupWithMultipleOutputEdges() { Vertex v1 = Vertex.create("v1", @@ -655,19 +655,19 @@ public void testVertexGroupWithMultipleOutputEdges() { Vertex v4 = Vertex.create("v4", ProcessorDescriptor.create("Processor"), dummyTaskCount, dummyTaskResource); - + DAG dag = DAG.create("testDag"); VertexGroup uv12 = dag.createVertexGroup("uv12", v1, v2); OutputDescriptor outDesc = new OutputDescriptor(); uv12.addDataSink("uvOut", DataSinkDescriptor.create(outDesc, null, null)); - + GroupInputEdge e1 = GroupInputEdge.create(uv12, v3, EdgeProperty.create(DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, OutputDescriptor.create("dummy output class"), InputDescriptor.create("dummy input class")), InputDescriptor.create("dummy input class")); - + GroupInputEdge e2 = GroupInputEdge.create(uv12, v4, EdgeProperty.create(DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, @@ -685,7 +685,7 @@ public void testVertexGroupWithMultipleOutputEdges() { for (int i = 0; i< 10;++i){ dag.verify(); // should be OK when called multiple times } - + Assert.assertEquals(2, v1.getOutputVertices().size()); Assert.assertEquals(2, v2.getOutputVertices().size()); Assert.assertTrue(v1.getOutputVertices().contains(v3)); @@ -693,7 +693,7 @@ public void testVertexGroupWithMultipleOutputEdges() { Assert.assertTrue(v2.getOutputVertices().contains(v3)); Assert.assertTrue(v2.getOutputVertices().contains(v4)); } - + @Test(timeout = 5000) public void testVertexGroup() { Vertex v1 = Vertex.create("v1", @@ -711,16 +711,16 @@ public void testVertexGroup() { Vertex v5 = Vertex.create("v5", ProcessorDescriptor.create("Processor"), dummyTaskCount, dummyTaskResource); - + DAG dag = DAG.create("testDag"); String groupName1 = "uv12"; VertexGroup uv12 = dag.createVertexGroup(groupName1, v1, v2); OutputDescriptor outDesc = new OutputDescriptor(); uv12.addDataSink("uvOut", DataSinkDescriptor.create(outDesc, null, null)); - + String groupName2 = "uv23"; VertexGroup uv23 = dag.createVertexGroup(groupName2, v2, v3); - + GroupInputEdge e1 = GroupInputEdge.create(uv12, v4, EdgeProperty.create(DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, @@ -733,7 +733,7 @@ public void testVertexGroup() { OutputDescriptor.create("dummy output class"), InputDescriptor.create("dummy input class")), InputDescriptor.create("dummy input class")); - + dag.addVertex(v1); dag.addVertex(v2); dag.addVertex(v3); @@ -744,7 +744,7 @@ public void testVertexGroup() { for (int i = 0; i< 10;++i){ dag.verify(); // should be OK when called multiple times } - + // for the first Group v1 and v2 should get connected to v4 and also have 1 output // for the second Group v2 and v3 should get connected to v5 // the Group place holders should disappear @@ -775,7 +775,7 @@ public void testVertexGroup() { Assert.assertTrue(v5.getGroupInputs().containsKey(groupName2)); Assert.assertEquals(2, dag.vertexGroups.size()); } - + @Test(timeout = 5000) public void testVertexGroupOneToOne() { Vertex v1 = Vertex.create("v1", @@ -793,16 +793,16 @@ public void testVertexGroupOneToOne() { Vertex v5 = Vertex.create("v5", ProcessorDescriptor.create("Processor"), -1, dummyTaskResource); - + DAG dag = DAG.create("testDag"); String groupName1 = "uv12"; VertexGroup uv12 = dag.createVertexGroup(groupName1, v1, v2); OutputDescriptor outDesc = new OutputDescriptor(); uv12.addDataSink("uvOut", DataSinkDescriptor.create(outDesc, null, null)); - + String groupName2 = "uv23"; VertexGroup uv23 = dag.createVertexGroup(groupName2, v2, v3); - + GroupInputEdge e1 = GroupInputEdge.create(uv12, v4, EdgeProperty.create(DataMovementType.ONE_TO_ONE, DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, @@ -815,7 +815,7 @@ public void testVertexGroupOneToOne() { OutputDescriptor.create("dummy output class"), InputDescriptor.create("dummy input class")), InputDescriptor.create("dummy input class")); - + dag.addVertex(v1); dag.addVertex(v2); dag.addVertex(v3); @@ -826,7 +826,7 @@ public void testVertexGroupOneToOne() { for (int i = 0; i< 10;++i){ dag.verify(); // should be OK when called multiple times } - + Assert.assertEquals(dummyTaskCount, v5.getParallelism()); } @@ -941,8 +941,8 @@ public void testMultipleRootInputsAllowed() { dag.createDag(new TezConfiguration(), null, null, null, true); } - - + + @Test(timeout = 5000) public void testDAGCreateDataInference() { Vertex v1 = Vertex.create("v1", ProcessorDescriptor.create(dummyProcessorClassName)); @@ -954,7 +954,7 @@ public void testDAGCreateDataInference() { String lrName2 = "LR2"; lrs2.put(lrName2, LocalResource.newInstance(URL.newInstance("file", "localhost", 0, "/test1"), LocalResourceType.FILE, LocalResourceVisibility.PUBLIC, 1, 1)); - + Set hosts = Sets.newHashSet(); hosts.add("h1"); hosts.add("h2"); @@ -962,10 +962,10 @@ public void testDAGCreateDataInference() { taskLocationHints.add(TaskLocationHint.createTaskLocationHint(hosts, null)); taskLocationHints.add(TaskLocationHint.createTaskLocationHint(hosts, null)); VertexLocationHint vLoc = VertexLocationHint.create(taskLocationHints); - DataSourceDescriptor ds = DataSourceDescriptor.create(InputDescriptor.create("I.class"), + DataSourceDescriptor ds = DataSourceDescriptor.create(InputDescriptor.create("I.class"), InputInitializerDescriptor.create(dummyInputInitClassName), dummyTaskCount, null, vLoc, lrs2); v1.addDataSource("i1", ds); - + DAG dag = DAG.create("testDag"); dag.addVertex(v1); dag.addTaskLocalFiles(lrs1); @@ -1003,10 +1003,10 @@ public void testInferredFilesFail() { Assert.assertTrue(e.getMessage().contains("Duplicate Resources found with different size")); } - DataSourceDescriptor ds = DataSourceDescriptor.create(InputDescriptor.create("I.class"), + DataSourceDescriptor ds = DataSourceDescriptor.create(InputDescriptor.create("I.class"), null, -1, null, null, lrs2); v1.addDataSource("i1", ds); - + DAG dag = DAG.create("testDag"); dag.addVertex(v1); dag.addTaskLocalFiles(lrs); @@ -1024,7 +1024,7 @@ public void testInferredFilesFail() { Assert.assertTrue(e.getMessage().contains("Duplicate Resources found with different size")); } } - + @Test(timeout = 5000) public void testDAGAccessControls() { DAG dag = DAG.create("testDag"); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index 177ba56014..2d2f23d031 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -71,6 +71,7 @@ import org.apache.tez.dag.api.NamedEntityDescriptor; import org.apache.tez.dag.api.SessionNotRunning; import org.apache.tez.dag.api.UserPayload; +import org.apache.tez.dag.api.records.DAGProtos; import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto; import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto; import org.apache.tez.dag.api.records.DAGProtos.TezNamedEntityDescriptorProto; @@ -298,7 +299,7 @@ public class DAGAppMaster extends AbstractService { private Path currentRecoveryDataDir; private Path tezSystemStagingDir; private FileSystem recoveryFS; - + private ExecutorService rawExecutor; private ListeningExecutorService execService; @@ -330,7 +331,7 @@ public class DAGAppMaster extends AbstractService { private String clientVersion; private boolean versionMismatch = false; private String versionMismatchDiagnostics; - + private ResourceCalculatorProcessTree cpuPlugin; private GcTimeUpdater gcPlugin; @@ -385,7 +386,7 @@ public static String getRunningLogURL(String nodeHttpAddress, return PATH_JOINER.join(nodeHttpAddress, "node", "containerlogs", containerId, user); } - + private void initResourceCalculatorPlugins() { Class clazz = amConf.getClass( TezConfiguration.TEZ_TASK_RESOURCE_CALCULATOR_PROCESS_TREE_CLASS, @@ -400,10 +401,10 @@ private void initResourceCalculatorPlugins() { pid = processName.split("@")[0]; } cpuPlugin = ResourceCalculatorProcessTree.getResourceCalculatorProcessTree(pid, clazz, amConf); - + gcPlugin = new GcTimeUpdater(null); } - + private long getAMCPUTime() { if (cpuPlugin != null) { cpuPlugin.updateProcessTree(); @@ -557,14 +558,14 @@ public synchronized void serviceInit(final Configuration conf) throws Exception dispatcher.register(TaskEventType.class, new TaskEventDispatcher()); dispatcher.register(TaskAttemptEventType.class, new TaskAttemptEventDispatcher()); } else { - int concurrency = conf.getInt(TezConfiguration.TEZ_AM_CONCURRENT_DISPATCHER_CONCURRENCY, + int concurrency = conf.getInt(TezConfiguration.TEZ_AM_CONCURRENT_DISPATCHER_CONCURRENCY, TezConfiguration.TEZ_AM_CONCURRENT_DISPATCHER_CONCURRENCY_DEFAULT); AsyncDispatcherConcurrent sharedDispatcher = dispatcher.registerAndCreateDispatcher( TaskEventType.class, new TaskEventDispatcher(), "TaskAndAttemptEventThread", concurrency); dispatcher.registerWithExistingDispatcher(TaskAttemptEventType.class, new TaskAttemptEventDispatcher(), sharedDispatcher); } - + // register other delegating dispatchers dispatcher.registerAndCreateDispatcher(SpeculatorEventType.class, new SpeculatorEventHandler(), "Speculator"); @@ -661,7 +662,7 @@ protected TaskSchedulerManager createTaskSchedulerManager( protected ContainerSignatureMatcher createContainerSignatureMatcher() { return new ContainerContextMatcher(); } - + @VisibleForTesting protected AsyncDispatcher createDispatcher() { return new AsyncDispatcher("Central"); @@ -680,7 +681,7 @@ protected void sysexit() { System.exit(0); } } - + @VisibleForTesting protected TaskSchedulerManager getTaskSchedulerManager() { return taskSchedulerManager; @@ -1342,7 +1343,7 @@ public void tryKillDAG(DAG dag, String message) throws TezException { } dispatcher.getEventHandler().handle(new DAGEventTerminateDag(dag.getID(), DAGTerminationCause.DAG_KILL, message)); } - + private Map getAdditionalLocalResourceDiff( DAG dag, Map additionalResources) throws TezException { if (additionalResources == null) { @@ -1494,7 +1495,7 @@ public long getStartTime() { public DAG getCurrentDAG() { return dag; } - + @Override public ListeningExecutorService getExecService() { return execService; @@ -1695,7 +1696,7 @@ public void setDAG(DAG dag) { public long getCumulativeCPUTime() { return getAMCPUTime(); } - + @Override public long getCumulativeGCTime() { return getAMGCTime(); @@ -1928,7 +1929,7 @@ private DAGRecoveryData recoverDAG() throws IOException, TezException { } return null; } - + @Override public synchronized void serviceStart() throws Exception { @@ -1971,8 +1972,19 @@ public synchronized void serviceStart() throws Exception { return; } + DAGPlan dagPlan = null; if (!isSession) { LOG.info("In Non-Session mode."); + dagPlan = readDAGPlanFile(); + if (hasConcurrentEdge(dagPlan)) { + // Currently a DAG with concurrent edge is deemed unrecoverable + // (run from scratch) on AM failover. Proper AM failover for DAG with + // concurrent edge is pending TEZ-4017 + if (recoveredDAGData != null) { + LOG.warn("Ignoring recoveredDAGData for a recovered DAG with concurrent edge."); + recoveredDAGData = null; + } + } } else { LOG.info("In Session mode. Waiting for DAG over RPC"); this.state = DAGAppMasterState.IDLE; @@ -2053,7 +2065,8 @@ public synchronized void serviceStart() throws Exception { if (!isSession) { // No dag recovered - in non-session, just restart the original DAG dagCounter.set(0); - startDAG(); + assert(dagPlan != null); + startDAG(dagPlan, null); } } @@ -2181,7 +2194,7 @@ private class TaskEventDispatcher implements EventHandler { @Override public void handle(TaskEvent event) { DAG dag = context.getCurrentDAG(); - int eventDagIndex = + int eventDagIndex = event.getTaskID().getVertexID().getDAGId().getId(); if (dag == null || eventDagIndex != dag.getID().getId()) { return; // event not relevant any more @@ -2192,7 +2205,7 @@ public void handle(TaskEvent event) { ((EventHandler)task).handle(event); } } - + private class SpeculatorEventHandler implements EventHandler { @Override public void handle(SpeculatorEvent event) { @@ -2211,7 +2224,7 @@ private class TaskAttemptEventDispatcher @Override public void handle(TaskAttemptEvent event) { DAG dag = context.getCurrentDAG(); - int eventDagIndex = + int eventDagIndex = event.getTaskAttemptID().getTaskID().getVertexID().getDAGId().getId(); if (dag == null || eventDagIndex != dag.getID().getId()) { return; // event not relevant any more @@ -2230,12 +2243,12 @@ private class VertexEventDispatcher @Override public void handle(VertexEvent event) { DAG dag = context.getCurrentDAG(); - int eventDagIndex = + int eventDagIndex = event.getVertexId().getDAGId().getId(); if (dag == null || eventDagIndex != dag.getID().getId()) { return; // event not relevant any more } - + Vertex vertex = dag.getVertex(event.getVertexId()); ((EventHandler) vertex).handle(event); @@ -2440,23 +2453,30 @@ public void run() { } } - private void startDAG() throws IOException, TezException { + private boolean hasConcurrentEdge(DAGPlan dagPlan) { + boolean hasConcurrentEdge = false; + for (DAGProtos.EdgePlan edge : dagPlan.getEdgeList()) { + if (DAGProtos.PlanEdgeSchedulingType.CONCURRENT.equals(edge.getSchedulingType())) { + return true; + } + } + return hasConcurrentEdge; + } + + private DAGPlan readDAGPlanFile() throws IOException, TezException { FileInputStream dagPBBinaryStream = null; + DAGPlan dagPlan = null; try { - DAGPlan dagPlan = null; - // Read the protobuf DAG dagPBBinaryStream = new FileInputStream(new File(workingDirectory, TezConstants.TEZ_PB_PLAN_BINARY_NAME)); dagPlan = DAGPlan.parseFrom(dagPBBinaryStream); - - startDAG(dagPlan, null); - } finally { if (dagPBBinaryStream != null) { dagPBBinaryStream.close(); } } + return dagPlan; } private void startDAG(DAGPlan dagPlan, Map additionalAMResources) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/edgemanager/SilentEdgeManager.java b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/edgemanager/SilentEdgeManager.java new file mode 100644 index 0000000000..db6bb5affe --- /dev/null +++ b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/edgemanager/SilentEdgeManager.java @@ -0,0 +1,89 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.library.edgemanager; + +import org.apache.tez.dag.api.EdgeManagerPlugin; +import org.apache.tez.dag.api.EdgeManagerPluginContext; +import org.apache.tez.runtime.api.events.DataMovementEvent; +import org.apache.tez.runtime.api.events.InputReadErrorEvent; + +import java.util.List; +import java.util.Map; + +/** + * A dummy edge manager used in scenarios where application will depend on + * the direct connection between containers/tasks to handle all data communications, + * including both routing and actual data transfers. + */ + +public class SilentEdgeManager extends EdgeManagerPlugin { + + /** + * Create an instance of the EdgeManagerPlugin. Classes extending this to + * create a EdgeManagerPlugin, must provide the same constructor so that Tez + * can create an instance of the class at runtime. + * + * @param context the context within which this EdgeManagerPlugin will run. Includes + * information like configuration which the user may have specified + * while setting up the edge. + */ + public SilentEdgeManager(EdgeManagerPluginContext context) { + super(context); + } + + @Override + public void initialize() throws Exception { + + } + + @Override + public int getNumDestinationTaskPhysicalInputs(int destinationTaskIndex) throws Exception { + return 0; + } + + @Override + public int getNumSourceTaskPhysicalOutputs(int sourceTaskIndex) throws Exception { + return 0; + } + + @Override + public void routeDataMovementEventToDestination( + DataMovementEvent event, int sourceTaskIndex, int sourceOutputIndex, + Map> destinationTaskAndInputIndices) throws Exception { + throw new UnsupportedOperationException( + "routeDataMovementEventToDestination not supported for SilentEdgeManager"); + } + + @Override + public void routeInputSourceTaskFailedEventToDestination( + int sourceTaskIndex, Map> destinationTaskAndInputIndices) throws Exception { + throw new UnsupportedOperationException( + "routeInputSourceTaskFailedEventToDestination not supported for SilentEdgeManager"); + } + + @Override + public int getNumDestinationConsumerTasks(int sourceTaskIndex) throws Exception { + return 0; + } + + @Override + public int routeInputErrorEventToSource(InputReadErrorEvent event, int destinationTaskIndex, int destinationFailedInputIndex) throws Exception { + return 0; + } +} diff --git a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/VertexManagerWithConcurrentInput.java b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/VertexManagerWithConcurrentInput.java new file mode 100644 index 0000000000..caf5acd588 --- /dev/null +++ b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/VertexManagerWithConcurrentInput.java @@ -0,0 +1,245 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.library.vertexmanager; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.hadoop.conf.Configuration; +import org.apache.tez.common.TezUtils; +import org.apache.tez.dag.api.EdgeProperty; +import org.apache.tez.dag.api.EdgeProperty.ConcurrentEdgeTriggerType; +import org.apache.tez.dag.api.InputDescriptor; +import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.api.TezUncheckedException; +import org.apache.tez.dag.api.UserPayload; +import org.apache.tez.dag.api.VertexManagerPlugin; +import org.apache.tez.dag.api.VertexManagerPluginContext; +import org.apache.tez.dag.api.VertexManagerPluginDescriptor; +import org.apache.tez.dag.api.event.VertexState; +import org.apache.tez.dag.api.event.VertexStateUpdate; +import org.apache.tez.runtime.api.Event; +import org.apache.tez.runtime.api.TaskAttemptIdentifier; +import org.apache.tez.runtime.api.events.VertexManagerEvent; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.EnumSet; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.apache.tez.dag.api.EdgeProperty.SchedulingType.CONCURRENT; + +public class VertexManagerWithConcurrentInput extends VertexManagerPlugin { + + private static final Logger LOG = LoggerFactory.getLogger(VertexManagerWithConcurrentInput.class); + + private final Map srcVerticesConfigured = Maps.newConcurrentMap(); + private int managedTasks; + private AtomicBoolean tasksScheduled = new AtomicBoolean(false); + private AtomicBoolean onVertexStartedDone = new AtomicBoolean(false); + private Configuration vertexConfig; + private String vertexName; + private ConcurrentEdgeTriggerType edgeTriggerType; + private volatile boolean allSrcVerticesConfigured; + + int completedUpstreamTasks; + + public VertexManagerWithConcurrentInput(VertexManagerPluginContext context) { + super(context); + } + + @Override + public void initialize() { + UserPayload userPayload = getContext().getUserPayload(); + if (userPayload == null || userPayload.getPayload() == null || + userPayload.getPayload().limit() == 0) { + throw new TezUncheckedException("Could not initialize VertexManagerWithConcurrentInput" + + " from provided user payload"); + } + managedTasks = getContext().getVertexNumTasks(getContext().getVertexName()); + Map edges = getContext().getInputVertexEdgeProperties(); + for (Map.Entry entry : edges.entrySet()) { + if (!CONCURRENT.equals(entry.getValue().getSchedulingType())) { + throw new TezUncheckedException("All input edges to vertex " + vertexName + + " must be CONCURRENT."); + } + String srcVertex = entry.getKey(); + srcVerticesConfigured.put(srcVertex, false); + getContext().registerForVertexStateUpdates(srcVertex, EnumSet.of(VertexState.CONFIGURED)); + } + + try { + vertexConfig = TezUtils.createConfFromUserPayload(getContext().getUserPayload()); + } catch (IOException e) { + throw new TezUncheckedException(e); + } + edgeTriggerType = ConcurrentEdgeTriggerType.valueOf( + vertexConfig.get(TezConfiguration.TEZ_CONCURRENT_EDGE_TRIGGER_TYPE, + TezConfiguration.TEZ_CONCURRENT_EDGE_TRIGGER_TYPE_DEFAULT)); + if (!ConcurrentEdgeTriggerType.SOURCE_VERTEX_CONFIGURED.equals(edgeTriggerType)) { + // pending TEZ-3999 + throw new TezUncheckedException("Only support SOURCE_VERTEX_CONFIGURED triggering type for now."); + } + LOG.info("VertexManagerWithConcurrentInput initialized with edgeTriggerType {}.", edgeTriggerType); + + vertexName = getContext().getVertexName(); + completedUpstreamTasks = 0; + } + + @Override + public synchronized void onVertexStarted(List completions) { + onVertexStartedDone.set(true); + scheduleTasks(); + } + + @Override + public synchronized void onVertexStateUpdated(VertexStateUpdate stateUpdate) { + VertexState state = stateUpdate.getVertexState(); + String fromVertex = stateUpdate.getVertexName(); + if (!srcVerticesConfigured.containsKey(fromVertex)) { + throw new IllegalArgumentException("Not expecting state update from vertex:" + + fromVertex + " in vertex: " + this.vertexName); + } + + if (!VertexState.CONFIGURED.equals(state)) { + throw new IllegalArgumentException("Received incorrect state notification : " + + state + " from vertex: " + fromVertex + " in vertex: " + this.vertexName); + } + + LOG.info("Received configured notification: " + state + " for vertex: " + + fromVertex + " in vertex: " + this.vertexName); + srcVerticesConfigured.put(fromVertex, true); + + // check for source vertices completely configured + boolean checkAllSrcVerticesConfigured = true; + for (Map.Entry entry : srcVerticesConfigured.entrySet()) { + if (!entry.getValue()) { + // vertex not configured + LOG.info("Waiting for vertex {} in vertex {} ", entry.getKey(), this.vertexName); + checkAllSrcVerticesConfigured = false; + break; + } + } + allSrcVerticesConfigured = checkAllSrcVerticesConfigured; + + scheduleTasks(); + } + + @Override + public synchronized void onSourceTaskCompleted(TaskAttemptIdentifier attempt) { + completedUpstreamTasks ++; + LOG.info("Source task attempt {} completion received at vertex {}", attempt, this.vertexName); + } + + @Override + public void onVertexManagerEventReceived(VertexManagerEvent vmEvent) { + } + + @Override + public void onRootVertexInitialized(String inputName, + InputDescriptor inputDescriptor, List events) { + } + + private void scheduleTasks() { + if (!onVertexStartedDone.get()) { + // vertex not started yet + return; + } + if (tasksScheduled.get()) { + // already scheduled + return; + } + + if (!canScheduleTasks()) { + return; + } + + tasksScheduled.compareAndSet(false, true); + List tasksToStart = Lists.newArrayListWithCapacity(managedTasks); + for (int i = 0; i < managedTasks; ++i) { + tasksToStart.add(VertexManagerPluginContext.ScheduleTaskRequest.create(i, null)); + } + + if (!tasksToStart.isEmpty()) { + LOG.info("Starting {} tasks in {}.", tasksToStart.size(), this.vertexName); + getContext().scheduleTasks(tasksToStart); + } + // all tasks scheduled. Can call vertexManagerDone(). + } + + private boolean canScheduleTasks() { + if (edgeTriggerType.equals(ConcurrentEdgeTriggerType.SOURCE_VERTEX_CONFIGURED)) { + return allSrcVerticesConfigured; + } else { + // pending TEZ-3999 + throw new TezUncheckedException("Only support SOURCE_VERTEX_CONFIGURED triggering type for now."); + } + } + + + /** + * Create a {@link VertexManagerPluginDescriptor} builder that can be used to + * configure the plugin. + * + * @param conf + * {@link Configuration} May be modified in place. May be null if the + * configuration parameters are to be set only via code. If + * configuration values may be changed at runtime via a config file + * then pass in a {@link Configuration} that is initialized from a + * config file. The parameters that are not overridden in code will + * be derived from the Configuration object. + * @return {@link ConcurrentInputVertexManagerConfigBuilder} + */ + public static ConcurrentInputVertexManagerConfigBuilder createConfigBuilder( + @Nullable Configuration conf) { + return new ConcurrentInputVertexManagerConfigBuilder(conf); + } + + /** + * Helper class to configure VertexManagerWithConcurrentInput + */ + public static final class ConcurrentInputVertexManagerConfigBuilder { + private final Configuration conf; + + private ConcurrentInputVertexManagerConfigBuilder(@Nullable Configuration conf) { + if (conf == null) { + this.conf = new Configuration(false); + } else { + this.conf = conf; + } + } + + public VertexManagerPluginDescriptor build() { + VertexManagerPluginDescriptor desc = + VertexManagerPluginDescriptor.create( + VertexManagerWithConcurrentInput.class.getName()); + + try { + return desc.setUserPayload(TezUtils + .createUserPayloadFromConf(this.conf)); + } catch (IOException e) { + throw new TezUncheckedException(e); + } + } + } + +} diff --git a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestVertexManagerWithConcurrentInput.java b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestVertexManagerWithConcurrentInput.java new file mode 100644 index 0000000000..619a4cdd1d --- /dev/null +++ b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestVertexManagerWithConcurrentInput.java @@ -0,0 +1,114 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.library.vertexmanager; + +import org.apache.tez.dag.api.EdgeManagerPluginDescriptor; +import org.apache.tez.dag.api.EdgeProperty; +import org.apache.tez.dag.api.InputDescriptor; +import org.apache.tez.dag.api.OutputDescriptor; +import org.apache.tez.dag.api.VertexManagerPluginContext; +import org.apache.tez.dag.api.VertexManagerPluginDescriptor; +import org.apache.tez.dag.api.event.VertexState; +import org.apache.tez.dag.api.event.VertexStateUpdate; +import org.apache.tez.dag.library.edgemanager.SilentEdgeManager; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Captor; +import org.mockito.MockitoAnnotations; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class TestVertexManagerWithConcurrentInput { + + @Captor + ArgumentCaptor> requestCaptor; + + @Before + public void init() { + MockitoAnnotations.initMocks(this); + } + + @Test(timeout = 5000) + public void testBasicVertexWithConcurrentInput() throws Exception { + HashMap mockInputVertices = + new HashMap(); + String mockSrcVertexId1 = "Vertex1"; + int srcVertex1Parallelism = 2; + EdgeProperty eProp1 = EdgeProperty.create( + EdgeManagerPluginDescriptor.create(SilentEdgeManager.class.getName()), + EdgeProperty.DataSourceType.EPHEMERAL, + EdgeProperty.SchedulingType.CONCURRENT, + OutputDescriptor.create("out"), + InputDescriptor.create("in")); + + String mockSrcVertexId2 = "Vertex2"; + int srcVertex2Parallelism = 3; + EdgeProperty eProp2 = EdgeProperty.create( + EdgeManagerPluginDescriptor.create(SilentEdgeManager.class.getName()), + EdgeProperty.DataSourceType.EPHEMERAL, + EdgeProperty.SchedulingType.CONCURRENT, + OutputDescriptor.create("out"), + InputDescriptor.create("in")); + + String mockManagedVertexId = "Vertex"; + int vertexParallelism = 2; + + VertexManagerWithConcurrentInput.ConcurrentInputVertexManagerConfigBuilder configurer = + VertexManagerWithConcurrentInput.createConfigBuilder(null); + VertexManagerPluginDescriptor pluginDesc = configurer.build(); + + VertexManagerPluginContext mockContext = mock(VertexManagerPluginContext.class); + when(mockContext.getUserPayload()).thenReturn(pluginDesc.getUserPayload()); + when(mockContext.getInputVertexEdgeProperties()).thenReturn(mockInputVertices); + when(mockContext.getVertexName()).thenReturn(mockManagedVertexId); + when(mockContext.getVertexNumTasks(mockManagedVertexId)).thenReturn(vertexParallelism); + when(mockContext.getVertexNumTasks(mockSrcVertexId1)).thenReturn(srcVertex1Parallelism); + when(mockContext.getVertexNumTasks(mockSrcVertexId2)).thenReturn(srcVertex2Parallelism); + mockInputVertices.put(mockSrcVertexId1, eProp1); + mockInputVertices.put(mockSrcVertexId2, eProp2); + + VertexManagerWithConcurrentInput manager = new VertexManagerWithConcurrentInput(mockContext); + when(mockContext.getUserPayload()).thenReturn(pluginDesc.getUserPayload()); + manager.initialize(); + when(mockContext.getUserPayload()).thenReturn(pluginDesc.getUserPayload()); + + // source vertex 1 configured + manager.onVertexStateUpdated(new VertexStateUpdate(mockSrcVertexId1, VertexState.CONFIGURED)); + verify(mockContext, times(0)).scheduleTasks(requestCaptor.capture()); + + // source vertex 2 configured + manager.onVertexStateUpdated(new VertexStateUpdate(mockSrcVertexId2, VertexState.CONFIGURED)); + verify(mockContext, times(0)).scheduleTasks(requestCaptor.capture()); + + // then own vertex started + manager.onVertexStarted(Collections.singletonList( + TestShuffleVertexManager.createTaskAttemptIdentifier(mockSrcVertexId1, 0))); + verify(mockContext, times(1)).scheduleTasks(requestCaptor.capture()); + Assert.assertEquals(0, manager.completedUpstreamTasks); + } +} From 9345c2b1d66a530c32e2a7258929b883fe9ec38e Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Tue, 27 Nov 2018 11:20:43 -0600 Subject: [PATCH 133/512] TEZ-4022. Upgrade Maven Surefire plugin to 3.0.0-M1 Signed-off-by: Jason Lowe --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 9d8797c256..2d46fb7b62 100644 --- a/pom.xml +++ b/pom.xml @@ -890,7 +890,7 @@ org.apache.maven.plugins maven-surefire-plugin - 2.21.0 + 3.0.0-M1 1 false From 282bb0a3fddf20260d71b0a6cd798fa5479e7038 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Tue, 27 Nov 2018 15:23:48 -0600 Subject: [PATCH 134/512] TEZ-4021. API incompatibility wro4j-maven-plugin Signed-off-by: Jason Lowe --- pom.xml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/pom.xml b/pom.xml index 2d46fb7b62..1d46ffd01e 100644 --- a/pom.xml +++ b/pom.xml @@ -1011,6 +1011,14 @@ ro.isdc.wro4j wro4j-maven-plugin 1.7.9 + + + + org.mockito + mockito-core + 2.18.0 + + From f49d665393a884295f529ffc7b9493cfa7bb3853 Mon Sep 17 00:00:00 2001 From: Sergey Shelukhin Date: Tue, 11 Dec 2018 11:45:58 -0800 Subject: [PATCH 135/512] TEZ-3957: Report TASK_DURATION_MILLIS as a Counter for completed tasks (Sergey Shelukhin, reviewed by Gopal V) Signed-off-by: Gopal V --- .../tez/common/counters/DAGCounter.java | 2 + .../tez/common/counters/TaskCounter.java | 2 + .../tez/dag/app/dag/impl/TaskAttemptImpl.java | 45 ++++++++++++++++++- .../LogicalIOProcessorRuntimeTask.java | 18 ++++++++ .../org/apache/tez/runtime/RuntimeTask.java | 4 ++ .../tez/runtime/task/TestTaskExecution2.java | 4 +- 6 files changed, 72 insertions(+), 3 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java index 5064c35c5a..0a32d388bc 100644 --- a/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java +++ b/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java @@ -38,5 +38,7 @@ public enum DAGCounter { NUM_UBER_SUBTASKS, NUM_FAILED_UBERTASKS, AM_CPU_MILLISECONDS, + /** Wall clock time taken by all the tasks. */ + WALL_CLOCK_MILLIS, AM_GC_TIME_MILLIS } diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java index 2f18bc677b..80424c7708 100644 --- a/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java +++ b/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java @@ -74,6 +74,8 @@ public enum TaskCounter { MERGED_MAP_OUTPUTS, GC_TIME_MILLIS, CPU_MILLISECONDS, + /** Wall clock time taken by the task initialization and execution. */ + WALL_CLOCK_MILLISECONDS, PHYSICAL_MEMORY_BYTES, VIRTUAL_MEMORY_BYTES, COMMITTED_HEAP_BYTES, diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java index 7399979e66..310733018e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java @@ -26,6 +26,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -179,6 +180,12 @@ public static DataEventDependencyInfo fromProto(DataEventDependencyInfoProto pro private TaskAttemptRecoveryData recoveryData; private long launchTime = 0; private long finishTime = 0; + /** System.nanoTime for task launch time, if recorded in this JVM. */ + private Long launchTimeNs; + /** System.nanoTime for task finish time, if recorded in this JVM. */ + private Long finishTimeNs; + /** Whether the task was recovered from a prior AM; see getDurationNs. */ + private boolean isRecoveredDuration; private String trackerName; private int httpPort; @@ -782,6 +789,25 @@ public long getLaunchTime() { } } + + /** @return task runtime duration in NS. */ + public long getDurationNs() { + readLock.lock(); + try { + if (isRecoveredDuration) { + // NS values are not mappable between JVMs (per documentation, at + // least), so just use the clock after recovery. + return TimeUnit.MILLISECONDS.toNanos(launchTime == 0 ? 0 + : (finishTime == 0 ? clock.getTime() : finishTime) - launchTime); + } else { + long ft = (finishTimeNs == null ? System.nanoTime() : finishTimeNs); + return (launchTimeNs == null) ? 0 : (ft - launchTimeNs); + } + } finally { + readLock.unlock(); + } + } + public long getCreationTime() { readLock.lock(); try { @@ -930,6 +956,8 @@ private void setFinishTime() { // set the finish time only if launch time is set if (launchTime != 0 && finishTime == 0) { finishTime = clock.getTime(); + // The default clock is not safe for measuring durations. + finishTimeNs = System.nanoTime(); } } @@ -957,6 +985,10 @@ private static DAGEventCounterUpdate createDAGCounterUpdateEventTAFinished( jce.addCounterUpdate(DAGCounter.NUM_SUCCEEDED_TASKS, 1); } + long amSideWallClockTimeMs = TimeUnit.NANOSECONDS.toMillis( + taskAttempt.getDurationNs()); + jce.addCounterUpdate(DAGCounter.WALL_CLOCK_MILLIS, amSideWallClockTimeMs); + return jce; } @@ -1032,6 +1064,14 @@ private JobHistoryEvent createTaskAttemptStartedEvent() { // */ // } + /** + * Records the launch time of the task. + */ + private void setLaunchTime() { + launchTime = clock.getTime(); + launchTimeNs = System.nanoTime(); + } + private void updateProgressSplits() { // double newProgress = reportedStatus.progress; // newProgress = Math.max(Math.min(newProgress, 1.0D), 0.0D); @@ -1215,6 +1255,7 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl ta, TaskAttemptEvent ta.recoveryData.getTaskAttemptStartedEvent(); if (taStartedEvent != null) { ta.launchTime = taStartedEvent.getStartTime(); + ta.isRecoveredDuration = true; TaskAttemptFinishedEvent taFinishedEvent = ta.recoveryData.getTaskAttemptFinishedEvent(); if (taFinishedEvent == null) { @@ -1383,6 +1424,7 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) { .getTaskAttemptState(), helper.getFailureType(event)); } else { ta.finishTime = ta.recoveryData.getTaskAttemptFinishedEvent().getFinishTime(); + ta.isRecoveredDuration = true; } if (event instanceof RecoveryEvent) { @@ -1419,7 +1461,7 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent origEvent) { .getNetworkLocation()); ta.lastNotifyProgressTimestamp = ta.clock.getTime(); - ta.launchTime = ta.clock.getTime(); + ta.setLaunchTime(); // TODO Resolve to host / IP in case of a local address. InetSocketAddress nodeHttpInetAddr = NetUtils @@ -1630,6 +1672,7 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) { ta.sendEvent(new VertexEventRouteEvent(ta.getVertexID(), tezEvents)); } ta.finishTime = taFinishedEvent.getFinishTime(); + ta.isRecoveredDuration = true; } else { ta.setFinishTime(); // Send out history event. diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java index 0ac916f10b..87ebb7b9d0 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java @@ -43,6 +43,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.tez.hadoop.shim.HadoopShim; @@ -57,6 +58,7 @@ import org.apache.tez.common.ReflectionUtils; import org.apache.tez.common.RunnableWithNdc; import org.apache.tez.common.TezExecutors; +import org.apache.tez.common.counters.TaskCounter; import org.apache.tez.dag.api.InputDescriptor; import org.apache.tez.dag.api.OutputDescriptor; import org.apache.tez.dag.api.ProcessorDescriptor; @@ -160,6 +162,8 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask { private final boolean initializeProcessorFirst; private final boolean initializeProcessorIOSerially; private final TezExecutors sharedExecutor; + /** nanoTime of the task initialization start. */ + private Long initStartTimeNs = null; public LogicalIOProcessorRuntimeTask(TaskSpec taskSpec, int appAttemptNumber, Configuration tezConf, String[] localDirs, TezUmbilical tezUmbilical, @@ -229,6 +233,9 @@ public LogicalIOProcessorRuntimeTask(TaskSpec taskSpec, int appAttemptNumber, public void initialize() throws Exception { Preconditions.checkState(this.state.get() == State.NEW, "Already initialized"); this.state.set(State.INITED); + if (this.tezCounters != null) { + this.initStartTimeNs = System.nanoTime(); + } this.processorContext = createProcessorContext(); this.processor = createProcessor(processorDescriptor.getClassName(), processorContext); @@ -1077,4 +1084,15 @@ public HadoopShim getHadoopShim() { public Configuration getTaskConf() { return tezConf; } + + @Override + public void setFrameworkCounters() { + super.setFrameworkCounters(); + if (tezCounters != null && isUpdatingSystemCounters()) { + long timeNs = initStartTimeNs == null ? 0 + : (System.nanoTime() - initStartTimeNs); + tezCounters.findCounter(TaskCounter.WALL_CLOCK_MILLISECONDS) + .setValue(TimeUnit.NANOSECONDS.toMillis(timeNs)); + } + } } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java index 7b86d4bc02..a53d0d2e7e 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java @@ -178,4 +178,8 @@ protected void setTaskDone() { } public abstract void abortTask(); + + protected final boolean isUpdatingSystemCounters() { + return counterUpdater != null; + } } diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution2.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution2.java index 07b9d33b46..6c25f0a245 100644 --- a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution2.java +++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution2.java @@ -675,8 +675,8 @@ private void verifySysCounters(TezCounters tezCounters, int minTaskCounterCount, // If Target <=0, assert counter count is exactly 0 if (minTaskCounterCount <= 0) { - assertEquals(0, numTaskCounters); - assertEquals(0, numFsCounters); + assertEquals(tezCounters.toString(), 0, numTaskCounters); + assertEquals(tezCounters.toString(), 0, numFsCounters); } else { assertTrue(numTaskCounters >= minTaskCounterCount); assertTrue(numFsCounters >= minFsCounterCount); From a03181742abbf064557d096b3d3cd231c64c1a2a Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Tue, 18 Dec 2018 09:14:45 -0600 Subject: [PATCH 136/512] TEZ-4027. DagAwareYarnTaskScheduler can miscompute blocked vertices and cause a hang Signed-off-by: Jason Lowe --- .../dag/app/rm/DagAwareYarnTaskScheduler.java | 4 +- .../app/rm/TestDagAwareYarnTaskScheduler.java | 109 ++++++++++++++++++ 2 files changed, 111 insertions(+), 2 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java index 167d879f8b..1cdc217e20 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java @@ -1899,12 +1899,12 @@ Resource getAmountToPreempt(int preemptionPercentage) { // scheduled due to outstanding requests from higher priority predecessor vertices. @GuardedBy("DagAwareYarnTaskScheduler.this") BitSet createVertexBlockedSet() { - BitSet blocked = new BitSet(); + BitSet blocked = new BitSet(vertexDescendants.size()); Entry entry = priorityStats.lastEntry(); if (entry != null) { RequestPriorityStats stats = entry.getValue(); blocked.or(stats.allowedVertices); - blocked.flip(0, blocked.length()); + blocked.flip(0, blocked.size()); blocked.or(stats.descendants); } return blocked; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java index 0910ed270b..911f4b1d3e 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java @@ -1228,6 +1228,115 @@ public void testPreemptionNoHeadroom() throws Exception { verify(mockRMClient).stop(); } + @Test (timeout = 50000L) + public void testPreemptionWhenBlocked() throws Exception { + AMRMClientAsyncWrapperForTest mockRMClient = spy(new AMRMClientAsyncWrapperForTest()); + + String appHost = "host"; + int appPort = 0; + String appUrl = "url"; + + Configuration conf = new Configuration(); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, true); + conf.setInt(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 100); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, true); + conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED, false); + conf.setInt(TezConfiguration.TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX, 100); + conf.setInt(TezConfiguration.TEZ_AM_PREEMPTION_PERCENTAGE, 10); + conf.setInt(TezConfiguration.TEZ_AM_PREEMPTION_HEARTBEATS_BETWEEN_PREEMPTIONS, 3); + conf.setInt(TezConfiguration.TEZ_AM_PREEMPTION_MAX_WAIT_TIME_MS, 60 * 1000); + + + DagInfo mockDagInfo = mock(DagInfo.class); + when(mockDagInfo.getTotalVertices()).thenReturn(3); + when(mockDagInfo.getVertexDescendants(0)).thenReturn(BitSet.valueOf(new long[] { 0x6 })); + when(mockDagInfo.getVertexDescendants(1)).thenReturn(BitSet.valueOf(new long[] { 0x2 })); + when(mockDagInfo.getVertexDescendants(2)).thenReturn(new BitSet()); + TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf); + when(mockApp.getCurrentDagInfo()).thenReturn(mockDagInfo); + TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp); + + MockClock clock = new MockClock(1000); + NewTaskSchedulerForTest scheduler = new NewTaskSchedulerForTest(drainableAppCallback, + mockRMClient, clock); + + scheduler.initialize(); + drainableAppCallback.drain(); + + scheduler.start(); + drainableAppCallback.drain(); + verify(mockRMClient).start(); + verify(mockRMClient).registerApplicationMaster(appHost, appPort, appUrl); + RegisterApplicationMasterResponse regResponse = mockRMClient.getRegistrationResponse(); + verify(mockApp).setApplicationRegistrationData(regResponse.getMaximumResourceCapability(), + regResponse.getApplicationACLs(), regResponse.getClientToAMTokenMasterKey(), + regResponse.getQueue()); + + assertEquals(scheduler.getClusterNodeCount(), mockRMClient.getClusterNodeCount()); + + Priority priorityv0 = Priority.newInstance(1); + Priority priorityv2 = Priority.newInstance(3); + String[] hostsv0t0 = { "host1", "host2" }; + MockTaskInfo taskv0t0 = new MockTaskInfo("taskv0t0", priorityv0, hostsv0t0); + when(mockApp.getVertexIndexForTask(taskv0t0.task)).thenReturn(0); + MockTaskInfo taskv0t1 = new MockTaskInfo("taskv0t1", priorityv0, hostsv0t0); + when(mockApp.getVertexIndexForTask(taskv0t1.task)).thenReturn(0); + MockTaskInfo taskv2t0 = new MockTaskInfo("taskv2t0", priorityv2, hostsv0t0); + when(mockApp.getVertexIndexForTask(taskv2t0.task)).thenReturn(2); + MockTaskInfo taskv2t1 = new MockTaskInfo("taskv2t1", priorityv2, hostsv0t0); + when(mockApp.getVertexIndexForTask(taskv2t1.task)).thenReturn(2); + when(mockApp.getVertexIndexForTask(taskv2t0.task)).thenReturn(2); + + // asks for one task for vertex 2 and start running + TaskRequestCaptor taskRequestCaptor = new TaskRequestCaptor(mockRMClient, + scheduler, drainableAppCallback); + TaskRequest reqv2t0 = taskRequestCaptor.scheduleTask(taskv2t0); + NodeId host1 = NodeId.newInstance("host1", 1); + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 1), 1); + ContainerId cid1 = ContainerId.newContainerId(attemptId, 1); + Container container1 = Container.newInstance(cid1, host1, null, taskv2t0.capability, priorityv2, null); + scheduler.onContainersAllocated(Collections.singletonList(container1)); + drainableAppCallback.drain(); + verify(mockApp).taskAllocated(taskv2t0.task, taskv2t0.cookie, container1); + verify(mockRMClient).removeContainerRequest(reqv2t0); + clock.incrementTime(1000); + + when(mockRMClient.getAvailableResources()).thenReturn(Resources.none()); + scheduler.getProgress(); + scheduler.getProgress(); + scheduler.getProgress(); + drainableAppCallback.drain(); + //ask another task for v2 + TaskRequest reqv2t1 = taskRequestCaptor.scheduleTask(taskv2t1); + scheduler.getProgress(); + scheduler.getProgress(); + scheduler.getProgress(); + drainableAppCallback.drain(); + + clock.incrementTime(1000); + // add a request for vertex 0 but there is no headroom, this should preempt + when(mockRMClient.getAvailableResources()).thenReturn(Resources.none()); + TaskRequest reqv0t0 = taskRequestCaptor.scheduleTask(taskv0t0); + + // should preempt after enough heartbeats to get past preemption interval + scheduler.getProgress(); + scheduler.getProgress(); + scheduler.getProgress(); + drainableAppCallback.drain(); + verify(mockApp, times(1)).preemptContainer(any(ContainerId.class)); + verify(mockApp).preemptContainer(cid1); + String appMsg = "success"; + AppFinalStatus finalStatus = + new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl); + when(mockApp.getFinalAppStatus()).thenReturn(finalStatus); + scheduler.shutdown(); + drainableAppCallback.drain(); + verify(mockRMClient). + unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, + appMsg, appUrl); + verify(mockRMClient).stop(); + } + @Test(timeout=50000) public void testContainerAssignmentReleaseNewContainers() throws Exception { AMRMClientAsyncWrapperForTest mockRMClient = spy(new AMRMClientAsyncWrapperForTest()); From c78d3487f5170321f985c058b8804d25b6eae48c Mon Sep 17 00:00:00 2001 From: Harish JP Date: Sun, 27 Jan 2019 19:27:46 -0800 Subject: [PATCH 137/512] TEZ-4028: Events not visible from proto history logging for s3a filesystem until dag completes (Harish JP, via Gopal V) Signed-off-by: Gopal V --- .../apache/tez/dag/api/TezConfiguration.java | 23 ++- .../proto/ProtoHistoryLoggingService.java | 14 +- .../proto/TestProtoHistoryLoggingService.java | 137 +++++++++++++++--- 3 files changed, 150 insertions(+), 24 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 791e634c90..7c521225f7 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -26,17 +26,17 @@ import java.util.Map; import java.util.Set; -import org.apache.tez.common.annotation.ConfigurationClass; -import org.apache.tez.common.annotation.ConfigurationProperty; -import org.apache.tez.dag.api.EdgeProperty.ConcurrentEdgeTriggerType; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.util.Shell; import org.apache.hadoop.yarn.api.records.LocalResource; +import org.apache.tez.common.annotation.ConfigurationClass; +import org.apache.tez.common.annotation.ConfigurationProperty; +import org.apache.tez.dag.api.EdgeProperty.ConcurrentEdgeTriggerType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import com.google.common.annotations.VisibleForTesting; @@ -56,7 +56,7 @@ public class TezConfiguration extends Configuration { private final static Logger LOG = LoggerFactory.getLogger(TezConfiguration.class); - private static Map PropertyScope = new HashMap(); + private static Map PropertyScope = new HashMap<>(); static { Configuration.addDeprecation("tez.am.counters.max.keys", TezConfiguration.TEZ_COUNTERS_MAX); @@ -1487,6 +1487,17 @@ public TezConfiguration(boolean loadDefaults) { TEZ_PREFIX + "history.logging.proto-sync-window-secs"; public static final long TEZ_HISTORY_LOGGING_PROTO_SYNC_WINDOWN_SECS_DEFAULT = 60L; + /** + * Boolean value. Set this to true, if the underlying file system does not support flush (Ex: s3). + * The dag submitted, initialized and started events are written into a file and closed. The rest + * of the events are written into another file. + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty(type="boolean") + public static final String TEZ_HISTORY_LOGGING_PROTO_SPLIT_DAG_START = + TEZ_PREFIX + "history.logging.split-dag-start"; + public static final boolean TEZ_HISTORY_LOGGING_PROTO_SPLIT_DAG_START_DEFAULT = false; + /** * Long value. The amount of time in seconds to wait to ensure all events for a day is synced * to disk. This should be maximum time variation b/w machines + maximum time to sync file diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java index 206b1c1030..d2e0b4defa 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java @@ -45,6 +45,9 @@ */ public class ProtoHistoryLoggingService extends HistoryLoggingService { private static final Logger LOG = LoggerFactory.getLogger(ProtoHistoryLoggingService.class); + // The file suffix used if we are writing start events and rest into different files. + static final String SPLIT_DAG_EVENTS_FILE_SUFFIX = "_1"; + private final HistoryEventProtoConverter converter = new HistoryEventProtoConverter(); private boolean loggingDisabled = false; @@ -64,6 +67,7 @@ public class ProtoHistoryLoggingService extends HistoryLoggingService { private String appEventsFile; private long appLaunchedEventOffset; + private boolean splitDagStartEvents; public ProtoHistoryLoggingService() { super(ProtoHistoryLoggingService.class.getName()); @@ -75,6 +79,8 @@ protected void serviceInit(Configuration conf) throws Exception { setConfig(conf); loggingDisabled = !conf.getBoolean(TezConfiguration.TEZ_AM_HISTORY_LOGGING_ENABLED, TezConfiguration.TEZ_AM_HISTORY_LOGGING_ENABLED_DEFAULT); + splitDagStartEvents = conf.getBoolean(TezConfiguration.TEZ_HISTORY_LOGGING_PROTO_SPLIT_DAG_START, + TezConfiguration.TEZ_HISTORY_LOGGING_PROTO_SPLIT_DAG_START_DEFAULT); LOG.info("Inited ProtoHistoryLoggingService"); } @@ -171,6 +177,13 @@ private void handleEvent(DAGHistoryEvent event) throws IOException { dagEventsWriter.writeProto(converter.convert(historyEvent)); } else if (dagEventsWriter != null) { dagEventsWriter.writeProto(converter.convert(historyEvent)); + if (splitDagStartEvents && type == HistoryEventType.DAG_STARTED) { + // Close the file and write submitted event offset into manifest. + finishCurrentDag(null); + dagEventsWriter = loggers.getDagEventsLogger().getWriter(dagId.toString() + + "_" + appContext.getApplicationAttemptId().getAttemptId() + + SPLIT_DAG_EVENTS_FILE_SUFFIX); + } } } } @@ -214,7 +227,6 @@ private void finishCurrentDag(DAGFinishedEvent event) throws IOException { // into another dag. IOUtils.closeQuietly(dagEventsWriter); dagEventsWriter = null; - currentDagId = null; dagSubmittedEventOffset = -1; } } diff --git a/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestProtoHistoryLoggingService.java b/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestProtoHistoryLoggingService.java index bc79b0746d..143faed516 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestProtoHistoryLoggingService.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestProtoHistoryLoggingService.java @@ -25,6 +25,7 @@ import java.io.IOException; import java.time.LocalDate; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -45,6 +46,8 @@ import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.events.AppLaunchedEvent; import org.apache.tez.dag.history.events.DAGFinishedEvent; +import org.apache.tez.dag.history.events.DAGInitializedEvent; +import org.apache.tez.dag.history.events.DAGStartedEvent; import org.apache.tez.dag.history.events.DAGSubmittedEvent; import org.apache.tez.dag.history.events.TaskAttemptStartedEvent; import org.apache.tez.dag.history.events.TaskStartedEvent; @@ -72,7 +75,7 @@ public class TestProtoHistoryLoggingService { @Test public void testService() throws Exception { - ProtoHistoryLoggingService service = createService(); + ProtoHistoryLoggingService service = createService(false); service.start(); TezDAGID dagId = TezDAGID.getInstance(appId, 0); List protos = new ArrayList<>(); @@ -87,19 +90,9 @@ public void testService() throws Exception { // Verify dag events are logged. DatePartitionedLogger dagLogger = loggers.getDagEventsLogger(); - Path dagFilePath = dagLogger.getPathForDate(LocalDate.ofEpochDay(0), dagId.toString() + "_" + 1); + Path dagFilePath = dagLogger.getPathForDate(LocalDate.ofEpochDay(0), dagId + "_1"); ProtoMessageReader reader = dagLogger.getReader(dagFilePath); - HistoryEventProto evt = reader.readEvent(); - int ind = 1; - while (evt != null) { - Assert.assertEquals(protos.get(ind), evt); - ind++; - try { - evt = reader.readEvent(); - } catch (EOFException e) { - evt = null; - } - } + assertEventsRead(reader, protos, 1, protos.size()); reader.close(); // Verify app events are logged. @@ -108,7 +101,7 @@ public void testService() throws Exception { ProtoMessageReader appReader = appLogger.getReader(appFilePath); long appOffset = appReader.getOffset(); Assert.assertEquals(protos.get(0), appReader.readEvent()); - reader.close(); + appReader.close(); // Verify manifest events are logged. DatePartitionedLogger manifestLogger = loggers.getManifestEventsLogger(); @@ -125,7 +118,7 @@ public void testService() throws Exception { // Verify offsets in manifest logger. reader = dagLogger.getReader(new Path(manifest.getDagFilePath())); reader.setOffset(manifest.getDagSubmittedEventOffset()); - evt = reader.readEvent(); + HistoryEventProto evt = reader.readEvent(); Assert.assertNotNull(evt); Assert.assertEquals(HistoryEventType.DAG_SUBMITTED.name(), evt.getEventType()); @@ -133,6 +126,7 @@ public void testService() throws Exception { evt = reader.readEvent(); Assert.assertNotNull(evt); Assert.assertEquals(HistoryEventType.DAG_FINISHED.name(), evt.getEventType()); + reader.close(); // Verify manifest file scanner. DagManifesFileScanner scanner = new DagManifesFileScanner(manifestLogger); @@ -141,6 +135,91 @@ public void testService() throws Exception { scanner.close(); } + @Test + public void testServiceSplitEvents() throws Exception { + ProtoHistoryLoggingService service = createService(true); + service.start(); + TezDAGID dagId = TezDAGID.getInstance(appId, 0); + List protos = new ArrayList<>(); + for (DAGHistoryEvent event : makeHistoryEvents(dagId, service)) { + protos.add(new HistoryEventProtoConverter().convert(event.getHistoryEvent())); + service.handle(event); + } + service.stop(); + + TezProtoLoggers loggers = new TezProtoLoggers(); + Assert.assertTrue(loggers.setup(service.getConfig(), clock)); + + // Verify dag events are logged. + DatePartitionedLogger dagLogger = loggers.getDagEventsLogger(); + Path dagFilePath1 = dagLogger.getPathForDate(LocalDate.ofEpochDay(0), dagId + "_1"); + Path dagFilePath2 = dagLogger.getPathForDate(LocalDate.ofEpochDay(0), dagId + "_1" + + ProtoHistoryLoggingService.SPLIT_DAG_EVENTS_FILE_SUFFIX); + + try (ProtoMessageReader reader = dagLogger.getReader(dagFilePath1)) { + assertEventsRead(reader, protos, 1, 1 + 3); + } + + try (ProtoMessageReader reader = dagLogger.getReader(dagFilePath2)) { + assertEventsRead(reader, protos, 4, protos.size()); + } + + // Verify app events are logged. + DatePartitionedLogger appLogger = loggers.getAppEventsLogger(); + Path appFilePath = appLogger.getPathForDate(LocalDate.ofEpochDay(0), attemptId.toString()); + ProtoMessageReader appReader = appLogger.getReader(appFilePath); + long appOffset = appReader.getOffset(); + Assert.assertEquals(protos.get(0), appReader.readEvent()); + appReader.close(); + + // Verify manifest events are logged. + DatePartitionedLogger manifestLogger = loggers.getManifestEventsLogger(); + DagManifesFileScanner scanner = new DagManifesFileScanner(manifestLogger); + Path manifestFilePath = manifestLogger.getPathForDate( + LocalDate.ofEpochDay(0), attemptId.toString()); + ProtoMessageReader manifestReader = manifestLogger.getReader( + manifestFilePath); + ManifestEntryProto manifest = manifestReader.readEvent(); + Assert.assertEquals(manifest, scanner.getNext()); + Assert.assertEquals(appId.toString(), manifest.getAppId()); + Assert.assertEquals(dagId.toString(), manifest.getDagId()); + Assert.assertEquals(dagFilePath1.toString(), manifest.getDagFilePath()); + Assert.assertEquals(appFilePath.toString(), manifest.getAppFilePath()); + Assert.assertEquals(appOffset, manifest.getAppLaunchedEventOffset()); + Assert.assertEquals(-1, manifest.getDagFinishedEventOffset()); + + HistoryEventProto evt = null; + // Verify offsets in manifest logger. + try (ProtoMessageReader reader = dagLogger.getReader( + new Path(manifest.getDagFilePath()))) { + reader.setOffset(manifest.getDagSubmittedEventOffset()); + evt = reader.readEvent(); + Assert.assertNotNull(evt); + Assert.assertEquals(HistoryEventType.DAG_SUBMITTED.name(), evt.getEventType()); + } + + manifest = manifestReader.readEvent(); + Assert.assertEquals(manifest, scanner.getNext()); + Assert.assertEquals(appId.toString(), manifest.getAppId()); + Assert.assertEquals(dagId.toString(), manifest.getDagId()); + Assert.assertEquals(dagFilePath2.toString(), manifest.getDagFilePath()); + Assert.assertEquals(appFilePath.toString(), manifest.getAppFilePath()); + Assert.assertEquals(appOffset, manifest.getAppLaunchedEventOffset()); + Assert.assertEquals(-1, manifest.getDagSubmittedEventOffset()); + + try (ProtoMessageReader reader = dagLogger.getReader( + new Path(manifest.getDagFilePath()))) { + reader.setOffset(manifest.getDagFinishedEventOffset()); + evt = reader.readEvent(); + Assert.assertNotNull(evt); + Assert.assertEquals(HistoryEventType.DAG_FINISHED.name(), evt.getEventType()); + } + + // Verify manifest file scanner. + Assert.assertNull(scanner.getNext()); + scanner.close(); + } + private List makeHistoryEvents(TezDAGID dagId, ProtoHistoryLoggingService service) { List historyEvents = new ArrayList<>(); @@ -152,6 +231,11 @@ private List makeHistoryEvents(TezDAGID dagId, new VersionInfo("component", "1.1.0", "rev1", "20120101", "git.apache.org") {}))); historyEvents.add(new DAGHistoryEvent(dagId, new DAGSubmittedEvent(dagId, time, DAGPlan.getDefaultInstance(), attemptId, null, user, conf, null, "default"))); + historyEvents.add(new DAGHistoryEvent(dagId, new DAGInitializedEvent(dagId, time + 1, user, + "test_dag", Collections.emptyMap()))); + historyEvents.add(new DAGHistoryEvent(dagId, new DAGStartedEvent(dagId, time + 2, user, + "test_dag"))); + TezVertexID vertexID = TezVertexID.getInstance(dagId, 1); historyEvents.add(new DAGHistoryEvent(dagId, new VertexStartedEvent(vertexID, time, time))); TezTaskID tezTaskID = TezTaskID.getInstance(vertexID, 1); @@ -168,7 +252,7 @@ private List makeHistoryEvents(TezDAGID dagId, } private static class FixedClock implements Clock { - final Clock clock = new SystemClock(); + final Clock clock = SystemClock.getInstance(); final long diff; public FixedClock(long startTime) { @@ -181,7 +265,7 @@ public long getTime() { } } - private ProtoHistoryLoggingService createService() throws IOException { + private ProtoHistoryLoggingService createService(boolean splitEvents) throws IOException { ProtoHistoryLoggingService service = new ProtoHistoryLoggingService(); clock = new FixedClock(0); // Start time is always first day, easier to write tests. AppContext appContext = mock(AppContext.class); @@ -194,7 +278,26 @@ private ProtoHistoryLoggingService createService() throws IOException { Configuration conf = new Configuration(false); String basePath = tempFolder.newFolder().getAbsolutePath(); conf.set(TezConfiguration.TEZ_HISTORY_LOGGING_PROTO_BASE_DIR, basePath); + conf.setBoolean(TezConfiguration.TEZ_HISTORY_LOGGING_PROTO_SPLIT_DAG_START, splitEvents); service.init(conf); return service; } + + private void assertEventsRead(ProtoMessageReader reader, + List protos, int start, int finish) throws Exception { + for (int i = start; i < finish; ++i) { + try { + HistoryEventProto evt = reader.readEvent(); + Assert.assertEquals(protos.get(i), evt); + } catch (EOFException e) { + Assert.fail("Unexpected eof"); + } + } + try { + HistoryEventProto evt = reader.readEvent(); + Assert.assertNull(evt); + } catch (EOFException e) { + // Expected. + } + } } From 3162aab3411f885db3e103f0c16c92de3e2ca46b Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Tue, 5 Feb 2019 11:33:43 -0600 Subject: [PATCH 138/512] TEZ-4036. TestMockDAGAppMaster#testInternalPreemption should assert for failed state (Kuhu Shukla via jeagles) --- .../test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java index 859537bb68..1a4659593f 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java @@ -215,7 +215,8 @@ public void testInternalPreemption() throws Exception { TezVertexID vertexId = TezVertexID.getInstance(dagImpl.getID(), 0); TezTaskAttemptID killedTaId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 0); TaskAttempt killedTa = dagImpl.getVertex(vA.getName()).getTask(0).getAttempt(killedTaId); - Assert.assertEquals(TaskAttemptState.KILLED, killedTa.getState()); + //Refer to TEZ-3950 + Assert.assertTrue(killedTa.getState().equals(TaskAttemptState.KILLED) || killedTa.getState().equals(TaskAttemptState.FAILED)); tezClient.stop(); } From ffaedf6b34c243b1d4395e04883c0f59c635d676 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Tue, 12 Feb 2019 15:26:25 -0600 Subject: [PATCH 139/512] TEZ-4037. Add back DAG search status KILLED (Jonathan Eagles via kshukla) --- .../main/webapp/app/templates/components/dags-page-search.hbs | 1 + 1 file changed, 1 insertion(+) diff --git a/tez-ui/src/main/webapp/app/templates/components/dags-page-search.hbs b/tez-ui/src/main/webapp/app/templates/components/dags-page-search.hbs index 7def955670..3a6fc9b323 100644 --- a/tez-ui/src/main/webapp/app/templates/components/dags-page-search.hbs +++ b/tez-ui/src/main/webapp/app/templates/components/dags-page-search.hbs @@ -52,6 +52,7 @@ +
From f9da6bbb83fcca59eff7a721f5d5f9e511f23f50 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Wed, 13 Feb 2019 13:07:00 -0600 Subject: [PATCH 140/512] TEZ-4040. Upgrade RoaringBitmap version to avoid NoSuchMethodError (Jonathan Eagles via kshukla) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 1d46ffd01e..6e18a7c671 100644 --- a/pom.xml +++ b/pom.xml @@ -213,7 +213,7 @@ org.roaringbitmap RoaringBitmap - 0.4.9 + 0.5.21 org.slf4j From 62a1140be8bac21c4f18c3bf6f9cc124ea1a1bfb Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Wed, 13 Feb 2019 14:59:57 -0600 Subject: [PATCH 141/512] TEZ-4041. TestExtServicesWithLocalMode fails in docker (Jonathan Eagles via kshukla) --- .../dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java index 732c81af67..377217b0cf 100644 --- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java +++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java @@ -61,8 +61,7 @@ public TezTestServiceTaskCommunicatorImpl( SubmitWorkRequestProto.Builder baseBuilder = SubmitWorkRequestProto.newBuilder(); - // TODO Avoid reading this from the environment - baseBuilder.setUser(System.getenv(ApplicationConstants.Environment.USER.name())); + baseBuilder.setUser(System.getProperty("user.name")); baseBuilder.setApplicationIdString( taskCommunicatorContext.getApplicationAttemptId().getApplicationId().toString()); baseBuilder From 1990438047fd92f73d3a8b571895ae3fc2c6ee05 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Fri, 15 Feb 2019 15:44:07 -0600 Subject: [PATCH 142/512] TEZ-4043. Create a yetus compatible checkstyle configuration (Jonathan Eagles via kshukla) --- pom.xml | 29 +++ tez-build-tools/pom.xml | 26 +++ .../main/resources/checkstyle/checkstyle.xml | 198 ++++++++++++++++++ .../resources/checkstyle/suppressions.xml | 21 ++ 4 files changed, 274 insertions(+) create mode 100644 tez-build-tools/pom.xml create mode 100644 tez-build-tools/src/main/resources/checkstyle/checkstyle.xml create mode 100644 tez-build-tools/src/main/resources/checkstyle/suppressions.xml diff --git a/pom.xml b/pom.xml index 6e18a7c671..4760af1402 100644 --- a/pom.xml +++ b/pom.xml @@ -61,6 +61,8 @@ 3.0.1 2.10.4 2.4.3 + 3.0.0 + 8.16 1.3.6 @@ -750,6 +752,7 @@ hadoop-shim tez-api + tez-build-tools tez-common tez-runtime-library tez-runtime-internals @@ -906,6 +909,32 @@ + + org.apache.maven.plugins + maven-checkstyle-plugin + ${maven-checkstyle-plugin.version} + + + org.apache.tez + tez-build-tools + ${project.version} + + + com.puppycrawl.tools + checkstyle + ${checkstyle.version} + + + + checkstyle/checkstyle.xml + checkstyle/suppressions.xml + true + false + xml + html + ${project.build.directory}/test/checkstyle-errors.xml + + org.owasp dependency-check-maven diff --git a/tez-build-tools/pom.xml b/tez-build-tools/pom.xml new file mode 100644 index 0000000000..baf75b18ec --- /dev/null +++ b/tez-build-tools/pom.xml @@ -0,0 +1,26 @@ + + + + + 4.0.0 + + org.apache.tez + tez + 0.10.1-SNAPSHOT + + tez-build-tools + diff --git a/tez-build-tools/src/main/resources/checkstyle/checkstyle.xml b/tez-build-tools/src/main/resources/checkstyle/checkstyle.xml new file mode 100644 index 0000000000..ef661c7a60 --- /dev/null +++ b/tez-build-tools/src/main/resources/checkstyle/checkstyle.xml @@ -0,0 +1,198 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/tez-build-tools/src/main/resources/checkstyle/suppressions.xml b/tez-build-tools/src/main/resources/checkstyle/suppressions.xml new file mode 100644 index 0000000000..ccc89c8bf0 --- /dev/null +++ b/tez-build-tools/src/main/resources/checkstyle/suppressions.xml @@ -0,0 +1,21 @@ + + + + + + + From 1234c028b5c088a4d3492a2aab6c91f4f0d3d19b Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Fri, 15 Feb 2019 16:29:21 -0600 Subject: [PATCH 143/512] TEZ-4034. Column selector filter should be case-insensitive (Jacob Tolar via jeagles) --- .../webapp/app/components/column-selector.js | 2 +- .../components/column-selector-test.js | 20 +++++++++++++++++++ 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/tez-ui/src/main/webapp/app/components/column-selector.js b/tez-ui/src/main/webapp/app/components/column-selector.js index 8f9ac13551..94739f3cb0 100644 --- a/tez-ui/src/main/webapp/app/components/column-selector.js +++ b/tez-ui/src/main/webapp/app/components/column-selector.js @@ -65,7 +65,7 @@ export default Ember.Component.extend({ } return options.filter(function (option) { - return option.get('displayText').match(searchText); + return option.get('displayText').match(new RegExp(searchText, 'i')); }); }), diff --git a/tez-ui/src/main/webapp/tests/integration/components/column-selector-test.js b/tez-ui/src/main/webapp/tests/integration/components/column-selector-test.js index 0034059eb6..9fe7d8a2d8 100644 --- a/tez-ui/src/main/webapp/tests/integration/components/column-selector-test.js +++ b/tez-ui/src/main/webapp/tests/integration/components/column-selector-test.js @@ -85,3 +85,23 @@ test('searchText test', function(assert) { assert.equal(this.$(".select-option").text().trim(), ''); }); + +test('case-insensitive searchText test', function(assert) { + + this.setProperties({ + searchText: "test", + content: { + visibleColumnIDs: { + testID: true, + }, + columns: [Ember.Object.create({ + id: "testID", + headerTitle: "Test Column" + })] + } + }); + + this.render(hbs`{{column-selector content=content searchText=searchText}}`); + + assert.equal(this.$(".select-option").text().trim(), 'Test Column'); +}); From ff999d8156a345503f5e3d680a36d6a95186bf79 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Fri, 15 Feb 2019 17:11:54 -0600 Subject: [PATCH 144/512] TEZ-3952. Allow Tez task speculation to grant greater customization of certain parameters (Nishant Dash via jeagles) --- .../apache/tez/dag/api/TezConfiguration.java | 52 +++++++++++++++++++ .../tez/dag/app/dag/impl/VertexImpl.java | 3 ++ .../speculation/legacy/LegacySpeculator.java | 51 ++++++++++++++---- .../apache/tez/dag/app/TestSpeculation.java | 17 ++++++ .../tez/mapreduce/hadoop/DeprecatedKeys.java | 10 ++++ .../tez/mapreduce/hadoop/MRJobConfig.java | 9 ++++ 6 files changed, 132 insertions(+), 10 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 7c521225f7..4566600525 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -551,6 +551,58 @@ public TezConfiguration(boolean loadDefaults) { TEZ_AM_PREFIX + "legacy.speculative.single.task.vertex.timeout"; public static final long TEZ_AM_LEGACY_SPECULATIVE_SINGLE_TASK_VERTEX_TIMEOUT_DEFAULT = -1; + /** + * Long value. Specifies amount of time (in ms) that needs to elapse to do the next round of + * speculation if there is no task speculated in this round. + */ + @Unstable + @ConfigurationScope(Scope.VERTEX) + @ConfigurationProperty(type="long") + public static final String TEZ_AM_SOONEST_RETRY_AFTER_NO_SPECULATE = + TEZ_AM_PREFIX + "soonest.retry.after.no.speculate"; + public static final long TEZ_AM_SOONEST_RETRY_AFTER_NO_SPECULATE_DEFAULT = 1000L * 1L; + + /** + * Long value. Specifies amount of time (in ms) that needs to elapse to do the next round of + * speculation if there are tasks speculated in this round. + */ + @Unstable + @ConfigurationScope(Scope.VERTEX) + @ConfigurationProperty(type="long") + public static final String TEZ_AM_SOONEST_RETRY_AFTER_SPECULATE= + TEZ_AM_PREFIX + "soonest.retry.after.speculate"; + public static final long TEZ_AM_SOONEST_RETRY_AFTER_SPECULATE_DEFAULT = 1000L * 15L; + + /** + * Double value. The max percent (0-1) of running tasks that can be speculatively re-executed at any time. + */ + @Unstable + @ConfigurationScope(Scope.VERTEX) + @ConfigurationProperty(type="double") + public static final String TEZ_AM_PROPORTION_RUNNING_TASKS_SPECULATABLE = + TEZ_AM_PREFIX + "proportion.running.tasks.speculatable"; + public static final double TEZ_AM_PROPORTION_RUNNING_TASKS_SPECULATABLE_DEFAULT = 0.1; + + /** + * Double value. The max percent (0-1) of all tasks that can be speculatively re-executed at any time. + */ + @Unstable + @ConfigurationScope(Scope.VERTEX) + @ConfigurationProperty(type="double") + public static final String TEZ_AM_PROPORTION_TOTAL_TASKS_SPECULATABLE = + TEZ_AM_PREFIX + "proportion.total.tasks.speculatable"; + public static final double TEZ_AM_PROPORTION_TOTAL_TASKS_SPECULATABLE_DEFAULT = 0.01; + + /** + * Integer value. The minimum allowed tasks that can be speculatively re-executed at any time. + */ + @Unstable + @ConfigurationScope(Scope.VERTEX) + @ConfigurationProperty(type="integer") + public static final String TEZ_AM_MINIMUM_ALLOWED_SPECULATIVE_TASKS = + TEZ_AM_PREFIX + "minimum.allowed.speculative.tasks"; + public static final int TEZ_AM_MINIMUM_ALLOWED_SPECULATIVE_TASKS_DEFAULT = 10; + /** * Int value. Upper limit on the number of threads user to launch containers in the app * master. Expert level setting. diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index a4d2de183a..a2ef475f5b 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -4762,4 +4762,7 @@ public boolean getTaskRescheduleRelaxedLocality() { return maxAllowedTimeForTaskReadErrorSec; } } + + @VisibleForTesting + public LegacySpeculator getSpeculator() { return speculator; } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java index c132fb16b8..3e7c2c0088 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java @@ -24,6 +24,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import com.google.common.annotations.VisibleForTesting; import org.apache.tez.dag.api.TezConfiguration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,12 +63,12 @@ public class LegacySpeculator { private static final long NOT_RUNNING = Long.MIN_VALUE + 4; private static final long TOO_LATE_TO_SPECULATE = Long.MIN_VALUE + 5; - private static final long SOONEST_RETRY_AFTER_NO_SPECULATE = 1000L * 1L; - private static final long SOONEST_RETRY_AFTER_SPECULATE = 1000L * 15L; + private final long soonestRetryAfterNoSpeculate; + private final long soonestRetryAfterSpeculate; - private static final double PROPORTION_RUNNING_TASKS_SPECULATABLE = 0.1; - private static final double PROPORTION_TOTAL_TASKS_SPECULATABLE = 0.01; - private static final int MINIMUM_ALLOWED_SPECULATIVE_TASKS = 10; + private final double proportionRunningTasksSpeculatable; + private final double proportionTotalTasksSpeculatable; + private final int minimumAllowedSpeculativeTasks; private static final int VERTEX_SIZE_THRESHOLD_FOR_TIMEOUT_SPECULATION = 1; private static final Logger LOG = LoggerFactory.getLogger(LegacySpeculator.class); @@ -93,6 +94,21 @@ public class LegacySpeculator { private final Clock clock; private long nextSpeculateTime = Long.MIN_VALUE; + @VisibleForTesting + public int getMinimumAllowedSpeculativeTasks() { return minimumAllowedSpeculativeTasks;} + + @VisibleForTesting + public double getProportionTotalTasksSpeculatable() { return proportionTotalTasksSpeculatable;} + + @VisibleForTesting + public double getProportionRunningTasksSpeculatable() { return proportionRunningTasksSpeculatable;} + + @VisibleForTesting + public long getSoonestRetryAfterNoSpeculate() { return soonestRetryAfterNoSpeculate;} + + @VisibleForTesting + public long getSoonestRetryAfterSpeculate() { return soonestRetryAfterSpeculate;} + public LegacySpeculator(Configuration conf, AppContext context, Vertex vertex) { this(conf, context.getClock(), vertex); } @@ -120,6 +136,21 @@ public LegacySpeculator(Configuration conf, Clock clock, Vertex vertex) { taskTimeout = conf.getLong( TezConfiguration.TEZ_AM_LEGACY_SPECULATIVE_SINGLE_TASK_VERTEX_TIMEOUT, TezConfiguration.TEZ_AM_LEGACY_SPECULATIVE_SINGLE_TASK_VERTEX_TIMEOUT_DEFAULT); + soonestRetryAfterNoSpeculate = conf.getLong( + TezConfiguration.TEZ_AM_SOONEST_RETRY_AFTER_NO_SPECULATE, + TezConfiguration.TEZ_AM_SOONEST_RETRY_AFTER_NO_SPECULATE_DEFAULT); + soonestRetryAfterSpeculate = conf.getLong( + TezConfiguration.TEZ_AM_SOONEST_RETRY_AFTER_SPECULATE, + TezConfiguration.TEZ_AM_SOONEST_RETRY_AFTER_SPECULATE_DEFAULT); + proportionRunningTasksSpeculatable = conf.getDouble( + TezConfiguration.TEZ_AM_PROPORTION_RUNNING_TASKS_SPECULATABLE, + TezConfiguration.TEZ_AM_PROPORTION_RUNNING_TASKS_SPECULATABLE_DEFAULT); + proportionTotalTasksSpeculatable = conf.getDouble( + TezConfiguration.TEZ_AM_PROPORTION_TOTAL_TASKS_SPECULATABLE, + TezConfiguration.TEZ_AM_PROPORTION_TOTAL_TASKS_SPECULATABLE_DEFAULT); + minimumAllowedSpeculativeTasks = conf.getInt( + TezConfiguration.TEZ_AM_MINIMUM_ALLOWED_SPECULATIVE_TASKS, + TezConfiguration.TEZ_AM_MINIMUM_ALLOWED_SPECULATIVE_TASKS_DEFAULT); } /* ************************************************************* */ @@ -133,8 +164,8 @@ void maybeSpeculate() { int speculations = maybeScheduleASpeculation(); long mininumRecomp - = speculations > 0 ? SOONEST_RETRY_AFTER_SPECULATE - : SOONEST_RETRY_AFTER_NO_SPECULATE; + = speculations > 0 ? soonestRetryAfterSpeculate + : soonestRetryAfterNoSpeculate; long wait = Math.max(mininumRecomp, clock.getTime() - now); @@ -358,8 +389,8 @@ private int maybeScheduleASpeculation() { Map tasks = vertex.getTasks(); int numberAllowedSpeculativeTasks - = (int) Math.max(MINIMUM_ALLOWED_SPECULATIVE_TASKS, - PROPORTION_TOTAL_TASKS_SPECULATABLE * tasks.size()); + = (int) Math.max(minimumAllowedSpeculativeTasks, + proportionTotalTasksSpeculatable * tasks.size()); TezTaskID bestTaskID = null; long bestSpeculationValue = -1L; @@ -388,7 +419,7 @@ private int maybeScheduleASpeculation() { } numberAllowedSpeculativeTasks = (int) Math.max(numberAllowedSpeculativeTasks, - PROPORTION_RUNNING_TASKS_SPECULATABLE * numberRunningTasks); + proportionRunningTasksSpeculatable * numberRunningTasks); // If we found a speculation target, fire it off if (bestTaskID != null diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java index 1df5af4fa3..e1aa44890b 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java @@ -44,6 +44,8 @@ import org.apache.tez.dag.app.dag.Task; import org.apache.tez.dag.app.dag.TaskAttempt; import org.apache.tez.dag.app.dag.impl.DAGImpl; +import org.apache.tez.dag.app.dag.impl.VertexImpl; +import org.apache.tez.dag.app.dag.speculation.legacy.LegacySpeculator; import org.apache.tez.dag.library.vertexmanager.ShuffleVertexManager; import org.apache.tez.dag.records.TaskAttemptTerminationCause; import org.apache.tez.dag.records.TezTaskAttemptID; @@ -148,6 +150,13 @@ public void testSingleTaskSpeculation() throws Exception { } public void testBasicSpeculation(boolean withProgress) throws Exception { + + defaultConf.setInt(TezConfiguration.TEZ_AM_MINIMUM_ALLOWED_SPECULATIVE_TASKS, 20); + defaultConf.setDouble(TezConfiguration.TEZ_AM_PROPORTION_TOTAL_TASKS_SPECULATABLE, 0.2); + defaultConf.setDouble(TezConfiguration.TEZ_AM_PROPORTION_RUNNING_TASKS_SPECULATABLE, 0.25); + defaultConf.setLong(TezConfiguration.TEZ_AM_SOONEST_RETRY_AFTER_NO_SPECULATE, 2000); + defaultConf.setLong(TezConfiguration.TEZ_AM_SOONEST_RETRY_AFTER_SPECULATE, 10000); + DAG dag = DAG.create("test"); Vertex vA = Vertex.create("A", ProcessorDescriptor.create("Proc.class"), 5); dag.addVertex(vA); @@ -185,6 +194,14 @@ public void testBasicSpeculation(boolean withProgress) throws Exception { Assert.assertEquals(1, v.getAllCounters().findCounter(TaskCounter.NUM_SPECULATIONS) .getValue()); } + + LegacySpeculator speculator = ((VertexImpl) dagImpl.getVertex(vA.getName())).getSpeculator(); + Assert.assertEquals(20, speculator.getMinimumAllowedSpeculativeTasks()); + Assert.assertEquals(.2, speculator.getProportionTotalTasksSpeculatable(), 0); + Assert.assertEquals(.25, speculator.getProportionRunningTasksSpeculatable(), 0); + Assert.assertEquals(2000, speculator.getSoonestRetryAfterNoSpeculate()); + Assert.assertEquals(10000, speculator.getSoonestRetryAfterSpeculate()); + tezClient.stop(); } diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java index 9ae58c0fce..d9b0930bb2 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java @@ -161,6 +161,16 @@ private static void populateMRToTezRuntimeParamMap() { registerMRToRuntimeKeyTranslation(MRJobConfig.MAP_OUTPUT_COMPRESS_CODEC, TezRuntimeConfiguration.TEZ_RUNTIME_COMPRESS_CODEC); registerMRToRuntimeKeyTranslation(MRJobConfig.MAPREDUCE_JOB_USER_CLASSPATH_FIRST, TezConfiguration.TEZ_USER_CLASSPATH_FIRST); + + registerMRToRuntimeKeyTranslation(MRJobConfig.RETRY_AFTER_NO_SPECULATE, TezConfiguration.TEZ_AM_SOONEST_RETRY_AFTER_NO_SPECULATE); + + registerMRToRuntimeKeyTranslation(MRJobConfig.RETRY_AFTER_SPECULATE, TezConfiguration.TEZ_AM_SOONEST_RETRY_AFTER_SPECULATE); + + registerMRToRuntimeKeyTranslation(MRJobConfig.SPECULATIVECAP_RUNNING_TASKS, TezConfiguration.TEZ_AM_PROPORTION_RUNNING_TASKS_SPECULATABLE); + + registerMRToRuntimeKeyTranslation(MRJobConfig.SPECULATIVECAP_TOTAL_TASKS, TezConfiguration.TEZ_AM_PROPORTION_TOTAL_TASKS_SPECULATABLE); + + registerMRToRuntimeKeyTranslation(MRJobConfig.MINIMUM_ALLOWED_TASKS, TezConfiguration.TEZ_AM_MINIMUM_ALLOWED_SPECULATIVE_TASKS); } private static void addDeprecatedKeys() { diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRJobConfig.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRJobConfig.java index 334a7dba3d..cd6fd443a6 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRJobConfig.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRJobConfig.java @@ -85,6 +85,15 @@ public interface MRJobConfig { public static final String SKIP_OUTDIR = "mapreduce.job.skip.outdir"; + public static final String SPECULATIVECAP_RUNNING_TASKS = "mapreduce.job.speculative.speculative-cap-running-tasks"; + + public static final String RETRY_AFTER_NO_SPECULATE = "mapreduce.job.speculative.retry-after-no-speculate"; + + public static final String RETRY_AFTER_SPECULATE = "mapreduce.job.speculative.retry-after-speculate"; + + public static final String MINIMUM_ALLOWED_TASKS = "mapreduce.job.speculative.minimum-allowed-tasks"; + + public static final String SPECULATIVECAP_TOTAL_TASKS = "mapreduce.job.speculative.speculative-cap-total-tasks"; public static final String JOB_LOCAL_DIR = "mapreduce.job.local.dir"; From 4876dc26ec2a1c365be9e5cc76a016ae715d9caa Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Tue, 19 Feb 2019 11:17:15 -0600 Subject: [PATCH 145/512] TEZ-4035. Tez master breaks with YARN 3.2.0 ApplicationReport API change (jeagles) --- .../apache/tez/mapreduce/client/NotRunningJob.java | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/NotRunningJob.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/NotRunningJob.java index e178948041..6cd92361cd 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/NotRunningJob.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/NotRunningJob.java @@ -84,11 +84,14 @@ private ApplicationReport getUnknownApplicationReport() { ApplicationAttemptId unknownAttemptId = recordFactory .newRecordInstance(ApplicationAttemptId.class); - // Setting AppState to NEW and finalStatus to UNDEFINED as they are never - // used for a non running job - return ApplicationReport.newInstance(unknownAppId, unknownAttemptId, "N/A", - "N/A", "N/A", "N/A", 0, null, YarnApplicationState.NEW, "N/A", "N/A", - 0, 0, FinalApplicationStatus.UNDEFINED, null, "N/A", 0.0f, "TEZ_MRR", null); + ApplicationReport report = recordFactory.newRecordInstance(ApplicationReport.class); + report.setUser("N/A"); + report.setName("N/A"); + report.setDiagnostics("N/A"); + report.setTrackingUrl("N/A"); + report.setStartTime(0); + report.setFinishTime(0); + return report; } NotRunningJob(ApplicationReport applicationReport, JobState jobState) { From 0093f8b4d9ea38fcdfeb802fe3206c4b2ae8f077 Mon Sep 17 00:00:00 2001 From: Ying Han Date: Mon, 25 Feb 2019 22:38:01 -0600 Subject: [PATCH 146/512] TEZ-4042. Speculative attempts should avoid running on the same node --- .../apache/tez/dag/app/dag/TaskAttempt.java | 5 +++ .../tez/dag/app/dag/impl/TaskAttemptImpl.java | 16 ++++++++-- .../apache/tez/dag/app/dag/impl/TaskImpl.java | 15 ++++++--- .../dag/app/rm/DagAwareYarnTaskScheduler.java | 32 ++++++++++++++++--- .../dag/app/rm/YarnTaskSchedulerService.java | 12 +++++-- .../tez/dag/app/dag/impl/TestTaskAttempt.java | 2 +- .../tez/dag/app/dag/impl/TestTaskImpl.java | 7 +++- .../app/rm/TestDagAwareYarnTaskScheduler.java | 19 +++++++++++ .../tez/dag/app/rm/TestTaskScheduler.java | 18 +++++++++++ 9 files changed, 110 insertions(+), 16 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java index ba09bd9589..d0fec5c948 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java @@ -19,6 +19,7 @@ package org.apache.tez.dag.app.dag; import java.util.List; +import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.hadoop.yarn.api.records.Container; @@ -136,4 +137,8 @@ public void setLocalityCounter(DAGCounter localityCounter) { */ long getFinishTime(); + /** + * @return the set of nodes on which sibling attempts were running on. + */ + Set getNodesWithSiblingRunningAttempts(); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java index 310733018e..46080528f1 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java @@ -228,6 +228,12 @@ public static DataEventDependencyInfo fromProto(DataEventDependencyInfoProto pro private final boolean leafVertex; private TezTaskAttemptID creationCausalTA; + // Record the set of nodes on which sibling attempts were running on, at the time of + // this attempt being scheduled. This set is empty for original task attempt, and + // non-empty when current task attempt is a speculative one, in which case scheduler + // should try to schedule the speculative attempt on to a node other than the one(s) + // recorded in this set. + private Set nodesWithSiblingRunningAttempts; private long creationTime; private long scheduledTime; @@ -540,7 +546,7 @@ public TaskAttemptImpl(TezTaskAttemptID attemptId, EventHandler eventHandler, Vertex vertex, TaskLocationHint locationHint, TaskSpec taskSpec) { this(attemptId, eventHandler, taskCommunicatorManagerInterface, conf, clock, taskHeartbeatHandler, appContext, isRescheduled, resource, containerContext, leafVertex, - vertex, locationHint, taskSpec, null); + vertex, locationHint, taskSpec, null, null); } @SuppressWarnings("rawtypes") @@ -550,7 +556,7 @@ public TaskAttemptImpl(TezTaskAttemptID attemptId, EventHandler eventHandler, boolean isRescheduled, Resource resource, ContainerContext containerContext, boolean leafVertex, Vertex vertex, TaskLocationHint locationHint, TaskSpec taskSpec, - TezTaskAttemptID schedulingCausalTA) { + TezTaskAttemptID schedulingCausalTA, Set nodesWithSiblingRunningAttempts) { ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock(); this.readLock = rwLock.readLock(); @@ -566,6 +572,7 @@ public TaskAttemptImpl(TezTaskAttemptID attemptId, EventHandler eventHandler, this.locationHint = locationHint; this.taskSpec = taskSpec; this.creationCausalTA = schedulingCausalTA; + this.nodesWithSiblingRunningAttempts = nodesWithSiblingRunningAttempts; this.creationTime = clock.getTime(); this.reportedStatus = new TaskAttemptStatus(this.attemptId); @@ -608,6 +615,11 @@ public TezTaskAttemptID getSchedulingCausalTA() { return creationCausalTA; } + @Override + public Set getNodesWithSiblingRunningAttempts() { + return nodesWithSiblingRunningAttempts; + } + @Override public TaskAttemptReport getReport() { TaskAttemptReport result = Records.newRecord(TaskAttemptReport.class); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java index 9e1d85f9e9..9e3c5a82ff 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java @@ -26,6 +26,8 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -37,6 +39,7 @@ import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.tez.dag.app.dag.event.TaskEventTAFailed; import org.apache.tez.runtime.api.TaskFailureType; import org.slf4j.Logger; @@ -149,6 +152,9 @@ public class TaskImpl implements Task, EventHandler { // track the status of TaskAttempt (true mean completed, false mean uncompleted) private final Map taskAttemptStatus = new HashMap(); + private final Set nodesWithRunningAttempts = Collections + .newSetFromMap(new ConcurrentHashMap()); + private static final SingleArcTransition ATTEMPT_KILLED_TRANSITION = new AttemptKilledTransition(); private static final SingleArcTransition @@ -744,7 +750,7 @@ TaskAttemptImpl createAttempt(int attemptNumber, TezTaskAttemptID schedulingCaus return new TaskAttemptImpl(attemptId, eventHandler, taskCommunicatorManagerInterface, conf, clock, taskHeartbeatHandler, appContext, (failedAttempts > 0), taskResource, containerContext, leafVertex, getVertex(), - locationHint, taskSpec, schedulingCausalTA); + locationHint, taskSpec, schedulingCausalTA, nodesWithRunningAttempts); } @Override @@ -1009,14 +1015,15 @@ private static class RedundantScheduleTransition public void transition(TaskImpl task, TaskEvent event) { LOG.info("Scheduling a redundant attempt for task " + task.taskId); task.counters.findCounter(TaskCounter.NUM_SPECULATIONS).increment(1); - TezTaskAttemptID earliestUnfinishedAttempt = null; + TaskAttempt earliestUnfinishedAttempt = null; for (TaskAttempt ta : task.attempts.values()) { // find the oldest running attempt if (!ta.isFinished()) { - earliestUnfinishedAttempt = ta.getID(); + earliestUnfinishedAttempt = ta; + task.nodesWithRunningAttempts.add(ta.getNodeId()); } } - task.addAndScheduleAttempt(earliestUnfinishedAttempt); + task.addAndScheduleAttempt(earliestUnfinishedAttempt.getID()); } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java index 1cdc217e20..3191c48758 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java @@ -45,6 +45,7 @@ import org.apache.tez.common.ContainerSignatureMatcher; import org.apache.tez.common.TezUtils; import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.app.dag.TaskAttempt; import org.apache.tez.serviceplugins.api.DagInfo; import org.apache.tez.serviceplugins.api.TaskAttemptEndReason; import org.apache.tez.serviceplugins.api.TaskScheduler; @@ -398,6 +399,9 @@ private void tryAssignNewContainer(HeldContainer hc, String location, for (Collection requests : results) { if (!requests.isEmpty()) { TaskRequest request = requests.iterator().next(); + if (maybeChangeNode(request, hc.getContainer().getNodeId())) { + continue; + } assignContainer(request, hc, location); assignments.add(new Assignment(request, hc.getContainer())); return; @@ -515,6 +519,9 @@ private TaskRequest tryAssignReuseContainerForAffinity(HeldContainer hc) { if (requestTracker.isRequestBlocked(request)) { LOG.debug("Cannot assign task {} to container {} since vertex {} is a descendant of pending tasks", request.getTask(), hc.getId(), request.getVertexIndex()); + } else if (maybeChangeNode(request, hc.getContainer().getNodeId())) { + LOG.debug("Cannot assign task {} to container {} since node {} is running sibling attempts", + request.getTask(), hc.getId(), request.getVertexIndex()); } else { assignContainer(request, hc, hc.getId()); return request; @@ -543,8 +550,10 @@ private TaskRequest tryAssignReuseContainerForPriority(HeldContainer hc, String Object signature = hc.getSignature(); if (signature == null || signatureMatcher.isSuperSet(signature, request.getContainerSignature())) { - assignContainer(request, hc, matchLocation); - return request; + if (!maybeChangeNode(request, hc.getContainer().getNodeId())) { + assignContainer(request, hc, matchLocation); + return request; + } } } } @@ -1050,7 +1059,7 @@ private HeldContainer tryAssignTaskToIdleContainer(TaskRequest request) { ContainerId affinity = request.getAffinity(); if (affinity != null) { HeldContainer hc = heldContainers.get(affinity); - if (hc != null && hc.isAssignable()) { + if (hc != null && hc.isAssignable() && !maybeChangeNode(request, hc.getContainer().getNodeId())) { assignContainer(request, hc, affinity); return hc; } @@ -1099,12 +1108,13 @@ private HeldContainer tryAssignTaskToIdleContainer(TaskRequest request, if (eligibleStates.contains(hc.getState())) { Object csig = hc.getSignature(); if (csig == null || signatureMatcher.isSuperSet(csig, request.getContainerSignature())) { + boolean needToChangeNode = maybeChangeNode(request, hc.getContainer().getNodeId()); int numAffinities = hc.getNumAffinities(); - if (numAffinities == 0) { + if (numAffinities == 0 && !needToChangeNode) { bestMatch = hc; break; } - if (bestMatch == null || numAffinities < bestMatch.getNumAffinities()) { + if ((bestMatch == null || numAffinities < bestMatch.getNumAffinities()) && !needToChangeNode) { bestMatch = hc; } } else { @@ -1119,6 +1129,18 @@ private HeldContainer tryAssignTaskToIdleContainer(TaskRequest request, return bestMatch; } + private boolean maybeChangeNode(TaskRequest request, NodeId nodeId) { + Object task = request.getTask(); + if (task instanceof TaskAttempt) { + Set nodesWithSiblingRunningAttempts = ((TaskAttempt) task).getNodesWithSiblingRunningAttempts(); + if (nodesWithSiblingRunningAttempts != null + && nodesWithSiblingRunningAttempts.contains(nodeId)) { + return true; + } + } + return false; + } + @Override public void setShouldUnregister() { shouldUnregister = true; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java index 95cd85be18..f128ec9aca 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java @@ -63,6 +63,7 @@ import org.apache.hadoop.yarn.util.RackResolver; import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.tez.serviceplugins.api.TaskAttemptEndReason; +import org.apache.tez.dag.app.dag.TaskAttempt; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.TezUncheckedException; import org.apache.tez.common.ContainerSignatureMatcher; @@ -118,7 +119,7 @@ public class YarnTaskSchedulerService extends TaskScheduler new HashMap(); Set priorityHasAffinity = Sets.newHashSet(); - + Set blacklistedNodes = Collections .newSetFromMap(new ConcurrentHashMap()); @@ -1533,6 +1534,11 @@ private CookieContainerRequest getMatchingRequestWithoutPriority( private boolean canAssignTaskToContainer( CookieContainerRequest cookieContainerRequest, Container container) { HeldContainer heldContainer = heldContainers.get(container.getId()); + Object task = getTask(cookieContainerRequest); + if (task instanceof TaskAttempt + && ((TaskAttempt) task).getNodesWithSiblingRunningAttempts().contains(container.getNodeId())) { + return false; + } if (heldContainer == null || heldContainer.isNew()) { // New container. return true; } else { @@ -1784,9 +1790,9 @@ private void informAppAboutAssignments( Container container = entry.getValue(); // check for blacklisted nodes. There may be race conditions between // setting blacklist and receiving allocations + CookieContainerRequest request = entry.getKey(); + Object task = getTask(request); if (blacklistedNodes.contains(container.getNodeId())) { - CookieContainerRequest request = entry.getKey(); - Object task = getTask(request); LOG.info("Container: " + container.getId() + " allocated on blacklisted node: " + container.getNodeId() + " for task: " + task); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java index 503881039a..ce3e7e52b5 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java @@ -2194,7 +2194,7 @@ public MockTaskAttemptImpl(TezTaskID taskId, int attemptNumber, eventHandler, tal, conf, clock, taskHeartbeatHandler, appContext, isRescheduled, resource, containerContext, leafVertex, mockVertex, - locationHint, null, null); + locationHint, null, null, null); } boolean inputFailedReported = false; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java index b142bb9b08..51a4bdfb99 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java @@ -1366,7 +1366,7 @@ public MockTaskAttemptImpl(TezTaskAttemptID attemptId, Resource resource, ContainerContext containerContext, TezTaskAttemptID schedCausalTA) { super(attemptId, eventHandler, tal, conf, clock, thh, appContext, isRescheduled, resource, containerContext, false, null, - locationHint, mockTaskSpec, schedCausalTA); + locationHint, mockTaskSpec, schedCausalTA, null); } @Override @@ -1401,6 +1401,11 @@ public TaskAttemptState getStateNoLock() { public ContainerId getAssignedContainerID() { return mockContainerId; } + + @Override + public NodeId getNodeId() { + return mockNodeId; + } } public class ServiceBusyEvent extends TezAbstractEvent diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java index 911f4b1d3e..c979a7ab7f 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java @@ -18,6 +18,7 @@ package org.apache.tez.dag.app.rm; +import com.google.common.collect.Sets; import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; @@ -39,6 +40,7 @@ import org.apache.tez.common.MockDNSToSwitchMapping; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.app.MockClock; +import org.apache.tez.dag.app.dag.TaskAttempt; import org.apache.tez.dag.app.rm.DagAwareYarnTaskScheduler.AMRMClientAsyncWrapper; import org.apache.tez.dag.app.rm.DagAwareYarnTaskScheduler.HeldContainer; import org.apache.tez.dag.app.rm.DagAwareYarnTaskScheduler.TaskRequest; @@ -343,6 +345,23 @@ public void testNoReuse() throws Exception { verify(mockRMClient, times(8)).addContainerRequest(any(TaskRequest.class)); assertFalse(scheduler.deallocateTask(mockTask1, true, null, null)); + // test speculative node adjustment + String speculativeNode = "host8"; + NodeId speculativeNodeId = mock(NodeId.class); + when(speculativeNodeId.getHost()).thenReturn(speculativeNode); + TaskAttempt mockTask5 = mock(TaskAttempt.class); + when(mockTask5.getNodesWithSiblingRunningAttempts()).thenReturn(Sets.newHashSet(speculativeNodeId)); + Object mockCookie5 = new Object(); + scheduler.allocateTask(mockTask5, mockCapability, hosts, racks, + mockPriority, null, mockCookie5); + drainableAppCallback.drain(); + // no new allocation + verify(mockApp, times(4)).taskAllocated(any(), any(), (Container) any()); + // verify container released + verify(mockRMClient, times(5)).releaseAssignedContainer((ContainerId) any()); + // verify request added back + verify(mockRMClient, times(9)).addContainerRequest(requestCaptor.capture()); + List mockUpdatedNodes = mock(List.class); scheduler.onNodesUpdated(mockUpdatedNodes); drainableAppCallback.drain(); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java index 49f8fe3a5d..21f4c52be7 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java @@ -60,6 +60,7 @@ import org.apache.tez.common.ContainerSignatureMatcher; import org.apache.tez.common.MockDNSToSwitchMapping; import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.app.dag.TaskAttempt; import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.AMRMClientAsyncForTest; import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.AMRMClientForTest; import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.AlwaysMatchesContainerMatcher; @@ -346,6 +347,23 @@ public void testTaskSchedulerNoReuse() throws Exception { (CookieContainerRequest) any()); assertFalse(scheduler.deallocateTask(mockTask1, true, null, null)); + // test speculative node adjustment + String speculativeNode = "host8"; + NodeId speculativeNodeId = mock(NodeId.class); + when(speculativeNodeId.getHost()).thenReturn(speculativeNode); + TaskAttempt mockTask5 = mock(TaskAttempt.class); + when(mockTask5.getNodesWithSiblingRunningAttempts()).thenReturn(Sets.newHashSet(speculativeNodeId)); + Object mockCookie5 = new Object(); + scheduler.allocateTask(mockTask5, mockCapability, hosts, racks, + mockPriority, null, mockCookie5); + drainableAppCallback.drain(); + // no new allocation + verify(mockApp, times(4)).taskAllocated(any(), any(), (Container) any()); + // verify container released + verify(mockRMClient, times(5)).releaseAssignedContainer((ContainerId) any()); + // verify request added back + verify(mockRMClient, times(9)).addContainerRequest(requestCaptor.capture()); + List mockUpdatedNodes = mock(List.class); scheduler.onNodesUpdated(mockUpdatedNodes); drainableAppCallback.drain(); From e88e824fccb0e3dac0ce854738040e885f0d606b Mon Sep 17 00:00:00 2001 From: Zhang Butao Date: Wed, 27 Feb 2019 21:57:41 -0600 Subject: [PATCH 147/512] TEZ-4032. TEZ will throw Client cannot authenticate via:[TOKEN, KERBEROS] when used with HDFS federation(non viewfs, only hdfs schema used). --- .../org/apache/tez/client/TezClientUtils.java | 19 +++++++++++++ .../tez/common/security/TokenCache.java | 27 ++++++++++++++++--- .../apache/tez/dag/api/TezConfiguration.java | 14 ++++++++++ .../tez/common/security/TestTokenCache.java | 7 ++++- .../tez/mapreduce/hadoop/DeprecatedKeys.java | 4 +++ .../tez/mapreduce/hadoop/MRJobConfig.java | 2 ++ 6 files changed, 68 insertions(+), 5 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index 689d94797a..cd3ae6bbf3 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -27,6 +27,7 @@ import java.nio.ByteBuffer; import java.security.PrivilegedExceptionAction; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; @@ -495,6 +496,8 @@ public static ApplicationSubmissionContext createApplicationSubmissionContext( // Add Staging dir creds to the list of session credentials. TokenCache.obtainTokensForFileSystems(sessionCreds, new Path[]{binaryConfPath}, conf); + populateTokenCache(conf, sessionCreds); + // Add session specific credentials to the AM credentials. amLaunchCredentials.mergeAll(sessionCreds); @@ -716,6 +719,22 @@ public static ApplicationSubmissionContext createApplicationSubmissionContext( return appContext; } + + //get secret keys and tokens and store them into TokenCache + private static void populateTokenCache(TezConfiguration conf, Credentials credentials) + throws IOException{ + // add the delegation tokens from configuration + String[] nameNodes = conf.getStrings(TezConfiguration.TEZ_JOB_FS_SERVERS); + LOG.debug("adding the following namenodes' delegation tokens:" + + Arrays.toString(nameNodes)); + if(nameNodes != null) { + Path[] ps = new Path[nameNodes.length]; + for(int i = 0; i < nameNodes.length; i++) { + ps[i] = new Path(nameNodes[i]); + } + TokenCache.obtainTokensForFileSystems(credentials, ps, conf); + } + } static DAGPlan prepareAndCreateDAGPlan(DAG dag, AMConfiguration amConfig, Map tezJarResources, boolean tezLrsAsArchive, diff --git a/tez-api/src/main/java/org/apache/tez/common/security/TokenCache.java b/tez-api/src/main/java/org/apache/tez/common/security/TokenCache.java index fc2c07dc1c..e56ef61270 100644 --- a/tez-api/src/main/java/org/apache/tez/common/security/TokenCache.java +++ b/tez-api/src/main/java/org/apache/tez/common/security/TokenCache.java @@ -34,6 +34,7 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; +import org.apache.tez.dag.api.TezConfiguration; /** @@ -101,6 +102,20 @@ static void obtainTokensForFileSystemsInternal(Credentials credentials, } } + static boolean isTokenRenewalExcluded(FileSystem fs, Configuration conf) { + String[] nns = + conf.getStrings(TezConfiguration.TEZ_JOB_FS_SERVERS_TOKEN_RENEWAL_EXCLUDE); + if (nns != null) { + String host = fs.getUri().getHost(); + for(int i = 0; i < nns.length; i++) { + if (nns[i].equals(host)) { + return true; + } + } + } + return false; + } + /** * get delegation token for a specific FS * @param fs @@ -112,10 +127,14 @@ static void obtainTokensForFileSystemsInternal(Credentials credentials, static void obtainTokensForFileSystemsInternal(FileSystem fs, Credentials credentials, Configuration conf) throws IOException { // TODO Change this to use YARN utilities once YARN-1664 is fixed. - String delegTokenRenewer = Master.getMasterPrincipal(conf); - if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { - throw new IOException( - "Can't get Master Kerberos principal for use as renewer"); + // RM skips renewing token with empty renewer + String delegTokenRenewer = ""; + if (!isTokenRenewalExcluded(fs, conf)) { + delegTokenRenewer = Master.getMasterPrincipal(conf); + if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { + throw new IOException( + "Can't get Master Kerberos principal for use as renewer"); + } } final Token tokens[] = fs.addDelegationTokens(delegTokenRenewer, diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 4566600525..8ce8f7c796 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -2010,4 +2010,18 @@ static Set getPropertySet() { public static final String TEZ_SHARED_EXECUTOR_MAX_THREADS = "tez.shared-executor.max-threads"; public static final int TEZ_SHARED_EXECUTOR_MAX_THREADS_DEFAULT = -1; + /** + * Acquire all FileSystems info. e.g., all namenodes info of HDFS federation cluster. + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty + public static final String TEZ_JOB_FS_SERVERS = "tez.job.fs-servers"; + + /** + * Skip delegation token renewal for specified FileSystems. + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty + public static final String TEZ_JOB_FS_SERVERS_TOKEN_RENEWAL_EXCLUDE = "tez.job.fs-servers.token-renewal.exclude"; + } diff --git a/tez-api/src/test/java/org/apache/tez/common/security/TestTokenCache.java b/tez-api/src/test/java/org/apache/tez/common/security/TestTokenCache.java index 59488b6fb8..fcb1e98aee 100644 --- a/tez-api/src/test/java/org/apache/tez/common/security/TestTokenCache.java +++ b/tez-api/src/test/java/org/apache/tez/common/security/TestTokenCache.java @@ -113,6 +113,11 @@ public void testObtainTokensForFileSystems() throws Exception { conf.setBoolean("fs.test.impl.disable.cache", true); TokenCache.obtainTokensForFileSystemsInternal(creds, paths, conf); verify(TestFileSystem.fs, times(paths.length + 1)).addDelegationTokens(renewer, creds); + + // Excluded filesystem tokens should not be obtained. + conf.set("tez.job.fs-servers.token-renewal.exclude", "dir"); + TokenCache.obtainTokensForFileSystemsInternal(creds, paths, conf); + verify(TestFileSystem.fs, times(paths.length + 1)).addDelegationTokens(renewer, creds); } private Path[] makePaths(int count, String prefix) throws Exception { @@ -127,7 +132,7 @@ public static class TestFileSystem extends FilterFileSystem { static final FileSystem fs = mock(FileSystem.class); static { try { - when(fs.getUri()).thenReturn(new URI("test:///")); + when(fs.getUri()).thenReturn(new URI("test://dir")); } catch (URISyntaxException e) { throw new RuntimeException(e); } diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java index d9b0930bb2..b8d491a4b0 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java @@ -88,6 +88,10 @@ private static void populateMRToDagParamMap() { TezConfiguration.TEZ_APPLICATION_TAGS); mrParamToDAGParamMap.put(MRJobConfig.MAPREDUCE_JOB_USER_CLASSPATH_FIRST, TezConfiguration.TEZ_USER_CLASSPATH_FIRST); + mrParamToDAGParamMap.put(MRJobConfig.JOB_NAMENODES, + TezConfiguration.TEZ_JOB_FS_SERVERS); + mrParamToDAGParamMap.put(MRJobConfig.JOB_NAMENODES_TOKEN_RENEWAL_EXCLUDE, + TezConfiguration.TEZ_JOB_FS_SERVERS_TOKEN_RENEWAL_EXCLUDE); } // TODO TEZAM4 Sometime, make sure this gets loaded by default. Instead of the current initialization in MRAppMaster, TezChild. diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRJobConfig.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRJobConfig.java index cd6fd443a6..ca954d92ff 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRJobConfig.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRJobConfig.java @@ -302,6 +302,8 @@ public interface MRJobConfig { public static final String JOB_NAMENODES = "mapreduce.job.hdfs-servers"; + public static final String JOB_NAMENODES_TOKEN_RENEWAL_EXCLUDE = "mapreduce.job.hdfs-servers.token-renewal.exclude"; + public static final String JOB_JOBTRACKER_ID = "mapreduce.job.kerberos.jtprinicipal"; public static final String JOB_CANCEL_DELEGATION_TOKEN = "mapreduce.job.complete.cancel.delegation.tokens"; From aeb90b46bc69c183754f5cc4f8f29af4946b4559 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Thu, 28 Feb 2019 10:25:42 -0600 Subject: [PATCH 148/512] TEZ-4049. Fix findbugs issues in NotRunningJob (Jonathan Eagles via kshukla) --- .../java/org/apache/tez/mapreduce/client/NotRunningJob.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/NotRunningJob.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/NotRunningJob.java index 6cd92361cd..8837df6818 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/NotRunningJob.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/NotRunningJob.java @@ -64,8 +64,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; -import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; -import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; @@ -85,6 +83,8 @@ private ApplicationReport getUnknownApplicationReport() { .newRecordInstance(ApplicationAttemptId.class); ApplicationReport report = recordFactory.newRecordInstance(ApplicationReport.class); + report.setApplicationId(unknownAppId); + report.setCurrentApplicationAttemptId(unknownAttemptId); report.setUser("N/A"); report.setName("N/A"); report.setDiagnostics("N/A"); From 05b1038fa45c4931ef4e264e538f1049ab3531f3 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Thu, 28 Feb 2019 11:06:07 -0600 Subject: [PATCH 149/512] TEZ-4050. maven site is failing due to missing configuration. (Jonathan Eagles via kshukla) --- pom.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pom.xml b/pom.xml index 4760af1402..3ffbe2a719 100644 --- a/pom.xml +++ b/pom.xml @@ -80,6 +80,10 @@ ${distMgmtSnapshotsName} ${distMgmtSnapshotsUrl} + + tez.apache.org + scp://tez.apache.org + From 35acf1524f8ca83187b1bb47586811aa29cc2f0b Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Thu, 28 Feb 2019 11:16:22 -0600 Subject: [PATCH 150/512] TEZ-4047. Tez trademark in xml is causing xml parsing issue (Jonathan Eagles via kshukla) --- docs/src/site/site.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/src/site/site.xml b/docs/src/site/site.xml index 79a613beb1..3984bac545 100644 --- a/docs/src/site/site.xml +++ b/docs/src/site/site.xml @@ -94,7 +94,7 @@ - + @@ -120,7 +120,7 @@ - + From 8bc2e3703ed3682d48a3f781828a4391c8d14e3a Mon Sep 17 00:00:00 2001 From: Prasanth Jayachandran Date: Mon, 4 Mar 2019 12:26:05 -0600 Subject: [PATCH 151/512] TEZ-4048. Make proto history logger queue size configurable Signed-off-by: Jonathan Eagles --- .../java/org/apache/tez/dag/api/TezConfiguration.java | 10 ++++++++++ .../logging/proto/ProtoHistoryLoggingService.java | 9 ++++++--- 2 files changed, 16 insertions(+), 3 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 8ce8f7c796..7b00cf6475 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -1539,6 +1539,16 @@ public TezConfiguration(boolean loadDefaults) { TEZ_PREFIX + "history.logging.proto-sync-window-secs"; public static final long TEZ_HISTORY_LOGGING_PROTO_SYNC_WINDOWN_SECS_DEFAULT = 60L; + /** + * Int value. Maximum queue size for proto history event logger. + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty(type="integer") + public static final String TEZ_HISTORY_LOGGING_PROTO_QUEUE_SIZE = + TEZ_PREFIX + "history.logging.queue.size"; + public static final int TEZ_HISTORY_LOGGING_PROTO_QUEUE_SIZE_DEFAULT = 100000; + + /** * Boolean value. Set this to true, if the underlying file system does not support flush (Ex: s3). * The dag submitted, initialized and started events are written into a file and closed. The rest diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java index d2e0b4defa..008b05d616 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java @@ -52,8 +52,7 @@ public class ProtoHistoryLoggingService extends HistoryLoggingService { new HistoryEventProtoConverter(); private boolean loggingDisabled = false; - private final LinkedBlockingQueue eventQueue = - new LinkedBlockingQueue<>(10000); + private LinkedBlockingQueue eventQueue; private Thread eventHandlingThread; private final AtomicBoolean stopped = new AtomicBoolean(false); @@ -81,7 +80,11 @@ protected void serviceInit(Configuration conf) throws Exception { TezConfiguration.TEZ_AM_HISTORY_LOGGING_ENABLED_DEFAULT); splitDagStartEvents = conf.getBoolean(TezConfiguration.TEZ_HISTORY_LOGGING_PROTO_SPLIT_DAG_START, TezConfiguration.TEZ_HISTORY_LOGGING_PROTO_SPLIT_DAG_START_DEFAULT); - LOG.info("Inited ProtoHistoryLoggingService"); + final int queueSize = conf.getInt(TezConfiguration.TEZ_HISTORY_LOGGING_PROTO_QUEUE_SIZE, + TezConfiguration.TEZ_HISTORY_LOGGING_PROTO_QUEUE_SIZE_DEFAULT); + eventQueue = new LinkedBlockingQueue<>(queueSize); + LOG.info("Inited ProtoHistoryLoggingService. loggingDisabled: {} splitDagStartEvents: {} queueSize: {}", + loggingDisabled, splitDagStartEvents, queueSize); } @Override From 4b4aeca67299e10d6f9fcdaa4d4ffaf7a4110cf3 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Wed, 6 Mar 2019 15:24:57 -0600 Subject: [PATCH 152/512] TEZ-4031. Support tez gitbox migration (Jonathan Eagles via kshukla) --- Tez_DOAP.rdf | 4 +- docs/src/site/site.xml | 2 +- pom.xml | 6 +-- .../apache/tez/common/TestVersionInfo.java | 2 +- .../resources/test1-version-info.properties | 2 +- .../resources/test3-version-info.properties | 2 +- tez-ui/src/main/webapp/bower-shrinkwrap.json | 44 +++++++++---------- tez-ui/src/main/webapp/package.json | 2 +- 8 files changed, 32 insertions(+), 32 deletions(-) diff --git a/Tez_DOAP.rdf b/Tez_DOAP.rdf index 7e32221f2f..25bc3f53a3 100644 --- a/Tez_DOAP.rdf +++ b/Tez_DOAP.rdf @@ -162,8 +162,8 @@ - - + + diff --git a/docs/src/site/site.xml b/docs/src/site/site.xml index 3984bac545..4b2d3a4571 100644 --- a/docs/src/site/site.xml +++ b/docs/src/site/site.xml @@ -133,7 +133,7 @@ - + diff --git a/pom.xml b/pom.xml index 3ffbe2a719..52b873c553 100644 --- a/pom.xml +++ b/pom.xml @@ -55,7 +55,7 @@ 2.5.0 ${env.PROTOC_PATH} UTF-8 - scm:git:https://git-wip-us.apache.org/repos/asf/tez.git + scm:git:https://gitbox.apache.org/repos/asf/tez.git ${maven.build.timestamp} 1.4 3.0.1 @@ -960,7 +960,7 @@ org.apache.maven.plugins maven-site-plugin - 3.4 + 3.7.1 org.apache.maven.plugins @@ -1030,7 +1030,7 @@ org.apache.maven.plugins maven-project-info-reports-plugin - 2.7 + 3.0.0 false diff --git a/tez-api/src/test/java/org/apache/tez/common/TestVersionInfo.java b/tez-api/src/test/java/org/apache/tez/common/TestVersionInfo.java index 17ff3d1655..6b937cd87a 100644 --- a/tez-api/src/test/java/org/apache/tez/common/TestVersionInfo.java +++ b/tez-api/src/test/java/org/apache/tez/common/TestVersionInfo.java @@ -29,7 +29,7 @@ public class TestVersionInfo { final String version = "0.6.0-SNAPSHOT"; final String revision = "d523db65804a5742ce50824e6fcfb8a04d184c0d"; final String buildTime = "20141024-1052"; - final String scmUrl = "scm:git:https://git-wip-us.apache.org/repos/asf/tez.git"; + final String scmUrl = "scm:git:https://gitbox.apache.org/repos/asf/tez.git"; @Test(timeout = 5000) public void testTest1File() { diff --git a/tez-api/src/test/resources/test1-version-info.properties b/tez-api/src/test/resources/test1-version-info.properties index ebb4c03ce3..e2563d6cec 100644 --- a/tez-api/src/test/resources/test1-version-info.properties +++ b/tez-api/src/test/resources/test1-version-info.properties @@ -19,4 +19,4 @@ version=0.6.0-SNAPSHOT revision=d523db65804a5742ce50824e6fcfb8a04d184c0d buildtime=20141024-1052 -scmurl=scm:git:https://git-wip-us.apache.org/repos/asf/tez.git +scmurl=scm:git:https://gitbox.apache.org/repos/asf/tez.git diff --git a/tez-api/src/test/resources/test3-version-info.properties b/tez-api/src/test/resources/test3-version-info.properties index 401f382872..9def3fbf72 100644 --- a/tez-api/src/test/resources/test3-version-info.properties +++ b/tez-api/src/test/resources/test3-version-info.properties @@ -19,4 +19,4 @@ version=0.6.0-SNAPSHOT revision=d523db65804a5742ce50824e6fcfb8a04d184c0d buildtime= -scmurl=scm:git:https://git-wip-us.apache.org/repos/asf/tez.git +scmurl=scm:git:https://gitbox.apache.org/repos/asf/tez.git diff --git a/tez-ui/src/main/webapp/bower-shrinkwrap.json b/tez-ui/src/main/webapp/bower-shrinkwrap.json index 357d57691a..edcbae33c1 100644 --- a/tez-ui/src/main/webapp/bower-shrinkwrap.json +++ b/tez-ui/src/main/webapp/bower-shrinkwrap.json @@ -1,72 +1,72 @@ { "https://github.com/FortAwesome/Font-Awesome.git": { - "4.5.0": "593ad563a987977f14102be935d0abc2a172903e" + "4.5.0": "4.5.0" }, "https://github.com/Teleborder/FileSaver.js.git": { - "1.20150507.2": "b7cf622909258086bc63ad764d08fcaed780ab42" + "1.20150507.2": "1.20150507.2" }, "https://github.com/adamwdraper/Numeral-js.git": { - "1.5.3": "f97f14bb8bab988f28f1d854525b4cfeff8ec9e1" + "1.5.3": "1.5.3" }, "https://github.com/components/codemirror.git": { - "5.11.0": "7d43f32bb56f83a9c47addb3f91170b3102f3ead" + "5.11.0": "5.11.0" }, "https://github.com/components/ember-data.git": { - "2.1.0": "d8b4d3092f67afe22d9d374c40d719d557915fa3" + "2.1.0": "2.1.0" }, "https://github.com/components/ember.git": { - "2.2.0": "49e042ca89922ed96b27488c2a98add280ae7123" + "2.2.0": "2.2.0" }, "https://github.com/components/jqueryui.git": { - "1.11.4": "c34f8dbf3ba57b3784b93f26119f436c0e8288e1" + "1.11.4": "1.11.4" }, "https://github.com/dockyard/ember-qunit-notifications.git": { - "0.1.0": "a83277aa7a1c0545c66e6d133caebb9a620e71ad" + "0.1.0": "0.1.0" }, "https://github.com/dockyard/qunit-notifications.git": { - "0.1.1": "7a13f6dba5a340e1cb9e0b64c1c711e4d7edaca1" + "0.1.1": "0.1.1" }, "https://github.com/ember-cli/ember-cli-shims.git": { - "0.0.6": "dcab43b58d5698690050bb9a46ead5c8663c7da1" + "0.0.6": "0.0.6" }, "https://github.com/ember-cli/ember-cli-test-loader.git": { - "0.2.1": "3348d801089279296c38f31ae14d9c4d115ce154" + "0.2.1": "0.2.1" }, "https://github.com/ember-cli/ember-load-initializers.git": { - "0.1.7": "7bb21488563bd1bba23e903a812bf5815beddd1a" + "0.1.7": "0.1.7" }, "https://github.com/jquery/jquery-dist.git": { - "2.1.4": "7751e69b615c6eca6f783a81e292a55725af6b85" + "2.1.4": "2.1.4" }, "https://github.com/jquery/jquery-mousewheel.git": { - "3.1.13": "67289b6b2aa0066d7d78a5807f520387135ffb22" + "3.1.13": "3.1.13" }, "https://github.com/jquery/qunit.git": { - "1.19.0": "467e7e34652ad7d5883ce9c568461cf8c5e172a8" + "1.19.0": "1.19.0" }, "https://github.com/moment/moment-timezone.git": { - "0.5.0": "74a2e9378ecf4a31a168f3049f086565c8d66814" + "0.5.0": "0.5.0" }, "https://github.com/moment/moment.git": { - "2.12.0": "d3d7488b4d60632854181cb0a9af325d57fb3d51" + "2.12.0": "2.12.0" }, "https://github.com/rwjblue/ember-qunit-builds.git": { - "0.4.16": "142c4066a5458bef9dfcb92b70152b9c01d79188" + "0.4.16": "0.4.16" }, "https://github.com/sreenaths/more-js.git": { "0.8.6": "f1d9ccdaf7ff74c26b6ee341067a5a5ef33bd64a", "0.8.8": "0.8.8" }, "https://github.com/sreenaths/snippet-ss.git": { - "1.11.0": "c1abc566f4e001b7f1939b6dbdd911eadc969cf9" + "1.11.0": "1.11.0" }, "https://github.com/sreenaths/zip.js.git": { - "1.0.0": "ec67ad22eba116083ea3ef2fe0b40ccc953513ce" + "1.0.0": "1.0.0" }, "https://github.com/stefanpenner/loader.js.git": { - "3.3.0": "ac909550c9544325632542bbea97531cc60bc628" + "3.3.0": "3.3.0" }, "https://github.com/twbs/bootstrap.git": { - "3.3.6": "81df608a40bf0629a1dc08e584849bb1e43e0b7a" + "3.3.6": "3.3.6" } } \ No newline at end of file diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json index a14d21583a..92f54d266d 100644 --- a/tez-ui/src/main/webapp/package.json +++ b/tez-ui/src/main/webapp/package.json @@ -19,7 +19,7 @@ }, "repository": { "type": "git", - "url": "https://git-wip-us.apache.org/repos/asf/tez.git" + "url": "https://gitbox.apache.org/repos/asf/tez.git" }, "engines": { "node": ">= 0.10.0" From f4b7137ddda3f7d3a127ca30eddcf7302d842c2a Mon Sep 17 00:00:00 2001 From: Jaume Marhuenda Date: Fri, 8 Mar 2019 12:02:16 -0600 Subject: [PATCH 153/512] TEZ-3995. Fix dot files produced by tests to prevent ASF license warnings in yetus (addendum) Signed-off-by: Jonathan Eagles --- tez-tests/pom.xml | 31 +++++++++++++++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/tez-tests/pom.xml b/tez-tests/pom.xml index c0bc884435..ad834e1fcd 100644 --- a/tez-tests/pom.xml +++ b/tez-tests/pom.xml @@ -24,6 +24,10 @@ tez-tests + + ${project.build.directory}/dagviz + + org.apache.tez @@ -129,6 +133,33 @@ + + maven-antrun-plugin + + + generate-sources + generate-sources + + + + + + + + run + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + ${test.tmp.dir} + + + org.apache.maven.plugins maven-jar-plugin From 3524e790fc24ee0d41f409441494ee160c68f2b3 Mon Sep 17 00:00:00 2001 From: Gopal V Date: Tue, 12 Mar 2019 10:25:08 -0500 Subject: [PATCH 154/512] TEZ-4044. Zookeeper: exclude jline from Zookeeper client from tez dist Signed-off-by: Jonathan Eagles --- tez-dist/src/main/assembly/tez-dist.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/tez-dist/src/main/assembly/tez-dist.xml b/tez-dist/src/main/assembly/tez-dist.xml index b8834a88a7..9456664f77 100644 --- a/tez-dist/src/main/assembly/tez-dist.xml +++ b/tez-dist/src/main/assembly/tez-dist.xml @@ -36,6 +36,7 @@ *:*:test-jar org.apache.tez:* + jline:jline From 96a5e3ec1007cd7203b42d1cc4abce41fa58c340 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Thu, 14 Mar 2019 14:54:58 -0500 Subject: [PATCH 155/512] TEZ-4052. Fit dot files ASF License issues - part 2 (Jonathan Eagles via kshukla) --- tez-dag/pom.xml | 8 ++++---- tez-ext-service-tests/pom.xml | 31 +++++++++++++++++++++++++++++++ tez-tests/pom.xml | 8 ++++---- 3 files changed, 39 insertions(+), 8 deletions(-) diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml index 5f63079e8b..c6962ae2ff 100644 --- a/tez-dag/pom.xml +++ b/tez-dag/pom.xml @@ -31,7 +31,7 @@ org.apache.tez.dag.app.rm.container.AMContainerImpl Tez Tez.gv - ${project.build.directory}/dagviz + ${project.build.directory}/logs tez-dag @@ -195,8 +195,8 @@ generate-sources - - + + @@ -210,7 +210,7 @@ maven-surefire-plugin - ${test.tmp.dir} + ${test.log.dir} diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml index f27e35c81a..c806bf7124 100644 --- a/tez-ext-service-tests/pom.xml +++ b/tez-ext-service-tests/pom.xml @@ -25,6 +25,10 @@ tez-ext-service-tests + + ${project.build.directory}/logs + + io.netty @@ -125,6 +129,33 @@ + + maven-antrun-plugin + + + generate-sources + generate-sources + + + + + + + + run + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + ${test.log.dir} + + + org.apache.rat apache-rat-plugin diff --git a/tez-tests/pom.xml b/tez-tests/pom.xml index ad834e1fcd..c744e6892d 100644 --- a/tez-tests/pom.xml +++ b/tez-tests/pom.xml @@ -25,7 +25,7 @@ tez-tests - ${project.build.directory}/dagviz + ${project.build.directory}/logs @@ -141,8 +141,8 @@ generate-sources - - + + @@ -156,7 +156,7 @@ maven-surefire-plugin - ${test.tmp.dir} + ${test.log.dir} From c4c66b53605f64c9c22f462cc9323bc42f84dd10 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Tue, 19 Mar 2019 15:09:20 -0500 Subject: [PATCH 156/512] Revert "TEZ-4031. Support tez gitbox migration (Jonathan Eagles via kshukla)" This reverts commit 4b4aeca67299e10d6f9fcdaa4d4ffaf7a4110cf3. --- Tez_DOAP.rdf | 4 +- docs/src/site/site.xml | 2 +- pom.xml | 6 +-- .../apache/tez/common/TestVersionInfo.java | 2 +- .../resources/test1-version-info.properties | 2 +- .../resources/test3-version-info.properties | 2 +- tez-ui/src/main/webapp/bower-shrinkwrap.json | 44 +++++++++---------- tez-ui/src/main/webapp/package.json | 2 +- 8 files changed, 32 insertions(+), 32 deletions(-) diff --git a/Tez_DOAP.rdf b/Tez_DOAP.rdf index 25bc3f53a3..7e32221f2f 100644 --- a/Tez_DOAP.rdf +++ b/Tez_DOAP.rdf @@ -162,8 +162,8 @@ - - + + diff --git a/docs/src/site/site.xml b/docs/src/site/site.xml index 4b2d3a4571..3984bac545 100644 --- a/docs/src/site/site.xml +++ b/docs/src/site/site.xml @@ -133,7 +133,7 @@ - + diff --git a/pom.xml b/pom.xml index 52b873c553..3ffbe2a719 100644 --- a/pom.xml +++ b/pom.xml @@ -55,7 +55,7 @@ 2.5.0 ${env.PROTOC_PATH} UTF-8 - scm:git:https://gitbox.apache.org/repos/asf/tez.git + scm:git:https://git-wip-us.apache.org/repos/asf/tez.git ${maven.build.timestamp} 1.4 3.0.1 @@ -960,7 +960,7 @@ org.apache.maven.plugins maven-site-plugin - 3.7.1 + 3.4 org.apache.maven.plugins @@ -1030,7 +1030,7 @@ org.apache.maven.plugins maven-project-info-reports-plugin - 3.0.0 + 2.7 false diff --git a/tez-api/src/test/java/org/apache/tez/common/TestVersionInfo.java b/tez-api/src/test/java/org/apache/tez/common/TestVersionInfo.java index 6b937cd87a..17ff3d1655 100644 --- a/tez-api/src/test/java/org/apache/tez/common/TestVersionInfo.java +++ b/tez-api/src/test/java/org/apache/tez/common/TestVersionInfo.java @@ -29,7 +29,7 @@ public class TestVersionInfo { final String version = "0.6.0-SNAPSHOT"; final String revision = "d523db65804a5742ce50824e6fcfb8a04d184c0d"; final String buildTime = "20141024-1052"; - final String scmUrl = "scm:git:https://gitbox.apache.org/repos/asf/tez.git"; + final String scmUrl = "scm:git:https://git-wip-us.apache.org/repos/asf/tez.git"; @Test(timeout = 5000) public void testTest1File() { diff --git a/tez-api/src/test/resources/test1-version-info.properties b/tez-api/src/test/resources/test1-version-info.properties index e2563d6cec..ebb4c03ce3 100644 --- a/tez-api/src/test/resources/test1-version-info.properties +++ b/tez-api/src/test/resources/test1-version-info.properties @@ -19,4 +19,4 @@ version=0.6.0-SNAPSHOT revision=d523db65804a5742ce50824e6fcfb8a04d184c0d buildtime=20141024-1052 -scmurl=scm:git:https://gitbox.apache.org/repos/asf/tez.git +scmurl=scm:git:https://git-wip-us.apache.org/repos/asf/tez.git diff --git a/tez-api/src/test/resources/test3-version-info.properties b/tez-api/src/test/resources/test3-version-info.properties index 9def3fbf72..401f382872 100644 --- a/tez-api/src/test/resources/test3-version-info.properties +++ b/tez-api/src/test/resources/test3-version-info.properties @@ -19,4 +19,4 @@ version=0.6.0-SNAPSHOT revision=d523db65804a5742ce50824e6fcfb8a04d184c0d buildtime= -scmurl=scm:git:https://gitbox.apache.org/repos/asf/tez.git +scmurl=scm:git:https://git-wip-us.apache.org/repos/asf/tez.git diff --git a/tez-ui/src/main/webapp/bower-shrinkwrap.json b/tez-ui/src/main/webapp/bower-shrinkwrap.json index edcbae33c1..357d57691a 100644 --- a/tez-ui/src/main/webapp/bower-shrinkwrap.json +++ b/tez-ui/src/main/webapp/bower-shrinkwrap.json @@ -1,72 +1,72 @@ { "https://github.com/FortAwesome/Font-Awesome.git": { - "4.5.0": "4.5.0" + "4.5.0": "593ad563a987977f14102be935d0abc2a172903e" }, "https://github.com/Teleborder/FileSaver.js.git": { - "1.20150507.2": "1.20150507.2" + "1.20150507.2": "b7cf622909258086bc63ad764d08fcaed780ab42" }, "https://github.com/adamwdraper/Numeral-js.git": { - "1.5.3": "1.5.3" + "1.5.3": "f97f14bb8bab988f28f1d854525b4cfeff8ec9e1" }, "https://github.com/components/codemirror.git": { - "5.11.0": "5.11.0" + "5.11.0": "7d43f32bb56f83a9c47addb3f91170b3102f3ead" }, "https://github.com/components/ember-data.git": { - "2.1.0": "2.1.0" + "2.1.0": "d8b4d3092f67afe22d9d374c40d719d557915fa3" }, "https://github.com/components/ember.git": { - "2.2.0": "2.2.0" + "2.2.0": "49e042ca89922ed96b27488c2a98add280ae7123" }, "https://github.com/components/jqueryui.git": { - "1.11.4": "1.11.4" + "1.11.4": "c34f8dbf3ba57b3784b93f26119f436c0e8288e1" }, "https://github.com/dockyard/ember-qunit-notifications.git": { - "0.1.0": "0.1.0" + "0.1.0": "a83277aa7a1c0545c66e6d133caebb9a620e71ad" }, "https://github.com/dockyard/qunit-notifications.git": { - "0.1.1": "0.1.1" + "0.1.1": "7a13f6dba5a340e1cb9e0b64c1c711e4d7edaca1" }, "https://github.com/ember-cli/ember-cli-shims.git": { - "0.0.6": "0.0.6" + "0.0.6": "dcab43b58d5698690050bb9a46ead5c8663c7da1" }, "https://github.com/ember-cli/ember-cli-test-loader.git": { - "0.2.1": "0.2.1" + "0.2.1": "3348d801089279296c38f31ae14d9c4d115ce154" }, "https://github.com/ember-cli/ember-load-initializers.git": { - "0.1.7": "0.1.7" + "0.1.7": "7bb21488563bd1bba23e903a812bf5815beddd1a" }, "https://github.com/jquery/jquery-dist.git": { - "2.1.4": "2.1.4" + "2.1.4": "7751e69b615c6eca6f783a81e292a55725af6b85" }, "https://github.com/jquery/jquery-mousewheel.git": { - "3.1.13": "3.1.13" + "3.1.13": "67289b6b2aa0066d7d78a5807f520387135ffb22" }, "https://github.com/jquery/qunit.git": { - "1.19.0": "1.19.0" + "1.19.0": "467e7e34652ad7d5883ce9c568461cf8c5e172a8" }, "https://github.com/moment/moment-timezone.git": { - "0.5.0": "0.5.0" + "0.5.0": "74a2e9378ecf4a31a168f3049f086565c8d66814" }, "https://github.com/moment/moment.git": { - "2.12.0": "2.12.0" + "2.12.0": "d3d7488b4d60632854181cb0a9af325d57fb3d51" }, "https://github.com/rwjblue/ember-qunit-builds.git": { - "0.4.16": "0.4.16" + "0.4.16": "142c4066a5458bef9dfcb92b70152b9c01d79188" }, "https://github.com/sreenaths/more-js.git": { "0.8.6": "f1d9ccdaf7ff74c26b6ee341067a5a5ef33bd64a", "0.8.8": "0.8.8" }, "https://github.com/sreenaths/snippet-ss.git": { - "1.11.0": "1.11.0" + "1.11.0": "c1abc566f4e001b7f1939b6dbdd911eadc969cf9" }, "https://github.com/sreenaths/zip.js.git": { - "1.0.0": "1.0.0" + "1.0.0": "ec67ad22eba116083ea3ef2fe0b40ccc953513ce" }, "https://github.com/stefanpenner/loader.js.git": { - "3.3.0": "3.3.0" + "3.3.0": "ac909550c9544325632542bbea97531cc60bc628" }, "https://github.com/twbs/bootstrap.git": { - "3.3.6": "3.3.6" + "3.3.6": "81df608a40bf0629a1dc08e584849bb1e43e0b7a" } } \ No newline at end of file diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json index 92f54d266d..a14d21583a 100644 --- a/tez-ui/src/main/webapp/package.json +++ b/tez-ui/src/main/webapp/package.json @@ -19,7 +19,7 @@ }, "repository": { "type": "git", - "url": "https://gitbox.apache.org/repos/asf/tez.git" + "url": "https://git-wip-us.apache.org/repos/asf/tez.git" }, "engines": { "node": ">= 0.10.0" From 6bbb41f5eaa83533bad2a51565f608d87eeacb61 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Tue, 19 Mar 2019 15:10:33 -0500 Subject: [PATCH 157/512] TEZ-4031. Support tez gitbox migration (Jonathan Eagles via kshukla) --- Tez_DOAP.rdf | 4 +-- docs/src/site/site.xml | 2 +- pom.xml | 4 +-- .../apache/tez/common/TestVersionInfo.java | 31 +++++++++---------- .../resources/test1-version-info.properties | 2 +- .../resources/test3-version-info.properties | 2 +- tez-ui/src/main/webapp/package.json | 2 +- 7 files changed, 22 insertions(+), 25 deletions(-) diff --git a/Tez_DOAP.rdf b/Tez_DOAP.rdf index 7e32221f2f..25bc3f53a3 100644 --- a/Tez_DOAP.rdf +++ b/Tez_DOAP.rdf @@ -162,8 +162,8 @@ - - + + diff --git a/docs/src/site/site.xml b/docs/src/site/site.xml index 3984bac545..4b2d3a4571 100644 --- a/docs/src/site/site.xml +++ b/docs/src/site/site.xml @@ -133,7 +133,7 @@ - + diff --git a/pom.xml b/pom.xml index 3ffbe2a719..28817025a1 100644 --- a/pom.xml +++ b/pom.xml @@ -55,7 +55,7 @@ 2.5.0 ${env.PROTOC_PATH} UTF-8 - scm:git:https://git-wip-us.apache.org/repos/asf/tez.git + scm:git:https://gitbox.apache.org/repos/asf/tez.git ${maven.build.timestamp} 1.4 3.0.1 @@ -1030,7 +1030,7 @@ org.apache.maven.plugins maven-project-info-reports-plugin - 2.7 + 2.9 false diff --git a/tez-api/src/test/java/org/apache/tez/common/TestVersionInfo.java b/tez-api/src/test/java/org/apache/tez/common/TestVersionInfo.java index 17ff3d1655..67f97b86a5 100644 --- a/tez-api/src/test/java/org/apache/tez/common/TestVersionInfo.java +++ b/tez-api/src/test/java/org/apache/tez/common/TestVersionInfo.java @@ -18,44 +18,41 @@ package org.apache.tez.common; -import java.io.IOException; - import org.junit.Assert; import org.junit.Test; - public class TestVersionInfo { - final String version = "0.6.0-SNAPSHOT"; - final String revision = "d523db65804a5742ce50824e6fcfb8a04d184c0d"; - final String buildTime = "20141024-1052"; - final String scmUrl = "scm:git:https://git-wip-us.apache.org/repos/asf/tez.git"; + private static final String VERSION = "0.6.0-SNAPSHOT"; + private static final String REVISION = "d523db65804a5742ce50824e6fcfb8a04d184c0d"; + private static final String BUILD_TIME = "20141024-1052"; + private static final String SCM_URL = "scm:git:https://gitbox.apache.org/repos/asf/tez.git"; @Test(timeout = 5000) public void testTest1File() { VersionInfo versionInfo = new VersionInfo("test1"); - Assert.assertEquals(version, versionInfo.getVersion()); - Assert.assertEquals(revision, versionInfo.getRevision()); - Assert.assertEquals(buildTime, versionInfo.getBuildTime()); - Assert.assertEquals(scmUrl, versionInfo.getSCMURL()); + Assert.assertEquals(VERSION, versionInfo.getVersion()); + Assert.assertEquals(REVISION, versionInfo.getRevision()); + Assert.assertEquals(BUILD_TIME, versionInfo.getBuildTime()); + Assert.assertEquals(SCM_URL, versionInfo.getSCMURL()); } @Test(timeout = 5000) public void testTest2File() { VersionInfo versionInfo = new VersionInfo("test2"); - Assert.assertEquals(version, versionInfo.getVersion()); - Assert.assertEquals(revision, versionInfo.getRevision()); - Assert.assertEquals(buildTime, versionInfo.getBuildTime()); + Assert.assertEquals(VERSION, versionInfo.getVersion()); + Assert.assertEquals(REVISION, versionInfo.getRevision()); + Assert.assertEquals(BUILD_TIME, versionInfo.getBuildTime()); Assert.assertEquals(VersionInfo.UNKNOWN, versionInfo.getSCMURL()); } @Test(timeout = 5000) public void testTest3File() { VersionInfo versionInfo = new VersionInfo("test3"); - Assert.assertEquals(version, versionInfo.getVersion()); - Assert.assertEquals(revision, versionInfo.getRevision()); + Assert.assertEquals(VERSION, versionInfo.getVersion()); + Assert.assertEquals(REVISION, versionInfo.getRevision()); Assert.assertEquals("", versionInfo.getBuildTime()); - Assert.assertEquals(scmUrl, versionInfo.getSCMURL()); + Assert.assertEquals(SCM_URL, versionInfo.getSCMURL()); } @Test(timeout = 5000) diff --git a/tez-api/src/test/resources/test1-version-info.properties b/tez-api/src/test/resources/test1-version-info.properties index ebb4c03ce3..e2563d6cec 100644 --- a/tez-api/src/test/resources/test1-version-info.properties +++ b/tez-api/src/test/resources/test1-version-info.properties @@ -19,4 +19,4 @@ version=0.6.0-SNAPSHOT revision=d523db65804a5742ce50824e6fcfb8a04d184c0d buildtime=20141024-1052 -scmurl=scm:git:https://git-wip-us.apache.org/repos/asf/tez.git +scmurl=scm:git:https://gitbox.apache.org/repos/asf/tez.git diff --git a/tez-api/src/test/resources/test3-version-info.properties b/tez-api/src/test/resources/test3-version-info.properties index 401f382872..9def3fbf72 100644 --- a/tez-api/src/test/resources/test3-version-info.properties +++ b/tez-api/src/test/resources/test3-version-info.properties @@ -19,4 +19,4 @@ version=0.6.0-SNAPSHOT revision=d523db65804a5742ce50824e6fcfb8a04d184c0d buildtime= -scmurl=scm:git:https://git-wip-us.apache.org/repos/asf/tez.git +scmurl=scm:git:https://gitbox.apache.org/repos/asf/tez.git diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json index a14d21583a..92f54d266d 100644 --- a/tez-ui/src/main/webapp/package.json +++ b/tez-ui/src/main/webapp/package.json @@ -19,7 +19,7 @@ }, "repository": { "type": "git", - "url": "https://git-wip-us.apache.org/repos/asf/tez.git" + "url": "https://gitbox.apache.org/repos/asf/tez.git" }, "engines": { "node": ">= 0.10.0" From d5675c332497c1ac1dedefdf91e87476b5c0d7a9 Mon Sep 17 00:00:00 2001 From: Ying Han Date: Fri, 22 Mar 2019 15:56:39 -0500 Subject: [PATCH 158/512] TEZ-4045. Task should be accessible from TaskAttempt Signed-off-by: Jonathan Eagles --- .../java/org/apache/tez/dag/app/dag/Task.java | 8 +++++ .../apache/tez/dag/app/dag/TaskAttempt.java | 9 ++---- .../tez/dag/app/dag/impl/TaskAttemptImpl.java | 32 ++++++++----------- .../apache/tez/dag/app/dag/impl/TaskImpl.java | 13 ++++++-- .../dag/app/rm/DagAwareYarnTaskScheduler.java | 2 +- .../dag/app/rm/YarnTaskSchedulerService.java | 7 ++-- .../tez/dag/app/dag/impl/TestTaskAttempt.java | 8 +++-- .../tez/dag/app/dag/impl/TestTaskImpl.java | 4 +-- .../app/rm/TestDagAwareYarnTaskScheduler.java | 5 ++- .../tez/dag/app/rm/TestTaskScheduler.java | 5 ++- 10 files changed, 56 insertions(+), 37 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java index d1b9b2a6ce..c1fe7f79ad 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java @@ -21,7 +21,9 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Set; +import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.TaskLocationHint; import org.apache.tez.dag.api.oldrecords.TaskReport; @@ -73,4 +75,10 @@ public ArrayList getTaskAttemptTezEvents(TezTaskAttemptID attemptID, long getFirstAttemptStartTime(); long getFinishTime(); + + /** + * @return set of nodes on which previous attempts were running on, at the time + * of latest attempt being scheduled. + */ + Set getNodesWithRunningAttempts(); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java index d0fec5c948..0fc7013fea 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java @@ -19,7 +19,6 @@ package org.apache.tez.dag.app.dag; import java.util.List; -import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.hadoop.yarn.api.records.Container; @@ -72,7 +71,8 @@ public void setLocalityCounter(DAGCounter localityCounter) { TezTaskID getTaskID(); TezVertexID getVertexID(); TezDAGID getDAGID(); - + + Task getTask(); TaskAttemptReport getReport(); List getDiagnostics(); TaskAttemptTerminationCause getTerminationCause(); @@ -136,9 +136,4 @@ public void setLocalityCounter(DAGCounter localityCounter) { * yet, returns 0. */ long getFinishTime(); - - /** - * @return the set of nodes on which sibling attempts were running on. - */ - Set getNodesWithSiblingRunningAttempts(); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java index 46080528f1..ade7bc742f 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java @@ -77,6 +77,7 @@ import org.apache.tez.dag.app.dag.TaskAttempt; import org.apache.tez.dag.app.dag.TaskAttemptStateInternal; import org.apache.tez.dag.app.dag.Vertex; +import org.apache.tez.dag.app.dag.Task; import org.apache.tez.dag.app.dag.event.DAGEvent; import org.apache.tez.dag.app.dag.event.DAGEventCounterUpdate; import org.apache.tez.dag.app.dag.event.DAGEventDiagnosticsUpdate; @@ -198,6 +199,7 @@ public static DataEventDependencyInfo fromProto(DataEventDependencyInfoProto pro private String nodeRackName; private final Vertex vertex; + private final Task task; private final TaskLocationHint locationHint; private final TaskSpec taskSpec; @@ -228,12 +230,6 @@ public static DataEventDependencyInfo fromProto(DataEventDependencyInfoProto pro private final boolean leafVertex; private TezTaskAttemptID creationCausalTA; - // Record the set of nodes on which sibling attempts were running on, at the time of - // this attempt being scheduled. This set is empty for original task attempt, and - // non-empty when current task attempt is a speculative one, in which case scheduler - // should try to schedule the speculative attempt on to a node other than the one(s) - // recorded in this set. - private Set nodesWithSiblingRunningAttempts; private long creationTime; private long scheduledTime; @@ -543,10 +539,10 @@ public TaskAttemptImpl(TezTaskAttemptID attemptId, EventHandler eventHandler, TaskHeartbeatHandler taskHeartbeatHandler, AppContext appContext, boolean isRescheduled, Resource resource, ContainerContext containerContext, boolean leafVertex, - Vertex vertex, TaskLocationHint locationHint, TaskSpec taskSpec) { + Task task, TaskLocationHint locationHint, TaskSpec taskSpec) { this(attemptId, eventHandler, taskCommunicatorManagerInterface, conf, clock, taskHeartbeatHandler, appContext, isRescheduled, resource, containerContext, leafVertex, - vertex, locationHint, taskSpec, null, null); + task, locationHint, taskSpec, null); } @SuppressWarnings("rawtypes") @@ -555,8 +551,8 @@ public TaskAttemptImpl(TezTaskAttemptID attemptId, EventHandler eventHandler, TaskHeartbeatHandler taskHeartbeatHandler, AppContext appContext, boolean isRescheduled, Resource resource, ContainerContext containerContext, boolean leafVertex, - Vertex vertex, TaskLocationHint locationHint, TaskSpec taskSpec, - TezTaskAttemptID schedulingCausalTA, Set nodesWithSiblingRunningAttempts) { + Task task, TaskLocationHint locationHint, TaskSpec taskSpec, + TezTaskAttemptID schedulingCausalTA) { ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock(); this.readLock = rwLock.readLock(); @@ -568,11 +564,11 @@ public TaskAttemptImpl(TezTaskAttemptID attemptId, EventHandler eventHandler, this.clock = clock; this.taskHeartbeatHandler = taskHeartbeatHandler; this.appContext = appContext; - this.vertex = vertex; + this.vertex = task.getVertex(); + this.task = task; this.locationHint = locationHint; this.taskSpec = taskSpec; this.creationCausalTA = schedulingCausalTA; - this.nodesWithSiblingRunningAttempts = nodesWithSiblingRunningAttempts; this.creationTime = clock.getTime(); this.reportedStatus = new TaskAttemptStatus(this.attemptId); @@ -615,11 +611,6 @@ public TezTaskAttemptID getSchedulingCausalTA() { return creationCausalTA; } - @Override - public Set getNodesWithSiblingRunningAttempts() { - return nodesWithSiblingRunningAttempts; - } - @Override public TaskAttemptReport getReport() { TaskAttemptReport result = Records.newRecord(TaskAttemptReport.class); @@ -866,7 +857,12 @@ public long getFinishTime() { readLock.unlock(); } } - + + @Override + public Task getTask() { + return task; + } + Vertex getVertex() { return vertex; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java index 9e3c5a82ff..9289d8f4cc 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java @@ -152,6 +152,10 @@ public class TaskImpl implements Task, EventHandler { // track the status of TaskAttempt (true mean completed, false mean uncompleted) private final Map taskAttemptStatus = new HashMap(); + // The set of nodes with active running attempts at the time of the latest attempt for + // this task was scheduled. This set is empty when scheduling original task attempt, and + // non-empty scheduling a speculative attempt, in which case scheduler should avoid + // scheduling the speculative attempt onto node(s) recorded in this set. private final Set nodesWithRunningAttempts = Collections .newSetFromMap(new ConcurrentHashMap()); @@ -584,6 +588,11 @@ public long getFinishTime() { } } + @Override + public Set getNodesWithRunningAttempts() { + return nodesWithRunningAttempts; + } + @VisibleForTesting public TaskStateInternal getInternalState() { readLock.lock(); @@ -749,8 +758,8 @@ TaskAttemptImpl createAttempt(int attemptNumber, TezTaskAttemptID schedulingCaus baseTaskSpec.getTaskConf()); return new TaskAttemptImpl(attemptId, eventHandler, taskCommunicatorManagerInterface, conf, clock, taskHeartbeatHandler, appContext, - (failedAttempts > 0), taskResource, containerContext, leafVertex, getVertex(), - locationHint, taskSpec, schedulingCausalTA, nodesWithRunningAttempts); + (failedAttempts > 0), taskResource, containerContext, leafVertex, this, + locationHint, taskSpec, schedulingCausalTA); } @Override diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java index 3191c48758..6a78425caa 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java @@ -1132,7 +1132,7 @@ private HeldContainer tryAssignTaskToIdleContainer(TaskRequest request, private boolean maybeChangeNode(TaskRequest request, NodeId nodeId) { Object task = request.getTask(); if (task instanceof TaskAttempt) { - Set nodesWithSiblingRunningAttempts = ((TaskAttempt) task).getNodesWithSiblingRunningAttempts(); + Set nodesWithSiblingRunningAttempts = ((TaskAttempt) task).getTask().getNodesWithRunningAttempts(); if (nodesWithSiblingRunningAttempts != null && nodesWithSiblingRunningAttempts.contains(nodeId)) { return true; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java index f128ec9aca..a3279676b7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java @@ -1536,7 +1536,8 @@ private boolean canAssignTaskToContainer( HeldContainer heldContainer = heldContainers.get(container.getId()); Object task = getTask(cookieContainerRequest); if (task instanceof TaskAttempt - && ((TaskAttempt) task).getNodesWithSiblingRunningAttempts().contains(container.getNodeId())) { + && ((TaskAttempt) task).getTask() != null + && ((TaskAttempt) task).getTask().getNodesWithRunningAttempts().contains(container.getNodeId())) { return false; } if (heldContainer == null || heldContainer.isNew()) { // New container. @@ -1790,9 +1791,9 @@ private void informAppAboutAssignments( Container container = entry.getValue(); // check for blacklisted nodes. There may be race conditions between // setting blacklist and receiving allocations - CookieContainerRequest request = entry.getKey(); - Object task = getTask(request); if (blacklistedNodes.contains(container.getNodeId())) { + CookieContainerRequest request = entry.getKey(); + Object task = getTask(request); LOG.info("Container: " + container.getId() + " allocated on blacklisted node: " + container.getNodeId() + " for task: " + task); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java index ce3e7e52b5..41cce3b60e 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java @@ -83,6 +83,7 @@ import org.apache.tez.dag.app.TaskCommunicatorWrapper; import org.apache.tez.dag.app.TaskHeartbeatHandler; import org.apache.tez.dag.app.dag.TaskAttemptStateInternal; +import org.apache.tez.dag.app.dag.Task; import org.apache.tez.dag.app.dag.Vertex; import org.apache.tez.dag.app.dag.event.DAGEvent; import org.apache.tez.dag.app.dag.event.DAGEventCounterUpdate; @@ -145,6 +146,7 @@ public FileStatus getFileStatus(Path f) throws IOException { TezConfiguration vertexConf = new TezConfiguration(); TaskLocationHint locationHint; Vertex mockVertex; + Task mockTask; ServicePluginInfo servicePluginInfo = new ServicePluginInfo() .setContainerLauncherName(TezConstants.getTezYarnServicePluginName()); @@ -161,6 +163,8 @@ public void setupTest() { TezConstants.getTezYarnServicePluginName()); createMockVertex(vertexConf); + mockTask = mock(Task.class); + when(mockTask.getVertex()).thenReturn(mockVertex); HistoryEventHandler mockHistHandler = mock(HistoryEventHandler.class); doReturn(mockHistHandler).when(appCtx).getHistoryHandler(); @@ -2193,8 +2197,8 @@ public MockTaskAttemptImpl(TezTaskID taskId, int attemptNumber, super(TezBuilderUtils.newTaskAttemptId(taskId, attemptNumber), eventHandler, tal, conf, clock, taskHeartbeatHandler, appContext, - isRescheduled, resource, containerContext, leafVertex, mockVertex, - locationHint, null, null, null); + isRescheduled, resource, containerContext, leafVertex, mockTask, + locationHint, null, null); } boolean inputFailedReported = false; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java index 51a4bdfb99..81cd675134 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java @@ -1365,8 +1365,8 @@ public MockTaskAttemptImpl(TezTaskAttemptID attemptId, boolean isRescheduled, Resource resource, ContainerContext containerContext, TezTaskAttemptID schedCausalTA) { super(attemptId, eventHandler, tal, conf, clock, thh, - appContext, isRescheduled, resource, containerContext, false, null, - locationHint, mockTaskSpec, schedCausalTA, null); + appContext, isRescheduled, resource, containerContext, false, mockTask, + locationHint, mockTaskSpec, schedCausalTA); } @Override diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java index c979a7ab7f..ad0cf079f2 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java @@ -40,6 +40,7 @@ import org.apache.tez.common.MockDNSToSwitchMapping; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.app.MockClock; +import org.apache.tez.dag.app.dag.Task; import org.apache.tez.dag.app.dag.TaskAttempt; import org.apache.tez.dag.app.rm.DagAwareYarnTaskScheduler.AMRMClientAsyncWrapper; import org.apache.tez.dag.app.rm.DagAwareYarnTaskScheduler.HeldContainer; @@ -350,7 +351,9 @@ public void testNoReuse() throws Exception { NodeId speculativeNodeId = mock(NodeId.class); when(speculativeNodeId.getHost()).thenReturn(speculativeNode); TaskAttempt mockTask5 = mock(TaskAttempt.class); - when(mockTask5.getNodesWithSiblingRunningAttempts()).thenReturn(Sets.newHashSet(speculativeNodeId)); + Task task = mock(Task.class); + when(mockTask5.getTask()).thenReturn(task); + when(task.getNodesWithRunningAttempts()).thenReturn(Sets.newHashSet(speculativeNodeId)); Object mockCookie5 = new Object(); scheduler.allocateTask(mockTask5, mockCapability, hosts, racks, mockPriority, null, mockCookie5); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java index 21f4c52be7..965b8d76a4 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java @@ -60,6 +60,7 @@ import org.apache.tez.common.ContainerSignatureMatcher; import org.apache.tez.common.MockDNSToSwitchMapping; import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.app.dag.Task; import org.apache.tez.dag.app.dag.TaskAttempt; import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.AMRMClientAsyncForTest; import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.AMRMClientForTest; @@ -352,7 +353,9 @@ public void testTaskSchedulerNoReuse() throws Exception { NodeId speculativeNodeId = mock(NodeId.class); when(speculativeNodeId.getHost()).thenReturn(speculativeNode); TaskAttempt mockTask5 = mock(TaskAttempt.class); - when(mockTask5.getNodesWithSiblingRunningAttempts()).thenReturn(Sets.newHashSet(speculativeNodeId)); + Task task = mock(Task.class); + when(mockTask5.getTask()).thenReturn(task); + when(task.getNodesWithRunningAttempts()).thenReturn(Sets.newHashSet(speculativeNodeId)); Object mockCookie5 = new Object(); scheduler.allocateTask(mockTask5, mockCapability, hosts, racks, mockPriority, null, mockCookie5); From 44344a09ef8e1425ab2f8912a72cbb501b68b19e Mon Sep 17 00:00:00 2001 From: Eric Wohlstadter Date: Thu, 28 Mar 2019 13:23:10 -0700 Subject: [PATCH 159/512] TEZ-4057: Fix Unsorted broadcast shuffle umasks (Eric Wohlstadter, reviewed by Gopal V) Signed-off-by: Gopal V --- .../library/common/writers/UnorderedPartitionedKVWriter.java | 3 +++ .../common/writers/TestUnorderedPartitionedKVWriter.java | 4 ++++ 2 files changed, 7 insertions(+) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java index 0486ddc664..29478dcda1 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java @@ -281,6 +281,9 @@ public UnorderedPartitionedKVWriter(OutputContext outputContext, Configuration c skipBuffers = true; writer = new IFile.Writer(conf, rfs, finalOutPath, keyClass, valClass, codec, outputRecordsCounter, outputRecordBytesCounter); + if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { + rfs.setPermission(finalOutPath, SPILL_FILE_PERMS); + } } else { skipBuffers = false; writer = null; diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java index dfd807bfe4..83bde7b355 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java @@ -1264,6 +1264,10 @@ private void baseTest(int numRecords, int numPartitions, Set skippedPar assertTrue(localFs.exists(outputFilePath)); assertTrue(localFs.exists(spillFilePath)); + assertEquals("Incorrect output permissions", (short)0640, + localFs.getFileStatus(outputFilePath).getPermission().toShort()); + assertEquals("Incorrect index permissions", (short)0640, + localFs.getFileStatus(spillFilePath).getPermission().toShort()); // verify no intermediate spill files have been left around synchronized (kvWriter.spillInfoList) { From 8395a9560a131799f1af49b26e1f10f12ef48752 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Fri, 29 Mar 2019 19:54:27 -0500 Subject: [PATCH 160/512] TEZ-4058. Changes for 0.9.2 release --- Tez_DOAP.rdf | 7 +++++ .../markdown/releases/apache-tez-0-9-2.md | 30 +++++++++++++++++++ docs/src/site/markdown/releases/index.md | 1 + docs/src/site/site.xml | 1 + 4 files changed, 39 insertions(+) create mode 100644 docs/src/site/markdown/releases/apache-tez-0-9-2.md diff --git a/Tez_DOAP.rdf b/Tez_DOAP.rdf index 25bc3f53a3..68142ec096 100644 --- a/Tez_DOAP.rdf +++ b/Tez_DOAP.rdf @@ -34,6 +34,13 @@ Java + + + Version 0.9.2 + 2019-03-29 + 0.9.2 + + Version 0.9.1 diff --git a/docs/src/site/markdown/releases/apache-tez-0-9-2.md b/docs/src/site/markdown/releases/apache-tez-0-9-2.md new file mode 100644 index 0000000000..d29cd6d4e0 --- /dev/null +++ b/docs/src/site/markdown/releases/apache-tez-0-9-2.md @@ -0,0 +1,30 @@ + + +Apache TEZ® 0.9.2 + +Apache TEZ® 0.9.2 +---------------------- + +- [Download Release Artifacts](http://www.apache.org/dyn/closer.lua/tez/0.9.2/) +- [Release Notes](0.9.2/release-notes.txt) +- Documentation + - [API Javadocs](0.9.2/tez-api-javadocs/index.html) : Documentation for the Tez APIs + - [Runtime Library Javadocs](0.9.2/tez-runtime-library-javadocs/index.html) : Documentation for built-in implementations of useful Inputs, Outputs, Processors etc. written based on the Tez APIs + - [Tez Mapreduce Javadocs](0.9.2/tez-mapreduce-javadocs/index.html) : Documentation for built-in implementations of Mapreduce compatible Inputs, Outputs, Processors etc. written based on the Tez APIs + - [Tez Configuration](0.9.2/tez-api-javadocs/configs/TezConfiguration.html) : Documentation for configurations of Tez. These configurations are typically specified in tez-site.xml. + - [Tez Runtime Configuration](0.9.2/tez-runtime-library-javadocs/configs/TezRuntimeConfiguration.html) : Documentation for runtime configurations of Tez. These configurations are typically specified by job submitters. diff --git a/docs/src/site/markdown/releases/index.md b/docs/src/site/markdown/releases/index.md index 3c2b393b1a..23ddd43f67 100644 --- a/docs/src/site/markdown/releases/index.md +++ b/docs/src/site/markdown/releases/index.md @@ -19,6 +19,7 @@ Releases ------------ +- [Apache TEZ® 0.9.2](./apache-tez-0-9-2.html) (Mar 29, 2019) - [Apache TEZ® 0.9.1](./apache-tez-0-9-1.html) (Jan 04, 2018) - [Apache TEZ® 0.9.0](./apache-tez-0-9-0.html) (Jul 27, 2017) - [Apache TEZ® 0.8.5](./apache-tez-0-8-5.html) (Mar 13, 2017) diff --git a/docs/src/site/site.xml b/docs/src/site/site.xml index 4b2d3a4571..b4fbfbf878 100644 --- a/docs/src/site/site.xml +++ b/docs/src/site/site.xml @@ -128,6 +128,7 @@ + From 46b4004d97dd2f2cde491a93abcdd48c9b82f68e Mon Sep 17 00:00:00 2001 From: Siddharth Seth Date: Fri, 26 Apr 2019 12:21:35 -0700 Subject: [PATCH 161/512] TEZ-1348. Allow Tez local mode to run against filesystems other than local FS. (Todd Lipcon via sseth) --- .../apache/tez/common/TezUtilsInternal.java | 41 +------- .../org/apache/tez/client/LocalClient.java | 53 +++++------ .../apache/tez/examples/TezExampleBase.java | 3 +- .../org/apache/tez/test/TestLocalMode.java | 93 +++++++++++++------ 4 files changed, 93 insertions(+), 97 deletions(-) diff --git a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java index 5d7aea3724..adcae8a964 100644 --- a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java +++ b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java @@ -71,20 +71,10 @@ public class TezUtilsInternal { public static ConfigurationProto readUserSpecifiedTezConfiguration(String baseDir) throws IOException { - FileInputStream confPBBinaryStream = null; - ConfigurationProto.Builder confProtoBuilder = ConfigurationProto.newBuilder(); - try { - confPBBinaryStream = - new FileInputStream(new File(baseDir, TezConstants.TEZ_PB_BINARY_CONF_NAME)); - confProtoBuilder.mergeFrom(confPBBinaryStream); - } finally { - if (confPBBinaryStream != null) { - confPBBinaryStream.close(); - } + File confPBFile = new File(baseDir, TezConstants.TEZ_PB_BINARY_CONF_NAME); + try (FileInputStream fis = new FileInputStream(confPBFile)) { + return ConfigurationProto.parseFrom(fis); } - - ConfigurationProto confProto = confProtoBuilder.build(); - return confProto; } public static void addUserSpecifiedTezConfiguration(Configuration conf, @@ -95,31 +85,6 @@ public static void addUserSpecifiedTezConfiguration(Configuration conf, } } } -// -// public static void addUserSpecifiedTezConfiguration(String baseDir, Configuration conf) throws -// IOException { -// FileInputStream confPBBinaryStream = null; -// ConfigurationProto.Builder confProtoBuilder = ConfigurationProto.newBuilder(); -// try { -// confPBBinaryStream = -// new FileInputStream(new File(baseDir, TezConstants.TEZ_PB_BINARY_CONF_NAME)); -// confProtoBuilder.mergeFrom(confPBBinaryStream); -// } finally { -// if (confPBBinaryStream != null) { -// confPBBinaryStream.close(); -// } -// } -// -// ConfigurationProto confProto = confProtoBuilder.build(); -// -// List kvPairList = confProto.getConfKeyValuesList(); -// if (kvPairList != null && !kvPairList.isEmpty()) { -// for (PlanKeyValuePair kvPair : kvPairList) { -// conf.set(kvPair.getKey(), kvPair.getValue()); -// } -// } -// } - public static byte[] compressBytes(byte[] inBytes) throws IOException { StopWatch sw = new StopWatch().start(); diff --git a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java index 6baea482e9..140ada1934 100644 --- a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java +++ b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java @@ -24,6 +24,8 @@ import java.nio.ByteBuffer; import java.util.List; +import javax.annotation.Nullable; + import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; import org.apache.hadoop.yarn.api.records.ContainerId; @@ -83,7 +85,6 @@ public LocalClient() { @Override public void init(TezConfiguration tezConf, YarnConfiguration yarnConf) { this.conf = tezConf; - tezConf.set("fs.defaultFS", "file:///"); // Tez libs already in the client's classpath this.conf.setBoolean(TezConfiguration.TEZ_IGNORE_LIB_URIS, true); this.conf.set(TezConfiguration.TEZ_AM_DAG_SCHEDULER_CLASS, localModeDAGSchedulerClassName); @@ -286,19 +287,34 @@ public void run() { try { ApplicationId appId = appContext.getApplicationId(); - // Set up working directory for DAGAppMaster + // Set up working directory for DAGAppMaster. + // The staging directory may be on the default file system, which may or may not + // be the local FS. For example, when using testing Hive against a pseudo-distributed + // cluster, it's useful for the default FS to be HDFS. Hive then puts its scratch + // directories on HDFS, and sets the Tez staging directory to be the session's + // scratch directory. + // + // To handle this case, we need to copy over the staging data back onto the + // local file system, where the rest of the Tez Child code expects it. + // + // NOTE: we base the local working directory path off of the staging path, even + // though it might be on a different file system. Typically they're both in a + // path starting with /tmp, but in the future we may want to use a different + // temp directory locally. Path staging = TezCommonUtils.getTezSystemStagingPath(conf, appId.toString()); - Path userDir = TezCommonUtils.getTezSystemStagingPath(conf, appId.toString()+"_wd"); + FileSystem stagingFs = staging.getFileSystem(conf); + + FileSystem localFs = FileSystem.getLocal(conf); + Path userDir = localFs.makeQualified(new Path(staging.toUri().getPath() + "_wd")); LOG.info("Using working directory: " + userDir.toUri().getPath()); - FileSystem fs = FileSystem.get(conf); // copy data from staging directory to working directory to simulate the resource localizing - FileUtil.copy(fs, staging, fs, userDir, false, conf); + FileUtil.copy(stagingFs, staging, localFs, userDir, false, conf); // Prepare Environment Path logDir = new Path(userDir, "localmode-log-dir"); Path localDir = new Path(userDir, "localmode-local-dir"); - fs.mkdirs(logDir); - fs.mkdirs(localDir); + localFs.mkdirs(logDir); + localFs.mkdirs(localDir); UserGroupInformation.setConfiguration(conf); // Add session specific credentials to the AM credentials. @@ -357,30 +373,11 @@ protected DAGAppMaster createDAGAppMaster(ApplicationAttemptId applicationAttemp // Read in additional information about external services AMPluginDescriptorProto amPluginDescriptorProto = - getPluginDescriptorInfo(conf, applicationAttemptId.getApplicationId().toString()); - + TezUtilsInternal.readUserSpecifiedTezConfiguration(userDir) + .getAmPluginDescriptor(); return new DAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort, new SystemClock(), appSubmitTime, isSession, userDir, localDirs, logDirs, versionInfo.getVersion(), credentials, jobUserName, amPluginDescriptorProto); } - - private AMPluginDescriptorProto getPluginDescriptorInfo(Configuration conf, - String applicationIdString) throws - IOException { - Path tezSysStagingPath = TezCommonUtils - .getTezSystemStagingPath(conf, applicationIdString); - // Remove the filesystem qualifier. - String unqualifiedPath = tezSysStagingPath.toUri().getPath(); - - DAGProtos.ConfigurationProto confProto = - TezUtilsInternal - .readUserSpecifiedTezConfiguration(unqualifiedPath); - AMPluginDescriptorProto amPluginDescriptorProto = null; - if (confProto.hasAmPluginDescriptor()) { - amPluginDescriptorProto = confProto.getAmPluginDescriptor(); - } - return amPluginDescriptorProto; - } - } diff --git a/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java b/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java index 6b626b189b..cb521055e4 100644 --- a/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java +++ b/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java @@ -276,8 +276,7 @@ private void _printUsage() { protected void printExtraOptionsUsage(PrintStream ps) { ps.println("Tez example extra options supported are"); - // TODO TEZ-1348 make it able to access dfs in tez local mode - ps.println("-" + LOCAL_MODE + "\t\trun it in tez local mode, currently it can only access local file system in tez local mode," + ps.println("-" + LOCAL_MODE + "\t\trun it in tez local mode, " + " run it in distributed mode without this option"); ps.println("-" + DISABLE_SPLIT_GROUPING + "\t\t disable split grouping for MRInput," + " enable split grouping without this option."); diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java b/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java index 2a5b65fa2c..318349ca4f 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java @@ -20,12 +20,16 @@ import java.io.File; import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; import java.util.List; import java.util.Map; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.tez.client.TezClient; import org.apache.tez.dag.api.DAG; import org.apache.tez.dag.api.ProcessorDescriptor; @@ -43,23 +47,73 @@ import org.apache.tez.runtime.api.ProcessorContext; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; import org.apache.tez.runtime.library.processor.SleepProcessor; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import static org.junit.Assert.*; +@RunWith(Parameterized.class) public class TestLocalMode { private static final File TEST_DIR = new File( System.getProperty("test.build.data", System.getProperty("java.io.tmpdir")), "TestLocalMode-tez-localmode"); + private static MiniDFSCluster dfsCluster; + private static FileSystem remoteFs; + + @Parameterized.Parameter + public boolean useDfs; + + @Parameterized.Parameters(name = "useDFS:{0}") + public static Collection params() { + return Arrays.asList(new Object[][]{{ false }, { true }}); + } + + + @BeforeClass + public static void beforeClass() throws Exception { + try { + Configuration conf = new Configuration(); + dfsCluster = + new MiniDFSCluster.Builder(conf).numDataNodes(3).format(true) + .racks(null).build(); + remoteFs = dfsCluster.getFileSystem(); + } catch (IOException io) { + throw new RuntimeException("problem starting mini dfs cluster", io); + } + } + + @AfterClass + public static void afterClass() throws InterruptedException { + if (dfsCluster != null) { + try { + dfsCluster.shutdown(); + } catch (Exception e) { + e.printStackTrace(); + } + } + } + + private TezConfiguration createConf() { + TezConfiguration conf = new TezConfiguration(); + conf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); + if (useDfs) { + conf.set("fs.defaultFS", remoteFs.getUri().toString()); + } else { + conf.set("fs.defaultFS", "file:///"); + } + conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); + return conf; + } + @Test(timeout = 30000) public void testMultipleClientsWithSession() throws TezException, InterruptedException, IOException { - TezConfiguration tezConf1 = new TezConfiguration(); - tezConf1.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); - tezConf1.set("fs.defaultFS", "file:///"); - tezConf1.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); + TezConfiguration tezConf1 = createConf(); TezClient tezClient1 = TezClient.create("commonName", tezConf1, true); tezClient1.start(); @@ -72,11 +126,7 @@ public void testMultipleClientsWithSession() throws TezException, InterruptedExc dagClient1.close(); tezClient1.stop(); - - TezConfiguration tezConf2 = new TezConfiguration(); - tezConf2.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); - tezConf2.set("fs.defaultFS", "file:///"); - tezConf2.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); + TezConfiguration tezConf2 = createConf(); DAG dag2 = createSimpleDAG("dag2", SleepProcessor.class.getName()); TezClient tezClient2 = TezClient.create("commonName", tezConf2, true); tezClient2.start(); @@ -91,10 +141,7 @@ public void testMultipleClientsWithSession() throws TezException, InterruptedExc @Test(timeout = 10000) public void testMultipleClientsWithoutSession() throws TezException, InterruptedException, IOException { - TezConfiguration tezConf1 = new TezConfiguration(); - tezConf1.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); - tezConf1.set("fs.defaultFS", "file:///"); - tezConf1.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); + TezConfiguration tezConf1 = createConf(); TezClient tezClient1 = TezClient.create("commonName", tezConf1, false); tezClient1.start(); @@ -108,10 +155,7 @@ public void testMultipleClientsWithoutSession() throws TezException, Interrupted tezClient1.stop(); - TezConfiguration tezConf2 = new TezConfiguration(); - tezConf2.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); - tezConf2.set("fs.defaultFS", "file:///"); - tezConf2.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); + TezConfiguration tezConf2 = createConf(); DAG dag2 = createSimpleDAG("dag2", SleepProcessor.class.getName()); TezClient tezClient2 = TezClient.create("commonName", tezConf2, false); tezClient2.start(); @@ -126,10 +170,7 @@ public void testMultipleClientsWithoutSession() throws TezException, Interrupted @Test(timeout = 20000) public void testNoSysExitOnSuccessfulDAG() throws TezException, InterruptedException, IOException { - TezConfiguration tezConf1 = new TezConfiguration(); - tezConf1.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); - tezConf1.set("fs.defaultFS", "file:///"); - tezConf1.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); + TezConfiguration tezConf1 = createConf(); // Run in non-session mode so that the AM terminates TezClient tezClient1 = TezClient.create("commonName", tezConf1, false); tezClient1.start(); @@ -150,10 +191,7 @@ public void testNoSysExitOnSuccessfulDAG() throws TezException, InterruptedExcep @Test(timeout = 20000) public void testNoSysExitOnFailinglDAG() throws TezException, InterruptedException, IOException { - TezConfiguration tezConf1 = new TezConfiguration(); - tezConf1.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); - tezConf1.set("fs.defaultFS", "file:///"); - tezConf1.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); + TezConfiguration tezConf1 = createConf(); // Run in non-session mode so that the AM terminates TezClient tezClient1 = TezClient.create("commonName", tezConf1, false); tezClient1.start(); @@ -211,10 +249,7 @@ public void testMultiDAGsOnSession() throws IOException, TezException, Interrupt String[] outputPaths = new String[dags]; DAGClient[] dagClients = new DAGClient[dags]; - TezConfiguration tezConf = new TezConfiguration(); - tezConf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); - tezConf.set("fs.defaultFS", "file:///"); - tezConf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); + TezConfiguration tezConf = createConf(); TezClient tezClient = TezClient.create("testMultiDAGOnSession", tezConf, true); tezClient.start(); From 0f71b0b0622f6a9af71bf9d8b3bf22c1a040cccb Mon Sep 17 00:00:00 2001 From: Ying Han Date: Fri, 26 Apr 2019 15:20:29 -0500 Subject: [PATCH 162/512] TEZ-4062. Speculative attempt scheduling should be aborted when Task has completed Signed-off-by: Jonathan Eagles --- .../apache/tez/dag/app/dag/impl/TaskImpl.java | 11 +++++++++++ .../tez/dag/app/dag/impl/TestTaskImpl.java | 17 +++++++++++++++++ 2 files changed, 28 insertions(+) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java index 9289d8f4cc..e563fe9cbf 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java @@ -1030,8 +1030,19 @@ public void transition(TaskImpl task, TaskEvent event) { if (!ta.isFinished()) { earliestUnfinishedAttempt = ta; task.nodesWithRunningAttempts.add(ta.getNodeId()); + } else { + if (TaskAttemptState.SUCCEEDED.equals(ta.getState())) { + LOG.info("Ignore speculation scheduling for task {} since it has succeeded with attempt {}.", + task.getTaskId(), ta.getID()); + return; + } } } + if (earliestUnfinishedAttempt == null) { + // no running (or SUCCEEDED) task attempt at this moment, no need to schedule speculative attempt either + LOG.info("Ignore speculation scheduling since there is no running attempt on task {}.", task.getTaskId()); + return; + } task.addAndScheduleAttempt(earliestUnfinishedAttempt.getID()); } } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java index 81cd675134..2d4adcc084 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java @@ -982,6 +982,23 @@ public void testSpeculatedThenRetroactiveFailure() { Assert.assertEquals(mockDestId, newAttempt.getSchedulingCausalTA()); } + @Test(timeout = 20000) + public void testIgnoreSpeculationOnSuccessfulOriginalAttempt() { + TezTaskID taskId = getNewTaskID(); + scheduleTaskAttempt(taskId); + MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); + launchTaskAttempt(firstAttempt.getID()); + // Mock success of the first task attempt + updateAttemptState(firstAttempt, TaskAttemptState.SUCCEEDED); + firstAttempt.handle(new TaskAttemptEvent(firstAttempt.getID(), TaskAttemptEventType.TA_DONE)); + + // Verify the speculation scheduling is ignored and no speculative attempt was added to the task + mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getID())); + MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); + launchTaskAttempt(specAttempt.getID()); + assertEquals(1, mockTask.getAttemptList().size()); + } + @SuppressWarnings("rawtypes") @Test public void testSucceededAttemptStatusWithRetroActiveFailures() throws InterruptedException { From 4bbd3c2e7600752cfe0d074d61e00e12ea0ee748 Mon Sep 17 00:00:00 2001 From: Siddharth Seth Date: Mon, 6 May 2019 10:51:43 -0700 Subject: [PATCH 163/512] Revert "TEZ-1348. Allow Tez local mode to run against filesystems other than" This reverts commit 46b4004d97dd2f2cde491a93abcdd48c9b82f68e. --- .../apache/tez/common/TezUtilsInternal.java | 41 +++++++- .../org/apache/tez/client/LocalClient.java | 53 ++++++----- .../apache/tez/examples/TezExampleBase.java | 3 +- .../org/apache/tez/test/TestLocalMode.java | 93 ++++++------------- 4 files changed, 97 insertions(+), 93 deletions(-) diff --git a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java index adcae8a964..5d7aea3724 100644 --- a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java +++ b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java @@ -71,10 +71,20 @@ public class TezUtilsInternal { public static ConfigurationProto readUserSpecifiedTezConfiguration(String baseDir) throws IOException { - File confPBFile = new File(baseDir, TezConstants.TEZ_PB_BINARY_CONF_NAME); - try (FileInputStream fis = new FileInputStream(confPBFile)) { - return ConfigurationProto.parseFrom(fis); + FileInputStream confPBBinaryStream = null; + ConfigurationProto.Builder confProtoBuilder = ConfigurationProto.newBuilder(); + try { + confPBBinaryStream = + new FileInputStream(new File(baseDir, TezConstants.TEZ_PB_BINARY_CONF_NAME)); + confProtoBuilder.mergeFrom(confPBBinaryStream); + } finally { + if (confPBBinaryStream != null) { + confPBBinaryStream.close(); + } } + + ConfigurationProto confProto = confProtoBuilder.build(); + return confProto; } public static void addUserSpecifiedTezConfiguration(Configuration conf, @@ -85,6 +95,31 @@ public static void addUserSpecifiedTezConfiguration(Configuration conf, } } } +// +// public static void addUserSpecifiedTezConfiguration(String baseDir, Configuration conf) throws +// IOException { +// FileInputStream confPBBinaryStream = null; +// ConfigurationProto.Builder confProtoBuilder = ConfigurationProto.newBuilder(); +// try { +// confPBBinaryStream = +// new FileInputStream(new File(baseDir, TezConstants.TEZ_PB_BINARY_CONF_NAME)); +// confProtoBuilder.mergeFrom(confPBBinaryStream); +// } finally { +// if (confPBBinaryStream != null) { +// confPBBinaryStream.close(); +// } +// } +// +// ConfigurationProto confProto = confProtoBuilder.build(); +// +// List kvPairList = confProto.getConfKeyValuesList(); +// if (kvPairList != null && !kvPairList.isEmpty()) { +// for (PlanKeyValuePair kvPair : kvPairList) { +// conf.set(kvPair.getKey(), kvPair.getValue()); +// } +// } +// } + public static byte[] compressBytes(byte[] inBytes) throws IOException { StopWatch sw = new StopWatch().start(); diff --git a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java index 140ada1934..6baea482e9 100644 --- a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java +++ b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java @@ -24,8 +24,6 @@ import java.nio.ByteBuffer; import java.util.List; -import javax.annotation.Nullable; - import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; import org.apache.hadoop.yarn.api.records.ContainerId; @@ -85,6 +83,7 @@ public LocalClient() { @Override public void init(TezConfiguration tezConf, YarnConfiguration yarnConf) { this.conf = tezConf; + tezConf.set("fs.defaultFS", "file:///"); // Tez libs already in the client's classpath this.conf.setBoolean(TezConfiguration.TEZ_IGNORE_LIB_URIS, true); this.conf.set(TezConfiguration.TEZ_AM_DAG_SCHEDULER_CLASS, localModeDAGSchedulerClassName); @@ -287,34 +286,19 @@ public void run() { try { ApplicationId appId = appContext.getApplicationId(); - // Set up working directory for DAGAppMaster. - // The staging directory may be on the default file system, which may or may not - // be the local FS. For example, when using testing Hive against a pseudo-distributed - // cluster, it's useful for the default FS to be HDFS. Hive then puts its scratch - // directories on HDFS, and sets the Tez staging directory to be the session's - // scratch directory. - // - // To handle this case, we need to copy over the staging data back onto the - // local file system, where the rest of the Tez Child code expects it. - // - // NOTE: we base the local working directory path off of the staging path, even - // though it might be on a different file system. Typically they're both in a - // path starting with /tmp, but in the future we may want to use a different - // temp directory locally. + // Set up working directory for DAGAppMaster Path staging = TezCommonUtils.getTezSystemStagingPath(conf, appId.toString()); - FileSystem stagingFs = staging.getFileSystem(conf); - - FileSystem localFs = FileSystem.getLocal(conf); - Path userDir = localFs.makeQualified(new Path(staging.toUri().getPath() + "_wd")); + Path userDir = TezCommonUtils.getTezSystemStagingPath(conf, appId.toString()+"_wd"); LOG.info("Using working directory: " + userDir.toUri().getPath()); + FileSystem fs = FileSystem.get(conf); // copy data from staging directory to working directory to simulate the resource localizing - FileUtil.copy(stagingFs, staging, localFs, userDir, false, conf); + FileUtil.copy(fs, staging, fs, userDir, false, conf); // Prepare Environment Path logDir = new Path(userDir, "localmode-log-dir"); Path localDir = new Path(userDir, "localmode-local-dir"); - localFs.mkdirs(logDir); - localFs.mkdirs(localDir); + fs.mkdirs(logDir); + fs.mkdirs(localDir); UserGroupInformation.setConfiguration(conf); // Add session specific credentials to the AM credentials. @@ -373,11 +357,30 @@ protected DAGAppMaster createDAGAppMaster(ApplicationAttemptId applicationAttemp // Read in additional information about external services AMPluginDescriptorProto amPluginDescriptorProto = - TezUtilsInternal.readUserSpecifiedTezConfiguration(userDir) - .getAmPluginDescriptor(); + getPluginDescriptorInfo(conf, applicationAttemptId.getApplicationId().toString()); + return new DAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort, new SystemClock(), appSubmitTime, isSession, userDir, localDirs, logDirs, versionInfo.getVersion(), credentials, jobUserName, amPluginDescriptorProto); } + + private AMPluginDescriptorProto getPluginDescriptorInfo(Configuration conf, + String applicationIdString) throws + IOException { + Path tezSysStagingPath = TezCommonUtils + .getTezSystemStagingPath(conf, applicationIdString); + // Remove the filesystem qualifier. + String unqualifiedPath = tezSysStagingPath.toUri().getPath(); + + DAGProtos.ConfigurationProto confProto = + TezUtilsInternal + .readUserSpecifiedTezConfiguration(unqualifiedPath); + AMPluginDescriptorProto amPluginDescriptorProto = null; + if (confProto.hasAmPluginDescriptor()) { + amPluginDescriptorProto = confProto.getAmPluginDescriptor(); + } + return amPluginDescriptorProto; + } + } diff --git a/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java b/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java index cb521055e4..6b626b189b 100644 --- a/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java +++ b/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java @@ -276,7 +276,8 @@ private void _printUsage() { protected void printExtraOptionsUsage(PrintStream ps) { ps.println("Tez example extra options supported are"); - ps.println("-" + LOCAL_MODE + "\t\trun it in tez local mode, " + // TODO TEZ-1348 make it able to access dfs in tez local mode + ps.println("-" + LOCAL_MODE + "\t\trun it in tez local mode, currently it can only access local file system in tez local mode," + " run it in distributed mode without this option"); ps.println("-" + DISABLE_SPLIT_GROUPING + "\t\t disable split grouping for MRInput," + " enable split grouping without this option."); diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java b/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java index 318349ca4f..2a5b65fa2c 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java @@ -20,16 +20,12 @@ import java.io.File; import java.io.IOException; -import java.util.Arrays; -import java.util.Collection; import java.util.List; import java.util.Map; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.tez.client.TezClient; import org.apache.tez.dag.api.DAG; import org.apache.tez.dag.api.ProcessorDescriptor; @@ -47,73 +43,23 @@ import org.apache.tez.runtime.api.ProcessorContext; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; import org.apache.tez.runtime.library.processor.SleepProcessor; -import org.junit.AfterClass; -import org.junit.BeforeClass; import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; import static org.junit.Assert.*; -@RunWith(Parameterized.class) public class TestLocalMode { private static final File TEST_DIR = new File( System.getProperty("test.build.data", System.getProperty("java.io.tmpdir")), "TestLocalMode-tez-localmode"); - private static MiniDFSCluster dfsCluster; - private static FileSystem remoteFs; - - @Parameterized.Parameter - public boolean useDfs; - - @Parameterized.Parameters(name = "useDFS:{0}") - public static Collection params() { - return Arrays.asList(new Object[][]{{ false }, { true }}); - } - - - @BeforeClass - public static void beforeClass() throws Exception { - try { - Configuration conf = new Configuration(); - dfsCluster = - new MiniDFSCluster.Builder(conf).numDataNodes(3).format(true) - .racks(null).build(); - remoteFs = dfsCluster.getFileSystem(); - } catch (IOException io) { - throw new RuntimeException("problem starting mini dfs cluster", io); - } - } - - @AfterClass - public static void afterClass() throws InterruptedException { - if (dfsCluster != null) { - try { - dfsCluster.shutdown(); - } catch (Exception e) { - e.printStackTrace(); - } - } - } - - private TezConfiguration createConf() { - TezConfiguration conf = new TezConfiguration(); - conf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); - if (useDfs) { - conf.set("fs.defaultFS", remoteFs.getUri().toString()); - } else { - conf.set("fs.defaultFS", "file:///"); - } - conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); - return conf; - } - @Test(timeout = 30000) public void testMultipleClientsWithSession() throws TezException, InterruptedException, IOException { - TezConfiguration tezConf1 = createConf(); + TezConfiguration tezConf1 = new TezConfiguration(); + tezConf1.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); + tezConf1.set("fs.defaultFS", "file:///"); + tezConf1.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); TezClient tezClient1 = TezClient.create("commonName", tezConf1, true); tezClient1.start(); @@ -126,7 +72,11 @@ public void testMultipleClientsWithSession() throws TezException, InterruptedExc dagClient1.close(); tezClient1.stop(); - TezConfiguration tezConf2 = createConf(); + + TezConfiguration tezConf2 = new TezConfiguration(); + tezConf2.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); + tezConf2.set("fs.defaultFS", "file:///"); + tezConf2.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); DAG dag2 = createSimpleDAG("dag2", SleepProcessor.class.getName()); TezClient tezClient2 = TezClient.create("commonName", tezConf2, true); tezClient2.start(); @@ -141,7 +91,10 @@ public void testMultipleClientsWithSession() throws TezException, InterruptedExc @Test(timeout = 10000) public void testMultipleClientsWithoutSession() throws TezException, InterruptedException, IOException { - TezConfiguration tezConf1 = createConf(); + TezConfiguration tezConf1 = new TezConfiguration(); + tezConf1.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); + tezConf1.set("fs.defaultFS", "file:///"); + tezConf1.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); TezClient tezClient1 = TezClient.create("commonName", tezConf1, false); tezClient1.start(); @@ -155,7 +108,10 @@ public void testMultipleClientsWithoutSession() throws TezException, Interrupted tezClient1.stop(); - TezConfiguration tezConf2 = createConf(); + TezConfiguration tezConf2 = new TezConfiguration(); + tezConf2.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); + tezConf2.set("fs.defaultFS", "file:///"); + tezConf2.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); DAG dag2 = createSimpleDAG("dag2", SleepProcessor.class.getName()); TezClient tezClient2 = TezClient.create("commonName", tezConf2, false); tezClient2.start(); @@ -170,7 +126,10 @@ public void testMultipleClientsWithoutSession() throws TezException, Interrupted @Test(timeout = 20000) public void testNoSysExitOnSuccessfulDAG() throws TezException, InterruptedException, IOException { - TezConfiguration tezConf1 = createConf(); + TezConfiguration tezConf1 = new TezConfiguration(); + tezConf1.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); + tezConf1.set("fs.defaultFS", "file:///"); + tezConf1.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); // Run in non-session mode so that the AM terminates TezClient tezClient1 = TezClient.create("commonName", tezConf1, false); tezClient1.start(); @@ -191,7 +150,10 @@ public void testNoSysExitOnSuccessfulDAG() throws TezException, InterruptedExcep @Test(timeout = 20000) public void testNoSysExitOnFailinglDAG() throws TezException, InterruptedException, IOException { - TezConfiguration tezConf1 = createConf(); + TezConfiguration tezConf1 = new TezConfiguration(); + tezConf1.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); + tezConf1.set("fs.defaultFS", "file:///"); + tezConf1.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); // Run in non-session mode so that the AM terminates TezClient tezClient1 = TezClient.create("commonName", tezConf1, false); tezClient1.start(); @@ -249,7 +211,10 @@ public void testMultiDAGsOnSession() throws IOException, TezException, Interrupt String[] outputPaths = new String[dags]; DAGClient[] dagClients = new DAGClient[dags]; - TezConfiguration tezConf = createConf(); + TezConfiguration tezConf = new TezConfiguration(); + tezConf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); + tezConf.set("fs.defaultFS", "file:///"); + tezConf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); TezClient tezClient = TezClient.create("testMultiDAGOnSession", tezConf, true); tezClient.start(); From 82c1d75c7b3abf7316ba265a599b910fcc2360c7 Mon Sep 17 00:00:00 2001 From: Siddharth Seth Date: Mon, 6 May 2019 10:52:19 -0700 Subject: [PATCH 164/512] EZ-1348. Allow Tez local mode to run against filesystems other than local FS. (Todd Lipcon via sseth) --- .../apache/tez/common/TezUtilsInternal.java | 41 +------- .../org/apache/tez/client/LocalClient.java | 53 +++++----- .../apache/tez/examples/TezExampleBase.java | 3 +- .../org/apache/tez/test/TestLocalMode.java | 98 +++++++++++++------ 4 files changed, 97 insertions(+), 98 deletions(-) diff --git a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java index 5d7aea3724..adcae8a964 100644 --- a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java +++ b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java @@ -71,20 +71,10 @@ public class TezUtilsInternal { public static ConfigurationProto readUserSpecifiedTezConfiguration(String baseDir) throws IOException { - FileInputStream confPBBinaryStream = null; - ConfigurationProto.Builder confProtoBuilder = ConfigurationProto.newBuilder(); - try { - confPBBinaryStream = - new FileInputStream(new File(baseDir, TezConstants.TEZ_PB_BINARY_CONF_NAME)); - confProtoBuilder.mergeFrom(confPBBinaryStream); - } finally { - if (confPBBinaryStream != null) { - confPBBinaryStream.close(); - } + File confPBFile = new File(baseDir, TezConstants.TEZ_PB_BINARY_CONF_NAME); + try (FileInputStream fis = new FileInputStream(confPBFile)) { + return ConfigurationProto.parseFrom(fis); } - - ConfigurationProto confProto = confProtoBuilder.build(); - return confProto; } public static void addUserSpecifiedTezConfiguration(Configuration conf, @@ -95,31 +85,6 @@ public static void addUserSpecifiedTezConfiguration(Configuration conf, } } } -// -// public static void addUserSpecifiedTezConfiguration(String baseDir, Configuration conf) throws -// IOException { -// FileInputStream confPBBinaryStream = null; -// ConfigurationProto.Builder confProtoBuilder = ConfigurationProto.newBuilder(); -// try { -// confPBBinaryStream = -// new FileInputStream(new File(baseDir, TezConstants.TEZ_PB_BINARY_CONF_NAME)); -// confProtoBuilder.mergeFrom(confPBBinaryStream); -// } finally { -// if (confPBBinaryStream != null) { -// confPBBinaryStream.close(); -// } -// } -// -// ConfigurationProto confProto = confProtoBuilder.build(); -// -// List kvPairList = confProto.getConfKeyValuesList(); -// if (kvPairList != null && !kvPairList.isEmpty()) { -// for (PlanKeyValuePair kvPair : kvPairList) { -// conf.set(kvPair.getKey(), kvPair.getValue()); -// } -// } -// } - public static byte[] compressBytes(byte[] inBytes) throws IOException { StopWatch sw = new StopWatch().start(); diff --git a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java index 6baea482e9..9006971a55 100644 --- a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java +++ b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java @@ -24,6 +24,7 @@ import java.nio.ByteBuffer; import java.util.List; + import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; import org.apache.hadoop.yarn.api.records.ContainerId; @@ -50,7 +51,6 @@ import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.api.client.DAGClientHandler; -import org.apache.tez.dag.api.records.DAGProtos; import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto; import org.apache.tez.dag.app.AppContext; import org.apache.tez.dag.app.DAGAppMaster; @@ -83,7 +83,6 @@ public LocalClient() { @Override public void init(TezConfiguration tezConf, YarnConfiguration yarnConf) { this.conf = tezConf; - tezConf.set("fs.defaultFS", "file:///"); // Tez libs already in the client's classpath this.conf.setBoolean(TezConfiguration.TEZ_IGNORE_LIB_URIS, true); this.conf.set(TezConfiguration.TEZ_AM_DAG_SCHEDULER_CLASS, localModeDAGSchedulerClassName); @@ -286,19 +285,34 @@ public void run() { try { ApplicationId appId = appContext.getApplicationId(); - // Set up working directory for DAGAppMaster + // Set up working directory for DAGAppMaster. + // The staging directory may be on the default file system, which may or may not + // be the local FS. For example, when using testing Hive against a pseudo-distributed + // cluster, it's useful for the default FS to be HDFS. Hive then puts its scratch + // directories on HDFS, and sets the Tez staging directory to be the session's + // scratch directory. + // + // To handle this case, we need to copy over the staging data back onto the + // local file system, where the rest of the Tez Child code expects it. + // + // NOTE: we base the local working directory path off of the staging path, even + // though it might be on a different file system. Typically they're both in a + // path starting with /tmp, but in the future we may want to use a different + // temp directory locally. Path staging = TezCommonUtils.getTezSystemStagingPath(conf, appId.toString()); - Path userDir = TezCommonUtils.getTezSystemStagingPath(conf, appId.toString()+"_wd"); + FileSystem stagingFs = staging.getFileSystem(conf); + + FileSystem localFs = FileSystem.getLocal(conf); + Path userDir = localFs.makeQualified(new Path(staging.toUri().getPath() + "_wd")); LOG.info("Using working directory: " + userDir.toUri().getPath()); - FileSystem fs = FileSystem.get(conf); // copy data from staging directory to working directory to simulate the resource localizing - FileUtil.copy(fs, staging, fs, userDir, false, conf); + FileUtil.copy(stagingFs, staging, localFs, userDir, false, conf); // Prepare Environment Path logDir = new Path(userDir, "localmode-log-dir"); Path localDir = new Path(userDir, "localmode-local-dir"); - fs.mkdirs(logDir); - fs.mkdirs(localDir); + localFs.mkdirs(logDir); + localFs.mkdirs(localDir); UserGroupInformation.setConfiguration(conf); // Add session specific credentials to the AM credentials. @@ -357,30 +371,11 @@ protected DAGAppMaster createDAGAppMaster(ApplicationAttemptId applicationAttemp // Read in additional information about external services AMPluginDescriptorProto amPluginDescriptorProto = - getPluginDescriptorInfo(conf, applicationAttemptId.getApplicationId().toString()); - + TezUtilsInternal.readUserSpecifiedTezConfiguration(userDir) + .getAmPluginDescriptor(); return new DAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort, new SystemClock(), appSubmitTime, isSession, userDir, localDirs, logDirs, versionInfo.getVersion(), credentials, jobUserName, amPluginDescriptorProto); } - - private AMPluginDescriptorProto getPluginDescriptorInfo(Configuration conf, - String applicationIdString) throws - IOException { - Path tezSysStagingPath = TezCommonUtils - .getTezSystemStagingPath(conf, applicationIdString); - // Remove the filesystem qualifier. - String unqualifiedPath = tezSysStagingPath.toUri().getPath(); - - DAGProtos.ConfigurationProto confProto = - TezUtilsInternal - .readUserSpecifiedTezConfiguration(unqualifiedPath); - AMPluginDescriptorProto amPluginDescriptorProto = null; - if (confProto.hasAmPluginDescriptor()) { - amPluginDescriptorProto = confProto.getAmPluginDescriptor(); - } - return amPluginDescriptorProto; - } - } diff --git a/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java b/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java index 6b626b189b..cb521055e4 100644 --- a/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java +++ b/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java @@ -276,8 +276,7 @@ private void _printUsage() { protected void printExtraOptionsUsage(PrintStream ps) { ps.println("Tez example extra options supported are"); - // TODO TEZ-1348 make it able to access dfs in tez local mode - ps.println("-" + LOCAL_MODE + "\t\trun it in tez local mode, currently it can only access local file system in tez local mode," + ps.println("-" + LOCAL_MODE + "\t\trun it in tez local mode, " + " run it in distributed mode without this option"); ps.println("-" + DISABLE_SPLIT_GROUPING + "\t\t disable split grouping for MRInput," + " enable split grouping without this option."); diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java b/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java index 2a5b65fa2c..ffc67fe51e 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java @@ -20,12 +20,16 @@ import java.io.File; import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; import java.util.List; import java.util.Map; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.tez.client.TezClient; import org.apache.tez.dag.api.DAG; import org.apache.tez.dag.api.ProcessorDescriptor; @@ -43,23 +47,78 @@ import org.apache.tez.runtime.api.ProcessorContext; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; import org.apache.tez.runtime.library.processor.SleepProcessor; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import static org.junit.Assert.*; +/** + * Tests for running Tez in local execution mode (without YARN). + */ +@RunWith(Parameterized.class) public class TestLocalMode { private static final File TEST_DIR = new File( System.getProperty("test.build.data", System.getProperty("java.io.tmpdir")), "TestLocalMode-tez-localmode"); + private static MiniDFSCluster dfsCluster; + private static FileSystem remoteFs; + + private final boolean useDfs; + + @Parameterized.Parameters(name = "useDFS:{0}") + public static Collection params() { + return Arrays.asList(new Object[][]{{false}, {true}}); + } + + public TestLocalMode(boolean useDfs) { + this.useDfs = useDfs; + } + + @BeforeClass + public static void beforeClass() throws Exception { + try { + Configuration conf = new Configuration(); + dfsCluster = + new MiniDFSCluster.Builder(conf).numDataNodes(3).format(true) + .racks(null).build(); + remoteFs = dfsCluster.getFileSystem(); + } catch (IOException io) { + throw new RuntimeException("problem starting mini dfs cluster", io); + } + } + + @AfterClass + public static void afterClass() throws InterruptedException { + if (dfsCluster != null) { + try { + dfsCluster.shutdown(); + } catch (Exception e) { + e.printStackTrace(); + } + } + } + + private TezConfiguration createConf() { + TezConfiguration conf = new TezConfiguration(); + conf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); + if (useDfs) { + conf.set("fs.defaultFS", remoteFs.getUri().toString()); + } else { + conf.set("fs.defaultFS", "file:///"); + } + conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); + return conf; + } + @Test(timeout = 30000) public void testMultipleClientsWithSession() throws TezException, InterruptedException, IOException { - TezConfiguration tezConf1 = new TezConfiguration(); - tezConf1.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); - tezConf1.set("fs.defaultFS", "file:///"); - tezConf1.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); + TezConfiguration tezConf1 = createConf(); TezClient tezClient1 = TezClient.create("commonName", tezConf1, true); tezClient1.start(); @@ -72,11 +131,7 @@ public void testMultipleClientsWithSession() throws TezException, InterruptedExc dagClient1.close(); tezClient1.stop(); - - TezConfiguration tezConf2 = new TezConfiguration(); - tezConf2.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); - tezConf2.set("fs.defaultFS", "file:///"); - tezConf2.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); + TezConfiguration tezConf2 = createConf(); DAG dag2 = createSimpleDAG("dag2", SleepProcessor.class.getName()); TezClient tezClient2 = TezClient.create("commonName", tezConf2, true); tezClient2.start(); @@ -91,10 +146,7 @@ public void testMultipleClientsWithSession() throws TezException, InterruptedExc @Test(timeout = 10000) public void testMultipleClientsWithoutSession() throws TezException, InterruptedException, IOException { - TezConfiguration tezConf1 = new TezConfiguration(); - tezConf1.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); - tezConf1.set("fs.defaultFS", "file:///"); - tezConf1.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); + TezConfiguration tezConf1 = createConf(); TezClient tezClient1 = TezClient.create("commonName", tezConf1, false); tezClient1.start(); @@ -108,10 +160,7 @@ public void testMultipleClientsWithoutSession() throws TezException, Interrupted tezClient1.stop(); - TezConfiguration tezConf2 = new TezConfiguration(); - tezConf2.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); - tezConf2.set("fs.defaultFS", "file:///"); - tezConf2.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); + TezConfiguration tezConf2 = createConf(); DAG dag2 = createSimpleDAG("dag2", SleepProcessor.class.getName()); TezClient tezClient2 = TezClient.create("commonName", tezConf2, false); tezClient2.start(); @@ -126,10 +175,7 @@ public void testMultipleClientsWithoutSession() throws TezException, Interrupted @Test(timeout = 20000) public void testNoSysExitOnSuccessfulDAG() throws TezException, InterruptedException, IOException { - TezConfiguration tezConf1 = new TezConfiguration(); - tezConf1.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); - tezConf1.set("fs.defaultFS", "file:///"); - tezConf1.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); + TezConfiguration tezConf1 = createConf(); // Run in non-session mode so that the AM terminates TezClient tezClient1 = TezClient.create("commonName", tezConf1, false); tezClient1.start(); @@ -150,10 +196,7 @@ public void testNoSysExitOnSuccessfulDAG() throws TezException, InterruptedExcep @Test(timeout = 20000) public void testNoSysExitOnFailinglDAG() throws TezException, InterruptedException, IOException { - TezConfiguration tezConf1 = new TezConfiguration(); - tezConf1.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); - tezConf1.set("fs.defaultFS", "file:///"); - tezConf1.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); + TezConfiguration tezConf1 = createConf(); // Run in non-session mode so that the AM terminates TezClient tezClient1 = TezClient.create("commonName", tezConf1, false); tezClient1.start(); @@ -211,10 +254,7 @@ public void testMultiDAGsOnSession() throws IOException, TezException, Interrupt String[] outputPaths = new String[dags]; DAGClient[] dagClients = new DAGClient[dags]; - TezConfiguration tezConf = new TezConfiguration(); - tezConf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); - tezConf.set("fs.defaultFS", "file:///"); - tezConf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); + TezConfiguration tezConf = createConf(); TezClient tezClient = TezClient.create("testMultiDAGOnSession", tezConf, true); tezClient.start(); From 2202f04d4bee465e529a734d9ec855dced170f00 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Tue, 7 May 2019 09:55:03 -0500 Subject: [PATCH 165/512] TEZ-4066. Upgrade servlet-api from 2.5 to 3.1.0 (Jonathan Eagles via kshukla) --- pom.xml | 4 ++-- tez-dag/pom.xml | 2 +- tez-dist/dist-files/full/LICENSE | 2 +- tez-dist/dist-files/minimal/LICENSE | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/pom.xml b/pom.xml index 28817025a1..89c081d3e6 100644 --- a/pom.xml +++ b/pom.xml @@ -266,8 +266,8 @@ javax.servlet - servlet-api - 2.5 + javax.servlet-api + 3.1.0 commons-codec diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml index c6962ae2ff..01f5623e4e 100644 --- a/tez-dag/pom.xml +++ b/tez-dag/pom.xml @@ -154,7 +154,7 @@ javax.servlet - servlet-api + javax.servlet-api compile diff --git a/tez-dist/dist-files/full/LICENSE b/tez-dist/dist-files/full/LICENSE index 2647d1399b..7b4c36c903 100644 --- a/tez-dist/dist-files/full/LICENSE +++ b/tez-dist/dist-files/full/LICENSE @@ -306,7 +306,7 @@ The Apache TEZ binary distribution bundles the following files under the CDDL Li - jersey-json-*.jar - jaxb-api-*.jar - jaxb-impl-*.jar - - servlet-api-*.jar + - javax.servlet-api-*.jar The text for this license can be found in the LICENSE-CDDLv1.1-GPLv2_withCPE file. diff --git a/tez-dist/dist-files/minimal/LICENSE b/tez-dist/dist-files/minimal/LICENSE index c89bc24176..11d7d0c210 100644 --- a/tez-dist/dist-files/minimal/LICENSE +++ b/tez-dist/dist-files/minimal/LICENSE @@ -261,7 +261,7 @@ The Apache TEZ binary distribution bundles the following files under the CDDL Li - jersey-client-*.jar - jersey-json-*.jar - - servlet-api-*.jar + - javax.servlet-api-*.jar The full text of the license is available in LICENSE-CDDLv1.1-GPLv2_withCPE. From 7d5a1830a51df64b23920901ec2e723999a9dfac Mon Sep 17 00:00:00 2001 From: Ying Han Date: Fri, 10 May 2019 04:55:29 -0500 Subject: [PATCH 166/512] TEZ-4062. Speculative attempt scheduling should be aborted when Task has completed Signed-off-by: Jonathan Eagles --- .../apache/tez/dag/app/dag/impl/TaskImpl.java | 5 +++++ .../tez/dag/app/dag/impl/TestTaskImpl.java | 17 +++++++++++++++++ 2 files changed, 22 insertions(+) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java index e563fe9cbf..2d0688f763 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java @@ -1043,6 +1043,11 @@ public void transition(TaskImpl task, TaskEvent event) { LOG.info("Ignore speculation scheduling since there is no running attempt on task {}.", task.getTaskId()); return; } + if (task.commitAttempt != null) { + LOG.info("Ignore speculation scheduling for task {} since commit has started with commitAttempt {}.", + task.getTaskId(), task.commitAttempt); + return; + } task.addAndScheduleAttempt(earliestUnfinishedAttempt.getID()); } } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java index 2d4adcc084..a3de9365de 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java @@ -999,6 +999,23 @@ public void testIgnoreSpeculationOnSuccessfulOriginalAttempt() { assertEquals(1, mockTask.getAttemptList().size()); } + @Test(timeout = 20000) + public void testIgnoreSpeculationAfterOriginalAttemptCommit() { + TezTaskID taskId = getNewTaskID(); + scheduleTaskAttempt(taskId); + MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); + launchTaskAttempt(firstAttempt.getID()); + updateAttemptState(firstAttempt, TaskAttemptState.RUNNING); + // Mock commit of the first task attempt + mockTask.canCommit(firstAttempt.getID()); + + // Verify the speculation scheduling is ignored and no speculative attempt was added to the task + mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getID())); + MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); + launchTaskAttempt(specAttempt.getID()); + assertEquals(1, mockTask.getAttemptList().size()); + } + @SuppressWarnings("rawtypes") @Test public void testSucceededAttemptStatusWithRetroActiveFailures() throws InterruptedException { From db5f462f9a6ac7faf5fde686bab401afb55ab66b Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Fri, 10 May 2019 04:58:13 -0500 Subject: [PATCH 167/512] Revert "TEZ-4062. Speculative attempt scheduling should be aborted when Task has completed" This reverts commit 7d5a1830a51df64b23920901ec2e723999a9dfac. --- .../apache/tez/dag/app/dag/impl/TaskImpl.java | 5 ----- .../tez/dag/app/dag/impl/TestTaskImpl.java | 17 ----------------- 2 files changed, 22 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java index 2d0688f763..e563fe9cbf 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java @@ -1043,11 +1043,6 @@ public void transition(TaskImpl task, TaskEvent event) { LOG.info("Ignore speculation scheduling since there is no running attempt on task {}.", task.getTaskId()); return; } - if (task.commitAttempt != null) { - LOG.info("Ignore speculation scheduling for task {} since commit has started with commitAttempt {}.", - task.getTaskId(), task.commitAttempt); - return; - } task.addAndScheduleAttempt(earliestUnfinishedAttempt.getID()); } } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java index a3de9365de..2d4adcc084 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java @@ -999,23 +999,6 @@ public void testIgnoreSpeculationOnSuccessfulOriginalAttempt() { assertEquals(1, mockTask.getAttemptList().size()); } - @Test(timeout = 20000) - public void testIgnoreSpeculationAfterOriginalAttemptCommit() { - TezTaskID taskId = getNewTaskID(); - scheduleTaskAttempt(taskId); - MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstAttempt.getID()); - updateAttemptState(firstAttempt, TaskAttemptState.RUNNING); - // Mock commit of the first task attempt - mockTask.canCommit(firstAttempt.getID()); - - // Verify the speculation scheduling is ignored and no speculative attempt was added to the task - mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getID())); - MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(specAttempt.getID()); - assertEquals(1, mockTask.getAttemptList().size()); - } - @SuppressWarnings("rawtypes") @Test public void testSucceededAttemptStatusWithRetroActiveFailures() throws InterruptedException { From 4125639619d5e3252fde62579d431ce5071414ea Mon Sep 17 00:00:00 2001 From: Ying Han Date: Fri, 10 May 2019 04:59:02 -0500 Subject: [PATCH 168/512] TEZ-4068. Prevent new speculative attempt after task has issued canCommit to an attempt Signed-off-by: Jonathan Eagles --- .../apache/tez/dag/app/dag/impl/TaskImpl.java | 5 +++++ .../tez/dag/app/dag/impl/TestTaskImpl.java | 17 +++++++++++++++++ 2 files changed, 22 insertions(+) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java index e563fe9cbf..2d0688f763 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java @@ -1043,6 +1043,11 @@ public void transition(TaskImpl task, TaskEvent event) { LOG.info("Ignore speculation scheduling since there is no running attempt on task {}.", task.getTaskId()); return; } + if (task.commitAttempt != null) { + LOG.info("Ignore speculation scheduling for task {} since commit has started with commitAttempt {}.", + task.getTaskId(), task.commitAttempt); + return; + } task.addAndScheduleAttempt(earliestUnfinishedAttempt.getID()); } } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java index 2d4adcc084..a3de9365de 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java @@ -999,6 +999,23 @@ public void testIgnoreSpeculationOnSuccessfulOriginalAttempt() { assertEquals(1, mockTask.getAttemptList().size()); } + @Test(timeout = 20000) + public void testIgnoreSpeculationAfterOriginalAttemptCommit() { + TezTaskID taskId = getNewTaskID(); + scheduleTaskAttempt(taskId); + MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); + launchTaskAttempt(firstAttempt.getID()); + updateAttemptState(firstAttempt, TaskAttemptState.RUNNING); + // Mock commit of the first task attempt + mockTask.canCommit(firstAttempt.getID()); + + // Verify the speculation scheduling is ignored and no speculative attempt was added to the task + mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getID())); + MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); + launchTaskAttempt(specAttempt.getID()); + assertEquals(1, mockTask.getAttemptList().size()); + } + @SuppressWarnings("rawtypes") @Test public void testSucceededAttemptStatusWithRetroActiveFailures() throws InterruptedException { From e62e85c0edd58e436653105dbcf26263f58e0071 Mon Sep 17 00:00:00 2001 From: Jesus Camacho Rodriguez Date: Tue, 16 Jul 2019 13:48:59 -0700 Subject: [PATCH 169/512] TEZ-4076. Add hadoop-cloud-storage jar to aws and azure mvn profiles (Jesus Camacho Rodriguez, reviewed by Gopal V) Signed-off-by: Gopal V --- pom.xml | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/pom.xml b/pom.xml index 89c081d3e6..883222ad9f 100644 --- a/pom.xml +++ b/pom.xml @@ -1334,6 +1334,12 @@ aws + + org.apache.hadoop + hadoop-cloud-storage + runtime + ${hadoop.version} + org.apache.hadoop hadoop-aws @@ -1348,6 +1354,12 @@ azure + + org.apache.hadoop + hadoop-cloud-storage + runtime + ${hadoop.version} + org.apache.hadoop hadoop-azure From 7a3e378b59dad2afe1a068669620846b87d6e732 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Tue, 27 Aug 2019 10:40:51 -0500 Subject: [PATCH 170/512] TEZ-4082. Reduce excessive getFileLinkInfo calls in Tez --- .../org/apache/tez/client/TezClientUtils.java | 57 +++++++------------ .../org/apache/tez/common/TezCommonUtils.java | 17 +----- .../apache/tez/common/TestTezCommonUtils.java | 36 ++++++++---- tez-ui/src/main/webapp/bower-shrinkwrap.json | 44 +++++++------- 4 files changed, 70 insertions(+), 84 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index cd3ae6bbf3..2b21024be1 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -19,6 +19,7 @@ package org.apache.tez.client; import java.io.File; +import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; import java.net.InetSocketAddress; @@ -121,31 +122,6 @@ public class TezClientUtils { private static Logger LOG = LoggerFactory.getLogger(TezClientUtils.class); private static final int UTF8_CHUNK_SIZE = 16 * 1024; - private static FileStatus[] getLRFileStatus(String fileName, Configuration conf) throws - IOException { - URI uri; - try { - uri = new URI(fileName); - } catch (URISyntaxException e) { - String message = "Invalid URI defined in configuration for" - + " location of TEZ jars. providedURI=" + fileName; - LOG.error(message); - throw new TezUncheckedException(message, e); - } - - Path p = new Path(uri); - FileSystem fs = p.getFileSystem(conf); - p = fs.resolvePath(p.makeQualified(fs.getUri(), - fs.getWorkingDirectory())); - FileSystem targetFS = p.getFileSystem(conf); - if (targetFS.isDirectory(p)) { - return targetFS.listStatus(p); - } else { - FileStatus fStatus = targetFS.getFileStatus(p); - return new FileStatus[]{fStatus}; - } - } - /** * Setup LocalResource map for Tez jars based on provided Configuration * @@ -217,8 +193,16 @@ private static boolean addLocalResources(Configuration conf, } Path p = new Path(u); FileSystem remoteFS = p.getFileSystem(conf); - p = remoteFS.resolvePath(p.makeQualified(remoteFS.getUri(), - remoteFS.getWorkingDirectory())); + FileStatus targetStatus = remoteFS.getFileLinkStatus(p); + p = targetStatus.getPath(); + + FileStatus[] fileStatuses; + FileSystem targetFS = p.getFileSystem(conf); + if (targetStatus.isDirectory()) { + fileStatuses = targetFS.listStatus(p); + } else { + fileStatuses = new FileStatus[]{targetStatus}; + } LocalResourceType type = null; @@ -232,8 +216,6 @@ private static boolean addLocalResources(Configuration conf, type = LocalResourceType.FILE; } - FileStatus [] fileStatuses = getLRFileStatus(configUri, conf); - for (FileStatus fStatus : fileStatuses) { String linkName; if (fStatus.isDirectory()) { @@ -329,13 +311,16 @@ public static FileSystem ensureStagingDirExists(Configuration conf, Path stagingArea) throws IOException { FileSystem fs = stagingArea.getFileSystem(conf); - String realUser; - String currentUser; UserGroupInformation ugi = UserGroupInformation.getLoginUser(); - realUser = ugi.getShortUserName(); - currentUser = UserGroupInformation.getCurrentUser().getShortUserName(); - if (fs.exists(stagingArea)) { - FileStatus fsStatus = fs.getFileStatus(stagingArea); + String realUser = ugi.getShortUserName(); + String currentUser = UserGroupInformation.getCurrentUser().getShortUserName(); + FileStatus fsStatus = null; + try { + fsStatus = fs.getFileStatus(stagingArea); + } catch (FileNotFoundException e) { + TezCommonUtils.mkDirForAM(fs, stagingArea); + } + if (fsStatus != null) { String owner = fsStatus.getOwner(); if (!(owner.equals(currentUser) || owner.equals(realUser))) { throw new IOException("The ownership on the staging directory " @@ -350,8 +335,6 @@ public static FileSystem ensureStagingDirExists(Configuration conf, + TezCommonUtils.TEZ_AM_DIR_PERMISSION); fs.setPermission(stagingArea, TezCommonUtils.TEZ_AM_DIR_PERMISSION); } - } else { - TezCommonUtils.mkDirForAM(fs, stagingArea); } return fs; } diff --git a/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java index 16165e2377..fc4789fc7d 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java @@ -79,21 +79,8 @@ public class TezCommonUtils { * @return Fully qualified staging directory */ public static Path getTezBaseStagingPath(Configuration conf) { - String stagingDirStr = conf.get(TezConfiguration.TEZ_AM_STAGING_DIR, - TezConfiguration.TEZ_AM_STAGING_DIR_DEFAULT); - Path baseStagingDir; - try { - Path p = new Path(stagingDirStr); - FileSystem fs = p.getFileSystem(conf); - if (!fs.exists(p)) { - mkDirForAM(fs, p); - LOG.info("Stage directory " + p + " doesn't exist and is created"); - } - baseStagingDir = fs.resolvePath(p); - } catch (IOException e) { - throw new TezUncheckedException(e); - } - return baseStagingDir; + return new Path(conf.get(TezConfiguration.TEZ_AM_STAGING_DIR, + TezConfiguration.TEZ_AM_STAGING_DIR_DEFAULT)); } /** diff --git a/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java b/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java index 3929c4bf5a..52df2c6d01 100644 --- a/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java +++ b/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java @@ -57,7 +57,7 @@ public static void setup() throws Exception { LOG.info("Starting mini clusters"); try { conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, TEST_ROOT_DIR); - dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).format(true).racks(null) + dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).format(true).racks(null) .build(); remoteFs = dfsCluster.getFileSystem(); RESOLVED_STAGE_DIR = remoteFs.getUri() + STAGE_DIR; @@ -80,19 +80,23 @@ public static void afterClass() throws InterruptedException { } // Testing base staging dir - @Test(timeout = 5000) + @Test public void testTezBaseStagingPath() throws Exception { Configuration localConf = new Configuration(); // Check if default works with localFS localConf.unset(TezConfiguration.TEZ_AM_STAGING_DIR); localConf.set("fs.defaultFS", "file:///"); Path stageDir = TezCommonUtils.getTezBaseStagingPath(localConf); - Assert.assertEquals(stageDir.toString(), "file:" + TezConfiguration.TEZ_AM_STAGING_DIR_DEFAULT); + TezCommonUtils.mkDirForAM(stageDir.getFileSystem(localConf), stageDir); + Path resolveStageDir = stageDir.getFileSystem(localConf).resolvePath(stageDir); + Assert.assertEquals(resolveStageDir.toString(), "file:" + TezConfiguration.TEZ_AM_STAGING_DIR_DEFAULT); // check if user set something, indeed works conf.set(TezConfiguration.TEZ_AM_STAGING_DIR, STAGE_DIR); stageDir = TezCommonUtils.getTezBaseStagingPath(conf); - Assert.assertEquals(stageDir.toString(), RESOLVED_STAGE_DIR); + TezCommonUtils.mkDirForAM(stageDir.getFileSystem(conf), stageDir); + resolveStageDir = stageDir.getFileSystem(conf).resolvePath(stageDir); + Assert.assertEquals(resolveStageDir.toString(), RESOLVED_STAGE_DIR); } // Testing System staging dir if createed @@ -111,7 +115,8 @@ public void testCreateTezSysStagingPath() throws Exception { } Assert.assertFalse(fs.exists(stagePath)); Path stageDir = TezCommonUtils.createTezSystemStagingPath(conf, strAppId); - Assert.assertEquals(stageDir.toString(), expectedStageDir); + Path resolveStageDir = stageDir.getFileSystem(conf).resolvePath(stageDir); + Assert.assertEquals(resolveStageDir.toString(), expectedStageDir); Assert.assertTrue(fs.exists(stagePath)); } @@ -122,7 +127,9 @@ public void testTezSysStagingPath() throws Exception { Path stageDir = TezCommonUtils.getTezSystemStagingPath(conf, strAppId); String expectedStageDir = RESOLVED_STAGE_DIR + Path.SEPARATOR + TezCommonUtils.TEZ_SYSTEM_SUB_DIR + Path.SEPARATOR + strAppId; - Assert.assertEquals(stageDir.toString(), expectedStageDir); + TezCommonUtils.mkDirForAM(stageDir.getFileSystem(conf), stageDir); + Path resolvedStageDir = stageDir.getFileSystem(conf).resolvePath(stageDir); + Assert.assertEquals(resolvedStageDir.toString(), expectedStageDir); } // Testing conf staging dir @@ -134,7 +141,9 @@ public void testTezConfStagingPath() throws Exception { String expectedDir = RESOLVED_STAGE_DIR + Path.SEPARATOR + TezCommonUtils.TEZ_SYSTEM_SUB_DIR + Path.SEPARATOR + strAppId + Path.SEPARATOR + TezConstants.TEZ_PB_BINARY_CONF_NAME; - Assert.assertEquals(confStageDir.toString(), expectedDir); + TezCommonUtils.mkDirForAM(confStageDir.getFileSystem(conf), confStageDir); + Path resolvedConfStageDir = confStageDir.getFileSystem(conf).resolvePath(confStageDir); + Assert.assertEquals(resolvedConfStageDir.toString(), expectedDir); } // Testing session jars staging dir @@ -146,7 +155,9 @@ public void testTezSessionJarStagingPath() throws Exception { String expectedDir = RESOLVED_STAGE_DIR + Path.SEPARATOR + TezCommonUtils.TEZ_SYSTEM_SUB_DIR + Path.SEPARATOR + strAppId + Path.SEPARATOR + TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME; - Assert.assertEquals(confStageDir.toString(), expectedDir); + TezCommonUtils.mkDirForAM(confStageDir.getFileSystem(conf), confStageDir); + Path resolvedConfStageDir = confStageDir.getFileSystem(conf).resolvePath(confStageDir); + Assert.assertEquals(resolvedConfStageDir.toString(), expectedDir); } // Testing bin plan staging dir @@ -158,7 +169,9 @@ public void testTezBinPlanStagingPath() throws Exception { String expectedDir = RESOLVED_STAGE_DIR + Path.SEPARATOR + TezCommonUtils.TEZ_SYSTEM_SUB_DIR + Path.SEPARATOR + strAppId + Path.SEPARATOR + TezConstants.TEZ_PB_PLAN_BINARY_NAME; - Assert.assertEquals(confStageDir.toString(), expectedDir); + TezCommonUtils.mkDirForAM(confStageDir.getFileSystem(conf), confStageDir); + Path resolvedConfStageDir = confStageDir.getFileSystem(conf).resolvePath(confStageDir); + Assert.assertEquals(resolvedConfStageDir.toString(), expectedDir); } // Testing text plan staging dir @@ -167,12 +180,15 @@ public void testTezTextPlanStagingPath() throws Exception { String strAppId = "testAppId"; String dagPBName = "testDagPBName"; Path tezSysStagingPath = TezCommonUtils.getTezSystemStagingPath(conf, strAppId); + TezCommonUtils.mkDirForAM(tezSysStagingPath.getFileSystem(conf), tezSysStagingPath); Path confStageDir = TezCommonUtils.getTezTextPlanStagingPath(tezSysStagingPath, strAppId, dagPBName); String expectedDir = RESOLVED_STAGE_DIR + Path.SEPARATOR + TezCommonUtils.TEZ_SYSTEM_SUB_DIR + Path.SEPARATOR + strAppId + Path.SEPARATOR + strAppId + "-" + dagPBName + "-" + TezConstants.TEZ_PB_PLAN_TEXT_NAME; - Assert.assertEquals(confStageDir.toString(), expectedDir); + TezCommonUtils.createFileForAM(confStageDir.getFileSystem(conf), confStageDir); + Path resolvedConfStageDir = confStageDir.getFileSystem(conf).resolvePath(confStageDir); + Assert.assertEquals(resolvedConfStageDir.toString(), expectedDir); } // Testing recovery path staging dir diff --git a/tez-ui/src/main/webapp/bower-shrinkwrap.json b/tez-ui/src/main/webapp/bower-shrinkwrap.json index 357d57691a..edcbae33c1 100644 --- a/tez-ui/src/main/webapp/bower-shrinkwrap.json +++ b/tez-ui/src/main/webapp/bower-shrinkwrap.json @@ -1,72 +1,72 @@ { "https://github.com/FortAwesome/Font-Awesome.git": { - "4.5.0": "593ad563a987977f14102be935d0abc2a172903e" + "4.5.0": "4.5.0" }, "https://github.com/Teleborder/FileSaver.js.git": { - "1.20150507.2": "b7cf622909258086bc63ad764d08fcaed780ab42" + "1.20150507.2": "1.20150507.2" }, "https://github.com/adamwdraper/Numeral-js.git": { - "1.5.3": "f97f14bb8bab988f28f1d854525b4cfeff8ec9e1" + "1.5.3": "1.5.3" }, "https://github.com/components/codemirror.git": { - "5.11.0": "7d43f32bb56f83a9c47addb3f91170b3102f3ead" + "5.11.0": "5.11.0" }, "https://github.com/components/ember-data.git": { - "2.1.0": "d8b4d3092f67afe22d9d374c40d719d557915fa3" + "2.1.0": "2.1.0" }, "https://github.com/components/ember.git": { - "2.2.0": "49e042ca89922ed96b27488c2a98add280ae7123" + "2.2.0": "2.2.0" }, "https://github.com/components/jqueryui.git": { - "1.11.4": "c34f8dbf3ba57b3784b93f26119f436c0e8288e1" + "1.11.4": "1.11.4" }, "https://github.com/dockyard/ember-qunit-notifications.git": { - "0.1.0": "a83277aa7a1c0545c66e6d133caebb9a620e71ad" + "0.1.0": "0.1.0" }, "https://github.com/dockyard/qunit-notifications.git": { - "0.1.1": "7a13f6dba5a340e1cb9e0b64c1c711e4d7edaca1" + "0.1.1": "0.1.1" }, "https://github.com/ember-cli/ember-cli-shims.git": { - "0.0.6": "dcab43b58d5698690050bb9a46ead5c8663c7da1" + "0.0.6": "0.0.6" }, "https://github.com/ember-cli/ember-cli-test-loader.git": { - "0.2.1": "3348d801089279296c38f31ae14d9c4d115ce154" + "0.2.1": "0.2.1" }, "https://github.com/ember-cli/ember-load-initializers.git": { - "0.1.7": "7bb21488563bd1bba23e903a812bf5815beddd1a" + "0.1.7": "0.1.7" }, "https://github.com/jquery/jquery-dist.git": { - "2.1.4": "7751e69b615c6eca6f783a81e292a55725af6b85" + "2.1.4": "2.1.4" }, "https://github.com/jquery/jquery-mousewheel.git": { - "3.1.13": "67289b6b2aa0066d7d78a5807f520387135ffb22" + "3.1.13": "3.1.13" }, "https://github.com/jquery/qunit.git": { - "1.19.0": "467e7e34652ad7d5883ce9c568461cf8c5e172a8" + "1.19.0": "1.19.0" }, "https://github.com/moment/moment-timezone.git": { - "0.5.0": "74a2e9378ecf4a31a168f3049f086565c8d66814" + "0.5.0": "0.5.0" }, "https://github.com/moment/moment.git": { - "2.12.0": "d3d7488b4d60632854181cb0a9af325d57fb3d51" + "2.12.0": "2.12.0" }, "https://github.com/rwjblue/ember-qunit-builds.git": { - "0.4.16": "142c4066a5458bef9dfcb92b70152b9c01d79188" + "0.4.16": "0.4.16" }, "https://github.com/sreenaths/more-js.git": { "0.8.6": "f1d9ccdaf7ff74c26b6ee341067a5a5ef33bd64a", "0.8.8": "0.8.8" }, "https://github.com/sreenaths/snippet-ss.git": { - "1.11.0": "c1abc566f4e001b7f1939b6dbdd911eadc969cf9" + "1.11.0": "1.11.0" }, "https://github.com/sreenaths/zip.js.git": { - "1.0.0": "ec67ad22eba116083ea3ef2fe0b40ccc953513ce" + "1.0.0": "1.0.0" }, "https://github.com/stefanpenner/loader.js.git": { - "3.3.0": "ac909550c9544325632542bbea97531cc60bc628" + "3.3.0": "3.3.0" }, "https://github.com/twbs/bootstrap.git": { - "3.3.6": "81df608a40bf0629a1dc08e584849bb1e43e0b7a" + "3.3.6": "3.3.6" } } \ No newline at end of file From 9f5c1b7f6d09884131aa2bab4e01e98768347def Mon Sep 17 00:00:00 2001 From: Siddharth Seth Date: Sun, 6 Oct 2019 10:15:07 -0700 Subject: [PATCH 171/512] TEZ-4086. Allow various examples to work when outputPath is on a FileSystem other than the default FileSystem. (#45) --- .../java/org/apache/tez/examples/HashJoinExample.java | 3 ++- .../main/java/org/apache/tez/examples/JoinDataGen.java | 10 ++++++---- .../org/apache/tez/examples/SortMergeJoinExample.java | 3 ++- .../tez/mapreduce/examples/FilterLinesByWord.java | 6 +++++- .../tez/mapreduce/examples/TestOrderedWordCount.java | 8 ++++---- .../apache/tez/mapreduce/examples/UnionExample.java | 5 ++++- .../src/test/java/org/apache/tez/test/TestTezJobs.java | 7 +++---- 7 files changed, 26 insertions(+), 16 deletions(-) diff --git a/tez-examples/src/main/java/org/apache/tez/examples/HashJoinExample.java b/tez-examples/src/main/java/org/apache/tez/examples/HashJoinExample.java index 935ccbca10..e762ac16dc 100644 --- a/tez-examples/src/main/java/org/apache/tez/examples/HashJoinExample.java +++ b/tez-examples/src/main/java/org/apache/tez/examples/HashJoinExample.java @@ -113,7 +113,8 @@ protected int runJob(String[] args, TezConfiguration tezConf, Path outputPath = new Path(outputDir); // Verify output path existence - FileSystem fs = FileSystem.get(tezConf); + FileSystem fs = outputPath.getFileSystem(tezConf); + outputPath = fs.makeQualified(outputPath); if (fs.exists(outputPath)) { System.err.println("Output directory: " + outputDir + " already exists"); return 3; diff --git a/tez-examples/src/main/java/org/apache/tez/examples/JoinDataGen.java b/tez-examples/src/main/java/org/apache/tez/examples/JoinDataGen.java index 4c0d2018af..d5b784b18e 100644 --- a/tez-examples/src/main/java/org/apache/tez/examples/JoinDataGen.java +++ b/tez-examples/src/main/java/org/apache/tez/examples/JoinDataGen.java @@ -102,10 +102,10 @@ protected int runJob(String[] args, TezConfiguration tezConf, Path expectedOutputPath = new Path(expectedOutputDir); // Verify output path existence - FileSystem fs = FileSystem.get(tezConf); int res = 0; - res = checkOutputDirectory(fs, largeOutPath) + checkOutputDirectory(fs, smallOutPath) - + checkOutputDirectory(fs, expectedOutputPath); + res = checkOutputDirectory(tezConf, largeOutPath) + + checkOutputDirectory(tezConf, smallOutPath) + + checkOutputDirectory(tezConf, expectedOutputPath); if (res != 0) { return 3; } @@ -279,7 +279,9 @@ private String createNonOverlaptring(int size, long count) { } - private int checkOutputDirectory(FileSystem fs, Path path) throws IOException { + private int checkOutputDirectory(Configuration conf, Path path) throws IOException { + FileSystem fs = path.getFileSystem(conf); + path = fs.makeQualified(path); if (fs.exists(path)) { System.err.println("Output directory: " + path + " already exists"); return 2; diff --git a/tez-examples/src/main/java/org/apache/tez/examples/SortMergeJoinExample.java b/tez-examples/src/main/java/org/apache/tez/examples/SortMergeJoinExample.java index 1054e007d1..820aaa5cba 100644 --- a/tez-examples/src/main/java/org/apache/tez/examples/SortMergeJoinExample.java +++ b/tez-examples/src/main/java/org/apache/tez/examples/SortMergeJoinExample.java @@ -105,7 +105,8 @@ protected int runJob(String[] args, TezConfiguration tezConf, Path outputPath = new Path(outputDir); // Verify output path existence - FileSystem fs = FileSystem.get(tezConf); + FileSystem fs = outputPath.getFileSystem(tezConf); + outputPath = fs.makeQualified(outputPath); if (fs.exists(outputPath)) { System.err.println("Output directory: " + outputDir + " already exists"); return 3; diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java index 36215a4b43..da4d9ea76c 100644 --- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java +++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java @@ -124,7 +124,11 @@ public int run(String[] args) throws Exception { String filterWord = otherArgs[2]; FileSystem fs = FileSystem.get(conf); - if (fs.exists(new Path(outputPath))) { + + Path outputPathAsPath = new Path(outputPath); + FileSystem outputFs = outputPathAsPath.getFileSystem(conf); + outputPathAsPath = outputFs.makeQualified(outputPathAsPath); + if (outputFs.exists(outputPathAsPath)) { System.err.println("Output directory : " + outputPath + " already exists"); return 2; } diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/TestOrderedWordCount.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/TestOrderedWordCount.java index 51e4be18ae..1b87e11713 100644 --- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/TestOrderedWordCount.java +++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/TestOrderedWordCount.java @@ -18,7 +18,6 @@ package org.apache.tez.mapreduce.examples; -import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.ArrayList; import java.util.EnumSet; @@ -441,8 +440,6 @@ public int run(String[] args) throws Exception { HadoopShim hadoopShim = new HadoopShimsLoader(tezConf).getHadoopShim(); TestOrderedWordCount instance = new TestOrderedWordCount(); - FileSystem fs = FileSystem.get(conf); - String stagingDirStr = conf.get(TezConfiguration.TEZ_AM_STAGING_DIR, TezConfiguration.TEZ_AM_STAGING_DIR_DEFAULT) + Path.SEPARATOR + Long.toString(System.currentTimeMillis()); @@ -498,7 +495,10 @@ public int run(String[] args) throws Exception { String inputPath = inputPaths.get(dagIndex-1); String outputPath = outputPaths.get(dagIndex-1); - if (fs.exists(new Path(outputPath))) { + Path outputPathAsPath = new Path(outputPath); + FileSystem fs = outputPathAsPath.getFileSystem(conf); + outputPathAsPath = fs.makeQualified(outputPathAsPath); + if (fs.exists(outputPathAsPath)) { throw new FileAlreadyExistsException("Output directory " + outputPath + " already exists"); } diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/UnionExample.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/UnionExample.java index 7688335d37..f78d162d31 100644 --- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/UnionExample.java +++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/UnionExample.java @@ -262,7 +262,10 @@ public boolean run(String inputPath, String outputPath, Configuration conf) thro DAGClient dagClient = null; try { - if (fs.exists(new Path(outputPath))) { + Path outputPathAsPath = new Path(outputPath); + FileSystem outputFs = outputPathAsPath.getFileSystem(tezConf); + outputPathAsPath = outputFs.makeQualified(outputPathAsPath); + if (outputFs.exists(outputPathAsPath)) { throw new FileAlreadyExistsException("Output directory " + outputPath + " already exists"); } diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java b/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java index 2dfc76ddff..2a94d9bf58 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java @@ -138,7 +138,6 @@ public static void setup() throws IOException { if (mrrTezCluster == null) { mrrTezCluster = new MiniTezCluster(TestTezJobs.class.getName(), 1, 1, 1); - Configuration conf = new Configuration(); conf.set("fs.defaultFS", remoteFs.getUri().toString()); // use HDFS conf.setLong(TezConfiguration.TEZ_AM_SLEEP_TIME_BEFORE_EXIT_MILLIS, 500); mrrTezCluster.init(conf); @@ -163,7 +162,7 @@ public static void tearDown() { @Test(timeout = 60000) public void testHashJoinExample() throws Exception { HashJoinExample hashJoinExample = new HashJoinExample(); - hashJoinExample.setConf(mrrTezCluster.getConfig()); + hashJoinExample.setConf(new Configuration(mrrTezCluster.getConfig())); Path stagingDirPath = new Path("/tmp/tez-staging-dir"); Path inPath1 = new Path("/tmp/hashJoin/inPath1"); Path inPath2 = new Path("/tmp/hashJoin/inPath2"); @@ -219,7 +218,7 @@ public boolean accept(Path p) { @Test(timeout = 60000) public void testHashJoinExampleDisableSplitGrouping() throws Exception { HashJoinExample hashJoinExample = new HashJoinExample(); - hashJoinExample.setConf(conf); + hashJoinExample.setConf(new Configuration(mrrTezCluster.getConfig())); Path stagingDirPath = new Path(TEST_ROOT_DIR + "/tmp/tez-staging-dir"); Path inPath1 = new Path(TEST_ROOT_DIR + "/tmp/hashJoin/inPath1"); Path inPath2 = new Path(TEST_ROOT_DIR + "/tmp/hashJoin/inPath2"); @@ -431,7 +430,7 @@ public void testPerIOCounterAggregation() throws Exception { @Test(timeout = 60000) public void testSortMergeJoinExampleDisableSplitGrouping() throws Exception { SortMergeJoinExample sortMergeJoinExample = new SortMergeJoinExample(); - sortMergeJoinExample.setConf(conf); + sortMergeJoinExample.setConf(new Configuration(mrrTezCluster.getConfig())); Path stagingDirPath = new Path(TEST_ROOT_DIR + "/tmp/tez-staging-dir"); Path inPath1 = new Path(TEST_ROOT_DIR + "/tmp/sortMerge/inPath1"); Path inPath2 = new Path(TEST_ROOT_DIR + "/tmp/sortMerge/inPath2"); From de019d5e31ba72659dd88a3d8fe5db8b4af0a3cd Mon Sep 17 00:00:00 2001 From: rbalamohan Date: Mon, 7 Oct 2019 12:27:38 +0530 Subject: [PATCH 172/512] TEZ-4075: Reimplement tez.runtime.transfer.data-via-events.enabled (#48) (Contributed by Richard Zhang) * TEZ-4075: Reimplement tez.runtime.transfer.data-via-events.enabled * TEZ-4075: Reimplement tez.runtime.transfer.data-via-events.enabled --- .../tez/common/counters/TaskCounter.java | 10 ++- .../library/api/TezRuntimeConfiguration.java | 14 ++++ .../library/common/shuffle/ShuffleUtils.java | 3 +- .../impl/ShuffleInputEventHandlerImpl.java | 49 ++++++++++++- .../common/shuffle/impl/ShuffleManager.java | 46 ++++++++++++ .../BaseUnorderedPartitionedKVWriter.java | 6 ++ .../writers/UnorderedPartitionedKVWriter.java | 63 ++++++++++++++--- .../library/output/UnorderedKVOutput.java | 2 + .../output/UnorderedPartitionedKVOutput.java | 2 + .../src/main/proto/ShufflePayloads.proto | 2 + .../TestUnorderedPartitionedKVWriter.java | 70 ++++++++++++------- ...estUnorderedPartitionedKVOutputConfig.java | 8 +++ .../java/org/apache/tez/test/TestTezJobs.java | 55 +++++++++++++++ 13 files changed, 293 insertions(+), 37 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java index 80424c7708..e604f37e63 100644 --- a/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java +++ b/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java @@ -220,5 +220,13 @@ public enum TaskCounter { * * Represented in milliseconds */ - LAST_EVENT_RECEIVED + LAST_EVENT_RECEIVED, + + + /** + * The size of the data that is transmitted via event. + * + * Represented in number of bytes + */ + DATA_BYTES_VIA_EVENT } \ No newline at end of file diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java index 86792e2fc3..d4532c99cc 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java @@ -484,6 +484,18 @@ public class TezRuntimeConfiguration { "empty.partitions.info-via-events.enabled"; public static final boolean TEZ_RUNTIME_EMPTY_PARTITION_INFO_VIA_EVENTS_ENABLED_DEFAULT = true; + @Private + public static final String TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_ENABLED = + TEZ_RUNTIME_PREFIX + "transfer.data-via-events.enabled"; + @Private + public static final boolean TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_ENABLED_DEFAULT = false; + + @Private + public static final String TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_MAX_SIZE = + TEZ_RUNTIME_PREFIX + "transfer.data-via-events.max-size"; + @Private + public static final int TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_MAX_SIZE_DEFAULT = 512; + /** * If the shuffle input is on the local host bypass the http fetch and access the files directly */ @@ -619,6 +631,8 @@ public class TezRuntimeConfiguration { tezRuntimeKeys.add(TEZ_RUNTIME_EMPTY_PARTITION_INFO_VIA_EVENTS_ENABLED); tezRuntimeKeys.add(TEZ_RUNTIME_PIPELINED_SHUFFLE_ENABLED); tezRuntimeKeys.add(TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT); + tezRuntimeKeys.add(TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_ENABLED); + tezRuntimeKeys.add(TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_MAX_SIZE); tezRuntimeKeys.add(TEZ_RUNTIME_RECORDS_BEFORE_PROGRESS); tezRuntimeKeys.add(TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH); tezRuntimeKeys.add(TEZ_RUNTIME_OPTIMIZE_SHARED_FETCH); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java index df4281a94e..00f37454fd 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java @@ -263,7 +263,8 @@ public static String stringify(DataMovementEventPayloadProto dmProto) { sb.append("host: " + dmProto.getHost()).append(", "); sb.append("port: " + dmProto.getPort()).append(", "); sb.append("pathComponent: " + dmProto.getPathComponent()).append(", "); - sb.append("runDuration: " + dmProto.getRunDuration()); + sb.append("runDuration: " + dmProto.getRunDuration()).append(", "); + sb.append("hasDataInEvent: " + dmProto.hasData()); sb.append("]"); return sb.toString(); } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java index 542ec34338..7ad1389170 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java @@ -43,7 +43,11 @@ import org.apache.tez.runtime.library.common.shuffle.FetchedInputAllocator; import org.apache.tez.runtime.library.common.shuffle.ShuffleEventHandler; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; +import org.apache.tez.runtime.library.common.shuffle.FetchedInput; +import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataProto; import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto; +import org.apache.tez.runtime.library.common.shuffle.DiskFetchedInput; +import org.apache.tez.runtime.library.common.shuffle.MemoryFetchedInput; import com.google.protobuf.InvalidProtocolBufferException; @@ -166,6 +170,9 @@ public void logProgress(boolean updateOnClose) { private void processDataMovementEvent(DataMovementEvent dme, DataMovementEventPayloadProto shufflePayload, BitSet emptyPartitionsBitSet) throws IOException { int srcIndex = dme.getSourceIndex(); + + String hostIdentifier = shufflePayload.getHost() + ":" + shufflePayload.getPort(); + if (LOG.isDebugEnabled()) { LOG.debug("DME srcIdx: " + srcIndex + ", targetIndex: " + dme.getTargetIndex() + ", attemptNum: " + dme.getVersion() + ", payload: " + ShuffleUtils @@ -189,7 +196,47 @@ private void processDataMovementEvent(DataMovementEvent dme, DataMovementEventPa CompositeInputAttemptIdentifier srcAttemptIdentifier = constructInputAttemptIdentifier(dme.getTargetIndex(), 1, dme.getVersion(), shufflePayload, (useSharedInputs && srcIndex == 0)); - shuffleManager.addKnownInput(shufflePayload.getHost(), shufflePayload.getPort(), srcAttemptIdentifier, srcIndex); + if (shufflePayload.hasData()) { + DataProto dataProto = shufflePayload.getData(); + + FetchedInput fetchedInput = + inputAllocator.allocate(dataProto.getRawLength(), + dataProto.getCompressedLength(), srcAttemptIdentifier); + moveDataToFetchedInput(dataProto, fetchedInput, hostIdentifier); + shuffleManager.addCompletedInputWithData(srcAttemptIdentifier, fetchedInput); + + LOG.debug("Payload via DME : " + srcAttemptIdentifier); + } else { + shuffleManager.addKnownInput(shufflePayload.getHost(), shufflePayload.getPort(), + srcAttemptIdentifier, srcIndex); + } + } + + private void moveDataToFetchedInput(DataProto dataProto, + FetchedInput fetchedInput, String hostIdentifier) throws IOException { + switch (fetchedInput.getType()) { + case DISK: + ShuffleUtils + .shuffleToDisk(((DiskFetchedInput) fetchedInput).getOutputStream(), + hostIdentifier, dataProto.getData().newInput(), + dataProto.getCompressedLength(), + dataProto.getUncompressedLength(), LOG, + fetchedInput.getInputAttemptIdentifier(), ifileReadAhead, + ifileReadAheadLength, true); + break; + case MEMORY: + ShuffleUtils + .shuffleToMemory(((MemoryFetchedInput) fetchedInput).getBytes(), + dataProto.getData().newInput(), dataProto.getRawLength(), + dataProto.getCompressedLength(), + codec, ifileReadAhead, ifileReadAheadLength, LOG, + fetchedInput.getInputAttemptIdentifier()); + break; + case WAIT: + default: + throw new TezUncheckedException("Unexpected type: " + + fetchedInput.getType()); + } } private void processCompositeRoutedDataMovementEvent(CompositeRoutedDataMovementEvent crdme, DataMovementEventPayloadProto shufflePayload, BitSet emptyPartitionsBitSet) throws IOException { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java index ba8592f8ae..5e03f08e23 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java @@ -665,6 +665,52 @@ public void addCompletedInputWithNoData( } } + public void addCompletedInputWithData( + InputAttemptIdentifier srcAttemptIdentifier, FetchedInput fetchedInput) + throws IOException { + //InputIdentifier inputIdentifier = srcAttemptIdentifier.getInputIdentifier(); + int inputIdentifier = srcAttemptIdentifier.getInputIdentifier(); + if (LOG.isDebugEnabled()) { + LOG.debug("Received Data via Event: " + srcAttemptIdentifier + " to " + + fetchedInput.getType()); + } + // Count irrespective of whether this is a copy of an already fetched input + lock.lock(); + try { + lastProgressTime = System.currentTimeMillis(); + } finally { + lock.unlock(); + } + + boolean committed = false; + if (!completedInputSet.get(inputIdentifier)) { + synchronized (completedInputSet) { + if (!completedInputSet.get(inputIdentifier)) { + fetchedInput.commit(); + committed = true; + if (!srcAttemptIdentifier.canRetrieveInputInChunks()) { + registerCompletedInput(fetchedInput); + } else { + registerCompletedInputForPipelinedShuffle(srcAttemptIdentifier, + fetchedInput); + } + } + } + } + if (!committed) { + fetchedInput.abort(); // If this fails, the fetcher may attempt another + // abort. + } else { + lock.lock(); + try { + // Signal the wakeLoop to check for termination. + wakeLoop.signal(); + } finally { + lock.unlock(); + } + } + } + protected synchronized void updateEventReceivedTime() { long relativeTime = System.currentTimeMillis() - startTime; if (firstEventReceived.getValue() == 0) { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/BaseUnorderedPartitionedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/BaseUnorderedPartitionedKVWriter.java index 30d1adb9fe..3467c82bb3 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/BaseUnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/BaseUnorderedPartitionedKVWriter.java @@ -101,6 +101,11 @@ public abstract class BaseUnorderedPartitionedKVWriter extends KeyValuesWriter { */ protected final TezCounter numAdditionalSpillsCounter; + /** + * Represents the number of bytes that is transmitted via the event. + */ + protected final TezCounter dataViaEventSize; + @SuppressWarnings("unchecked") public BaseUnorderedPartitionedKVWriter(OutputContext outputContext, Configuration conf, int numOutputs) { this.outputContext = outputContext; @@ -122,6 +127,7 @@ public BaseUnorderedPartitionedKVWriter(OutputContext outputContext, Configurati additionalSpillBytesWritternCounter = outputContext.getCounters().findCounter(TaskCounter.ADDITIONAL_SPILLS_BYTES_WRITTEN); additionalSpillBytesReadCounter = outputContext.getCounters().findCounter(TaskCounter.ADDITIONAL_SPILLS_BYTES_READ); numAdditionalSpillsCounter = outputContext.getCounters().findCounter(TaskCounter.ADDITIONAL_SPILL_COUNT); + dataViaEventSize = outputContext.getCounters().findCounter(TaskCounter.DATA_BYTES_VIA_EVENT); // compression if (ConfigUtils.shouldCompressIntermediateOutput(this.conf)) { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java index 29478dcda1..7165205e52 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java @@ -49,6 +49,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.io.DataInputBuffer; +import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.tez.common.CallableWithNdc; import org.apache.tez.common.TezCommonUtils; @@ -70,6 +71,7 @@ import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer; import org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; +import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads; import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -128,6 +130,8 @@ public class UnorderedPartitionedKVWriter extends BaseUnorderedPartitionedKVWrit // uncompressed size for each partition private final long[] sizePerPartition; private volatile long spilledSize = 0; + private boolean dataViaEventsEnabled; + private int dataViaEventsMaxSize; static final ThreadLocal deflater = new ThreadLocal() { @@ -210,6 +214,15 @@ public UnorderedPartitionedKVWriter(OutputContext outputContext, Configuration c this.pipelinedShuffle = pipelinedShuffleConf && !isFinalMergeEnabled; this.finalEvents = Lists.newLinkedList(); + this.dataViaEventsEnabled = conf.getBoolean( + TezRuntimeConfiguration.TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_ENABLED, + TezRuntimeConfiguration.TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_ENABLED_DEFAULT); + + // No max cap on size (intentional) + this.dataViaEventsMaxSize = conf.getInt( + TezRuntimeConfiguration.TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_MAX_SIZE, + TezRuntimeConfiguration.TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_MAX_SIZE_DEFAULT); + if (availableMemoryBytes == 0) { Preconditions.checkArgument(((numPartitions == 1) && !pipelinedShuffle), "availableMemory " + "can be set to 0 only when numPartitions=1 and " + TezRuntimeConfiguration @@ -270,14 +283,11 @@ public UnorderedPartitionedKVWriter(OutputContext outputContext, Configuration c outputLargeRecordsCounter = outputContext.getCounters().findCounter( TaskCounter.OUTPUT_LARGE_RECORDS); - - indexFileSizeEstimate = numPartitions * Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH; if (numPartitions == 1 && !pipelinedShuffle) { //special case, where in only one partition is available. finalOutPath = outputFileHandler.getOutputFileForWrite(); - finalIndexPath = outputFileHandler.getOutputIndexFileForWrite(indexFileSizeEstimate); skipBuffers = true; writer = new IFile.Writer(conf, rfs, finalOutPath, keyClass, valClass, codec, outputRecordsCounter, outputRecordBytesCounter); @@ -298,7 +308,10 @@ public UnorderedPartitionedKVWriter(OutputContext outputContext, Configuration c + ", pipelinedShuffle=" + pipelinedShuffle + ", isFinalMergeEnabled=" + isFinalMergeEnabled + ", numPartitions=" + numPartitions - + ", reportPartitionStats=" + reportPartitionStats); + + ", reportPartitionStats=" + reportPartitionStats + + ", dataViaEventsEnabled=" + dataViaEventsEnabled + + ", dataViaEventsMaxSize=" + dataViaEventsMaxSize + ); } private static final int ALLOC_OVERHEAD = 64; @@ -684,6 +697,21 @@ public static long getInitialMemoryRequirement(Configuration conf, long maxAvail return reqBytes; } + private boolean canSendDataOverDME() { + return (writer != null) && (dataViaEventsEnabled) + && (writer.getCompressedLength() <= dataViaEventsMaxSize); + } + + private byte[] readDataForDME() throws IOException { + // TODO: Not introducing a caching layer in IFile yet. + byte[] buf = null; + try (FSDataInputStream inStream = rfs.open(finalOutPath)) { + buf = new byte[(int) writer.getCompressedLength()]; + IOUtils.readFully(inStream, buf, 0, (int) writer.getCompressedLength()); + } + return buf; + } + @Override public List close() throws IOException, InterruptedException { // In case there are buffers to be spilled, schedule spilling @@ -721,10 +749,6 @@ public List close() throws IOException, InterruptedException { writer.close(); long rawLen = writer.getRawLength(); long compLen = writer.getCompressedLength(); - TezIndexRecord rec = new TezIndexRecord(0, rawLen, compLen); - TezSpillRecord sr = new TezSpillRecord(1); - sr.putIndex(rec, 0); - sr.writeToFile(finalIndexPath, conf); BitSet emptyPartitions = new BitSet(); if (outputRecordsCounter.getValue() == 0) { @@ -742,8 +766,17 @@ public List close() throws IOException, InterruptedException { fileOutputBytesCounter.increment(compLen + indexFileSizeEstimate); } eventList.add(generateVMEvent()); + + if (!canSendDataOverDME()) { + TezIndexRecord rec = new TezIndexRecord(0, rawLen, compLen); + TezSpillRecord sr = new TezSpillRecord(1); + sr.putIndex(rec, 0); + finalIndexPath = outputFileHandler.getOutputIndexFileForWrite(indexFileSizeEstimate); + sr.writeToFile(finalIndexPath, conf); + } eventList.add(generateDMEvent(false, -1, false, outputContext - .getUniqueIdentifier(), emptyPartitions)); + .getUniqueIdentifier(), emptyPartitions)); + return eventList; } @@ -856,6 +889,18 @@ private Event generateDMEvent(boolean addSpillDetails, int spillId, payloadBuilder.setLastEvent(isLastSpill); } + if (canSendDataOverDME()) { + ShuffleUserPayloads.DataProto.Builder dataProtoBuilder = ShuffleUserPayloads.DataProto.newBuilder(); + dataProtoBuilder.setData(ByteString.copyFrom(readDataForDME())); + dataProtoBuilder.setRawLength((int) this.writer.getRawLength()); + + dataProtoBuilder.setCompressedLength((int) this.writer.getCompressedLength()); + payloadBuilder.setData(dataProtoBuilder.build()); + + this.dataViaEventSize.increment(this.writer.getCompressedLength()); + LOG.debug("payload packed in DME, dataSize: " + this.writer.getCompressedLength()); + } + ByteBuffer payload = payloadBuilder.build().toByteString().asReadOnlyByteBuffer(); return CompositeDataMovementEvent.create(0, numPartitions, payload); } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java index c987024086..50e1e8b836 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java @@ -167,6 +167,8 @@ String getHost() { confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_COMPRESS); confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_COMPRESS_CODEC); confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_EMPTY_PARTITION_INFO_VIA_EVENTS_ENABLED); + confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_ENABLED); + confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_MAX_SIZE); confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_CONVERT_USER_PAYLOAD_TO_HISTORY_TEXT); confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_PIPELINED_SHUFFLE_ENABLED); confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java index 94312f7530..5c56083b82 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java @@ -140,6 +140,8 @@ public synchronized List close() throws Exception { confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_COMPRESS); confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_COMPRESS_CODEC); confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_EMPTY_PARTITION_INFO_VIA_EVENTS_ENABLED); + confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_ENABLED); + confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_MAX_SIZE); confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_CONVERT_USER_PAYLOAD_TO_HISTORY_TEXT); confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_PIPELINED_SHUFFLE_ENABLED); confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT); diff --git a/tez-runtime-library/src/main/proto/ShufflePayloads.proto b/tez-runtime-library/src/main/proto/ShufflePayloads.proto index 0a4f4a6147..0d13710d73 100644 --- a/tez-runtime-library/src/main/proto/ShufflePayloads.proto +++ b/tez-runtime-library/src/main/proto/ShufflePayloads.proto @@ -26,6 +26,7 @@ message DataMovementEventPayloadProto { optional int32 port = 3; optional string path_component = 4; optional int32 run_duration = 5; + optional DataProto data = 6; optional bool pipelined = 7; // Related to pipelined shuffle optional bool last_event = 8; // Related to pipelined shuffle optional int32 spill_id = 9; // Related to pipelined shuffle. @@ -35,6 +36,7 @@ message DataProto { optional int32 raw_length = 1; optional int32 compressed_length = 2; optional bytes data = 3; + optional int32 uncompressed_length = 4; } message InputInformationEventPayloadProto { diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java index 83bde7b355..4fa031161f 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java @@ -35,6 +35,7 @@ import java.io.ByteArrayInputStream; import java.io.DataInputStream; +import java.io.InputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; @@ -264,7 +265,7 @@ public void testMultipleSpills() throws IOException, InterruptedException { @Test(timeout = 10000) public void testMultipleSpillsWithSmallBuffer() throws IOException, InterruptedException { // numBuffers is much higher than available threads. - baseTest(200, 10, null, shouldCompress, 512, 0, 9600); + baseTest(200, 10, null, shouldCompress, 512, 0, 9600, false); } @Test(timeout = 10000) @@ -280,7 +281,9 @@ public void testNoRecords() throws IOException, InterruptedException { @Test(timeout = 10000) public void testNoRecords_SinglePartition() throws IOException, InterruptedException { // skipBuffers - baseTest(0, 1, null, shouldCompress, -1, 0); + baseTest(0, 1, null, shouldCompress, -1, 0, 2048, false); + // Check with data via events + baseTest(0, 1, null, shouldCompress, -1, 0, 2048, true); } @Test(timeout = 10000) @@ -488,6 +491,7 @@ public void textTest(int numRegularRecords, int numPartitions, long availableMem assertEquals(numPartitions, cdme.getCount()); DataMovementEventPayloadProto eventProto = DataMovementEventPayloadProto.parseFrom( ByteString.copyFrom(cdme.getUserPayload())); + assertFalse(eventProto.hasData()); BitSet emptyPartitionBits = null; if (partitionsWithData.cardinality() != numPartitions) { assertTrue(eventProto.hasEmptyPartitions()); @@ -1095,12 +1099,12 @@ private void baseTest(int numRecords, int numPartitions, Set skippedPar boolean shouldCompress, int maxSingleBufferSizeBytes, int bufferMergePercent) throws IOException, InterruptedException { baseTest(numRecords, numPartitions, skippedPartitions, shouldCompress, - maxSingleBufferSizeBytes, bufferMergePercent, 2048); + maxSingleBufferSizeBytes, bufferMergePercent, 2048, false); } private void baseTest(int numRecords, int numPartitions, Set skippedPartitions, boolean shouldCompress, int maxSingleBufferSizeBytes, int bufferMergePercent, int - availableMemory) + availableMemory, boolean dataViaEventEnabled) throws IOException, InterruptedException { PartitionerForTest partitioner = new PartitionerForTest(); ApplicationId appId = ApplicationId.newInstance(10000000, 1); @@ -1189,7 +1193,7 @@ private void baseTest(int numRecords, int numPartitions, Set skippedPar long fileOutputBytes = fileOutputBytesCounter.getValue(); if (numRecordsWritten > 0) { assertTrue(fileOutputBytes > 0); - if (!shouldCompress) { + if ((!shouldCompress) && (!dataViaEventEnabled)) { assertTrue(fileOutputBytes > outputRecordBytesCounter.getValue()); } } else { @@ -1262,37 +1266,53 @@ private void baseTest(int numRecords, int numPartitions, Set skippedPar return; } + boolean isInMem= eventProto.getData().hasData(); assertTrue(localFs.exists(outputFilePath)); - assertTrue(localFs.exists(spillFilePath)); - assertEquals("Incorrect output permissions", (short)0640, - localFs.getFileStatus(outputFilePath).getPermission().toShort()); - assertEquals("Incorrect index permissions", (short)0640, - localFs.getFileStatus(spillFilePath).getPermission().toShort()); - - // verify no intermediate spill files have been left around - synchronized (kvWriter.spillInfoList) { - for (SpillInfo spill : kvWriter.spillInfoList) { - assertFalse("lingering intermediate spill file " + spill.outPath, - localFs.exists(spill.outPath)); + assertEquals("Incorrect output permissions", (short) 0640, + localFs.getFileStatus(outputFilePath).getPermission().toShort()); + if( !isInMem ) { + assertTrue(localFs.exists(spillFilePath)); + assertEquals("Incorrect index permissions", (short) 0640, + localFs.getFileStatus(spillFilePath).getPermission().toShort()); + + // verify no intermediate spill files have been left around + synchronized (kvWriter.spillInfoList) { + for (SpillInfo spill : kvWriter.spillInfoList) { + assertFalse("lingering intermediate spill file " + spill.outPath, + localFs.exists(spill.outPath)); + } } } // Special case for 0 records. - TezSpillRecord spillRecord = new TezSpillRecord(spillFilePath, conf); DataInputBuffer keyBuffer = new DataInputBuffer(); DataInputBuffer valBuffer = new DataInputBuffer(); IntWritable keyDeser = new IntWritable(); LongWritable valDeser = new LongWritable(); for (int i = 0; i < numOutputs; i++) { - TezIndexRecord indexRecord = spillRecord.getIndex(i); - if (skippedPartitions != null && skippedPartitions.contains(i)) { - assertFalse("The Index Record for partition " + i + " should not have any data", indexRecord.hasData()); - continue; + IFile.Reader reader = null; + InputStream inStream; + if (isInMem) { + // Read from in memory payload + int dataLoadSize = eventProto.getData().getData().size(); + inStream = new ByteArrayInputStream(eventProto.getData().getData().toByteArray()); + reader = new IFile.Reader(inStream, dataLoadSize, codec, null, + null, false, 0, -1); + } else { + TezSpillRecord spillRecord = new TezSpillRecord(spillFilePath, conf); + TezIndexRecord indexRecord = spillRecord.getIndex(i); + if (skippedPartitions != null && skippedPartitions.contains(i)) { + assertFalse("The Index Record for partition " + i + " should not have any data", indexRecord.hasData()); + continue; + } + + FSDataInputStream tmpStream = FileSystem.getLocal(conf).open(outputFilePath); + tmpStream.seek(indexRecord.getStartOffset()); + inStream = tmpStream; + reader = new IFile.Reader(tmpStream, indexRecord.getPartLength(), codec, null, + null, false, 0, -1); } - FSDataInputStream inStream = FileSystem.getLocal(conf).open(outputFilePath); - inStream.seek(indexRecord.getStartOffset()); - IFile.Reader reader = new IFile.Reader(inStream, indexRecord.getPartLength(), codec, null, - null, false, 0, -1); + while (reader.nextRawKey(keyBuffer)) { reader.nextRawValue(valBuffer); keyDeser.readFields(keyBuffer); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/conf/TestUnorderedPartitionedKVOutputConfig.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/conf/TestUnorderedPartitionedKVOutputConfig.java index 5e49b5118e..4bcff88cc2 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/conf/TestUnorderedPartitionedKVOutputConfig.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/conf/TestUnorderedPartitionedKVOutputConfig.java @@ -82,6 +82,10 @@ public void testSetters() { .setAvailableBufferSize(1111) .setAdditionalConfiguration("fs.shouldExist", "fs") .setAdditionalConfiguration("test.key.1", "key1") + .setAdditionalConfiguration(TezRuntimeConfiguration + .TEZ_RUNTIME_EMPTY_PARTITION_INFO_VIA_EVENTS_ENABLED, "true") + .setAdditionalConfiguration(TezRuntimeConfiguration + .TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_MAX_SIZE, "5120") .setAdditionalConfiguration(TezRuntimeConfiguration .TEZ_RUNTIME_PIPELINED_SHUFFLE_ENABLED, "true") .setAdditionalConfiguration(TezRuntimeConfiguration @@ -121,6 +125,10 @@ public void testSetters() { TezRuntimeConfiguration.TEZ_RUNTIME_IFILE_READAHEAD_BYTES_DEFAULT)); assertEquals(2222, conf.getInt(TezRuntimeConfiguration.TEZ_RUNTIME_UNORDERED_OUTPUT_MAX_PER_BUFFER_SIZE_BYTES, 0)); + assertEquals(true, + conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_EMPTY_PARTITION_INFO_VIA_EVENTS_ENABLED, false)); + assertEquals(5120, + conf.getInt(TezRuntimeConfiguration.TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_MAX_SIZE, 512)); assertEquals("io", conf.get("io.shouldExist")); assertEquals("file", conf.get("file.shouldExist")); assertEquals("fs", conf.get("fs.shouldExist")); diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java b/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java index 2a94d9bf58..b66151960d 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java @@ -55,6 +55,7 @@ import org.apache.tez.dag.api.client.StatusGetOpts; import org.apache.tez.dag.api.client.VertexStatus; import org.apache.tez.mapreduce.examples.CartesianProduct; +import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; import org.apache.tez.runtime.library.cartesianproduct.CartesianProductVertexManager; import org.apache.tez.runtime.library.conf.OrderedPartitionedKVEdgeConfig; import org.apache.tez.runtime.library.partitioner.HashPartitioner; @@ -215,6 +216,60 @@ public boolean accept(Path p) { assertEquals(0, expectedResult.size()); } + /** + * test whole {@link HashJoinExample} pipeline as following:
+ * {@link JoinDataGen} -> {@link HashJoinExample} -> {@link JoinValidate} + * @throws Exception + */ + @Test(timeout = 120000) + public void testHashJoinExampleWithDataViaEvent() throws Exception { + + Path testDir = new Path("/tmp/testHashJoinExampleDataViaEvent"); + Path stagingDirPath = new Path("/tmp/tez-staging-dir"); + remoteFs.mkdirs(stagingDirPath); + remoteFs.mkdirs(testDir); + + Path dataPath1 = new Path(testDir, "inPath1"); + Path dataPath2 = new Path(testDir, "inPath2"); + Path expectedOutputPath = new Path(testDir, "expectedOutputPath"); + Path outPath = new Path(testDir, "outPath"); + + TezConfiguration tezConf = new TezConfiguration(mrrTezCluster.getConfig()); + tezConf.set(TezConfiguration.TEZ_AM_STAGING_DIR, stagingDirPath.toString()); + + //turn on the dataViaEvent + tezConf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_ENABLED, true); + + TezClient tezSession = null; + try { + tezSession = TezClient.create("HashJoinExampleSession", tezConf, true); + tezSession.start(); + + JoinDataGen dataGen = new JoinDataGen(); + String[] dataGenArgs = new String[] { + "-counter", + dataPath1.toString(), "1048576", dataPath2.toString(), "8", + expectedOutputPath.toString(), "2" }; + assertEquals(0, dataGen.run(tezConf, dataGenArgs, tezSession)); + + HashJoinExample joinExample = new HashJoinExample(); + String[] args = new String[] { + dataPath1.toString(), dataPath2.toString(), "1", outPath.toString(), + "doBroadcast"}; + + assertEquals(0, joinExample.run(tezConf, args, tezSession)); + + JoinValidate joinValidate = new JoinValidate(); + String[] validateArgs = new String[] { + "-counter", expectedOutputPath.toString(), outPath.toString(), "3" }; + assertEquals(0, joinValidate.run(tezConf, validateArgs, tezSession)); + } finally { + if (tezSession != null) { + tezSession.stop(); + } + } + } + @Test(timeout = 60000) public void testHashJoinExampleDisableSplitGrouping() throws Exception { HashJoinExample hashJoinExample = new HashJoinExample(); From 3dce6c9ae5e02d02dd84eecf0cd920ef07298b5b Mon Sep 17 00:00:00 2001 From: rbalamohan Date: Fri, 11 Oct 2019 09:58:09 +0530 Subject: [PATCH 173/512] * TEZ-4088: Create in-memory ifile writer for transferring smaller payloads (follow up of TEZ-4075) * TEZ-4088: Create in-memory ifile writer for transferring smaller payloads (follow up of TEZ-4075) (rbalamohan, reviewed by sseth) --- .../library/api/TezRuntimeConfiguration.java | 9 + .../library/common/sort/impl/IFile.java | 226 ++++++++++++++++-- .../common/sort/impl/IFileOutputStream.java | 4 + .../writers/UnorderedPartitionedKVWriter.java | 38 ++- .../library/output/UnorderedKVOutput.java | 1 + .../output/UnorderedPartitionedKVOutput.java | 1 + .../library/common/sort/impl/TestIFile.java | 124 ++++++++++ ...estUnorderedPartitionedKVOutputConfig.java | 5 + 8 files changed, 385 insertions(+), 23 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java index d4532c99cc..00bb20cd6c 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java @@ -496,6 +496,14 @@ public class TezRuntimeConfiguration { @Private public static final int TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_MAX_SIZE_DEFAULT = 512; + @Private + public static final String TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_SUPPORT_IN_MEM_FILE = + TEZ_RUNTIME_PREFIX + "transfer.data-via-events.support.in-mem.file"; + + @Private + public static final boolean TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_SUPPORT_IN_MEM_FILE_DEFAULT + = true; + /** * If the shuffle input is on the local host bypass the http fetch and access the files directly */ @@ -633,6 +641,7 @@ public class TezRuntimeConfiguration { tezRuntimeKeys.add(TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT); tezRuntimeKeys.add(TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_ENABLED); tezRuntimeKeys.add(TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_MAX_SIZE); + tezRuntimeKeys.add(TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_SUPPORT_IN_MEM_FILE); tezRuntimeKeys.add(TEZ_RUNTIME_RECORDS_BEFORE_PROGRESS); tezRuntimeKeys.add(TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH); tezRuntimeKeys.add(TEZ_RUNTIME_OPTIMIZE_SHARED_FETCH); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java index e460859582..ab82b01139 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java @@ -23,11 +23,14 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.nio.ByteBuffer; import java.util.Iterator; import java.util.concurrent.atomic.AtomicBoolean; import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.io.BoundedByteArrayOutputStream; +import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience; @@ -71,6 +74,195 @@ public class IFile { private static final String INCOMPLETE_READ = "Requested to read %d got %d"; private static final String REQ_BUFFER_SIZE_TOO_LARGE = "Size of data %d is greater than the max allowed of %d"; + /** + * IFileWriter which stores data in memory for specified limit, beyond + * which it falls back to file based writer. It creates files lazily on + * need basis and avoids any disk hit (in cases, where data fits entirely in mem). + *

+ * This class should not make any changes to IFile logic and should just flip streams + * from mem to disk on need basis. + * + * During write, it verifies whether uncompressed payload can fit in memory. If so, it would + * store in buffer. Otherwise, it falls back to file based writer. Note that data stored + * internally would be in compressed format (if codec is provided). However, for easier + * comparison and spill over, uncompressed payload check is done. This is + * done intentionally, as it is not possible to know compressed data length + * upfront. + */ + public static class FileBackedInMemIFileWriter extends Writer { + + private FileSystem fs; + private boolean bufferFull; + + // For lazy creation of file + private TezTaskOutput taskOutput; + private int totalSize; + + @VisibleForTesting + private Path outputPath; + private CompressionCodec fileCodec; + private BoundedByteArrayOutputStream cacheStream; + + private static final int checksumSize = IFileOutputStream.getCheckSumSize(); + + /** + * Note that we do not allow compression in in-mem stream. + * When spilled over to file, compression gets enabled. + * + * @param conf + * @param fs + * @param taskOutput + * @param keyClass + * @param valueClass + * @param codec + * @param writesCounter + * @param serializedBytesCounter + * @param cacheSize + * @throws IOException + */ + public FileBackedInMemIFileWriter(Configuration conf, FileSystem fs, + TezTaskOutput taskOutput, Class keyClass, Class valueClass, + CompressionCodec codec, + TezCounter writesCounter, + TezCounter serializedBytesCounter, + int cacheSize) throws IOException { + super(conf, new FSDataOutputStream(createBoundedBuffer(cacheSize), null), + keyClass, valueClass, null, writesCounter, serializedBytesCounter); + this.fs = fs; + this.cacheStream = (BoundedByteArrayOutputStream) this.rawOut.getWrappedStream(); + this.taskOutput = taskOutput; + this.bufferFull = (cacheStream == null); + this.totalSize = getBaseCacheSize(); + this.fileCodec = codec; + } + + /** + * For basic cache size checks: header + checksum + EOF marker + * + * @return size of the base cache needed + */ + static int getBaseCacheSize() { + return (HEADER.length + checksumSize + + (2 * WritableUtils.getVIntSize(EOF_MARKER))); + } + + boolean shouldWriteToDisk() { + return totalSize >= cacheStream.getLimit(); + } + + /** + * Create in mem stream. In it is too small, adjust it's size + * + * @param size + * @return in memory stream + */ + public static BoundedByteArrayOutputStream createBoundedBuffer(int size) { + int resize = Math.max(getBaseCacheSize(), size); + return new BoundedByteArrayOutputStream(resize); + } + + /** + * Flip over from memory to file based writer. + * + * 1. Content format: HEADER + real data + CHECKSUM. Checksum is for real + * data. + * 2. Before flipping, close checksum stream, so that checksum is written + * out. + * 3. Create relevant file based writer. + * 4. Write header and then real data. + * + * @throws IOException + */ + private void resetToFileBasedWriter() throws IOException { + // Close out stream, so that data checksums are written. + // Buf contents = HEADER + real data + CHECKSUM + this.out.close(); + + // Get the buffer which contains data in memory + BoundedByteArrayOutputStream bout = + (BoundedByteArrayOutputStream) this.rawOut.getWrappedStream(); + + // Create new file based writer + if (outputPath == null) { + outputPath = taskOutput.getOutputFileForWrite(); + } + LOG.info("Switching from mem stream to disk stream. File: " + outputPath); + FSDataOutputStream newRawOut = fs.create(outputPath); + this.rawOut = newRawOut; + this.ownOutputStream = true; + + setupOutputStream(fileCodec); + + // Write header to file + headerWritten = false; + writeHeader(newRawOut); + + // write real data + int sPos = HEADER.length; + int len = (bout.size() - checksumSize - HEADER.length); + this.out.write(bout.getBuffer(), sPos, len); + + bufferFull = true; + bout.reset(); + } + + + @Override + protected void writeKVPair(byte[] keyData, int keyPos, int keyLength, + byte[] valueData, int valPos, int valueLength) throws IOException { + if (!bufferFull) { + // Compute actual payload size: write RLE marker, length info and then entire data. + totalSize += ((prevKey == REPEAT_KEY) ? V_END_MARKER_SIZE : 0) + + WritableUtils.getVIntSize(keyLength) + keyLength + + WritableUtils.getVIntSize(valueLength) + valueLength; + + if (shouldWriteToDisk()) { + resetToFileBasedWriter(); + } + } + super.writeKVPair(keyData, keyPos, keyLength, valueData, valPos, valueLength); + } + + @Override + protected void writeValue(byte[] data, int offset, int length) throws IOException { + if (!bufferFull) { + totalSize += ((prevKey != REPEAT_KEY) ? RLE_MARKER_SIZE : 0) + + WritableUtils.getVIntSize(length) + length; + + if (shouldWriteToDisk()) { + resetToFileBasedWriter(); + } + } + super.writeValue(data, offset, length); + } + + /** + * Check if data was flushed to disk. + * + * @return whether data is flushed to disk ot not + */ + public boolean isDataFlushedToDisk() { + return bufferFull; + } + + /** + * Get cached data if any + * + * @return if data is not flushed to disk, it returns in-mem contents + */ + public ByteBuffer getData() { + if (!isDataFlushedToDisk()) { + return ByteBuffer.wrap(cacheStream.getBuffer(), 0, cacheStream.size()); + } + return null; + } + + @VisibleForTesting + void setOutputPath(Path outputPath) { + this.outputPath = outputPath; + } + } + /** * IFile.Writer to write out intermediate map-outputs. */ @@ -148,9 +340,28 @@ public Writer(Configuration conf, FSDataOutputStream outputStream, this.rawOut = outputStream; this.writtenRecordsCounter = writesCounter; this.serializedUncompressedBytes = serializedBytesCounter; - this.checksumOut = new IFileOutputStream(outputStream); this.start = this.rawOut.getPos(); this.rle = rle; + + setupOutputStream(codec); + + writeHeader(outputStream); + + if (keyClass != null) { + this.closeSerializers = true; + SerializationFactory serializationFactory = + new SerializationFactory(conf); + this.keySerializer = serializationFactory.getSerializer(keyClass); + this.keySerializer.open(buffer); + this.valueSerializer = serializationFactory.getSerializer(valueClass); + this.valueSerializer.open(buffer); + } else { + this.closeSerializers = false; + } + } + + void setupOutputStream(CompressionCodec codec) throws IOException { + this.checksumOut = new IFileOutputStream(this.rawOut); if (codec != null) { this.compressor = CodecPool.getCompressor(codec); if (this.compressor != null) { @@ -165,19 +376,6 @@ public Writer(Configuration conf, FSDataOutputStream outputStream, } else { this.out = new FSDataOutputStream(checksumOut,null); } - writeHeader(outputStream); - - if (keyClass != null) { - this.closeSerializers = true; - SerializationFactory serializationFactory = - new SerializationFactory(conf); - this.keySerializer = serializationFactory.getSerializer(keyClass); - this.keySerializer.open(buffer); - this.valueSerializer = serializationFactory.getSerializer(valueClass); - this.valueSerializer.open(buffer); - } else { - this.closeSerializers = false; - } } public Writer(Configuration conf, FileSystem fs, Path file) throws IOException { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileOutputStream.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileOutputStream.java index 319844680d..5ec0537288 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileOutputStream.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileOutputStream.java @@ -59,6 +59,10 @@ public IFileOutputStream(OutputStream out) { offset = 0; } + public static int getCheckSumSize() { + return DataChecksum.Type.CRC32.size; + } + @Override public void close() throws IOException { if (closed) { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java index 7165205e52..1197dde17b 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java @@ -196,6 +196,9 @@ public Deflater get() { private List filledBuffers = new ArrayList<>(); + // When enabled, uses in-mem ifile writer + private final boolean useCachedStream; + public UnorderedPartitionedKVWriter(OutputContext outputContext, Configuration conf, int numOutputs, long availableMemoryBytes) throws IOException { super(outputContext, conf, numOutputs); @@ -223,6 +226,13 @@ public UnorderedPartitionedKVWriter(OutputContext outputContext, Configuration c TezRuntimeConfiguration.TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_MAX_SIZE, TezRuntimeConfiguration.TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_MAX_SIZE_DEFAULT); + boolean useCachedStreamConfig = conf.getBoolean( + TezRuntimeConfiguration.TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_SUPPORT_IN_MEM_FILE, + TezRuntimeConfiguration.TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_SUPPORT_IN_MEM_FILE_DEFAULT); + + this.useCachedStream = useCachedStreamConfig && (this.dataViaEventsEnabled && (numPartitions == 1) + && !pipelinedShuffle); + if (availableMemoryBytes == 0) { Preconditions.checkArgument(((numPartitions == 1) && !pipelinedShuffle), "availableMemory " + "can be set to 0 only when numPartitions=1 and " + TezRuntimeConfiguration @@ -287,10 +297,16 @@ public UnorderedPartitionedKVWriter(OutputContext outputContext, Configuration c if (numPartitions == 1 && !pipelinedShuffle) { //special case, where in only one partition is available. - finalOutPath = outputFileHandler.getOutputFileForWrite(); skipBuffers = true; - writer = new IFile.Writer(conf, rfs, finalOutPath, keyClass, valClass, - codec, outputRecordsCounter, outputRecordBytesCounter); + if (this.useCachedStream) { + writer = new IFile.FileBackedInMemIFileWriter(conf, rfs, outputFileHandler, keyClass, + valClass, codec, outputRecordsCounter, outputRecordBytesCounter, + dataViaEventsMaxSize); + } else { + finalOutPath = outputFileHandler.getOutputFileForWrite(); + writer = new IFile.Writer(conf, rfs, finalOutPath, keyClass, valClass, + codec, outputRecordsCounter, outputRecordBytesCounter); + } if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { rfs.setPermission(finalOutPath, SPILL_FILE_PERMS); } @@ -311,6 +327,8 @@ public UnorderedPartitionedKVWriter(OutputContext outputContext, Configuration c + ", reportPartitionStats=" + reportPartitionStats + ", dataViaEventsEnabled=" + dataViaEventsEnabled + ", dataViaEventsMaxSize=" + dataViaEventsMaxSize + + ", useCachedStreamConfig=" + useCachedStreamConfig + + ", useCachedStream=" + useCachedStream ); } @@ -702,14 +720,16 @@ private boolean canSendDataOverDME() { && (writer.getCompressedLength() <= dataViaEventsMaxSize); } - private byte[] readDataForDME() throws IOException { - // TODO: Not introducing a caching layer in IFile yet. - byte[] buf = null; - try (FSDataInputStream inStream = rfs.open(finalOutPath)) { - buf = new byte[(int) writer.getCompressedLength()]; + private ByteBuffer readDataForDME() throws IOException { + if (this.useCachedStream) { + return ((IFile.FileBackedInMemIFileWriter) writer).getData(); + } else { + try (FSDataInputStream inStream = rfs.open(finalOutPath)) { + byte[] buf = new byte[(int) writer.getCompressedLength()]; IOUtils.readFully(inStream, buf, 0, (int) writer.getCompressedLength()); + return ByteBuffer.wrap(buf); + } } - return buf; } @Override diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java index 50e1e8b836..85368f6ea9 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java @@ -169,6 +169,7 @@ String getHost() { confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_EMPTY_PARTITION_INFO_VIA_EVENTS_ENABLED); confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_ENABLED); confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_MAX_SIZE); + confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_SUPPORT_IN_MEM_FILE); confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_CONVERT_USER_PAYLOAD_TO_HISTORY_TEXT); confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_PIPELINED_SHUFFLE_ENABLED); confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java index 5c56083b82..536ee3216f 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java @@ -142,6 +142,7 @@ public synchronized List close() throws Exception { confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_EMPTY_PARTITION_INFO_VIA_EVENTS_ENABLED); confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_ENABLED); confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_MAX_SIZE); + confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_SUPPORT_IN_MEM_FILE); confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_CONVERT_USER_PAYLOAD_TO_HISTORY_TEXT); confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_PIPELINED_SHUFFLE_ENABLED); confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java index 90f5374590..518f733802 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java @@ -29,6 +29,9 @@ import java.util.List; import java.util.Random; +import com.google.protobuf.ByteString; +import org.apache.tez.common.TezRuntimeFrameworkConfigs; +import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutputFiles; import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -83,6 +86,7 @@ public class TestIFile { new Path(System.getProperty("test.build.data", "/tmp")), TestIFile.class.getName()) .makeQualified(localFs.getUri(), localFs.getWorkingDirectory()); LOG.info("Using workDir: " + workDir); + defaultConf.set(TezRuntimeFrameworkConfigs.LOCAL_DIRS, workDir.toString()); } catch (IOException e) { throw new RuntimeException(e); } @@ -499,6 +503,126 @@ public void testAppendValues() throws IOException { readAndVerifyData(writer.getRawLength(), writer.getCompressedLength(), data, codec); } + @Test(timeout = 5000) + // Basic test + public void testFileBackedInMemIFileWriter() throws IOException { + List data = new ArrayList<>(); + List values = new ArrayList<>(); + Text key = new Text("key"); + IntWritable val = new IntWritable(1); + for(int i = 0; i < 5; i++) { + data.add(new KVPair(key, val)); + values.add(val); + } + + TezTaskOutputFiles tezTaskOutput = new TezTaskOutputFiles(defaultConf, "uniqueId", 1); + IFile.FileBackedInMemIFileWriter writer = new IFile.FileBackedInMemIFileWriter(defaultConf, localFs, tezTaskOutput, + Text.class, IntWritable.class, codec, null, null, + 200); + + writer.appendKeyValues(data.get(0).getKey(), values.iterator()); + Text lastKey = new Text("key3"); + IntWritable lastVal = new IntWritable(10); + data.add(new KVPair(lastKey, lastVal)); + writer.append(lastKey, lastVal); + writer.close(); + + byte[] bytes = new byte[(int) writer.getRawLength()]; + IFile.Reader.readToMemory(bytes, + new ByteArrayInputStream(ByteString.copyFrom(writer.getData()).toByteArray()), + (int) writer.getCompressedLength(), codec, false, -1); + readUsingInMemoryReader(bytes, data); + } + + @Test(timeout = 5000) + // Basic test + public void testFileBackedInMemIFileWriterWithSmallBuffer() throws IOException { + List data = new ArrayList<>(); + TezTaskOutputFiles tezTaskOutput = new TezTaskOutputFiles(defaultConf, "uniqueId", 1); + IFile.FileBackedInMemIFileWriter writer = new IFile.FileBackedInMemIFileWriter(defaultConf, localFs, tezTaskOutput, + Text.class, IntWritable.class, codec, null, null, + 2); + + // empty ifile + writer.close(); + + // Buffer should have self adjusted. So for this empty file, it shouldn't + // hit disk. + assertFalse("Data should have been flushed to disk", writer.isDataFlushedToDisk()); + + byte[] bytes = new byte[(int) writer.getRawLength()]; + IFile.Reader.readToMemory(bytes, + new ByteArrayInputStream(ByteString.copyFrom(writer.getData()).toByteArray()), + (int) writer.getCompressedLength(), codec, false, -1); + + readUsingInMemoryReader(bytes, data); + } + + @Test(timeout = 20000) + // Test file spill over scenario + public void testFileBackedInMemIFileWriter_withSpill() throws IOException { + List data = new ArrayList<>(); + List values = new ArrayList<>(); + + Text key = new Text("key"); + IntWritable val = new IntWritable(1); + for(int i = 0; i < 5; i++) { + data.add(new KVPair(key, val)); + values.add(val); + } + + // Setting cache limit to 20. Actual data would be around 43 bytes, so it would spill over. + TezTaskOutputFiles tezTaskOutput = new TezTaskOutputFiles(defaultConf, "uniqueId", 1); + IFile.FileBackedInMemIFileWriter writer = new IFile.FileBackedInMemIFileWriter(defaultConf, localFs, tezTaskOutput, + Text.class, IntWritable.class, codec, null, null, + 20); + writer.setOutputPath(outputPath); + + writer.appendKeyValues(data.get(0).getKey(), values.iterator()); + Text lastKey = new Text("key3"); + IntWritable lastVal = new IntWritable(10); + + data.add(new KVPair(lastKey, lastVal)); + writer.append(lastKey, lastVal); + writer.close(); + + assertTrue("Data should have been flushed to disk", writer.isDataFlushedToDisk()); + + // Read output content to memory + FSDataInputStream inStream = localFs.open(outputPath); + byte[] bytes = new byte[(int) writer.getRawLength()]; + + IFile.Reader.readToMemory(bytes, inStream, + (int) writer.getCompressedLength(), codec, false, -1); + inStream.close(); + + readUsingInMemoryReader(bytes, data); + } + + @Test(timeout = 5000) + // Test empty file case + public void testEmptyFileBackedInMemIFileWriter() throws IOException { + List data = new ArrayList<>(); + TezTaskOutputFiles + tezTaskOutput = new TezTaskOutputFiles(defaultConf, "uniqueId", 1); + + IFile.FileBackedInMemIFileWriter writer = new IFile.FileBackedInMemIFileWriter(defaultConf, localFs, tezTaskOutput, + Text.class, IntWritable.class, codec, null, null, + 100); + + // empty ifile + writer.close(); + + byte[] bytes = new byte[(int) writer.getRawLength()]; + + IFile.Reader.readToMemory(bytes, + new ByteArrayInputStream(ByteString.copyFrom(writer.getData()).toByteArray()), + (int) writer.getCompressedLength(), codec, false, -1); + + readUsingInMemoryReader(bytes, data); + } + + @Test(timeout = 5000) //Test appendKeyValues feature public void testAppendKeyValues() throws IOException { diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/conf/TestUnorderedPartitionedKVOutputConfig.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/conf/TestUnorderedPartitionedKVOutputConfig.java index 4bcff88cc2..bff2868020 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/conf/TestUnorderedPartitionedKVOutputConfig.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/conf/TestUnorderedPartitionedKVOutputConfig.java @@ -86,6 +86,9 @@ public void testSetters() { .TEZ_RUNTIME_EMPTY_PARTITION_INFO_VIA_EVENTS_ENABLED, "true") .setAdditionalConfiguration(TezRuntimeConfiguration .TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_MAX_SIZE, "5120") + .setAdditionalConfiguration(TezRuntimeConfiguration + .TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_SUPPORT_IN_MEM_FILE, + "false") .setAdditionalConfiguration(TezRuntimeConfiguration .TEZ_RUNTIME_PIPELINED_SHUFFLE_ENABLED, "true") .setAdditionalConfiguration(TezRuntimeConfiguration @@ -129,6 +132,8 @@ public void testSetters() { conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_EMPTY_PARTITION_INFO_VIA_EVENTS_ENABLED, false)); assertEquals(5120, conf.getInt(TezRuntimeConfiguration.TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_MAX_SIZE, 512)); + assertEquals(false, + conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_SUPPORT_IN_MEM_FILE, true)); assertEquals("io", conf.get("io.shouldExist")); assertEquals("file", conf.get("file.shouldExist")); assertEquals("fs", conf.get("fs.shouldExist")); From b99c7ce938def1ed98a2d3ca135457f3300ddcf4 Mon Sep 17 00:00:00 2001 From: rbalamohan Date: Fri, 18 Oct 2019 05:24:39 +0530 Subject: [PATCH 174/512] TEZ-4091: UnorderedPartitionedKVWriter::readDataForDME should check if in-mem file is flushed or not (#53) --- .../library/common/sort/impl/IFile.java | 5 ++- .../writers/UnorderedPartitionedKVWriter.java | 32 ++++++++++++++---- .../TestUnorderedPartitionedKVWriter.java | 33 +++++++++++++++---- 3 files changed, 57 insertions(+), 13 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java index ab82b01139..09abf27bc8 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java @@ -98,7 +98,6 @@ public static class FileBackedInMemIFileWriter extends Writer { private TezTaskOutput taskOutput; private int totalSize; - @VisibleForTesting private Path outputPath; private CompressionCodec fileCodec; private BoundedByteArrayOutputStream cacheStream; @@ -261,6 +260,10 @@ public ByteBuffer getData() { void setOutputPath(Path outputPath) { this.outputPath = outputPath; } + + public Path getOutputPath() { + return this.outputPath; + } } /** diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java index 1197dde17b..db0c30e660 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java @@ -178,7 +178,8 @@ public Deflater get() { Path finalOutPath; //for single partition cases (e.g UnorderedKVOutput) - private final IFile.Writer writer; + @VisibleForTesting + final IFile.Writer writer; @VisibleForTesting final boolean skipBuffers; @@ -306,9 +307,9 @@ public UnorderedPartitionedKVWriter(OutputContext outputContext, Configuration c finalOutPath = outputFileHandler.getOutputFileForWrite(); writer = new IFile.Writer(conf, rfs, finalOutPath, keyClass, valClass, codec, outputRecordsCounter, outputRecordBytesCounter); - } - if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { - rfs.setPermission(finalOutPath, SPILL_FILE_PERMS); + if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { + rfs.setPermission(finalOutPath, SPILL_FILE_PERMS); + } } } else { skipBuffers = false; @@ -715,18 +716,37 @@ public static long getInitialMemoryRequirement(Configuration conf, long maxAvail return reqBytes; } - private boolean canSendDataOverDME() { + private boolean canSendDataOverDME() throws IOException { + if (dataViaEventsEnabled + && this.useCachedStream + && this.finalOutPath == null) { + + // It is possible that in-mem writer spilled over to disk. Need to use + // that path as finalOutPath and set its permission. + + if (((IFile.FileBackedInMemIFileWriter) writer).isDataFlushedToDisk()) { + this.finalOutPath = + ((IFile.FileBackedInMemIFileWriter) writer).getOutputPath(); + if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { + rfs.setPermission(finalOutPath, SPILL_FILE_PERMS); + } + additionalSpillBytesWritternCounter.increment(writer.getCompressedLength()); + } + } + return (writer != null) && (dataViaEventsEnabled) && (writer.getCompressedLength() <= dataViaEventsMaxSize); } private ByteBuffer readDataForDME() throws IOException { - if (this.useCachedStream) { + if (this.useCachedStream + && !((IFile.FileBackedInMemIFileWriter) writer).isDataFlushedToDisk()) { return ((IFile.FileBackedInMemIFileWriter) writer).getData(); } else { try (FSDataInputStream inStream = rfs.open(finalOutPath)) { byte[] buf = new byte[(int) writer.getCompressedLength()]; IOUtils.readFully(inStream, buf, 0, (int) writer.getCompressedLength()); + additionalSpillBytesReadCounter.increment(writer.getCompressedLength()); return ByteBuffer.wrap(buf); } } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java index 4fa031161f..3692392518 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java @@ -296,6 +296,10 @@ public void testNoSpill_SinglePartition() throws IOException, InterruptedExcepti baseTest(10, 1, null, shouldCompress, -1, 0); } + @Test(timeout = 10000) + public void testSpill_SinglePartition() throws IOException, InterruptedException { + baseTest(1000, 1, null, shouldCompress, -1, 0, 2048, true); + } @Test(timeout = 10000) public void testRandomText() throws IOException, InterruptedException { @@ -1120,6 +1124,9 @@ private void baseTest(int numRecords, int numPartitions, Set skippedPar conf.setInt( TezRuntimeConfiguration.TEZ_RUNTIME_UNORDERED_PARTITIONED_KVWRITER_BUFFER_MERGE_PERCENT, bufferMergePercent); + conf.setBoolean( + TezRuntimeConfiguration.TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_ENABLED, + dataViaEventEnabled); CompressionCodec codec = null; if (shouldCompress) { @@ -1199,7 +1206,10 @@ private void baseTest(int numRecords, int numPartitions, Set skippedPar } else { assertEquals(0, fileOutputBytes); } - assertEquals(recordsPerBuffer * numExpectedSpills, spilledRecordsCounter.getValue()); + if (!dataViaEventEnabled) { + assertEquals(recordsPerBuffer * numExpectedSpills, + spilledRecordsCounter.getValue()); + } long additionalSpillBytesWritten = additionalSpillBytesWritternCounter.getValue(); long additionalSpillBytesRead = additionalSpillBytesReadCounter.getValue(); if (numExpectedSpills == 0) { @@ -1207,13 +1217,24 @@ private void baseTest(int numRecords, int numPartitions, Set skippedPar assertEquals(0, additionalSpillBytesRead); } else { assertTrue(additionalSpillBytesWritten > 0); - assertTrue(additionalSpillBytesRead > 0); - if (!shouldCompress) { - assertTrue(additionalSpillBytesWritten > (recordsPerBuffer * numExpectedSpills * sizePerRecord)); - assertTrue(additionalSpillBytesRead > (recordsPerBuffer * numExpectedSpills * sizePerRecord)); + if (!dataViaEventEnabled) { + assertTrue(additionalSpillBytesRead > 0); + if (!shouldCompress) { + assertTrue(additionalSpillBytesWritten > + (recordsPerBuffer * numExpectedSpills * sizePerRecord)); + assertTrue(additionalSpillBytesRead > + (recordsPerBuffer * numExpectedSpills * sizePerRecord)); + } + } else { + if (kvWriter.writer.getCompressedLength() > + TezRuntimeConfiguration.TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_MAX_SIZE_DEFAULT) { + assertTrue(additionalSpillBytesWritten > 0); + } } } - assertEquals(additionalSpillBytesWritten, additionalSpillBytesRead); + if (!dataViaEventEnabled) { + assertEquals(additionalSpillBytesWritten, additionalSpillBytesRead); + } // due to multiple threads, buffers could be merged in chunks in scheduleSpill. assertTrue(numExpectedSpills >= numAdditionalSpillsCounter.getValue()); From 8c8458f69fec315837feede20bcabb1131ab3835 Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Sat, 9 Nov 2019 05:50:09 +0530 Subject: [PATCH 175/512] TEZ-4096: SSLFactory should pickup configs from incoming conf payload (rbalamohan, reviewed by gopalv) --- .../src/main/java/org/apache/tez/http/SSLFactory.java | 6 +++--- .../tez/runtime/library/api/TezRuntimeConfiguration.java | 3 ++- .../runtime/library/conf/TestUnorderedKVInputConfig.java | 2 ++ .../test/java/org/apache/tez/test/TestSecureShuffle.java | 4 ++++ 4 files changed, 11 insertions(+), 4 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java b/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java index e7a2dd0051..203eb40dcd 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java @@ -85,13 +85,13 @@ public SSLFactory(Mode mode, Configuration conf) { this.mode = mode; requireClientCert = conf.getBoolean(SSL_REQUIRE_CLIENT_CERT_KEY, DEFAULT_SSL_REQUIRE_CLIENT_CERT); - Configuration sslConf = readSSLConfiguration(mode); + // Rest of ssl configs are pre-populated in incoming conf payload + conf.setBoolean(SSL_REQUIRE_CLIENT_CERT_KEY, requireClientCert); Class klass = conf.getClass(KEYSTORES_FACTORY_CLASS_KEY, FileBasedKeyStoresFactory.class, KeyStoresFactory.class); - keystoresFactory = ReflectionUtils.newInstance(klass, sslConf); - + keystoresFactory = ReflectionUtils.newInstance(klass, conf); enabledProtocols = conf.getStrings(SSL_ENABLED_PROTOCOLS, DEFAULT_SSL_ENABLED_PROTOCOLS); } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java index 00bb20cd6c..9c2f7c3dcf 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java @@ -668,10 +668,11 @@ public class TezRuntimeConfiguration { } // Do NOT need all prefixes from the following list. Only specific ones are allowed - // "hadoop.", "hadoop.security", "io.", "fs.", "ipc.", "net.", "file.", "dfs.", "ha.", "s3.", "nfs3.", "rpc." + // "hadoop.", "hadoop.security", "io.", "fs.", "ipc.", "net.", "file.", "dfs.", "ha.", "s3.", "nfs3.", "rpc.", "ssl." allowedPrefixes.add("io."); allowedPrefixes.add("file."); allowedPrefixes.add("fs."); + allowedPrefixes.add("ssl."); umnodifiableTezRuntimeKeySet = Collections.unmodifiableSet(tezRuntimeKeys); unmodifiableOtherKeySet = Collections.unmodifiableSet(otherKeys); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/conf/TestUnorderedKVInputConfig.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/conf/TestUnorderedKVInputConfig.java index bb754429b4..d04fa6d298 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/conf/TestUnorderedKVInputConfig.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/conf/TestUnorderedKVInputConfig.java @@ -58,6 +58,7 @@ public void testSetters() { fromConf.set("test.conf.key.1", "confkey1"); fromConf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_IFILE_READAHEAD_BYTES, 1111); fromConf.set("io.shouldExist", "io"); + fromConf.set("ssl.shouldExist", "ssl"); Map additionalConf = new HashMap(); additionalConf.put("test.key.2", "key2"); additionalConf.put(TezRuntimeConfiguration.TEZ_RUNTIME_IO_SORT_FACTOR, "3"); @@ -105,6 +106,7 @@ public void testSetters() { assertEquals("io", conf.get("io.shouldExist")); assertEquals("file", conf.get("file.shouldExist")); assertEquals("fs", conf.get("fs.shouldExist")); + assertEquals("ssl", conf.get("ssl.shouldExist")); assertNull(conf.get("test.conf.key.1")); assertNull(conf.get("test.key.1")); assertNull(conf.get("test.key.2")); diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java b/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java index 0fb07fc481..6d34464c04 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java @@ -18,6 +18,7 @@ package org.apache.tez.test; +import static org.apache.hadoop.security.ssl.SSLFactory.SSL_CLIENT_CONF_KEY; import static org.junit.Assert.assertEquals; import java.io.BufferedWriter; @@ -133,6 +134,9 @@ public void setupTezCluster() throws Exception { conf.setLong(TezConfiguration.TEZ_AM_SLEEP_TIME_BEFORE_EXIT_MILLIS, 500); + String sslConf = conf.get(SSL_CLIENT_CONF_KEY, "ssl-client.xml"); + conf.addResource(sslConf); + miniTezCluster = new MiniTezCluster(TestSecureShuffle.class.getName() + "-" + (enableSSLInCluster ? "withssl" : "withoutssl"), 1, 1, 1); From cf73f435bee18022804bded8ae9ca526983b6cee Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Fri, 15 Nov 2019 13:13:27 -0600 Subject: [PATCH 176/512] TEZ-4085. Tez UI resources vendor.js and tez-ui.js not getting minified in tez releases (Himanshu Mishra via jeagles) --- tez-ui/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tez-ui/pom.xml b/tez-ui/pom.xml index 78d59fd7d6..365249fbdd 100644 --- a/tez-ui/pom.xml +++ b/tez-ui/pom.xml @@ -213,7 +213,7 @@ true tez-ui,vendor ${basedir}/target/minified-resources/assets - ${webappDir}/dist/assets + ${basedir}/src/main/webapp/dist/assets ${basedir}/src/main/webapp/WEB-INF/wro.xml From 616229e65038ec82eafbcf17537e9c2d46751b23 Mon Sep 17 00:00:00 2001 From: Jacob Tolar Date: Fri, 15 Nov 2019 14:38:29 -0600 Subject: [PATCH 177/512] TEZ-4084. Tez local mode fails when distributed cache creates link with parent Signed-off-by: Jonathan Eagles --- .../app/launcher/TezLocalCacheManager.java | 36 +++-- .../launcher/TestTezLocalCacheManager.java | 128 +++++++++--------- 2 files changed, 90 insertions(+), 74 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java index 45e55403a1..9bcbb1530a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java @@ -96,11 +96,13 @@ public void localize() throws IOException { Path linkPath = new Path(cwd, entry.getKey()); if (resourceInfo.containsKey(resource)) { - // We've already downloaded this resource and just need to add another link. - resourceInfo.get(resource).linkPaths.add(linkPath); + // We've already downloaded this resource and just need to add another link. + resourceInfo.get(resource).getLinkPaths().add(linkPath); } else { // submit task to download the object - java.nio.file.Path downloadDir = Files.createTempDirectory(tempDir, resourceName); + java.nio.file.Path fp = Paths.get(resourceName).getFileName(); + String prefix = fp == null ? "" : fp.toString(); // The null case is unexpected, but FindBugs complains + java.nio.file.Path downloadDir = Files.createTempDirectory(tempDir, prefix); Path dest = new Path(downloadDir.toAbsolutePath().toString()); FSDownload downloader = new FSDownload(fileContext, ugi, conf, dest, resource); Future downloadedPath = threadPool.submit(downloader); @@ -113,12 +115,12 @@ public void localize() throws IOException { LocalResource resource = entry.getKey(); ResourceInfo resourceMeta = entry.getValue(); - for (Path linkPath : resourceMeta.linkPaths) { + for (Path linkPath : resourceMeta.getLinkPaths()) { Path targetPath; try { // this blocks on the download completing - targetPath = resourceMeta.downloadPath.get(); + targetPath = resourceMeta.getDownloadPath().get(); } catch (InterruptedException | ExecutionException e) { throw new IOException(e); } @@ -144,7 +146,7 @@ public void localize() throws IOException { */ public void cleanup() throws IOException { for (ResourceInfo info : resourceInfo.values()) { - for (Path linkPath : info.linkPaths) { + for (Path linkPath : info.getLinkPaths()) { if (fileContext.util().exists(linkPath)) { fileContext.delete(linkPath, true); } @@ -172,23 +174,31 @@ private boolean createSymlink(Path target, Path link) throws IOException { try { Files.createSymbolicLink(Paths.get(linkPath), Paths.get(targetPath)); return true; - } catch (UnsupportedOperationException e) { - LOG.warn("Unable to create symlink {} <- {}: UnsupportedOperationException", target, link); + } catch (UnsupportedOperationException | IOException e) { + LOG.warn("Unable to create symlink {} <- {}: {}", target, link, e); return false; } } } /** - * Wrapper to keep track of download path and link path + * Wrapper to keep track of download path and link path. */ private static class ResourceInfo { - final Future downloadPath; - final Set linkPaths = new HashSet<>(); + private final Future downloadPath; + private final Set linkPaths = new HashSet<>(); - public ResourceInfo(Future downloadPath, Path linkPath) { + ResourceInfo(Future downloadPath, Path linkPath) { this.downloadPath = downloadPath; - this.linkPaths.add(linkPath); + this.getLinkPaths().add(linkPath); + } + + Future getDownloadPath() { + return downloadPath; + } + + Set getLinkPaths() { + return linkPaths; } } } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestTezLocalCacheManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestTezLocalCacheManager.java index fb23a1d403..beca047349 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestTezLocalCacheManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestTezLocalCacheManager.java @@ -39,69 +39,75 @@ import java.util.HashMap; import java.util.Map; +/** + * Test local cache manager. + */ public class TestTezLocalCacheManager { - @Test - public void testManager() throws URISyntaxException, IOException { - Map resources = new HashMap<>(); - - // Test that localization works for regular files and verify that if multiple symlinks are created, - // they all work - LocalResource resourceOne = createFile("content-one"); - LocalResource resourceTwo = createFile("content-two"); - - resources.put("file-one", resourceOne); - resources.put("file-two", resourceTwo); - resources.put("file-three", resourceTwo); - - TezLocalCacheManager manager = new TezLocalCacheManager(resources, new Configuration()); - - try { - manager.localize(); - - Assert.assertEquals( - "content-one", - new String(Files.readAllBytes(Paths.get("./file-one"))) - ); - - Assert.assertEquals( - "content-two", - new String(Files.readAllBytes(Paths.get("./file-two"))) - ); - - Assert.assertEquals( - "content-two", - new String(Files.readAllBytes(Paths.get("./file-three"))) - ); - } finally { - manager.cleanup(); - } - - // verify that symlinks were removed - Assert.assertFalse(Files.exists(Paths.get("./file-one"))); - Assert.assertFalse(Files.exists(Paths.get("./file-two"))); - Assert.assertFalse(Files.exists(Paths.get("./file-three"))); - } + @Test + public void testManager() throws URISyntaxException, IOException { + Map resources = new HashMap<>(); + + // Test that localization works for regular files and verify that if multiple symlinks are created, + // they all work + LocalResource resourceOne = createFile("content-one"); + LocalResource resourceTwo = createFile("content-two"); + + resources.put("file-one", resourceOne); + resources.put("file-two", resourceTwo); + resources.put("file-three", resourceTwo); + + // Not currently supported, but shouldn't throw an exception... + resources.put("some-subdir/file-three", resourceTwo); - // create a temporary file with the given content and return a LocalResource - private static LocalResource createFile(String content) throws IOException { - FileContext fs = FileContext.getLocalFSFileContext(); - - java.nio.file.Path tempFile = Files.createTempFile("test-cache-manager", ".txt"); - File temp = tempFile.toFile(); - temp.deleteOnExit(); - Path p = new Path("file:///" + tempFile.toAbsolutePath().toString()); - - Files.write(tempFile, content.getBytes()); - - RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); - LocalResource ret = recordFactory.newRecordInstance(LocalResource.class); - URL yarnUrlFromPath = ConverterUtils.getYarnUrlFromPath(p); - ret.setResource(yarnUrlFromPath); - ret.setSize(content.getBytes().length); - ret.setType(LocalResourceType.FILE); - ret.setVisibility(LocalResourceVisibility.PRIVATE); - ret.setTimestamp(fs.getFileStatus(p).getModificationTime()); - return ret; + TezLocalCacheManager manager = new TezLocalCacheManager(resources, new Configuration()); + + try { + manager.localize(); + + Assert.assertEquals( + "content-one", + new String(Files.readAllBytes(Paths.get("./file-one"))) + ); + + Assert.assertEquals( + "content-two", + new String(Files.readAllBytes(Paths.get("./file-two"))) + ); + + Assert.assertEquals( + "content-two", + new String(Files.readAllBytes(Paths.get("./file-three"))) + ); + } finally { + manager.cleanup(); } + + // verify that symlinks were removed + Assert.assertFalse(Files.exists(Paths.get("./file-one"))); + Assert.assertFalse(Files.exists(Paths.get("./file-two"))); + Assert.assertFalse(Files.exists(Paths.get("./file-three"))); + } + + // create a temporary file with the given content and return a LocalResource + private static LocalResource createFile(String content) throws IOException { + FileContext fs = FileContext.getLocalFSFileContext(); + + java.nio.file.Path tempFile = Files.createTempFile("test-cache-manager", ".txt"); + File temp = tempFile.toFile(); + temp.deleteOnExit(); + Path p = new Path("file:///" + tempFile.toAbsolutePath().toString()); + + Files.write(tempFile, content.getBytes()); + + RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); + LocalResource ret = recordFactory.newRecordInstance(LocalResource.class); + URL yarnUrlFromPath = ConverterUtils.getYarnUrlFromPath(p); + ret.setResource(yarnUrlFromPath); + ret.setSize(content.getBytes().length); + ret.setType(LocalResourceType.FILE); + ret.setVisibility(LocalResourceVisibility.PRIVATE); + ret.setTimestamp(fs.getFileStatus(p).getModificationTime()); + return ret; + } } From 32e80e7d8f1ee7fae645c6cf0a3d86c9b9563c9b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Fri, 15 Nov 2019 15:54:47 -0600 Subject: [PATCH 178/512] TEZ-3992. Update commons-codec from 1.4 to 1.11 Signed-off-by: Jonathan Eagles --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 883222ad9f..cd526fa9f7 100644 --- a/pom.xml +++ b/pom.xml @@ -272,7 +272,7 @@ commons-codec commons-codec - 1.4 + 1.11 commons-collections From 9c66f974b2932a2d9551e6e7b6b1ed7e75c02f6d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Mon, 18 Nov 2019 10:05:45 -0600 Subject: [PATCH 179/512] TEZ-4083. Upgrade to latest 9.3.x Jetty version Signed-off-by: Jonathan Eagles --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index cd526fa9f7..1a5148ac64 100644 --- a/pom.xml +++ b/pom.xml @@ -38,7 +38,7 @@ true ${user.home}/clover.license 3.0.3 - 9.3.24.v20180605 + 9.3.28.v20191105 3.10.5.Final 0.13.0 1.19 From 47f0f354ab38ea2b96ed6ed008f158527b30903f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Mon, 18 Nov 2019 13:24:32 -0600 Subject: [PATCH 180/512] TEZ-3860. JDK9: ReflectionUtils may not use URLClassLoader Signed-off-by: Jonathan Eagles --- .../apache/tez/common/ReflectionUtils.java | 8 ++- .../org/apache/tez/common/TezClassLoader.java | 62 +++++++++++++++++++ .../apache/tez/client/TestTezClientUtils.java | 12 ++-- .../org/apache/tez/dag/app/DAGAppMaster.java | 3 +- 4 files changed, 76 insertions(+), 9 deletions(-) create mode 100644 tez-api/src/main/java/org/apache/tez/common/TezClassLoader.java diff --git a/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java b/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java index 4d89ed4f9d..9f7c5d35f0 100644 --- a/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java @@ -122,9 +122,9 @@ public static synchronized void addResourcesToClasspath(List urls) { @Private public static synchronized void addResourcesToSystemClassLoader(List urls) { - URLClassLoader sysLoader = (URLClassLoader)ClassLoader.getSystemClassLoader(); + ClassLoader sysLoader = getSystemClassLoader(); if (sysClassLoaderMethod == null) { - Class sysClass = URLClassLoader.class; + Class sysClass = TezClassLoader.class; Method method; try { method = sysClass.getDeclaredMethod("addURL", parameters); @@ -148,4 +148,8 @@ public static synchronized void addResourcesToSystemClassLoader(List urls) } } } + + private static ClassLoader getSystemClassLoader() { + return TezClassLoader.getInstance(); + } } diff --git a/tez-api/src/main/java/org/apache/tez/common/TezClassLoader.java b/tez-api/src/main/java/org/apache/tez/common/TezClassLoader.java new file mode 100644 index 0000000000..2679efa692 --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/common/TezClassLoader.java @@ -0,0 +1,62 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.common; + +import java.net.MalformedURLException; +import java.net.URL; +import java.net.URLClassLoader; +import java.security.AccessController; +import java.security.PrivilegedAction; +import java.util.Arrays; + +public class TezClassLoader extends URLClassLoader { + private static TezClassLoader INSTANCE; + + static { + INSTANCE = AccessController.doPrivileged(new PrivilegedAction() { + ClassLoader sysLoader = TezClassLoader.class.getClassLoader(); + + public TezClassLoader run() { + return new TezClassLoader( + sysLoader instanceof URLClassLoader ? ((URLClassLoader) sysLoader).getURLs() : extractClassPathEntries(), + sysLoader); + } + }); + } + + public TezClassLoader(URL[] urls, ClassLoader classLoader) { + super(urls, classLoader); + } + + public void addURL(URL url) { + super.addURL(url); + } + + public static TezClassLoader getInstance() { + return INSTANCE; + } + + private static URL[] extractClassPathEntries() { + String pathSeparator = System.getProperty("path.separator"); + String[] classPathEntries = System.getProperty("java.class.path").split(pathSeparator); + URL[] cp = Arrays.asList(classPathEntries).stream().map(s -> { + try { + return new URL("file://" + s); + } catch (MalformedURLException e) { + throw new RuntimeException(e); + } + }).toArray(URL[]::new); + return cp; + } +} diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java index 7ff8125768..581d7225a3 100644 --- a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java +++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java @@ -27,7 +27,6 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.net.URL; -import java.net.URLClassLoader; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; @@ -60,6 +59,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.util.Records; +import org.apache.tez.common.TezClassLoader; import org.apache.tez.common.security.JobTokenIdentifier; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.common.security.TokenCache; @@ -131,9 +131,9 @@ public void validateSetTezJarLocalResourcesDefinedNonExistingDirectory() throws /** * */ - @Test (timeout=10000) + @Test (timeout=20000) public void validateSetTezJarLocalResourcesDefinedExistingDirectory() throws Exception { - URL[] cp = ((URLClassLoader)ClassLoader.getSystemClassLoader()).getURLs(); + URL[] cp = TezClassLoader.getInstance().getURLs(); StringBuffer buffer = new StringBuffer(); for (URL url : cp) { buffer.append(url.toExternalForm()); @@ -171,7 +171,7 @@ public void validateSetTezJarLocalResourcesDefinedExistingDirectory() throws Exc */ @Test (timeout=5000) public void validateSetTezJarLocalResourcesDefinedExistingDirectoryIgnored() throws Exception { - URL[] cp = ((URLClassLoader)ClassLoader.getSystemClassLoader()).getURLs(); + URL[] cp = TezClassLoader.getInstance().getURLs(); StringBuffer buffer = new StringBuffer(); for (URL url : cp) { buffer.append(url.toExternalForm()); @@ -190,9 +190,9 @@ public void validateSetTezJarLocalResourcesDefinedExistingDirectoryIgnored() thr * * @throws Exception */ - @Test (timeout=5000) + @Test (timeout=20000) public void validateSetTezJarLocalResourcesDefinedExistingDirectoryIgnoredSetToFalse() throws Exception { - URL[] cp = ((URLClassLoader)ClassLoader.getSystemClassLoader()).getURLs(); + URL[] cp = TezClassLoader.getInstance().getURLs(); StringBuffer buffer = new StringBuffer(); for (URL url : cp) { buffer.append(url.toExternalForm()); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index 2d2f23d031..6636fb63cd 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -115,6 +115,7 @@ import org.apache.tez.common.AsyncDispatcher; import org.apache.tez.common.AsyncDispatcherConcurrent; import org.apache.tez.common.GcTimeUpdater; +import org.apache.tez.common.TezClassLoader; import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.TezConverterUtils; import org.apache.tez.common.TezUtilsInternal; @@ -1404,7 +1405,7 @@ public Boolean run() throws Exception { } private static Path findLocalFileForResource(String fileName) { - URL localResource = ClassLoader.getSystemClassLoader().getResource(fileName); + URL localResource = TezClassLoader.getInstance().getResource(fileName); if (localResource == null) return null; return new Path(localResource.getPath()); } From 1af0897c88fb32d39606d21b4c0abcee11b1896c Mon Sep 17 00:00:00 2001 From: Ahmed Hussein Date: Tue, 26 Nov 2019 13:59:54 -0600 Subject: [PATCH 181/512] TEZ-4067. Tez Speculation decision is calculated on each update by the dispatcher Signed-off-by: Jonathan Eagles --- .../apache/tez/dag/api/TezConfiguration.java | 8 + .../org/apache/tez/dag/app/DAGAppMaster.java | 30 ++- .../org/apache/tez/dag/app/dag/Vertex.java | 7 +- .../tez/dag/app/dag/impl/VertexImpl.java | 141 +++++++++++- .../speculation/legacy/LegacySpeculator.java | 212 ++++++++++++------ .../apache/tez/dag/app/TestSpeculation.java | 51 +++-- 6 files changed, 343 insertions(+), 106 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 7b00cf6475..f087e3a91f 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -530,6 +530,14 @@ public TezConfiguration(boolean loadDefaults) { public static final String TEZ_AM_SPECULATION_ENABLED = TEZ_AM_PREFIX + "speculation.enabled"; public static final boolean TEZ_AM_SPECULATION_ENABLED_DEFAULT = false; + /** + * Class used to estimate task resource needs. + */ + @ConfigurationScope(Scope.VERTEX) + @ConfigurationProperty + public static final String TEZ_AM_SPECULATION_ESTIMATOR_CLASS = + TEZ_AM_PREFIX + "speculation.estimator.class"; + /** * Float value. Specifies how many standard deviations away from the mean task execution time * should be considered as an outlier/slow task. diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index 6636fb63cd..f29d19995b 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -757,6 +757,8 @@ protected synchronized void handle(DAGAppMasterEvent event) { String timeStamp = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss").format(Calendar.getInstance().getTime()); System.err.println(timeStamp + " Completed Dag: " + finishEvt.getDAGId().toString()); System.out.println(timeStamp + " Completed Dag: " + finishEvt.getDAGId().toString()); + // Stop vertex services if any + stopVertexServices(currentDAG); if (!isSession) { LOG.info("Not a session, AM will unregister as DAG has completed"); this.taskSchedulerManager.setShouldUnregisterFlag(); @@ -1293,7 +1295,6 @@ public String submitDAGToAppMaster(DAGPlan dagPlan, throw new SessionNotRunning("AM unable to accept new DAG submissions." + " In the process of shutting down"); } - // dag is in cleanup when dag state is completed but AM state is still RUNNING synchronized (idleStateLock) { while (currentDAG != null && currentDAG.isComplete() && state == DAGAppMasterState.RUNNING) { @@ -1840,7 +1841,7 @@ private void notifyDependentServices() { } } - void startServices(){ + void startServices() { try { Throwable firstError = null; List threads = new ArrayList(); @@ -1888,12 +1889,16 @@ void initServices(Configuration conf) { } void stopServices() { + Exception firstException = null; // stop in reverse order of start + if (currentDAG != null) { + stopVertexServices(currentDAG); + } List serviceList = new ArrayList(services.size()); for (ServiceWithDependency sd : services.values()) { serviceList.add(sd.service); } - Exception firstException = null; + for (int i = services.size() - 1; i >= 0; i--) { Service service = serviceList.get(i); if (LOG.isDebugEnabled()) { @@ -1933,7 +1938,6 @@ private DAGRecoveryData recoverDAG() throws IOException, TezException { @Override public synchronized void serviceStart() throws Exception { - //start all the components startServices(); super.serviceStart(); @@ -2060,6 +2064,9 @@ public synchronized void serviceStart() throws Exception { DAGEventRecoverEvent recoverDAGEvent = new DAGEventRecoverEvent( recoveredDAGData.recoveredDAG.getID(), recoveredDAGData); dagEventDispatcher.handle(recoverDAGEvent); + // If we reach here, then we have recoverable DAG and we need to + // reinitialize the vertex services including speculators. + startVertexServices(currentDAG); this.state = DAGAppMasterState.RUNNING; } } else { @@ -2543,6 +2550,18 @@ public Void run() throws Exception { this.state = DAGAppMasterState.RUNNING; } + private void startVertexServices(DAG dag) { + for (Vertex v : dag.getVertices().values()) { + v.startServices(); + } + } + + void stopVertexServices(DAG dag) { + for (Vertex v: dag.getVertices().values()) { + v.stopServices(); + } + } + private void startDAGExecution(DAG dag, final Map additionalAmResources) throws TezException { currentDAG = dag; @@ -2574,7 +2593,8 @@ public List run() throws Exception { // This is a synchronous call, not an event through dispatcher. We want // job-init to be done completely here. dagEventDispatcher.handle(initDagEvent); - + // Start the vertex services + startVertexServices(dag); // All components have started, start the job. /** create a job-start event to get this ball rolling */ DAGEvent startDagEvent = new DAGEventStartDag(currentDAG.getID(), additionalUrlsForClasspath); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java index 0b2406fad6..f3ef72bbc4 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java @@ -26,6 +26,7 @@ import javax.annotation.Nullable; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.AbstractService; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.EdgeManagerPluginDescriptor; @@ -73,7 +74,6 @@ public interface Vertex extends Comparable { LinkedHashMap getIOIndices(); String getName(); VertexState getState(); - /** * Get all the counters of this vertex. * @return aggregate task-counters @@ -169,7 +169,10 @@ public TaskAttemptEventInfo getTaskAttemptTezEvents(TezTaskAttemptID attemptID, int fromEventId, int nextPreRoutedFromEventId, int maxEvents); void handleSpeculatorEvent(SpeculatorEvent event); - + AbstractService getSpeculator(); + void initServices(); + void startServices(); + void stopServices(); ProcessorDescriptor getProcessorDescriptor(); public DAG getDAG(); @Nullable diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index a2ef475f5b..9a59e884f4 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -49,6 +49,9 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.service.ServiceOperations; +import org.apache.hadoop.service.ServiceStateException; import org.apache.hadoop.util.StringInterner; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.Resource; @@ -306,8 +309,10 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl @VisibleForTesting final List pendingVmEvents = new LinkedList<>(); - - LegacySpeculator speculator; + + private final AtomicBoolean servicesInited; + private LegacySpeculator speculator; + private List services; @VisibleForTesting Map> commitFutures = new ConcurrentHashMap>(); @@ -869,6 +874,94 @@ void resetCompletedTaskStatsCache(boolean recompute) { } } + @Override + public void initServices() { + if (servicesInited.get()) { + if (LOG.isDebugEnabled()) { + LOG.debug("Skipping Initing services for vertex because already" + + " Initialized, name=" + this.vertexName); + } + return; + } + writeLock.lock(); + try { + List servicesToAdd = new ArrayList<>(); + if (isSpeculationEnabled()) { + // Initialize the speculator + if (LOG.isDebugEnabled()) { + LOG.debug( + "Initing service vertex speculator, name=" + this.vertexName); + } + speculator = new LegacySpeculator(vertexConf, getAppContext(), this); + speculator.init(vertexConf); + servicesToAdd.add(speculator); + } + services = Collections.synchronizedList(servicesToAdd); + servicesInited.set(true); + } finally { + writeLock.unlock(); + } + if (LOG.isDebugEnabled()) { + LOG.debug("Initing service vertex, name=" + this.vertexName); + } + } + + @Override + public void startServices() { + writeLock.lock(); + try { + if (!servicesInited.get()) { + initServices(); + } + for (AbstractService srvc : services) { + if (LOG.isDebugEnabled()) { + LOG.debug("starting service : " + srvc.getName() + + ", for vertex: " + getName()); + } + srvc.start(); + } + } finally { + writeLock.unlock(); + } + } + + @Override + public void stopServices() { + Exception firstException = null; + List stoppedServices = new ArrayList<>(); + writeLock.lock(); + try { + if (servicesInited.get()) { + for (AbstractService srvc : services) { + if (LOG.isDebugEnabled()) { + LOG.debug("Stopping service : " + srvc); + } + Exception ex = ServiceOperations.stopQuietly(srvc); + if (ex != null && firstException == null) { + LOG.warn(String.format( + "Failed to stop service=(%s) for vertex name=(%s)", + srvc.getName(), getName()), ex); + firstException = ex; + } else { + stoppedServices.add(srvc); + } + } + services.clear(); + } + servicesInited.set(false); + } finally { + writeLock.unlock(); + } + // wait for services to stop + for (AbstractService srvc : stoppedServices) { + srvc.waitForServiceToStop(60000L); + } + // After stopping all services, rethrow the first exception raised + if (firstException != null) { + throw ServiceStateException.convert(firstException); + } + } + public VertexImpl(TezVertexID vertexId, VertexPlan vertexPlan, String vertexName, Configuration dagConf, EventHandler eventHandler, TaskCommunicatorManagerInterface taskCommunicatorManagerInterface, Clock clock, @@ -972,11 +1065,11 @@ public VertexImpl(TezVertexID vertexId, VertexPlan vertexPlan, this.dagVertexGroups = dagVertexGroups; - isSpeculationEnabled = vertexConf.getBoolean(TezConfiguration.TEZ_AM_SPECULATION_ENABLED, - TezConfiguration.TEZ_AM_SPECULATION_ENABLED_DEFAULT); - if (isSpeculationEnabled()) { - speculator = new LegacySpeculator(vertexConf, getAppContext(), this); - } + isSpeculationEnabled = + vertexConf.getBoolean(TezConfiguration.TEZ_AM_SPECULATION_ENABLED, + TezConfiguration.TEZ_AM_SPECULATION_ENABLED_DEFAULT); + servicesInited = new AtomicBoolean(false); + initServices(); maxFailuresPercent = vertexConf.getFloat(TezConfiguration.TEZ_VERTEX_FAILURES_MAXPERCENT, TezConfiguration.TEZ_VERTEX_FAILURES_MAXPERCENT_DEFAULT); @@ -2329,6 +2422,11 @@ VertexState finished(VertexState finalState, abortVertex(VertexStatus.State.valueOf(finalState.name())); eventHandler.handle(new DAGEvent(getDAGId(), DAGEventType.INTERNAL_ERROR)); + if (LOG.isDebugEnabled()) { + LOG.debug("stopping services attached to the succeeded Vertex," + + "name=" + getName()); + } + stopServices(); try { logJobHistoryVertexFailedEvent(finalState); } catch (IOException e) { @@ -2344,6 +2442,11 @@ VertexState finished(VertexState finalState, abortVertex(VertexStatus.State.valueOf(finalState.name())); eventHandler.handle(new DAGEventVertexCompleted(getVertexId(), finalState, terminationCause)); + if (LOG.isDebugEnabled()) { + LOG.debug("stopping services attached to the succeeded Vertex," + + "name=" + getName()); + } + stopServices(); try { logJobHistoryVertexFailedEvent(finalState); } catch (IOException e) { @@ -2356,6 +2459,12 @@ VertexState finished(VertexState finalState, logJobHistoryVertexFinishedEvent(); eventHandler.handle(new DAGEventVertexCompleted(getVertexId(), finalState)); + // Stop related services + if (LOG.isDebugEnabled()) { + LOG.debug("stopping services attached to the succeeded Vertex," + + "name=" + getName()); + } + stopServices(); } catch (LimitExceededException e) { LOG.error("Counter limits exceeded for vertex: " + getLogIdentifier(), e); finalState = VertexState.FAILED; @@ -2374,6 +2483,12 @@ VertexState finished(VertexState finalState, } break; default: + // Stop related services + if (LOG.isDebugEnabled()) { + LOG.debug("stopping services attached with Unexpected State," + + "name=" + getName()); + } + stopServices(); throw new TezUncheckedException("Unexpected VertexState: " + finalState); } return finalState; @@ -2458,6 +2573,8 @@ private boolean initializeVertex() { } else { initedTime = clock.getTime(); } + // set the vertex services to be initialized. + initServices(); // Only initialize committer when it is in non-recovery mode or vertex is not recovered to completed // state in recovery mode if (recoveryData == null || recoveryData.getVertexFinishedEvent() == null) { @@ -3316,6 +3433,12 @@ public Void run() { if (finishTime == 0) { setFinishTime(); } + // Stop related services + if (LOG.isDebugEnabled()) { + LOG.debug("stopping services attached to the aborted Vertex, name=" + + getName()); + } + stopServices(); } private void mayBeConstructFinalFullCounters() { @@ -4763,6 +4886,6 @@ public boolean getTaskRescheduleRelaxedLocality() { } } - @VisibleForTesting - public LegacySpeculator getSpeculator() { return speculator; } + @Override + public AbstractService getSpeculator() { return speculator; } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java index 3e7c2c0088..0a0e9a20dd 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java @@ -18,13 +18,22 @@ package org.apache.tez.dag.app.dag.speculation.legacy; +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; import java.util.HashSet; import java.util.Map; import java.util.Set; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import com.google.common.annotations.VisibleForTesting; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.service.ServiceOperations; import org.apache.tez.dag.api.TezConfiguration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,8 +50,6 @@ import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.dag.records.TezTaskID; -import com.google.common.base.Preconditions; - /** * Maintains runtime estimation statistics. Makes periodic updates * estimates based on progress and decides on when to trigger a @@ -54,7 +61,7 @@ * because it may be likely a wasted attempt. There is a delay between * successive speculations. */ -public class LegacySpeculator { +public class LegacySpeculator extends AbstractService { private static final long ON_SCHEDULE = Long.MIN_VALUE; private static final long ALREADY_SPECULATING = Long.MIN_VALUE + 1; @@ -75,7 +82,7 @@ public class LegacySpeculator { private final ConcurrentMap runningTasks = new ConcurrentHashMap(); - + private ReadWriteLock lock = new ReentrantReadWriteLock(); // Used to track any TaskAttempts that aren't heart-beating for a while, so // that we can aggressively speculate instead of waiting for task-timeout. private final ConcurrentMap @@ -86,13 +93,17 @@ public class LegacySpeculator { // in progress. private static final long MAX_WAITTING_TIME_FOR_HEARTBEAT = 9 * 1000; - private final Set waitingToSpeculate = new HashSet(); + private final Set mayHaveSpeculated = new HashSet(); private Vertex vertex; private TaskRuntimeEstimator estimator; private final long taskTimeout; private final Clock clock; private long nextSpeculateTime = Long.MIN_VALUE; + private Thread speculationBackgroundThread = null; + private volatile boolean stopped = false; + /* Allow the speculator to wait on a blockingQueue in case we use it for event notification */ + private BlockingQueue scanControl = new LinkedBlockingQueue(); @VisibleForTesting public int getMinimumAllowedSpeculativeTasks() { return minimumAllowedSpeculativeTasks;} @@ -119,17 +130,72 @@ public LegacySpeculator(Configuration conf, Clock clock, Vertex vertex) { static private TaskRuntimeEstimator getEstimator (Configuration conf, Vertex vertex) { - TaskRuntimeEstimator estimator = new LegacyTaskRuntimeEstimator(); - estimator.contextualize(conf, vertex); - + TaskRuntimeEstimator estimator; + // "tez.am.speculation.estimator.class" + Class estimatorClass = + conf.getClass(TezConfiguration.TEZ_AM_SPECULATION_ESTIMATOR_CLASS, + LegacyTaskRuntimeEstimator.class, + TaskRuntimeEstimator.class); + try { + Constructor estimatorConstructor + = estimatorClass.getConstructor(); + estimator = estimatorConstructor.newInstance(); + estimator.contextualize(conf, vertex); + } catch (NoSuchMethodException e) { + LOG.error("Can't make a speculation runtime estimator", e); + throw new RuntimeException(e); + } catch (IllegalAccessException e) { + LOG.error("Can't make a speculation runtime estimator", e); + throw new RuntimeException(e); + } catch (InstantiationException e) { + LOG.error("Can't make a speculation runtime estimator", e); + throw new RuntimeException(e); + } catch (InvocationTargetException e) { + LOG.error("Can't make a speculation runtime estimator", e); + throw new RuntimeException(e); + } return estimator; } + @Override + protected void serviceStart() throws Exception { + lock.writeLock().lock(); + try { + assert (speculationBackgroundThread == null); + + if (speculationBackgroundThread == null) { + speculationBackgroundThread = + new Thread(createThread(), + "DefaultSpeculator background processing"); + speculationBackgroundThread.start(); + } + super.serviceStart(); + } catch (Exception e) { + LOG.warn("Speculator thread could not launch", e); + } finally { + lock.writeLock().unlock(); + } + } + + public boolean isStarted() { + boolean result = false; + lock.readLock().lock(); + try { + if (this.speculationBackgroundThread != null) { + result = getServiceState().equals(STATE.STARTED); + } + } finally { + lock.readLock().unlock(); + } + return result; + } + // This constructor is designed to be called by other constructors. // However, it's public because we do use it in the test cases. // Normally we figure out our own estimator. public LegacySpeculator (Configuration conf, TaskRuntimeEstimator estimator, Clock clock, Vertex vertex) { + super(LegacySpeculator.class.getName()); this.vertex = vertex; this.estimator = estimator; this.clock = clock; @@ -153,30 +219,48 @@ public LegacySpeculator(Configuration conf, Clock clock, Vertex vertex) { TezConfiguration.TEZ_AM_MINIMUM_ALLOWED_SPECULATIVE_TASKS_DEFAULT); } -/* ************************************************************* */ - - void maybeSpeculate() { - long now = clock.getTime(); - - if (now < nextSpeculateTime) { - return; - } - - int speculations = maybeScheduleASpeculation(); - long mininumRecomp - = speculations > 0 ? soonestRetryAfterSpeculate - : soonestRetryAfterNoSpeculate; - - long wait = Math.max(mininumRecomp, - clock.getTime() - now); - nextSpeculateTime = now + wait; - - if (speculations > 0) { - LOG.info("We launched " + speculations - + " speculations. Waiting " + wait + " milliseconds."); + @Override + protected void serviceStop() throws Exception { + lock.writeLock().lock(); + try { + stopped = true; + // this could be called before background thread is established + if (speculationBackgroundThread != null) { + speculationBackgroundThread.interrupt(); + } + super.serviceStop(); + speculationBackgroundThread = null; + } finally { + lock.writeLock().unlock(); } } + public Runnable createThread() { + return new Runnable() { + @Override + public void run() { + while (!stopped && !Thread.currentThread().isInterrupted()) { + long backgroundRunStartTime = clock.getTime(); + try { + int speculations = computeSpeculations(); + long nextRecompTime = speculations > 0 ? soonestRetryAfterSpeculate + : soonestRetryAfterNoSpeculate; + long wait = Math.max(nextRecompTime, clock.getTime() - backgroundRunStartTime); + if (speculations > 0) { + LOG.info("We launched " + speculations + + " speculations. Waiting " + wait + " milliseconds."); + } + Object pollResult = scanControl.poll(wait, TimeUnit.MILLISECONDS); + } catch (InterruptedException ie) { + if (!stopped) { + LOG.warn("Speculator thread interrupted", ie); + } + } + } + } + }; + } + /* ************************************************************* */ public void notifyAttemptStarted(TezTaskAttemptID taId, long timestamp) { @@ -186,7 +270,6 @@ public void notifyAttemptStarted(TezTaskAttemptID taId, long timestamp) { public void notifyAttemptStatusUpdate(TezTaskAttemptID taId, TaskAttemptState reportedState, long timestamp) { statusUpdate(taId, reportedState, timestamp); - maybeSpeculate(); } /** @@ -197,12 +280,15 @@ public void notifyAttemptStatusUpdate(TezTaskAttemptID taId, TaskAttemptState re * @param timestamp the time this status corresponds to. This matters * because statuses contain progress. */ - private void statusUpdate(TezTaskAttemptID attemptID, TaskAttemptState reportedState, long timestamp) { + private void statusUpdate(TezTaskAttemptID attemptID, + TaskAttemptState reportedState, long timestamp) { TezTaskID taskID = attemptID.getTaskID(); Task task = vertex.getTask(taskID); - Preconditions.checkState(task != null, "Null task for attempt: " + attemptID); + if (task == null) { + return; + } estimator.updateAttempt(attemptID, reportedState, timestamp); @@ -257,46 +343,28 @@ private long speculationValue(Task task, long now, boolean shouldUseTimeout) { // short circuit completed tasks. no need to spend time on them if (task.getState() == TaskState.SUCCEEDED) { + // remove the task from may have speculated if it exists + mayHaveSpeculated.remove(taskID); return NOT_RUNNING; } - int numberRunningAttempts = 0; - - for (TaskAttempt taskAttempt : attempts.values()) { - TaskAttemptState taskAttemptState = taskAttempt.getState(); - if (taskAttemptState == TaskAttemptState.RUNNING - || taskAttemptState == TaskAttemptState.STARTING) { - if (++numberRunningAttempts > 1) { - waitingToSpeculate.remove(taskID); - return ALREADY_SPECULATING; - } - } - } - - // If we are here, there's at most one task attempt. - if (numberRunningAttempts == 0) { - return NOT_RUNNING; - } - - if ((numberRunningAttempts == 1) && waitingToSpeculate.contains(taskID)) { - return ALREADY_SPECULATING; - } - else { - if (!shouldUseTimeout) { - acceptableRuntime = estimator.thresholdRuntime(taskID); - if (acceptableRuntime == Long.MAX_VALUE) { - return ON_SCHEDULE; - } + if (!mayHaveSpeculated.contains(taskID) && !shouldUseTimeout) { + acceptableRuntime = estimator.thresholdRuntime(taskID); + if (acceptableRuntime == Long.MAX_VALUE) { + return ON_SCHEDULE; } } TezTaskAttemptID runningTaskAttemptID = null; + int numberRunningAttempts = 0; for (TaskAttempt taskAttempt : attempts.values()) { TaskAttemptState taskAttemptState = taskAttempt.getState(); if (taskAttemptState == TaskAttemptState.RUNNING || taskAttemptState == TaskAttemptState.STARTING) { - + if (++numberRunningAttempts > 1) { + return ALREADY_SPECULATING; + } runningTaskAttemptID = taskAttempt.getID(); long taskAttemptStartTime @@ -338,7 +406,8 @@ private long speculationValue(Task task, long now, boolean shouldUseTimeout) { if (data.notHeartbeatedInAWhile(now)) { // Stats have stagnated for a while, simulate heart-beat. // Now simulate the heart-beat - statusUpdate(taskAttempt.getID(), taskAttempt.getState(), clock.getTime()); + statusUpdate(taskAttempt.getID(), taskAttempt.getState(), + clock.getTime()); } } else { // Stats have changed - update our data structure @@ -361,6 +430,11 @@ private long speculationValue(Task task, long now, boolean shouldUseTimeout) { } } + // If we are here, there's at most one task attempt. + if (numberRunningAttempts == 0) { + return NOT_RUNNING; + } + if ((acceptableRuntime == Long.MIN_VALUE) && !shouldUseTimeout) { acceptableRuntime = estimator.thresholdRuntime(taskID); if (acceptableRuntime == Long.MAX_VALUE) { @@ -371,14 +445,14 @@ private long speculationValue(Task task, long now, boolean shouldUseTimeout) { return result; } - //Add attempt to a given Task. + // Add attempt to a given Task. protected void addSpeculativeAttempt(TezTaskID taskID) { LOG.info("DefaultSpeculator.addSpeculativeAttempt -- we are speculating " + taskID); vertex.scheduleSpeculativeTask(taskID); - waitingToSpeculate.add(taskID); + mayHaveSpeculated.add(taskID); } - private int maybeScheduleASpeculation() { + int computeSpeculations() { int successes = 0; long now = clock.getTime(); @@ -390,19 +464,18 @@ private int maybeScheduleASpeculation() { int numberAllowedSpeculativeTasks = (int) Math.max(minimumAllowedSpeculativeTasks, - proportionTotalTasksSpeculatable * tasks.size()); - + proportionTotalTasksSpeculatable * tasks.size()); TezTaskID bestTaskID = null; long bestSpeculationValue = -1L; boolean shouldUseTimeout = - (tasks.size() <= VERTEX_SIZE_THRESHOLD_FOR_TIMEOUT_SPECULATION) && + (tasks.size() <= VERTEX_SIZE_THRESHOLD_FOR_TIMEOUT_SPECULATION) && (taskTimeout >= 0); // this loop is potentially pricey. // TODO track the tasks that are potentially worth looking at for (Map.Entry taskEntry : tasks.entrySet()) { long mySpeculationValue = speculationValue(taskEntry.getValue(), now, - shouldUseTimeout); + shouldUseTimeout); if (mySpeculationValue == ALREADY_SPECULATING) { ++numberSpeculationsAlready; @@ -419,7 +492,7 @@ private int maybeScheduleASpeculation() { } numberAllowedSpeculativeTasks = (int) Math.max(numberAllowedSpeculativeTasks, - proportionRunningTasksSpeculatable * numberRunningTasks); + proportionRunningTasksSpeculatable * numberRunningTasks); // If we found a speculation target, fire it off if (bestTaskID != null @@ -427,7 +500,6 @@ private int maybeScheduleASpeculation() { addSpeculativeAttempt(bestTaskID); ++successes; } - return successes; } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java index e1aa44890b..a81d4d3b9f 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java @@ -44,7 +44,6 @@ import org.apache.tez.dag.app.dag.Task; import org.apache.tez.dag.app.dag.TaskAttempt; import org.apache.tez.dag.app.dag.impl.DAGImpl; -import org.apache.tez.dag.app.dag.impl.VertexImpl; import org.apache.tez.dag.app.dag.speculation.legacy.LegacySpeculator; import org.apache.tez.dag.library.vertexmanager.ShuffleVertexManager; import org.apache.tez.dag.records.TaskAttemptTerminationCause; @@ -78,7 +77,7 @@ public class TestSpeculation { throw new RuntimeException("init failure", e); } } - + MockTezClient createTezSession() throws Exception { TezConfiguration tezconf = new TezConfiguration(defaultConf); AtomicBoolean mockAppLauncherGoFlag = new AtomicBoolean(false); @@ -109,11 +108,12 @@ public void testSingleTaskSpeculation() throws Exception { confToExpected.put(Long.MAX_VALUE >> 1, 1); // Really long time to speculate confToExpected.put(100L, 2); confToExpected.put(-1L, 1); // Don't speculate - + defaultConf.setLong(TezConfiguration.TEZ_AM_SOONEST_RETRY_AFTER_NO_SPECULATE, 50); for(Map.Entry entry : confToExpected.entrySet()) { defaultConf.setLong( TezConfiguration.TEZ_AM_LEGACY_SPECULATIVE_SINGLE_TASK_VERTEX_TIMEOUT, entry.getKey()); + DAG dag = DAG.create("test"); Vertex vA = Vertex.create("A", ProcessorDescriptor.create("Proc.class"), @@ -154,15 +154,14 @@ public void testBasicSpeculation(boolean withProgress) throws Exception { defaultConf.setInt(TezConfiguration.TEZ_AM_MINIMUM_ALLOWED_SPECULATIVE_TASKS, 20); defaultConf.setDouble(TezConfiguration.TEZ_AM_PROPORTION_TOTAL_TASKS_SPECULATABLE, 0.2); defaultConf.setDouble(TezConfiguration.TEZ_AM_PROPORTION_RUNNING_TASKS_SPECULATABLE, 0.25); - defaultConf.setLong(TezConfiguration.TEZ_AM_SOONEST_RETRY_AFTER_NO_SPECULATE, 2000); - defaultConf.setLong(TezConfiguration.TEZ_AM_SOONEST_RETRY_AFTER_SPECULATE, 10000); + defaultConf.setLong(TezConfiguration.TEZ_AM_SOONEST_RETRY_AFTER_NO_SPECULATE, 25); + defaultConf.setLong(TezConfiguration.TEZ_AM_SOONEST_RETRY_AFTER_SPECULATE, 50); DAG dag = DAG.create("test"); Vertex vA = Vertex.create("A", ProcessorDescriptor.create("Proc.class"), 5); dag.addVertex(vA); MockTezClient tezClient = createTezSession(); - DAGClient dagClient = tezClient.submitDAG(dag); DAGImpl dagImpl = (DAGImpl) mockApp.getContext().getCurrentDAG(); TezVertexID vertexId = TezVertexID.getInstance(dagImpl.getID(), 0); @@ -195,12 +194,13 @@ public void testBasicSpeculation(boolean withProgress) throws Exception { .getValue()); } - LegacySpeculator speculator = ((VertexImpl) dagImpl.getVertex(vA.getName())).getSpeculator(); + LegacySpeculator speculator = + (LegacySpeculator)(dagImpl.getVertex(vA.getName())).getSpeculator(); Assert.assertEquals(20, speculator.getMinimumAllowedSpeculativeTasks()); Assert.assertEquals(.2, speculator.getProportionTotalTasksSpeculatable(), 0); Assert.assertEquals(.25, speculator.getProportionRunningTasksSpeculatable(), 0); - Assert.assertEquals(2000, speculator.getSoonestRetryAfterNoSpeculate()); - Assert.assertEquals(10000, speculator.getSoonestRetryAfterSpeculate()); + Assert.assertEquals(25, speculator.getSoonestRetryAfterNoSpeculate()); + Assert.assertEquals(50, speculator.getSoonestRetryAfterSpeculate()); tezClient.stop(); } @@ -214,15 +214,18 @@ public void testBasicSpeculationWithProgress() throws Exception { public void testBasicSpeculationWithoutProgress() throws Exception { testBasicSpeculation(false); } - - @Test (timeout=10000) + + @Test (timeout=100000) public void testBasicSpeculationPerVertexConf() throws Exception { DAG dag = DAG.create("test"); String vNameNoSpec = "A"; String vNameSpec = "B"; + String speculatorSleepTime = "50"; Vertex vA = Vertex.create(vNameNoSpec, ProcessorDescriptor.create("Proc.class"), 5); Vertex vB = Vertex.create(vNameSpec, ProcessorDescriptor.create("Proc.class"), 5); vA.setConf(TezConfiguration.TEZ_AM_SPECULATION_ENABLED, "false"); + vB.setConf(TezConfiguration.TEZ_AM_SOONEST_RETRY_AFTER_NO_SPECULATE, + speculatorSleepTime); dag.addVertex(vA); dag.addVertex(vB); // min/max src fraction is set to 1. So vertices will run sequentially @@ -233,14 +236,14 @@ public void testBasicSpeculationPerVertexConf() throws Exception { InputDescriptor.create("I")))); MockTezClient tezClient = createTezSession(); - + DAGClient dagClient = tezClient.submitDAG(dag); DAGImpl dagImpl = (DAGImpl) mockApp.getContext().getCurrentDAG(); TezVertexID vertexId = dagImpl.getVertex(vNameSpec).getVertexId(); TezVertexID vertexIdNoSpec = dagImpl.getVertex(vNameNoSpec).getVertexId(); // original attempt is killed and speculative one is successful - TezTaskAttemptID killedTaId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), - 0); + TezTaskAttemptID killedTaId = + TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 0); TezTaskAttemptID noSpecTaId = TezTaskAttemptID .getInstance(TezTaskID.getInstance(vertexIdNoSpec, 0), 0); @@ -249,15 +252,23 @@ public void testBasicSpeculationPerVertexConf() throws Exception { mockLauncher.setStatusUpdatesForTask(noSpecTaId, 100); mockLauncher.startScheduling(true); - dagClient.waitForCompletion(); - Assert.assertEquals(DAGStatus.State.SUCCEEDED, dagClient.getDAGStatus(null).getState()); org.apache.tez.dag.app.dag.Vertex vSpec = dagImpl.getVertex(vertexId); org.apache.tez.dag.app.dag.Vertex vNoSpec = dagImpl.getVertex(vertexIdNoSpec); + // Wait enough time to give chance for the speculator to trigger + // speculation on VB. + // This would fail because of JUnit time out. + do { + Thread.sleep(100); + } while (vSpec.getAllCounters().findCounter(TaskCounter.NUM_SPECULATIONS) + .getValue() <= 0); + dagClient.waitForCompletion(); // speculation for vA but not for vB - Assert.assertTrue(vSpec.getAllCounters().findCounter(TaskCounter.NUM_SPECULATIONS) - .getValue() > 0); - Assert.assertEquals(0, vNoSpec.getAllCounters().findCounter(TaskCounter.NUM_SPECULATIONS) - .getValue()); + Assert.assertTrue("Num Speculations is not higher than 0", + vSpec.getAllCounters().findCounter(TaskCounter.NUM_SPECULATIONS) + .getValue() > 0); + Assert.assertEquals(0, + vNoSpec.getAllCounters().findCounter(TaskCounter.NUM_SPECULATIONS) + .getValue()); tezClient.stop(); } From c9358108b411c6d168943853bbec3fd7c356e770 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Fri, 13 Dec 2019 13:17:03 -0600 Subject: [PATCH 182/512] TEZ-4107. PreCommit-TEZ-Build fails - Docker failed to build yetus/tez Signed-off-by: Jonathan Eagles --- build-tools/docker/Dockerfile | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/build-tools/docker/Dockerfile b/build-tools/docker/Dockerfile index 8d76861b5a..1edd8ae5ee 100644 --- a/build-tools/docker/Dockerfile +++ b/build-tools/docker/Dockerfile @@ -146,9 +146,9 @@ ENV FINDBUGS_HOME /usr # Install shellcheck (0.4.6, the latest as of 2017-09-26) #### # hadolint ignore=DL3008 -RUN add-apt-repository -y ppa:jonathonf/ghc-8.0.2 \ +RUN add-apt-repository -y ppa:hvr/ghc \ && apt-get -q update \ - && apt-get -q install -y --no-install-recommends shellcheck \ + && apt-get -q install -y --no-install-recommends shellcheck ghc-8.0.2 \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* From 09c6f902fdab527ca537f6b727e70c1579f2a2cf Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Mon, 16 Dec 2019 10:40:10 -0600 Subject: [PATCH 183/512] TEZ-1869. Exclude tez-ui war / other dependencies from tez.tar.gz Signed-off-by: Jonathan Eagles --- tez-dist/src/main/assembly/tez-dist-minimal.xml | 1 + tez-dist/src/main/assembly/tez-dist.xml | 1 + 2 files changed, 2 insertions(+) diff --git a/tez-dist/src/main/assembly/tez-dist-minimal.xml b/tez-dist/src/main/assembly/tez-dist-minimal.xml index fbd1782273..35f9be76e8 100644 --- a/tez-dist/src/main/assembly/tez-dist-minimal.xml +++ b/tez-dist/src/main/assembly/tez-dist-minimal.xml @@ -25,6 +25,7 @@ org.apache.tez:tez-aux-services org.apache.tez:tez-ext-service-tests + org.apache.tez:tez-ui / diff --git a/tez-dist/src/main/assembly/tez-dist.xml b/tez-dist/src/main/assembly/tez-dist.xml index 9456664f77..665121aee1 100644 --- a/tez-dist/src/main/assembly/tez-dist.xml +++ b/tez-dist/src/main/assembly/tez-dist.xml @@ -24,6 +24,7 @@ true org.apache.tez:tez-aux-services + org.apache.tez:tez-ui / From 689d7388592196c320319f026961b3fef0167dc5 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Tue, 17 Dec 2019 09:06:05 -0600 Subject: [PATCH 184/512] TEZ-4108. NullPointerException during speculative execution race condition --- .../apache/tez/dag/app/dag/impl/TaskImpl.java | 4 +- .../tez/dag/app/dag/impl/TestTaskImpl.java | 39 +++++++++++++++++++ 2 files changed, 42 insertions(+), 1 deletion(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java index 2d0688f763..39e2b4c1f7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java @@ -1029,7 +1029,9 @@ public void transition(TaskImpl task, TaskEvent event) { // find the oldest running attempt if (!ta.isFinished()) { earliestUnfinishedAttempt = ta; - task.nodesWithRunningAttempts.add(ta.getNodeId()); + if (ta.getNodeId() != null) { + task.nodesWithRunningAttempts.add(ta.getNodeId()); + } } else { if (TaskAttemptState.SUCCEEDED.equals(ta.getState())) { LOG.info("Ignore speculation scheduling for task {} since it has succeeded with attempt {}.", diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java index a3de9365de..1af6092d1e 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java @@ -887,6 +887,45 @@ public void testFailedThenSpeculativeSucceeded() { assertEquals(2, mockTask.getAttemptList().size()); } + @Test + public void testKilledBeforeSpeculatedSucceeded() { + conf.setInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS, 1); + Vertex vertex = mock(Vertex.class); + doReturn(new VertexImpl.VertexConfigImpl(conf)).when(vertex).getVertexConfig(); + mockTask = new MockTaskImpl(vertexId, partition, + eventHandler, conf, taskCommunicatorManagerInterface, clock, + taskHeartbeatHandler, appContext, leafVertex, + taskResource, containerContext, vertex); + TezTaskID taskId = getNewTaskID(); + scheduleTaskAttempt(taskId); + MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); + launchTaskAttempt(firstAttempt.getID()); + updateAttemptState(firstAttempt, TaskAttemptState.RUNNING); + + mockTask.handle(createTaskTAKilledEvent(firstAttempt.getID())); + assertEquals(TaskStateInternal.RUNNING, mockTask.getInternalState()); + + // We need to manually override the current node id + // to induce NPE in the state machine transition + // simulating killed before speculated scenario + NodeId nodeId = mockNodeId; + mockNodeId = null; + + // Add a speculative task attempt + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + mockNodeId = nodeId; + MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); + launchTaskAttempt(specAttempt.getID()); + updateAttemptState(specAttempt, TaskAttemptState.RUNNING); + assertEquals(3, mockTask.getAttemptList().size()); + + // Now succeed the speculative attempt + updateAttemptState(specAttempt, TaskAttemptState.SUCCEEDED); + mockTask.handle(createTaskTASucceededEvent(specAttempt.getID())); + assertEquals(TaskState.SUCCEEDED, mockTask.getState()); + assertEquals(3, mockTask.getAttemptList().size()); + } + @Test(timeout = 20000) public void testKilledAttemptUpdatesDAGScheduler() { TezTaskID taskId = getNewTaskID(); From a6219ede1c0eaaec97c69e8b74ba7fe0458d5f0f Mon Sep 17 00:00:00 2001 From: Ahmed Hussein Date: Tue, 17 Dec 2019 11:37:48 -0600 Subject: [PATCH 185/512] TEZ-4103. Progress in DAG, Vertex, and tasks is incorrect Signed-off-by: Jonathan Eagles --- .../org/apache/tez/common/ProgressHelper.java | 165 +++++++++++++----- .../tez/runtime/api/ProcessorContext.java | 22 ++- .../apache/tez/dag/app/dag/impl/DAGImpl.java | 28 ++- .../tez/dag/app/dag/impl/VertexImpl.java | 36 ++-- .../speculation/legacy/LegacySpeculator.java | 7 + .../apache/tez/dag/app/TestDAGAppMaster.java | 10 +- .../mapreduce/processor/MRTaskReporter.java | 4 + .../api/impl/TezProcessorContextImpl.java | 4 +- 8 files changed, 209 insertions(+), 67 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/ProgressHelper.java b/tez-api/src/main/java/org/apache/tez/common/ProgressHelper.java index 07b066c2ea..1518ccdaf1 100644 --- a/tez-api/src/main/java/org/apache/tez/common/ProgressHelper.java +++ b/tez-api/src/main/java/org/apache/tez/common/ProgressHelper.java @@ -19,74 +19,155 @@ package org.apache.tez.common; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.atomic.AtomicReference; import org.apache.tez.runtime.api.AbstractLogicalInput; import org.apache.tez.runtime.api.LogicalInput; import org.apache.tez.runtime.api.ProcessorContext; -import org.apache.tez.runtime.api.ProgressFailedException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import java.util.Map; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; public class ProgressHelper { - private static final Logger LOG = LoggerFactory.getLogger(ProgressHelper.class); - private String processorName; + private static final Logger LOG = + LoggerFactory.getLogger(ProgressHelper.class); + private static final float MIN_PROGRESS_VAL = 0.0f; + private static final float MAX_PROGRESS_VAL = 1.0f; + private final String processorName; protected final Map inputs; - final ProcessorContext processorContext; + private final ProcessorContext processorContext; + private final AtomicReference> periodicMonitorTaskRef; + private long monitorExecPeriod; + private volatile ScheduledExecutorService scheduledExecutorService; - volatile ScheduledExecutorService scheduledExecutorService; - Runnable monitorProgress = new Runnable() { - @Override - public void run() { - try { - float progSum = 0.0f; - float progress; - if (inputs != null && inputs.size() != 0) { - for (LogicalInput input : inputs.values()) { - if (input instanceof AbstractLogicalInput) { - float inputProgress = ((AbstractLogicalInput) input).getProgress(); - if (inputProgress >= 0.0f && inputProgress <= 1.0f) { - progSum += inputProgress; + public static final float processProgress(float val) { + return (Float.isNaN(val)) ? MIN_PROGRESS_VAL + : Math.max(MIN_PROGRESS_VAL, Math.min(MAX_PROGRESS_VAL, val)); + } + + public static final boolean isProgressWithinRange(float val) { + return (val <= MAX_PROGRESS_VAL && val >= MIN_PROGRESS_VAL); + } + + public ProgressHelper(Map inputsParam, + ProcessorContext context, String processorName) { + this.periodicMonitorTaskRef = new AtomicReference<>(null); + this.inputs = inputsParam; + this.processorContext = context; + this.processorName = processorName; + } + + public void scheduleProgressTaskService(long delay, long period) { + monitorExecPeriod = period; + scheduledExecutorService = + Executors.newScheduledThreadPool(1, + new ThreadFactoryBuilder().setDaemon(true).setNameFormat( + "TaskProgressService{" + processorName + ":" + processorContext + .getTaskVertexName() + + "} #%d").build()); + try { + createPeriodicTask(delay); + } catch (RejectedExecutionException | IllegalArgumentException ex) { + LOG.error("Could not create periodic scheduled task for processor={}", + processorName, ex); + } + } + + private Runnable createRunnableMonitor() { + return new Runnable() { + @Override + public void run() { + try { + float progSum = MIN_PROGRESS_VAL; + int invalidInput = 0; + float progressVal = MIN_PROGRESS_VAL; + if (inputs != null && !inputs.isEmpty()) { + for (LogicalInput input : inputs.values()) { + if (!(input instanceof AbstractLogicalInput)) { + /** + * According to javdoc in + * {@link org.apache.tez.runtime.api.AbstractLogicalInput} all + * implementations must extend AbstractLogicalInput. + */ + continue; } + final float inputProgress = + ((AbstractLogicalInput) input).getProgress(); + if (!isProgressWithinRange(inputProgress)) { + final int invalidSnapshot = ++invalidInput; + if (LOG.isDebugEnabled()) { + LOG.debug( + "progress update: Incorrect value in progress helper in " + + "processor={}, inputProgress={}, inputsSize={}, " + + "invalidInput={}", + processorName, inputProgress, inputs.size(), + invalidSnapshot); + } + } + progSum += processProgress(inputProgress); } + // No need to process the average within the valid range since the + // processorContext validates the value before being set. + progressVal = progSum / inputs.size(); + } + // Report progress as 0.0f when if are errors. + processorContext.setProgress(progressVal); + } catch (Throwable th) { + if (LOG.isDebugEnabled()) { + LOG.debug("progress update: Encountered InterruptedException during" + + " Processor={}", processorName, th); + } + if (th instanceof InterruptedException) { + // set interrupt flag to true sand exit + Thread.currentThread().interrupt(); + return; } - progress = (1.0f) * progSum / inputs.size(); - } else { - progress = 1.0f; } - processorContext.setProgress(progress); - } catch (ProgressFailedException pe) { - LOG.warn("Encountered ProgressFailedException during Processor progress update" - + pe); - } catch (InterruptedException ie) { - LOG.warn("Encountered InterruptedException during Processor progress update" - + ie); } - } - }; + }; + } - public ProgressHelper(Map _inputs, ProcessorContext context, String processorName) { - this.inputs = _inputs; - this.processorContext = context; - this.processorName = processorName; + private boolean createPeriodicTask(long delay) + throws RejectedExecutionException, IllegalArgumentException { + stopPeriodicMonitor(); + final Runnable runnableMonitor = createRunnableMonitor(); + ScheduledFuture futureTask = scheduledExecutorService + .scheduleWithFixedDelay(runnableMonitor, delay, monitorExecPeriod, + TimeUnit.MILLISECONDS); + periodicMonitorTaskRef.set(futureTask); + return true; } - public void scheduleProgressTaskService(long delay, long period) { - scheduledExecutorService = Executors.newScheduledThreadPool(1, new ThreadFactoryBuilder() - .setDaemon(true).setNameFormat("TaskProgressService{" + processorName+ ":" + processorContext.getTaskVertexName() - + "} #%d").build()); - scheduledExecutorService.scheduleWithFixedDelay(monitorProgress, delay, period, - TimeUnit.MILLISECONDS); + private void stopPeriodicMonitor() { + ScheduledFuture scheduledMonitorRes = + this.periodicMonitorTaskRef.get(); + if (scheduledMonitorRes != null && !scheduledMonitorRes.isCancelled()) { + scheduledMonitorRes.cancel(true); + this.periodicMonitorTaskRef.set(null); + } } public void shutDownProgressTaskService() { + stopPeriodicMonitor(); if (scheduledExecutorService != null) { + scheduledExecutorService.shutdown(); + try { + if (!scheduledExecutorService.awaitTermination(monitorExecPeriod, + TimeUnit.MILLISECONDS)) { + scheduledExecutorService.shutdownNow(); + } + } catch (InterruptedException e) { + if (LOG.isDebugEnabled()) { + LOG.debug("Interrupted exception while shutting down the " + + "executor service for the processor name={}", processorName); + } + } scheduledExecutorService.shutdownNow(); - scheduledExecutorService = null; } + scheduledExecutorService = null; } - } diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/ProcessorContext.java b/tez-api/src/main/java/org/apache/tez/runtime/api/ProcessorContext.java index acb2a57da7..3782a8d8d7 100644 --- a/tez-api/src/main/java/org/apache/tez/runtime/api/ProcessorContext.java +++ b/tez-api/src/main/java/org/apache/tez/runtime/api/ProcessorContext.java @@ -22,6 +22,7 @@ import java.util.Collection; import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.tez.common.ProgressHelper; /** * Context handle for the Processor to initialize itself. @@ -30,13 +31,32 @@ @Public public interface ProcessorContext extends TaskContext { + /** + * validate that progress is the valid range. + * @param progress + * @return the processed value of the progress that is guaranteed to be within + * the valid range. + */ + static float preProcessProgress(float progress) { + return ProgressHelper.processProgress(progress); + } + /** * Set the overall progress of this Task Attempt. * This automatically results in invocation of {@link ProcessorContext#notifyProgress()} * and so invoking that separately is not required. * @param progress Progress in the range from [0.0 - 1.0f] */ - public void setProgress(float progress); + default void setProgress(float progress) { + setProgressInternally(preProcessProgress(progress)); + } + + /** + * The actual implementation of the taskAttempt progress. + * All implementations needs to override this method + * @param progress + */ + void setProgressInternally(float progress); /** * Check whether this attempt can commit its output diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java index db51ceea95..18b7128e1d 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java @@ -43,6 +43,7 @@ import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.tez.Utils; +import org.apache.tez.common.ProgressHelper; import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.common.counters.LimitExceededException; import org.apache.tez.dag.app.dag.event.DAGEventTerminateDag; @@ -804,19 +805,30 @@ public DAGReport getReport() { public float getProgress() { this.readLock.lock(); try { - float progress = 0.0f; + float accProg = 0.0f; + float dagProgress = 0.0f; + int verticesCount = getVertices().size(); for (Vertex v : getVertices().values()) { float vertexProgress = v.getProgress(); - if (vertexProgress >= 0.0f && vertexProgress <= 1.0f) { - progress += vertexProgress; + if (LOG.isDebugEnabled()) { + if (!ProgressHelper.isProgressWithinRange(vertexProgress)) { + LOG.debug("progress update: Vertex progress is invalid range" + + "; v={}, progress={}", v.getName(), vertexProgress); + } } + accProg += ProgressHelper.processProgress(vertexProgress); } - float dagProgress = progress / getTotalVertices(); - if (dagProgress >= 0.0f && progress <= 1.0f) { - return dagProgress; - } else { - return 0.0f; + if (LOG.isDebugEnabled()) { + if (verticesCount == 0) { + LOG.debug("progress update: DAGImpl getProgress() returns 0.0f: " + + "vertices count is 0"); + } + } + if (verticesCount > 0) { + dagProgress = + ProgressHelper.processProgress(accProg / verticesCount); } + return dagProgress; } finally { this.readLock.unlock(); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index 9a59e884f4..52fe9325f3 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -64,6 +64,7 @@ import org.apache.hadoop.yarn.util.Clock; import org.apache.tez.client.TezClientUtils; import org.apache.tez.common.ATSConstants; +import org.apache.tez.common.ProgressHelper; import org.apache.tez.common.ReflectionUtils; import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.common.counters.AggregateTezCounters; @@ -1572,20 +1573,31 @@ public TaskLocationHint getTaskLocationHint(TezTaskID taskId) { List getOnDemandRouteEvents() { return onDemandRouteEvents; } - + + /** + * Updates the progress value in the vertex. + * This should be called only when the vertex is running state. + * No need to acquire the lock since this is nested inside + * {@link #getProgress() getProgress} method. + */ private void computeProgress() { - this.readLock.lock(); - try { - float progress = 0f; - for (Task task : this.tasks.values()) { - progress += (task.getProgress()); - } - if (this.numTasks != 0) { - progress /= this.numTasks; + + float accProg = 0.0f; + int tasksCount = this.tasks.size(); + for (Task task : this.tasks.values()) { + float taskProg = task.getProgress(); + if (LOG.isDebugEnabled()) { + if (!ProgressHelper.isProgressWithinRange(taskProg)) { + LOG.debug("progress update: vertex={}, task={} incorrect; range={}", + getName(), task.getTaskId().toString(), taskProg); + } } - this.progress = progress; - } finally { - this.readLock.unlock(); + accProg += ProgressHelper.processProgress(taskProg); + } + // tasksCount is 0, do not reset the current progress. + if (tasksCount > 0) { + // force the progress to be below within the range + progress = ProgressHelper.processProgress(accProg / tasksCount); } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java index 0a0e9a20dd..23b057af2b 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java @@ -34,6 +34,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.hadoop.service.AbstractService; import org.apache.hadoop.service.ServiceOperations; +import org.apache.tez.common.ProgressHelper; import org.apache.tez.dag.api.TezConfiguration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -529,6 +530,12 @@ public void setEstimatedRunTime(long estimatedRunTime) { } public void setProgress(float progress) { + if (LOG.isDebugEnabled()) { + if (!ProgressHelper.isProgressWithinRange(progress)) { + LOG.debug("Progress update: speculator received progress in invalid " + + "range={}", progress); + } + } this.progress = progress; } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java index 7a7dfe2b04..92e43aab20 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java @@ -465,9 +465,15 @@ public void testBadProgress() throws Exception { when(mockVertex.getProgress()).thenReturn(-10f); Assert.assertEquals("Progress was negative and should be reported as 0", 0, am.getProgress(), 0); + when(mockVertex.getProgress()).thenReturn(1.0000567f); + Assert.assertEquals( + "Progress was greater than 1 by a small float precision " + + "1.0000567 and should be reported as 1", + 1.0f, am.getProgress(), 0.0f); when(mockVertex.getProgress()).thenReturn(10f); - Assert.assertEquals("Progress was greater than 1 and should be reported as 0", - 0, am.getProgress(), 0); + Assert.assertEquals( + "Progress was greater than 1 and should be reported as 1", + 1.0f, am.getProgress(), 0.0f); } @SuppressWarnings("deprecation") diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java index 2fa75bf6f4..e3fdc27a93 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java @@ -23,6 +23,7 @@ import org.apache.hadoop.mapred.Counters; import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.Reporter; +import org.apache.tez.common.ProgressHelper; import org.apache.tez.common.counters.TezCounter; import org.apache.tez.mapreduce.hadoop.mapred.MRCounters; import org.apache.tez.mapreduce.hadoop.mapred.MRReporter; @@ -62,6 +63,9 @@ public MRTaskReporter(InputContext context) { } public void setProgress(float progress) { + // Validate that the progress is within the valid range. This guarantees + // that reporter and processorContext gets the same value. + progress = ProgressHelper.processProgress(progress); reporter.setProgress(progress); if (isProcessorContext) { ((ProcessorContext)context).setProgress(progress); diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java index beae693053..54605c8645 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java @@ -93,8 +93,8 @@ public UserPayload getUserPayload() { } @Override - public void setProgress(float progress) { - if (Math.abs(progress - runtimeTask.getProgress()) >= 0.001f) { + public void setProgressInternally(float progress) { + if (Float.compare(progress, runtimeTask.getProgress()) != 0) { runtimeTask.setProgress(progress); notifyProgress(); } From 271351ba29382da08a24adb2ba2327cc68b95efb Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Mon, 6 Jan 2020 11:41:57 +0530 Subject: [PATCH 186/512] =?UTF-8?q?TEZ-4098:=20tez-tools=20improvements:?= =?UTF-8?q?=20log-split,=20swimlane=20(L=C3=A1szl=C3=B3=20Bodor,=20reviewe?= =?UTF-8?q?d=20by=20rbalamohan)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- tez-tools/swimlanes/swimlane.py | 1 + tez-tools/swimlanes/yarn-swimlanes.sh | 17 ++- tez-tools/tez-log-split/README.md | 77 ++++++++++++++ tez-tools/tez-log-split/logsplit.py | 111 ++++++++++++++++++++ tez-tools/tez-log-split/tez-log-splitter.sh | 35 ++++++ 5 files changed, 236 insertions(+), 5 deletions(-) create mode 100644 tez-tools/tez-log-split/README.md create mode 100644 tez-tools/tez-log-split/logsplit.py create mode 100644 tez-tools/tez-log-split/tez-log-splitter.sh diff --git a/tez-tools/swimlanes/swimlane.py b/tez-tools/swimlanes/swimlane.py index bbd54df46d..11976daab9 100644 --- a/tez-tools/swimlanes/swimlane.py +++ b/tez-tools/swimlanes/swimlane.py @@ -195,6 +195,7 @@ def main(argv): svg.text(marginRight+xdomain(percentX), y+marginTop+12, "%d%% (%0.1fs)" % (int(fraction*100), (percentX - dag.start)/1000.0), style="font-size:12px; text-anchor: middle") out.write(svg.flush()) out.close() + print("Output svg is written into: " + str(out)) if __name__ == "__main__": sys.exit(main(sys.argv[1:])) diff --git a/tez-tools/swimlanes/yarn-swimlanes.sh b/tez-tools/swimlanes/yarn-swimlanes.sh index df4d071a7c..02465b0129 100644 --- a/tez-tools/swimlanes/yarn-swimlanes.sh +++ b/tez-tools/swimlanes/yarn-swimlanes.sh @@ -19,10 +19,17 @@ set -e APPID=$1 - -YARN=$(which yarn); TMP=$(mktemp) +DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" + +if [[ -f $APPID ]]; then + echo "Reading yarn logs from local file: $APPID" + cat "$APPID" | grep HISTORY > "$TMP" +else + YARN=$(which yarn); + echo "Fetching yarn logs for $APPID" + $YARN logs -applicationId "$APPID" | grep HISTORY > "$TMP" +fi +echo "History was written into $TMP" -echo "Fetching yarn logs for $APPID" -$YARN logs -applicationId $APPID | grep HISTORY > $TMP -python swimlane.py -o $APPID.svg $TMP +python "$DIR/swimlane.py" -o "$APPID.svg" "$TMP" \ No newline at end of file diff --git a/tez-tools/tez-log-split/README.md b/tez-tools/tez-log-split/README.md new file mode 100644 index 0000000000..a7341a757d --- /dev/null +++ b/tez-tools/tez-log-split/README.md @@ -0,0 +1,77 @@ + + +Tez log splitter +========= + +This is a post-hoc analysis tool for Apache Tez which splits +an aggregated yarn log file to separate files into a hierarchical folder structure. + +``` +. +├── vc0525.your.domain.com_8041 +│ └── container_e10_1575565459633_0004_01_000001 +│ ├── container-localizer-syslog +│ ├── dag_1575565459633_0004_1-tez-dag.pb.txt +│ ├── dag_1575565459633_0004_1.dot +│ ├── prelaunch.err +│ ├── prelaunch.out +│ ├── stderr +│ ├── stdout +│ ├── syslog +│ ├── syslog_dag_1575565459633_0004_1 +│ └── syslog_dag_1575565459633_0004_1_post +├── vc0526.your.domain.com_8041 +│ └── container_e10_1575565459633_0004_01_000004 +│ ├── container-localizer-syslog +│ ├── prelaunch.err +│ ├── prelaunch.out +│ ├── stderr +│ ├── stdout +│ ├── syslog +│ └── syslog_attempt_1575565459633_0004_1_00_000000_2 +├── vc0528.your.domain.com_8041 +│ └── container_e10_1575565459633_0004_01_000002 +│ ├── container-localizer-syslog +│ ├── prelaunch.err +│ ├── prelaunch.out +│ ├── stderr +│ ├── stdout +│ ├── syslog +│ └── syslog_attempt_1575565459633_0004_1_00_000000_0 +├── vc0529.your.domain.com_8041 +│ └── container_e10_1575565459633_0004_01_000005 +│ ├── container-localizer-syslog +│ ├── prelaunch.err +│ ├── prelaunch.out +│ ├── stderr +│ ├── stdout +│ ├── syslog +│ └── syslog_attempt_1575565459633_0004_1_00_000000_3 +└── vc0536.your.domain.com_8041 + └── container_e10_1575565459633_0004_01_000003 + ├── container-localizer-syslog + ├── prelaunch.err + ├── prelaunch.out + ├── stderr + ├── stdout + ├── syslog + └── syslog_attempt_1575565459633_0004_1_00_000000_1 +``` + +To use the tool, run e.g. + +`tez-log-splitter.sh application_1576254620247_0010` (app log is fetched from yarn) +`tez-log-splitter.sh ~/path/to/application_1576254620247_0010.log` (...when app log is already on your computer) +`tez-log-splitter.sh ~/path/to/application_1576254620247_0010.log.gz` (...when app log is already on your computer in gz) diff --git a/tez-tools/tez-log-split/logsplit.py b/tez-tools/tez-log-split/logsplit.py new file mode 100644 index 0000000000..47e17da11b --- /dev/null +++ b/tez-tools/tez-log-split/logsplit.py @@ -0,0 +1,111 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +import sys +import os +import re +from gzip import GzipFile as GZFile +from getopt import getopt + +def usage(): + sys.stderr.write(""" +usage: logsplit.py + +Input files for this tool can be prepared by "yarn logs -applicationId ". +""") + +def open_file(f): + if f.endswith(".gz"): + return GZFile(f) + return open(f) + +class AggregatedLog(object): + def __init__(self): + self.in_container = False + self.in_logfile = False + self.current_container_header = None + self.current_container_name = None + self.current_host_name = None # as read from log line: "hello.my.host.com_8041" + self.current_file = None + self.HEADER_CONTAINER_RE = re.compile("Container: (container_[a-z0-9_]+) on (.*)") + self.HEADER_LAST_ROW_RE = re.compile("^LogContents:$") + self.HEADER_LOG_TYPE_RE = re.compile("^LogType:(.*)") + self.LAST_LOG_LINE_RE = re.compile("^End of LogType:.*") + + def process(self, input_file): + self.output_folder = input_file.name + "_splitlogs" + os.mkdir(self.output_folder) + + for line in input_file: + self.parse(line) + + def parse(self, line): + if self.in_container: + if self.in_logfile: + m = self.LAST_LOG_LINE_RE.match(line) + if m: + self.in_container = False + self.in_logfile = False + self.current_file.close() + else: + self.write_to_current_file(line) + else: + m = self.HEADER_LOG_TYPE_RE.match(line) + if m: + file_name = m.group(1) + self.create_file_in_current_container(file_name) + elif self.HEADER_LAST_ROW_RE.match(line): + self.in_logfile = True + self.write_to_current_file(self.current_container_header) #for host reference + else: + m = self.HEADER_CONTAINER_RE.match(line) + self.current_container_header = line + if m: + self.in_container = True + self.current_container_name = m.group(1) + self.current_host_name = m.group(2) + self.start_container_folder() + + def start_container_folder(self): + container_dir = os.path.join(self.output_folder, self.get_current_container_dir_name()) + if not os.path.exists(container_dir): + os.makedirs(container_dir) + + def create_file_in_current_container(self, file_name): + file_to_be_created = os.path.join(self.output_folder, self.get_current_container_dir_name(), file_name) + file = open(file_to_be_created, "w+") + self.current_file = file + + def write_to_current_file(self, line): + self.current_file.write(line) + + def get_current_container_dir_name(self): + return os.path.join(self.current_host_name, self.current_container_name) + +def main(argv): + (opts, args) = getopt(argv, "") + input_file = args[0] + fp = open_file(input_file) + aggregated_log = AggregatedLog() + aggregated_log.process(fp) + print ("Split application logs was written into folder " + aggregated_log.output_folder) + fp.close() + +if __name__ == "__main__": + sys.exit(main(sys.argv[1:])) diff --git a/tez-tools/tez-log-split/tez-log-splitter.sh b/tez-tools/tez-log-split/tez-log-splitter.sh new file mode 100644 index 0000000000..712e499a4f --- /dev/null +++ b/tez-tools/tez-log-split/tez-log-splitter.sh @@ -0,0 +1,35 @@ +#!/bin/bash + +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + + +#set -e + +APPID=$1 +TMP=$(mktemp) +DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" + +if [[ -f $APPID ]]; then + echo "Reading yarn logs from local file: $APPID" + TMP=$APPID +else + YARN=$(which yarn); + echo "Fetching yarn logs for $APPID" + $YARN logs -applicationId "$APPID" > "$TMP" + echo "Application log was written into $TMP" +fi + +python "$DIR/logsplit.py" "$TMP" \ No newline at end of file From bf4942b49a3d9c0d326f64715b3e76952b77382d Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Mon, 20 Jan 2020 14:36:43 +0530 Subject: [PATCH 187/512] TEZ-4115: turn on data-via-events as default (Contributed by Richard Zhang, reviewed by rbalamohan) --- .../apache/tez/runtime/library/api/TezRuntimeConfiguration.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java index 9c2f7c3dcf..64e6a71d7e 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java @@ -488,7 +488,7 @@ public class TezRuntimeConfiguration { public static final String TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_ENABLED = TEZ_RUNTIME_PREFIX + "transfer.data-via-events.enabled"; @Private - public static final boolean TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_ENABLED_DEFAULT = false; + public static final boolean TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_ENABLED_DEFAULT = true; @Private public static final String TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_MAX_SIZE = From 5b81017a4d11c09cfa0180e3d56e22c22744559b Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Wed, 22 Jan 2020 03:55:46 +0530 Subject: [PATCH 188/512] TEZ-4113: TezUtils::createByteStringFromConf should use snappy instead of DeflaterOutputStream (rbalamohan, reviewed by Ashutosh Chauhan) --- .../main/java/org/apache/tez/common/TezUtils.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java index 072c02f97b..192d2a0e19 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java @@ -25,9 +25,6 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.zip.Deflater; -import java.util.zip.DeflaterOutputStream; -import java.util.zip.InflaterInputStream; import com.google.common.base.Preconditions; import com.google.protobuf.ByteString; @@ -42,6 +39,8 @@ import org.apache.tez.dag.api.records.DAGProtos; import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; +import org.xerial.snappy.SnappyInputStream; +import org.xerial.snappy.SnappyOutputStream; /** * Utility methods for setting up a DAG. Has helpers for setting up log4j configuration, converting @@ -77,8 +76,7 @@ public static void addLog4jSystemProperties(String logLevel, public static ByteString createByteStringFromConf(Configuration conf) throws IOException { Preconditions.checkNotNull(conf, "Configuration must be specified"); ByteString.Output os = ByteString.newOutput(); - DeflaterOutputStream compressOs = new DeflaterOutputStream(os, - new Deflater(Deflater.BEST_SPEED)); + SnappyOutputStream compressOs = new SnappyOutputStream(os); try { writeConfInPB(compressOs, conf); } finally { @@ -111,9 +109,7 @@ public static UserPayload createUserPayloadFromConf(Configuration conf) throws I */ public static Configuration createConfFromByteString(ByteString byteString) throws IOException { Preconditions.checkNotNull(byteString, "ByteString must be specified"); - // SnappyInputStream uncompressIs = new - // SnappyInputStream(byteString.newInput()); - try(InflaterInputStream uncompressIs = new InflaterInputStream(byteString.newInput())) { + try(SnappyInputStream uncompressIs = new SnappyInputStream(byteString.newInput());) { DAGProtos.ConfigurationProto confProto = DAGProtos.ConfigurationProto.parseFrom(uncompressIs); Configuration conf = new Configuration(false); readConfFromPB(confProto, conf); From b27e5ddfe40fab0fc493e93dd43a22c9d9f3782c Mon Sep 17 00:00:00 2001 From: Ahmed Hussein Date: Wed, 22 Jan 2020 16:51:13 -0600 Subject: [PATCH 189/512] TEZ-3391. Optimize single split MR split reader Signed-off-by: Jonathan Eagles --- .../split/SplitMetaInfoReaderTez.java | 115 +++++++++++++----- .../apache/tez/mapreduce/input/MRInput.java | 4 +- .../tez/mapreduce/lib/MRInputUtils.java | 9 +- 3 files changed, 92 insertions(+), 36 deletions(-) diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/SplitMetaInfoReaderTez.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/SplitMetaInfoReaderTez.java index 0d703e073b..db156d2ce2 100644 --- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/SplitMetaInfoReaderTez.java +++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/SplitMetaInfoReaderTez.java @@ -48,24 +48,19 @@ public class SplitMetaInfoReaderTez { public static final int META_SPLIT_VERSION = JobSplit.META_SPLIT_VERSION; public static final byte[] META_SPLIT_FILE_HEADER = JobSplit.META_SPLIT_FILE_HEADER; - - // Forked from the MR variant so that the metaInfo file as well as the split - // file can be read from local fs - relying on these files being localized. - public static TaskSplitMetaInfo[] readSplitMetaInfo(Configuration conf, + private static FSDataInputStream getFSDataIS(Configuration conf, FileSystem fs) throws IOException { - long maxMetaInfoSize = conf.getLong( MRJobConfig.SPLIT_METAINFO_MAXSIZE, MRJobConfig.DEFAULT_SPLIT_METAINFO_MAXSIZE); - + FSDataInputStream in = null; // TODO NEWTEZ Figure out how this can be improved. i.e. access from context instead of setting in conf ? String basePath = conf.get(MRFrameworkConfigs.TASK_LOCAL_RESOURCE_DIR, "."); LOG.info("Attempting to find splits in dir: " + basePath); - + Path metaSplitFile = new Path( basePath, MRJobConfig.JOB_SPLIT_METAINFO); - String jobSplitFile = MRJobConfig.JOB_SPLIT; File file = new File(metaSplitFile.toUri().getPath()).getAbsoluteFile(); if (LOG.isDebugEnabled()) { @@ -74,34 +69,96 @@ public static TaskSplitMetaInfo[] readSplitMetaInfo(Configuration conf, + FileSystem.getDefaultUri(conf)); } - FileStatus fStatus = fs.getFileStatus(metaSplitFile); - if (maxMetaInfoSize > 0 && fStatus.getLen() > maxMetaInfoSize) { - throw new IOException("Split metadata size exceeded " + maxMetaInfoSize - + ". Aborting job "); - } - FSDataInputStream in = fs.open(metaSplitFile); - byte[] header = new byte[JobSplit.META_SPLIT_FILE_HEADER.length]; - in.readFully(header); - if (!Arrays.equals(JobSplit.META_SPLIT_FILE_HEADER, header)) { - throw new IOException("Invalid header on split file"); + FileStatus fStatus = null; + try { + fStatus = fs.getFileStatus(metaSplitFile); + if (maxMetaInfoSize > 0 && fStatus.getLen() > maxMetaInfoSize) { + throw new IOException("Split metadata size exceeded " + maxMetaInfoSize + + ". Aborting job "); + } + in = fs.open(metaSplitFile); + byte[] header = new byte[JobSplit.META_SPLIT_FILE_HEADER.length]; + in.readFully(header); + if (!Arrays.equals(JobSplit.META_SPLIT_FILE_HEADER, header)) { + throw new IOException("Invalid header on split file"); + } + int vers = WritableUtils.readVInt(in); + if (vers != JobSplit.META_SPLIT_VERSION) { + throw new IOException("Unsupported split version " + vers); + } + } catch (IOException e) { + if (in != null) { + in.close(); + } + throw e; } - int vers = WritableUtils.readVInt(in); - if (vers != JobSplit.META_SPLIT_VERSION) { - in.close(); - throw new IOException("Unsupported split version " + vers); + return in; + } + + // Forked from the MR variant so that the metaInfo file as well as the split + // file can be read from local fs - relying on these files being localized. + public static TaskSplitMetaInfo[] readSplitMetaInfo(Configuration conf, + FileSystem fs) throws IOException { + FSDataInputStream in = null; + try { + in = getFSDataIS(conf, fs); + final String jobSplitFile = MRJobConfig.JOB_SPLIT; + final String basePath = conf.get(MRFrameworkConfigs.TASK_LOCAL_RESOURCE_DIR, "."); + int numSplits = WritableUtils.readVInt(in); // TODO: check for insane values + JobSplit.TaskSplitMetaInfo[] allSplitMetaInfo = new JobSplit.TaskSplitMetaInfo[numSplits]; + for (int i = 0; i < numSplits; i++) { + JobSplit.SplitMetaInfo splitMetaInfo = new JobSplit.SplitMetaInfo(); + splitMetaInfo.readFields(in); + JobSplit.TaskSplitIndex splitIndex = new JobSplit.TaskSplitIndex( + new Path(basePath, jobSplitFile) + .toUri().toString(), splitMetaInfo.getStartOffset()); + allSplitMetaInfo[i] = new JobSplit.TaskSplitMetaInfo(splitIndex, + splitMetaInfo.getLocations(), splitMetaInfo.getInputDataLength()); + } + return allSplitMetaInfo; + } finally { + if (in != null) { + in.close(); + } } - int numSplits = WritableUtils.readVInt(in); // TODO: check for insane values - JobSplit.TaskSplitMetaInfo[] allSplitMetaInfo = new JobSplit.TaskSplitMetaInfo[numSplits]; - for (int i = 0; i < numSplits; i++) { + } + + /** + * Get the split meta info for the task with a specific index. This method + * reduces the overhead of creating meta objects below the index of the task. + * + * @param conf job configuration. + * @param fs FileSystem. + * @param index the index of the task. + * @return split meta info object of the task. + * @throws IOException + */ + public static TaskSplitMetaInfo getSplitMetaInfo(Configuration conf, + FileSystem fs, int index) throws IOException { + FSDataInputStream in = null; + try { + in = getFSDataIS(conf, fs); + final String jobSplitFile = MRJobConfig.JOB_SPLIT; + final String basePath = + conf.get(MRFrameworkConfigs.TASK_LOCAL_RESOURCE_DIR, "."); + final int numSplits = WritableUtils.readVInt(in); // TODO: check for insane values + if (numSplits <= index) { + throw new IOException("Index is larger than the number of splits"); + } JobSplit.SplitMetaInfo splitMetaInfo = new JobSplit.SplitMetaInfo(); - splitMetaInfo.readFields(in); + int iter = 0; + while (iter++ <= index) { + splitMetaInfo.readFields(in); + } JobSplit.TaskSplitIndex splitIndex = new JobSplit.TaskSplitIndex( new Path(basePath, jobSplitFile) .toUri().toString(), splitMetaInfo.getStartOffset()); - allSplitMetaInfo[i] = new JobSplit.TaskSplitMetaInfo(splitIndex, + return new JobSplit.TaskSplitMetaInfo(splitIndex, splitMetaInfo.getLocations(), splitMetaInfo.getInputDataLength()); + } finally { + if (in != null) { + in.close(); + } } - in.close(); - return allSplitMetaInfo; } } diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java index 248a92ac4b..317f6ebe55 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java @@ -490,8 +490,8 @@ void initializeInternal() throws IOException { getContext()); } } else { - TaskSplitMetaInfo[] allMetaInfo = MRInputUtils.readSplits(jobConf); - TaskSplitMetaInfo thisTaskMetaInfo = allMetaInfo[getContext().getTaskIndex()]; + TaskSplitMetaInfo thisTaskMetaInfo = MRInputUtils.getSplits(jobConf, + getContext().getTaskIndex()); TaskSplitIndex splitMetaInfo = new TaskSplitIndex(thisTaskMetaInfo.getSplitLocation(), thisTaskMetaInfo.getStartOffset()); long splitLength = -1; diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRInputUtils.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRInputUtils.java index bc96e388da..a2b87e04c9 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRInputUtils.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRInputUtils.java @@ -47,11 +47,10 @@ public class MRInputUtils { private static final Logger LOG = LoggerFactory.getLogger(MRInputUtils.class); - public static TaskSplitMetaInfo[] readSplits(Configuration conf) throws IOException { - TaskSplitMetaInfo[] allTaskSplitMetaInfo; - allTaskSplitMetaInfo = SplitMetaInfoReaderTez - .readSplitMetaInfo(conf, FileSystem.getLocal(conf)); - return allTaskSplitMetaInfo; + public static TaskSplitMetaInfo getSplits(Configuration conf, int index) throws IOException { + TaskSplitMetaInfo taskSplitMInfo = SplitMetaInfoReaderTez + .getSplitMetaInfo(conf, FileSystem.getLocal(conf), index); + return taskSplitMInfo; } public static org.apache.hadoop.mapreduce.InputSplit getNewSplitDetailsFromEvent( From 8b5d40fe831a33c05464c84d40a0f622cfd64d35 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Thu, 23 Jan 2020 11:21:21 +0100 Subject: [PATCH 190/512] =?UTF-8?q?TEZ-4102:=20Let=20session=20credentials?= =?UTF-8?q?=20be=20merged=20before=20merging=20am=20launch=20credentials?= =?UTF-8?q?=20(L=C3=A1szl=C3=B3=20Bodor=20reviewed=20by=20Ashutosh=20Chauh?= =?UTF-8?q?an)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../org/apache/tez/client/TezClientUtils.java | 34 ++++++++++++------- .../apache/tez/client/TestTezClientUtils.java | 26 ++++++++++++++ 2 files changed, 47 insertions(+), 13 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index 2b21024be1..2aeabe451a 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -470,19 +470,8 @@ public static ApplicationSubmissionContext createApplicationSubmissionContext( // Setup required Credentials for the AM launch. DAG specific credentials // are handled separately. ByteBuffer securityTokens = null; - // Setup security tokens - Credentials amLaunchCredentials = new Credentials(); - if (amConfig.getCredentials() != null) { - amLaunchCredentials.addAll(amConfig.getCredentials()); - } - - // Add Staging dir creds to the list of session credentials. - TokenCache.obtainTokensForFileSystems(sessionCreds, new Path[]{binaryConfPath}, conf); - - populateTokenCache(conf, sessionCreds); - - // Add session specific credentials to the AM credentials. - amLaunchCredentials.mergeAll(sessionCreds); + Credentials amLaunchCredentials = + prepareAmLaunchCredentials(amConfig, sessionCreds, conf, binaryConfPath); DataOutputBuffer dob = new DataOutputBuffer(); amLaunchCredentials.writeTokenStorageToStream(dob); @@ -703,6 +692,25 @@ public static ApplicationSubmissionContext createApplicationSubmissionContext( } + static Credentials prepareAmLaunchCredentials(AMConfiguration amConfig, Credentials sessionCreds, + TezConfiguration conf, Path binaryConfPath) throws IOException { + // Setup security tokens + Credentials amLaunchCredentials = new Credentials(); + + // Add Staging dir creds to the list of session credentials. + TokenCache.obtainTokensForFileSystems(sessionCreds, new Path[] {binaryConfPath }, conf); + + populateTokenCache(conf, sessionCreds); + + // Add session specific credentials to the AM credentials. + amLaunchCredentials.mergeAll(sessionCreds); + + if (amConfig.getCredentials() != null) { + amLaunchCredentials.mergeAll(amConfig.getCredentials()); + } + return amLaunchCredentials; + } + //get secret keys and tokens and store them into TokenCache private static void populateTokenCache(TezConfiguration conf, Credentials credentials) throws IOException{ diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java index 581d7225a3..edcec49ff2 100644 --- a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java +++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java @@ -46,8 +46,10 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.io.DataInputByteBuffer; +import org.apache.hadoop.io.Text; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.api.ApplicationConstants.Environment; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -890,5 +892,29 @@ public void testClusterTaskLaunchCmdOptsSetup() throws TezException { } + @Test + public void testSessionCredentialsMergedBeforeAmConfigCredentials() throws Exception { + TezConfiguration conf = new TezConfiguration(); + Text tokenType = new Text("TEST_TOKEN_TYPE"); + Text tokenKind = new Text("TEST_TOKEN_KIND"); + Text tokenService = new Text("TEST_TOKEN_SERVICE"); + + Credentials amConfigCredentials = new Credentials(); + amConfigCredentials.addToken(tokenType, + new Token<>("id1".getBytes(), null, tokenKind, tokenService)); + + Credentials sessionCredentials = new Credentials(); + Token sessionToken = + new Token<>("id2".getBytes(), null, tokenKind, tokenService); + sessionCredentials.addToken(tokenType, sessionToken); + AMConfiguration amConfig = new AMConfiguration(conf, null, amConfigCredentials); + + Credentials amLaunchCredentials = + TezClientUtils.prepareAmLaunchCredentials(amConfig, sessionCredentials, conf, null); + + // if there is another token in am conf creds of the same token type, + // session token should be applied while creating ContainerLaunchContext + Assert.assertEquals(sessionToken, amLaunchCredentials.getToken(tokenType)); + } } From a3b2dbb7bb16ff9261baf6b2245bd273a5b1122b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Thu, 23 Jan 2020 17:52:46 +0100 Subject: [PATCH 191/512] =?UTF-8?q?TEZ-4117:=20Fix=20minor=20issues=20in?= =?UTF-8?q?=20docs/pom.xml=20(L=C3=A1szl=C3=B3=20Bodor=20reviewed=20by=20J?= =?UTF-8?q?onathan=20Turner=20Eagles)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- docs/pom.xml | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/docs/pom.xml b/docs/pom.xml index 67a913adf7..c618cec4ba 100644 --- a/docs/pom.xml +++ b/docs/pom.xml @@ -238,6 +238,15 @@ PMC + + abstractdog + Laszlo Bodor + abstractdog@apache.org + +1 + + Committer + + kamrul Mohammad Kamrul Islam From cf111b1c896db94d40ea92fc6da2a6fcce356114 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Mon, 27 Jan 2020 12:44:43 -0600 Subject: [PATCH 192/512] TEZ-4101. Eliminate some guava dependencies by Java8+ features - Preconditions Signed-off-by: Jonathan Eagles --- .../org/apache/tez/client/CallerContext.java | 2 +- .../java/org/apache/tez/client/TezClient.java | 5 +- .../org/apache/tez/client/TezClientUtils.java | 10 +- .../org/apache/tez/common/Preconditions.java | 115 +++++++++++++++++ .../java/org/apache/tez/common/TezUtils.java | 6 +- .../counters/FileSystemCounterGroup.java | 5 +- .../counters/FrameworkCounterGroup.java | 5 +- .../main/java/org/apache/tez/dag/api/DAG.java | 11 +- .../tez/dag/api/DataSinkDescriptor.java | 4 +- .../tez/dag/api/DataSourceDescriptor.java | 4 +- .../org/apache/tez/dag/api/EdgeProperty.java | 2 +- .../apache/tez/dag/api/EntityDescriptor.java | 5 +- .../tez/dag/api/NamedEntityDescriptor.java | 2 +- .../apache/tez/dag/api/TaskLocationHint.java | 5 +- .../java/org/apache/tez/dag/api/Vertex.java | 2 +- .../dag/api/VertexManagerPluginContext.java | 2 +- .../tez/dag/api/client/DAGClientImpl.java | 2 +- .../api/events/InputInitializerEvent.java | 7 +- .../api/events/InputUpdatePayloadEvent.java | 5 +- .../api/events/VertexManagerEvent.java | 2 +- .../api/ServicePluginsDescriptor.java | 2 +- .../apache/tez/common/AsyncDispatcher.java | 1 - .../tez/common/AsyncDispatcherConcurrent.java | 1 - .../org/apache/tez/common/Preconditions.java | 116 ++++++++++++++++++ .../apache/tez/common/TezSharedExecutor.java | 1 - .../org/apache/tez/dag/records/TezDAGID.java | 2 +- .../org/apache/tez/dag/records/TezTaskID.java | 2 +- .../apache/tez/dag/records/TezVertexID.java | 2 +- .../InitialMemoryRequestContext.java | 9 +- .../apache/tez/dag/app/ContainerContext.java | 21 ++-- .../dag/app/ContainerLauncherContextImpl.java | 11 +- .../org/apache/tez/dag/app/DAGAppMaster.java | 9 +- .../apache/tez/dag/app/RecoveryParser.java | 2 +- .../dag/app/TaskCommunicatorContextImpl.java | 9 +- .../tez/dag/app/TaskCommunicatorManager.java | 2 +- .../tez/dag/app/TezTaskCommunicatorImpl.java | 7 +- .../app/dag/RootInputInitializerManager.java | 9 +- .../tez/dag/app/dag/StateChangeNotifier.java | 10 +- ...AGAppMasterEventUserServiceFatalError.java | 2 +- .../event/TaskAttemptEventAttemptFailed.java | 5 +- .../dag/app/dag/event/TaskEventTAFailed.java | 5 +- .../apache/tez/dag/app/dag/impl/DAGImpl.java | 2 +- .../org/apache/tez/dag/app/dag/impl/Edge.java | 2 +- .../dag/impl/ImmediateStartVertexManager.java | 2 +- .../dag/app/dag/impl/OneToOneEdgeManager.java | 2 +- .../dag/impl/OneToOneEdgeManagerOnDemand.java | 2 +- .../dag/impl/OutputCommitterContextImpl.java | 10 +- .../app/dag/impl/RootInputVertexManager.java | 2 +- .../dag/impl/ScatterGatherEdgeManager.java | 2 +- .../tez/dag/app/dag/impl/TaskAttemptImpl.java | 2 +- .../apache/tez/dag/app/dag/impl/TaskImpl.java | 2 +- .../TezRootInputInitializerContextImpl.java | 11 +- .../tez/dag/app/dag/impl/VertexImpl.java | 2 +- .../tez/dag/app/dag/impl/VertexManager.java | 15 +-- .../launcher/ContainerLauncherManager.java | 2 +- .../tez/dag/app/launcher/ContainerOp.java | 2 +- .../app/launcher/LocalContainerLauncher.java | 2 +- .../dag/app/rm/DagAwareYarnTaskScheduler.java | 2 +- .../dag/app/rm/TaskSchedulerContextImpl.java | 4 +- .../tez/dag/app/rm/TaskSchedulerManager.java | 8 +- .../dag/app/rm/YarnTaskSchedulerService.java | 2 +- .../dag/app/rm/container/AMContainerImpl.java | 2 +- .../dag/app/rm/container/AMContainerTask.java | 4 +- .../rm/container/ContainerContextMatcher.java | 11 +- .../apache/tez/dag/app/web/WebUIService.java | 2 +- .../tez/dag/history/utils/DAGUtils.java | 2 +- .../apache/tez/dag/app/MockDAGAppMaster.java | 2 +- .../apache/tez/dag/app/TestDAGAppMaster.java | 2 +- .../dag/app/rm/TestTaskSchedulerHelpers.java | 10 +- .../GraceShuffleVertexManagerForTest.java | 5 +- .../apache/tez/examples/CartesianProduct.java | 2 +- .../apache/tez/examples/HashJoinExample.java | 2 +- .../org/apache/tez/examples/JoinDataGen.java | 2 +- .../org/apache/tez/examples/JoinValidate.java | 2 +- .../apache/tez/examples/OrderedWordCount.java | 2 +- .../tez/examples/SortMergeJoinExample.java | 2 +- .../org/apache/tez/examples/WordCount.java | 2 +- .../TezTestServiceContainerLauncher.java | 2 +- .../TezTestServiceTaskSchedulerService.java | 2 +- .../service/MiniTezTestServiceCluster.java | 2 +- .../tez/service/impl/ContainerRunnerImpl.java | 2 +- .../tez/service/impl/TezTestService.java | 2 +- .../tez/shufflehandler/ShuffleHandler.java | 2 +- .../tests/ExternalTezServiceTestHelper.java | 4 +- .../split/TezGroupedSplitsInputFormat.java | 2 +- .../mapred/split/TezMapredSplitsGrouper.java | 2 +- .../split/TezGroupedSplitsInputFormat.java | 2 +- .../split/TezMapReduceSplitsGrouper.java | 2 +- .../apache/tez/mapreduce/common/Utils.java | 5 +- .../grouper/MapReduceSplitContainer.java | 4 +- .../grouper/MapredSplitContainer.java | 4 +- .../mapreduce/grouper/TezSplitGrouper.java | 2 +- .../mapreduce/hadoop/InputSplitInfoMem.java | 2 +- .../tez/mapreduce/hadoop/MRInputHelpers.java | 11 +- .../apache/tez/mapreduce/input/MRInput.java | 2 +- .../tez/mapreduce/input/MultiMRInput.java | 2 +- .../tez/mapreduce/input/base/MRInputBase.java | 2 +- .../tez/mapreduce/lib/MRReaderMapReduce.java | 5 +- .../tez/mapreduce/lib/MRReaderMapred.java | 5 +- .../apache/tez/mapreduce/output/MROutput.java | 2 +- .../org/apache/tez/history/ATSImportTool.java | 2 +- .../tez/history/parser/ATSFileParser.java | 2 +- .../history/parser/SimpleHistoryParser.java | 2 +- .../history/parser/datamodel/BaseParser.java | 2 +- .../tez/history/parser/datamodel/DagInfo.java | 2 +- .../parser/datamodel/TaskAttemptInfo.java | 2 +- .../history/parser/datamodel/TaskInfo.java | 2 +- .../history/parser/datamodel/VertexInfo.java | 2 +- .../apache/tez/runtime/InputReadyTracker.java | 2 +- .../LogicalIOProcessorRuntimeTask.java | 2 +- .../tez/runtime/api/impl/EventMetaData.java | 7 +- .../apache/tez/runtime/api/impl/TaskSpec.java | 25 ++-- .../tez/runtime/api/impl/TaskStatistics.java | 2 +- .../runtime/api/impl/TezInputContextImpl.java | 15 +-- .../api/impl/TezMergedInputContextImpl.java | 9 +- .../api/impl/TezOutputContextImpl.java | 11 +- .../api/impl/TezProcessorContextImpl.java | 9 +- .../runtime/api/impl/TezTaskContextImpl.java | 29 ++--- .../common/resources/MemoryDistributor.java | 11 +- .../common/resources/ScalingAllocator.java | 2 +- .../org/apache/tez/runtime/task/TezChild.java | 2 +- .../tez/runtime/task/TezTaskRunner2.java | 2 +- .../tez/runtime/task/TestTaskExecution2.java | 2 +- .../java/org/apache/hadoop/io/FileChunk.java | 5 +- .../FairShuffleVertexManager.java | 2 +- .../InputReadyVertexManager.java | 2 +- .../vertexmanager/ShuffleVertexManager.java | 2 +- .../ShuffleVertexManagerBase.java | 2 +- .../org/apache/tez/http/HttpConnection.java | 2 +- .../http/async/netty/AsyncHttpConnection.java | 2 +- .../CartesianProductCombination.java | 2 +- .../CartesianProductConfig.java | 2 +- .../CartesianProductEdgeManager.java | 2 +- .../CartesianProductVertexManager.java | 2 +- .../runtime/library/common/ConfigUtils.java | 2 +- .../common/MemoryUpdateCallbackHandler.java | 2 +- .../library/common/ValuesIterator.java | 2 +- .../common/shuffle/DiskFetchedInput.java | 2 +- .../library/common/shuffle/Fetcher.java | 2 +- .../common/shuffle/LocalDiskFetchedInput.java | 2 +- .../common/shuffle/MemoryFetchedInput.java | 2 +- .../library/common/shuffle/ShuffleUtils.java | 2 +- .../common/shuffle/impl/ShuffleManager.java | 2 +- .../shuffle/orderedgrouped/MergeManager.java | 2 +- .../shuffle/orderedgrouped/Shuffle.java | 2 +- .../orderedgrouped/ShuffleScheduler.java | 2 +- .../common/sort/impl/ExternalSorter.java | 2 +- .../common/sort/impl/PipelinedSorter.java | 2 +- .../common/sort/impl/dflt/DefaultSorter.java | 2 +- .../task/local/output/TezTaskOutputFiles.java | 2 +- .../writers/UnorderedPartitionedKVWriter.java | 2 +- .../conf/OrderedGroupedKVInputConfig.java | 17 +-- .../conf/OrderedPartitionedKVEdgeConfig.java | 5 +- .../OrderedPartitionedKVOutputConfig.java | 23 ++-- .../library/conf/UnorderedKVEdgeConfig.java | 5 +- .../library/conf/UnorderedKVInputConfig.java | 15 +-- .../library/conf/UnorderedKVOutputConfig.java | 15 +-- .../UnorderedPartitionedKVEdgeConfig.java | 5 +- .../UnorderedPartitionedKVOutputConfig.java | 19 +-- .../library/input/OrderedGroupedKVInput.java | 2 +- .../library/input/UnorderedKVInput.java | 2 +- .../output/OrderedPartitionedKVOutput.java | 2 +- .../output/UnorderedPartitionedKVOutput.java | 2 +- .../WeightedScalingMemoryDistributor.java | 2 +- .../tez/runtime/library/utils/Grouper.java | 2 +- .../common/sort/impl/TestTezMerger.java | 2 +- .../examples/BroadcastAndOneToOneExample.java | 2 +- .../mapreduce/examples/BroadcastLoadGen.java | 2 +- .../mapreduce/examples/CartesianProduct.java | 2 +- .../tez/mapreduce/examples/UnionExample.java | 2 +- .../helpers/SplitsInClientOptionParser.java | 2 +- .../tez/mapreduce/TestMRRJobsDAGApi.java | 2 +- .../RecoveryServiceWithEventHandlingHook.java | 2 +- .../apache/tez/test/TestPipelinedShuffle.java | 2 +- .../java/org/apache/tez/test/TestTezJobs.java | 2 +- .../org/apache/tez/analyzer/CSVResult.java | 2 +- .../plugins/CriticalPathAnalyzer.java | 2 +- .../tez/analyzer/plugins/SkewAnalyzer.java | 2 +- .../tez/analyzer/plugins/TezAnalyzerBase.java | 2 +- 179 files changed, 609 insertions(+), 371 deletions(-) create mode 100644 tez-api/src/main/java/org/apache/tez/common/Preconditions.java create mode 100644 tez-common/src/main/java/org/apache/tez/common/Preconditions.java diff --git a/tez-api/src/main/java/org/apache/tez/client/CallerContext.java b/tez-api/src/main/java/org/apache/tez/client/CallerContext.java index 809cf47d8e..f6c67cfe9c 100644 --- a/tez-api/src/main/java/org/apache/tez/client/CallerContext.java +++ b/tez-api/src/main/java/org/apache/tez/client/CallerContext.java @@ -24,7 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; @Public @Unstable diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java index ad00592076..a615179e85 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java @@ -30,6 +30,7 @@ import java.util.concurrent.TimeUnit; import java.util.HashMap; import java.util.Map; +import java.util.Objects; import javax.annotation.Nullable; @@ -83,7 +84,7 @@ import org.apache.tez.dag.api.records.DAGProtos.DAGPlan; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Maps; import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.protobuf.ServiceException; @@ -325,7 +326,7 @@ public static TezClient create(String name, TezConfiguration tezConf, boolean is * @param localFiles the files to be made available in the AM */ public synchronized void addAppMasterLocalFiles(Map localFiles) { - Preconditions.checkNotNull(localFiles); + Objects.requireNonNull(localFiles); if (isSession && sessionStarted.get()) { additionalLocalResources.putAll(localFiles); } diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index 2aeabe451a..b7d46f8a61 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -39,6 +39,7 @@ import java.util.TreeMap; import java.util.Vector; import java.util.Map.Entry; +import java.util.Objects; import com.google.common.base.Strings; import org.apache.commons.codec.digest.DigestUtils; @@ -112,7 +113,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; -import com.google.common.base.Preconditions; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; @@ -137,7 +137,7 @@ public class TezClientUtils { static boolean setupTezJarsLocalResources(TezConfiguration conf, Credentials credentials, Map tezJarResources) throws IOException { - Preconditions.checkNotNull(credentials, "A non-null credentials object should be specified"); + Objects.requireNonNull(credentials, "A non-null credentials object should be specified"); boolean usingTezArchive = false; if (conf.getBoolean(TezConfiguration.TEZ_IGNORE_LIB_URIS, false)){ @@ -379,7 +379,7 @@ public Path apply(URI input) { static Credentials setupDAGCredentials(DAG dag, Credentials sessionCredentials, Configuration conf) throws IOException { - Preconditions.checkNotNull(sessionCredentials); + Objects.requireNonNull(sessionCredentials); TezCommonUtils.logCredentials(LOG, sessionCredentials, "session"); Credentials dagCredentials = new Credentials(); @@ -445,7 +445,7 @@ public static ApplicationSubmissionContext createApplicationSubmissionContext( ServicePluginsDescriptor servicePluginsDescriptor, JavaOptsChecker javaOptsChecker) throws IOException, YarnException { - Preconditions.checkNotNull(sessionCreds); + Objects.requireNonNull(sessionCreds); TezConfiguration conf = amConfig.getTezConfiguration(); FileSystem fs = TezClientUtils.ensureStagingDirExists(conf, @@ -738,7 +738,7 @@ static DAGPlan prepareAndCreateDAGPlan(DAG dag, AMConfiguration amConfig, } static void maybeAddDefaultLoggingJavaOpts(String logLevel, List vargs) { - Preconditions.checkNotNull(vargs); + Objects.requireNonNull(vargs); if (!vargs.isEmpty()) { for (String arg : vargs) { if (arg.contains(TezConstants.TEZ_ROOT_LOGGER_NAME)) { diff --git a/tez-api/src/main/java/org/apache/tez/common/Preconditions.java b/tez-api/src/main/java/org/apache/tez/common/Preconditions.java new file mode 100644 index 0000000000..b32d951101 --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/common/Preconditions.java @@ -0,0 +1,115 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.common; + +import javax.annotation.Nullable; + +/** + * A simplified version of Guava's Preconditions for making it easy to handle its usage in Tez project. + */ +//TODO remove this and make Preconditions class in common module available everywhere +public class Preconditions { + + private Preconditions() { + } + + public static void checkArgument(boolean expression) { + if (!expression) { + throw new IllegalArgumentException(); + } + } + + public static void checkArgument(boolean expression, @Nullable Object message) { + if (!expression) { + throw new IllegalArgumentException(String.valueOf(message)); + } + } + + public static void checkArgument(boolean expression, @Nullable String template, @Nullable Object... args) { + if (!expression) { + throw new IllegalArgumentException(format(template, args)); + } + } + + public static void checkState(boolean expression) { + if (!expression) { + throw new IllegalStateException(); + } + } + + public static void checkState(boolean expression, @Nullable Object message) { + if (!expression) { + throw new IllegalStateException(String.valueOf(message)); + } + } + + public static void checkState(boolean expression, @Nullable String template, @Nullable Object... args) { + if (!expression) { + throw new IllegalStateException(format(template, args)); + } + } + + private static String format(@Nullable String template, @Nullable Object... args) { + template = String.valueOf(template); // null -> "null" + + if (args == null) { + args = new Object[] { "(Object[])null" }; + } else { + for (int i = 0; i < args.length; i++) { + args[i] = lenientToString(args[i]); + } + } + + // start substituting the arguments into the '%s' placeholders + StringBuilder builder = new StringBuilder(template.length() + 16 * args.length); + int templateStart = 0; + int i = 0; + while (i < args.length) { + int placeholderStart = template.indexOf("%s", templateStart); + if (placeholderStart == -1) { + break; + } + builder.append(template, templateStart, placeholderStart); + builder.append(args[i++]); + templateStart = placeholderStart + 2; + } + builder.append(template, templateStart, template.length()); + + // if we run out of placeholders, append the extra args in square braces + if (i < args.length) { + builder.append(" ["); + builder.append(args[i++]); + while (i < args.length) { + builder.append(", "); + builder.append(args[i++]); + } + builder.append(']'); + } + + return builder.toString(); + } + + private static String lenientToString(@Nullable Object o) { + try { + return String.valueOf(o); + } catch (Exception e) { + String objectToString = o.getClass().getName() + '@' + Integer.toHexString(System.identityHashCode(o)); + return "<" + objectToString + " threw " + e.getClass().getName() + ">"; + } + } +} diff --git a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java index 192d2a0e19..536eda47db 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java @@ -25,8 +25,8 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Objects; -import com.google.common.base.Preconditions; import com.google.protobuf.ByteString; import org.slf4j.Logger; @@ -74,7 +74,7 @@ public static void addLog4jSystemProperties(String logLevel, * @throws java.io.IOException */ public static ByteString createByteStringFromConf(Configuration conf) throws IOException { - Preconditions.checkNotNull(conf, "Configuration must be specified"); + Objects.requireNonNull(conf, "Configuration must be specified"); ByteString.Output os = ByteString.newOutput(); SnappyOutputStream compressOs = new SnappyOutputStream(os); try { @@ -108,7 +108,7 @@ public static UserPayload createUserPayloadFromConf(Configuration conf) throws I * @throws java.io.IOException */ public static Configuration createConfFromByteString(ByteString byteString) throws IOException { - Preconditions.checkNotNull(byteString, "ByteString must be specified"); + Objects.requireNonNull(byteString, "ByteString must be specified"); try(SnappyInputStream uncompressIs = new SnappyInputStream(byteString.newInput());) { DAGProtos.ConfigurationProto confProto = DAGProtos.ConfigurationProto.parseFrom(uncompressIs); Configuration conf = new Configuration(false); diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounterGroup.java b/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounterGroup.java index 3ea4acd153..76e5235d27 100644 --- a/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounterGroup.java +++ b/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounterGroup.java @@ -27,11 +27,10 @@ import java.util.Iterator; import java.util.Locale; import java.util.Map; +import java.util.Objects; import com.google.common.base.Joiner; -import static com.google.common.base.Preconditions.*; - import com.google.common.collect.AbstractIterator; import com.google.common.collect.Iterators; import com.google.common.collect.Maps; @@ -231,7 +230,7 @@ public void incrAllCounters(CounterGroupBase rightGroup) { @Override public void aggrAllCounters(CounterGroupBase other) { - if (checkNotNull(other.getUnderlyingGroup(), "other group") + if (Objects.requireNonNull(other.getUnderlyingGroup(), "other group") instanceof FileSystemCounterGroup) { for (TezCounter counter : other) { FSCounter c = (FSCounter) ((TezCounter)counter).getUnderlyingCounter(); diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/FrameworkCounterGroup.java b/tez-api/src/main/java/org/apache/tez/common/counters/FrameworkCounterGroup.java index bcb6454146..cb27741283 100644 --- a/tez-api/src/main/java/org/apache/tez/common/counters/FrameworkCounterGroup.java +++ b/tez-api/src/main/java/org/apache/tez/common/counters/FrameworkCounterGroup.java @@ -18,13 +18,14 @@ package org.apache.tez.common.counters; -import static com.google.common.base.Preconditions.checkNotNull; + import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; import java.util.Arrays; import java.util.Iterator; +import java.util.Objects; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.io.WritableUtils; @@ -199,7 +200,7 @@ public void incrAllCounters(CounterGroupBase rightGroup) { @SuppressWarnings("rawtypes") @Override public void aggrAllCounters(CounterGroupBase other) { - if (checkNotNull(other, "other counter group") + if (Objects.requireNonNull(other, "other counter group") instanceof FrameworkCounterGroup) { for (TezCounter counter : other) { findCounter(((FrameworkCounter) counter).key.name()) diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java index f8a2ddc5dc..0864b82e80 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java @@ -33,6 +33,7 @@ import java.util.Map.Entry; import java.util.Set; import java.util.Stack; +import java.util.Objects; import org.apache.commons.collections4.BidiMap; import org.apache.commons.collections4.bidimap.DualLinkedHashBidiMap; @@ -73,7 +74,7 @@ import org.apache.tez.dag.api.records.DAGProtos.VertexPlan; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; @@ -128,7 +129,7 @@ public static DAG create(String name) { * @return {@link DAG} */ public synchronized DAG addTaskLocalFiles(Map localFiles) { - Preconditions.checkNotNull(localFiles); + Objects.requireNonNull(localFiles); TezCommonUtils.addAdditionalLocalResources(localFiles, commonTaskLocalFiles, "DAG " + getName()); return this; } @@ -178,7 +179,7 @@ public synchronized DAG setCredentials(Credentials credentials) { */ @Deprecated public synchronized DAG setDAGInfo(String dagInfo) { - Preconditions.checkNotNull(dagInfo); + Objects.requireNonNull(dagInfo); this.dagInfo = dagInfo; return this; } @@ -190,7 +191,7 @@ public synchronized DAG setDAGInfo(String dagInfo) { * @return {@link DAG} */ public synchronized DAG setCallerContext(CallerContext callerContext) { - Preconditions.checkNotNull(callerContext); + Objects.requireNonNull(callerContext); this.callerContext = callerContext; return this; } @@ -257,7 +258,7 @@ public synchronized DAGAccessControls getDagAccessControls() { * @return {@link DAG} */ public synchronized DAG addURIsForCredentials(Collection uris) { - Preconditions.checkNotNull(uris, "URIs cannot be null"); + Objects.requireNonNull(uris, "URIs cannot be null"); urisForCredentials.addAll(uris); return this; } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DataSinkDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/DataSinkDescriptor.java index 4d0d615db4..fec2bf6eed 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/DataSinkDescriptor.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/DataSinkDescriptor.java @@ -21,13 +21,13 @@ import java.net.URI; import java.util.Collection; import java.util.Collections; +import java.util.Objects; import javax.annotation.Nullable; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.security.Credentials; -import com.google.common.base.Preconditions; import com.google.common.collect.Sets; /** @@ -121,7 +121,7 @@ public OutputDescriptor getOutputDescriptor() { * @return this */ public synchronized DataSinkDescriptor addURIsForCredentials(Collection uris) { - Preconditions.checkNotNull(uris, "URIs cannot be null"); + Objects.requireNonNull(uris, "URIs cannot be null"); urisForCredentials.addAll(uris); return this; } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DataSourceDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/DataSourceDescriptor.java index 1c5c16db5f..db43c91f7a 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/DataSourceDescriptor.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/DataSourceDescriptor.java @@ -24,6 +24,7 @@ import java.util.Collection; import java.util.Collections; import java.util.Map; +import java.util.Objects; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience.Private; @@ -34,7 +35,6 @@ import org.apache.tez.runtime.api.InputInitializer; import org.apache.tez.runtime.api.events.InputDataInformationEvent; -import com.google.common.base.Preconditions; import com.google.common.collect.Sets; /** @@ -157,7 +157,7 @@ public InputDescriptor getInputDescriptor() { * @return this */ public synchronized DataSourceDescriptor addURIsForCredentials(Collection uris) { - Preconditions.checkNotNull(uris, "URIs cannot be null"); + Objects.requireNonNull(uris, "URIs cannot be null"); urisForCredentials.addAll(uris); return this; } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java b/tez-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java index c203f8c6b6..1850060ece 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java @@ -21,7 +21,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; /** * An @link {@link EdgeProperty} defines the relation between the source and diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/EntityDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/EntityDescriptor.java index 13d4a93f0a..2c7b834c1c 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/EntityDescriptor.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/EntityDescriptor.java @@ -22,6 +22,7 @@ import java.io.DataOutput; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Objects; import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.classification.InterfaceAudience.Private; @@ -30,8 +31,6 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; -import com.google.common.base.Preconditions; - /** * Describes a given user code entity. Consists of the name of the class implementing * the user logic and a payload that can be used to configure an object instance of @@ -66,7 +65,7 @@ public UserPayload getUserPayload() { * @return this object for further chained method calls */ public T setUserPayload(UserPayload userPayload) { - Preconditions.checkNotNull(userPayload); + Objects.requireNonNull(userPayload); this.userPayload = userPayload; return (T) this; } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java index 426d4eb1b0..452deaa77b 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java @@ -18,7 +18,7 @@ import java.io.DataOutput; import java.io.IOException; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import org.apache.hadoop.classification.InterfaceAudience; @SuppressWarnings("unchecked") diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TaskLocationHint.java b/tez-api/src/main/java/org/apache/tez/dag/api/TaskLocationHint.java index d1a1359977..3070c338cc 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TaskLocationHint.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TaskLocationHint.java @@ -20,12 +20,13 @@ import java.util.Collections; import java.util.Set; +import java.util.Objects; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Evolving; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; /** * Describes the placements hints for tasks. @@ -67,7 +68,7 @@ public String toString() { private TaskBasedLocationAffinity affinitizedTask; private TaskLocationHint(String vertexName, int taskIndex) { - Preconditions.checkNotNull(vertexName); + Objects.requireNonNull(vertexName); Preconditions.checkArgument(taskIndex >= 0); this.affinitizedTask = new TaskBasedLocationAffinity(vertexName, taskIndex); } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java b/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java index bf3a59ba24..ccfa92a9f6 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java @@ -35,7 +35,7 @@ import org.apache.tez.dag.api.VertexGroup.GroupInfo; import org.apache.tez.runtime.api.LogicalIOProcessor; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/VertexManagerPluginContext.java b/tez-api/src/main/java/org/apache/tez/dag/api/VertexManagerPluginContext.java index b89b279fad..ba43ecd9c1 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/VertexManagerPluginContext.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/VertexManagerPluginContext.java @@ -35,7 +35,7 @@ import org.apache.tez.runtime.api.events.CustomProcessorEvent; import org.apache.tez.runtime.api.events.InputDataInformationEvent; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; /** * Object with API's to interact with the Tez execution engine diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java index 9e17b9b1e2..52f66e3c63 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java @@ -29,7 +29,7 @@ import java.util.Set; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException; import org.slf4j.Logger; diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputInitializerEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputInitializerEvent.java index 3037e619b5..21ad130c88 100644 --- a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputInitializerEvent.java +++ b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputInitializerEvent.java @@ -21,8 +21,7 @@ package org.apache.tez.runtime.api.events; import java.nio.ByteBuffer; - -import com.google.common.base.Preconditions; +import java.util.Objects; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience.Public; @@ -53,8 +52,8 @@ public class InputInitializerEvent extends Event { private InputInitializerEvent(String targetVertexName, String targetInputName, ByteBuffer eventPayload) { - Preconditions.checkNotNull(targetVertexName, "TargetVertexName cannot be null"); - Preconditions.checkNotNull(targetInputName, "TargetInputName cannot be null"); + Objects.requireNonNull(targetVertexName, "TargetVertexName cannot be null"); + Objects.requireNonNull(targetInputName, "TargetInputName cannot be null"); this.targetVertexName = targetVertexName; this.targetInputName = targetInputName; this.eventPayload = eventPayload; diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputUpdatePayloadEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputUpdatePayloadEvent.java index 2cfec69589..8c336354d9 100644 --- a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputUpdatePayloadEvent.java +++ b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputUpdatePayloadEvent.java @@ -19,14 +19,13 @@ package org.apache.tez.runtime.api.events; import java.nio.ByteBuffer; +import java.util.Objects; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.InputInitializer; -import com.google.common.base.Preconditions; - /** * Events used by {@link InputInitializer} implementations to update the * shared user payload for the Input that is being initialized.

@@ -41,7 +40,7 @@ public class InputUpdatePayloadEvent extends Event { private final ByteBuffer userPayload; private InputUpdatePayloadEvent(ByteBuffer userPayload) { - Preconditions.checkNotNull(userPayload); + Objects.requireNonNull(userPayload); this.userPayload = userPayload; } diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/events/VertexManagerEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/VertexManagerEvent.java index 9e73fe5d00..6e5738918c 100644 --- a/tez-api/src/main/java/org/apache/tez/runtime/api/events/VertexManagerEvent.java +++ b/tez-api/src/main/java/org/apache/tez/runtime/api/events/VertexManagerEvent.java @@ -27,7 +27,7 @@ import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.TaskAttemptIdentifier; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; /** * Event used to send information from a Task to the VertexManager for a vertex. diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java index 39d2cb8102..c942a3ad3a 100644 --- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java +++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java @@ -16,7 +16,7 @@ import java.util.Arrays; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.tez.dag.api.TezConfiguration; diff --git a/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcher.java b/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcher.java index 3a59ff6449..c197f1d32e 100644 --- a/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcher.java +++ b/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcher.java @@ -36,7 +36,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.base.Preconditions; import com.google.common.collect.Maps; /** diff --git a/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcherConcurrent.java b/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcherConcurrent.java index 4a632f5a61..f22159b46b 100644 --- a/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcherConcurrent.java +++ b/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcherConcurrent.java @@ -37,7 +37,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.util.concurrent.ThreadFactoryBuilder; diff --git a/tez-common/src/main/java/org/apache/tez/common/Preconditions.java b/tez-common/src/main/java/org/apache/tez/common/Preconditions.java new file mode 100644 index 0000000000..42d862cd22 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/common/Preconditions.java @@ -0,0 +1,116 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.common; + +import javax.annotation.Nullable; + +/** + * A simplified version of Guava's Preconditions for making it easy to handle its usage in Tez project. + */ +public class Preconditions { + + private Preconditions() { + } + + public static void checkArgument(boolean expression) { + if (!expression) { + throw new IllegalArgumentException(); + } + } + + public static void checkArgument(boolean expression, @Nullable Object message) { + if (!expression) { + throw new IllegalArgumentException(String.valueOf(message)); + } + } + + public static void checkArgument(boolean expression, @Nullable String template, + @Nullable Object... args) { + if (!expression) { + throw new IllegalArgumentException(format(template, args)); + } + } + + public static void checkState(boolean expression) { + if (!expression) { + throw new IllegalStateException(); + } + } + + public static void checkState(boolean expression, @Nullable Object message) { + if (!expression) { + throw new IllegalStateException(String.valueOf(message)); + } + } + + public static void checkState(boolean expression, @Nullable String template, + @Nullable Object... args) { + if (!expression) { + throw new IllegalStateException(format(template, args)); + } + } + + private static String format(@Nullable String template, @Nullable Object... args) { + template = String.valueOf(template); // null -> "null" + + if (args == null) { + args = new Object[] { "(Object[])null" }; + } else { + for (int i = 0; i < args.length; i++) { + args[i] = lenientToString(args[i]); + } + } + + // start substituting the arguments into the '%s' placeholders + StringBuilder builder = new StringBuilder(template.length() + 16 * args.length); + int templateStart = 0; + int i = 0; + while (i < args.length) { + int placeholderStart = template.indexOf("%s", templateStart); + if (placeholderStart == -1) { + break; + } + builder.append(template, templateStart, placeholderStart); + builder.append(args[i++]); + templateStart = placeholderStart + 2; + } + builder.append(template, templateStart, template.length()); + + // if we run out of placeholders, append the extra args in square braces + if (i < args.length) { + builder.append(" ["); + builder.append(args[i++]); + while (i < args.length) { + builder.append(", "); + builder.append(args[i++]); + } + builder.append(']'); + } + + return builder.toString(); + } + + private static String lenientToString(@Nullable Object o) { + try { + return String.valueOf(o); + } catch (Exception e) { + String objectToString = o.getClass().getName() + '@' + Integer.toHexString(System.identityHashCode(o)); + return "<" + objectToString + " threw " + e.getClass().getName() + ">"; + } + } +} diff --git a/tez-common/src/main/java/org/apache/tez/common/TezSharedExecutor.java b/tez-common/src/main/java/org/apache/tez/common/TezSharedExecutor.java index 3cc72d528f..bf8eb4f861 100644 --- a/tez-common/src/main/java/org/apache/tez/common/TezSharedExecutor.java +++ b/tez-common/src/main/java/org/apache/tez/common/TezSharedExecutor.java @@ -40,7 +40,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.tez.dag.api.TezConfiguration; -import com.google.common.base.Preconditions; import com.google.common.util.concurrent.ThreadFactoryBuilder; /** diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezDAGID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezDAGID.java index 2e3309e4dc..24365c95e2 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezDAGID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezDAGID.java @@ -25,7 +25,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.yarn.api.records.ApplicationId; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import org.apache.tez.util.FastNumberFormat; /** diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java index 3295f6a707..15b695c7a5 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java @@ -25,7 +25,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import org.apache.tez.util.FastNumberFormat; /** diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java index b56c9adfa5..b5a36abe33 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java @@ -25,7 +25,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import org.apache.tez.util.FastNumberFormat; /** diff --git a/tez-common/src/main/java/org/apache/tez/runtime/common/resources/InitialMemoryRequestContext.java b/tez-common/src/main/java/org/apache/tez/runtime/common/resources/InitialMemoryRequestContext.java index aec3795435..da361a1e59 100644 --- a/tez-common/src/main/java/org/apache/tez/runtime/common/resources/InitialMemoryRequestContext.java +++ b/tez-common/src/main/java/org/apache/tez/runtime/common/resources/InitialMemoryRequestContext.java @@ -18,7 +18,8 @@ package org.apache.tez.runtime.common.resources; -import com.google.common.base.Preconditions; +import java.util.Objects; + import org.apache.hadoop.classification.InterfaceAudience.Private; @@ -38,9 +39,9 @@ public static enum ComponentType { public InitialMemoryRequestContext(long requestedSize, String componentClassName, ComponentType componentType, String componentVertexName) { - Preconditions.checkNotNull(componentClassName, "componentClassName is null"); - Preconditions.checkNotNull(componentType, "componentType is null"); - Preconditions.checkNotNull(componentVertexName, "componentVertexName is null"); + Objects.requireNonNull(componentClassName, "componentClassName is null"); + Objects.requireNonNull(componentType, "componentType is null"); + Objects.requireNonNull(componentVertexName, "componentVertexName is null"); this.requestedSize = requestedSize; this.componentClassName = componentClassName; this.componentType = componentType; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerContext.java b/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerContext.java index d88daff8e4..f2c7d5f700 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerContext.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerContext.java @@ -21,6 +21,7 @@ import java.util.HashMap; import java.util.Map; import java.util.Map.Entry; +import java.util.Objects; import javax.annotation.Nullable; @@ -33,8 +34,6 @@ import org.apache.hadoop.yarn.api.records.LocalResourceType; import org.apache.tez.dag.app.dag.Vertex; -import com.google.common.base.Preconditions; - public class ContainerContext { private static final Logger LOG = LoggerFactory.getLogger(ContainerContext.class); @@ -49,11 +48,11 @@ public class ContainerContext { public ContainerContext(Map localResources, Credentials credentials, Map environment, String javaOpts) { - Preconditions.checkNotNull(localResources, + Objects.requireNonNull(localResources, "localResources should not be null"); - Preconditions.checkNotNull(credentials, "credentials should not be null"); - Preconditions.checkNotNull(environment, "environment should not be null"); - Preconditions.checkNotNull(javaOpts, "javaOpts should not be null"); + Objects.requireNonNull(credentials, "credentials should not be null"); + Objects.requireNonNull(environment, "environment should not be null"); + Objects.requireNonNull(javaOpts, "javaOpts should not be null"); this.localResources = localResources; this.credentials = credentials; this.environment = environment; @@ -64,11 +63,11 @@ public ContainerContext(Map localResources, public ContainerContext(Map localResources, Credentials credentials, Map environment, String javaOpts, @Nullable Vertex vertex) { - Preconditions.checkNotNull(localResources, + Objects.requireNonNull(localResources, "localResources should not be null"); - Preconditions.checkNotNull(credentials, "credentials should not be null"); - Preconditions.checkNotNull(environment, "environment should not be null"); - Preconditions.checkNotNull(javaOpts, "javaOpts should not be null"); + Objects.requireNonNull(credentials, "credentials should not be null"); + Objects.requireNonNull(environment, "environment should not be null"); + Objects.requireNonNull(javaOpts, "javaOpts should not be null"); this.localResources = localResources; this.credentials = credentials; this.environment = environment; @@ -97,7 +96,7 @@ public String getJavaOpts() { * container context. */ public boolean isSuperSet(ContainerContext otherContext) { - Preconditions.checkNotNull(otherContext, "otherContext should not null"); + Objects.requireNonNull(otherContext, "otherContext should not null"); // Assumptions: // Credentials are the same for all containers belonging to a DAG. // Matching can be added if containers are used across DAGs diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java index 7e68675b24..694de075d0 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java @@ -16,7 +16,8 @@ import javax.annotation.Nullable; -import com.google.common.base.Preconditions; +import java.util.Objects; + import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.tez.common.TezUtilsInternal; @@ -52,9 +53,9 @@ public class ContainerLauncherContextImpl implements ContainerLauncherContext { public ContainerLauncherContextImpl(AppContext appContext, ContainerLauncherManager containerLauncherManager, TaskCommunicatorManagerInterface tal, UserPayload initialUserPayload, int containerLauncherIndex) { - Preconditions.checkNotNull(appContext, "AppContext cannot be null"); - Preconditions.checkNotNull(appContext, "ContainerLauncherManager cannot be null"); - Preconditions.checkNotNull(tal, "TaskCommunicator cannot be null"); + Objects.requireNonNull(appContext, "AppContext cannot be null"); + Objects.requireNonNull(appContext, "ContainerLauncherManager cannot be null"); + Objects.requireNonNull(tal, "TaskCommunicator cannot be null"); this.context = appContext; this.containerLauncherManager = containerLauncherManager; this.tal = tal; @@ -140,7 +141,7 @@ public Object getTaskCommunicatorMetaInfo(String taskCommName) { @Override public void reportError(ServicePluginError servicePluginError, String message, DagInfo dagInfo) { - Preconditions.checkNotNull(servicePluginError, "ServiceError must be specified"); + Objects.requireNonNull(servicePluginError, "ServiceError must be specified"); containerLauncherManager.reportError(containerLauncherIndex, servicePluginError, message, dagInfo); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index f29d19995b..99b01d6eae 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -18,7 +18,7 @@ package org.apache.tez.dag.app; -import static com.google.common.base.Preconditions.checkNotNull; + import java.io.File; import java.io.FileInputStream; @@ -55,6 +55,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.Objects; import com.google.common.collect.BiMap; import com.google.common.collect.HashBiMap; @@ -191,7 +192,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Joiner; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Maps; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; @@ -1458,7 +1459,7 @@ private class RunningAppContext implements AppContext { private volatile String queueName; public RunningAppContext(Configuration config) { - checkNotNull(config, "config is null"); + Objects.requireNonNull(config, "config is null"); this.conf = config; this.eventHandler = dispatcher.getEventHandler(); } @@ -1684,7 +1685,7 @@ public TezDAGID getCurrentDAGID() { @Override public void setDAG(DAG dag) { - Preconditions.checkNotNull(dag, "dag is null"); + Objects.requireNonNull(dag, "dag is null"); try { wLock.lock(); this.dag = dag; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java index 99ac2837ae..bab614259d 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java @@ -77,7 +77,7 @@ import org.apache.tez.runtime.api.impl.TezEvent; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; /** diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java index 1adbf6edf8..b09eac7663 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java @@ -19,9 +19,10 @@ import java.io.IOException; import java.util.Set; import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.Objects; import com.google.common.base.Function; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Iterables; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.security.Credentials; @@ -155,7 +156,7 @@ public void taskFailed(TezTaskAttemptID taskAttemptId, TaskFailureType taskFailu @Override public void registerForVertexStateUpdates(String vertexName, @Nullable Set stateSet) { - Preconditions.checkNotNull(vertexName, "VertexName cannot be null: " + vertexName); + Objects.requireNonNull(vertexName, "VertexName cannot be null: " + vertexName); DAG dag = getDag(); dag.getStateChangeNotifier().registerForVertexUpdates(vertexName, stateSet, this); @@ -174,7 +175,7 @@ public DagInfo getCurrentDagInfo() { @Override public Iterable getInputVertexNames(String vertexName) { - Preconditions.checkNotNull(vertexName, "VertexName cannot be null: " + vertexName); + Objects.requireNonNull(vertexName, "VertexName cannot be null: " + vertexName); DAG dag = getDag(); Vertex vertex = dag.getVertex(vertexName); Set sources = vertex.getInputVertices().keySet(); @@ -227,7 +228,7 @@ public long getDagStartTime() { @Override public void reportError(@Nonnull ServicePluginError servicePluginError, String message, DagInfo dagInfo) { - Preconditions.checkNotNull(servicePluginError, "ServicePluginError must be set"); + Objects.requireNonNull(servicePluginError, "ServicePluginError must be set"); taskCommunicatorManager.reportError(taskCommunicatorIndex, servicePluginError, message, dagInfo); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java index d1b0349eec..55b2d1b021 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java @@ -27,7 +27,7 @@ import java.util.concurrent.ConcurrentMap; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import org.apache.commons.collections4.ListUtils; import org.apache.hadoop.yarn.event.Event; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java index d52df4f44d..9c8fb6c67e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java @@ -20,8 +20,8 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.Objects; -import com.google.common.base.Preconditions; import com.google.common.collect.Maps; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; @@ -221,8 +221,9 @@ public void registerRunningTaskAttempt(ContainerId containerId, TaskSpec taskSpe int priority) { ContainerInfo containerInfo = registeredContainers.get(containerId); - Preconditions.checkNotNull(containerInfo, "Cannot register task attempt %s to unknown container %s", - taskSpec.getTaskAttemptID(), containerId); + Objects.requireNonNull(containerInfo, + String.format("Cannot register task attempt %s to unknown container %s", + taskSpec.getTaskAttemptID(), containerId)); synchronized (containerInfo) { if (containerInfo.taskSpec != null) { throw new TezUncheckedException( diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java index e03b469a21..b2a0c0bb59 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java @@ -34,8 +34,9 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.Objects; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.LinkedListMultimap; import com.google.common.collect.ListMultimap; import com.google.common.collect.Lists; @@ -175,7 +176,7 @@ public void handleInitializerEvents(List events) { InputInitializerEvent event = (InputInitializerEvent)tezEvent.getEvent(); Preconditions.checkState(vertex.getName().equals(event.getTargetVertexName()), "Received event for incorrect vertex"); - Preconditions.checkNotNull(event.getTargetInputName(), "target input name must be set"); + Objects.requireNonNull(event.getTargetInputName(), "target input name must be set"); InitializerWrapper initializer = initializerMap.get(event.getTargetInputName()); Preconditions.checkState(initializer != null, "Received event for unknown input : " + event.getTargetInputName()); @@ -220,8 +221,8 @@ private VertexUpdateRegistrationHolder(String vertexName, Set stateSet) { - Preconditions.checkNotNull(vertexName, "VertexName cannot be null: " + vertexName); - Preconditions.checkNotNull(inputName, "InputName cannot be null"); + Objects.requireNonNull(vertexName, "VertexName cannot be null: " + vertexName); + Objects.requireNonNull(inputName, "InputName cannot be null"); InitializerWrapper initializer = initializerMap.get(inputName); if (initializer == null) { pendingVertexRegistrations.put(inputName, new VertexUpdateRegistrationHolder(vertexName, stateSet)); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/StateChangeNotifier.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/StateChangeNotifier.java index bd04fd8913..7d13f930c0 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/StateChangeNotifier.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/StateChangeNotifier.java @@ -25,9 +25,9 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.Objects; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; import com.google.common.collect.HashMultimap; import com.google.common.collect.LinkedListMultimap; import com.google.common.collect.ListMultimap; @@ -268,7 +268,7 @@ public int hashCode() { public void registerForTaskSuccessUpdates(String vertexName, TaskStateUpdateListener listener) { TezVertexID vertexId = validateAndGetVertexId(vertexName); - Preconditions.checkNotNull(listener, "listener cannot be null"); + Objects.requireNonNull(listener, "listener cannot be null"); taskWriteLock.lock(); try { taskListeners.put(vertexId, listener); @@ -279,7 +279,7 @@ public void registerForTaskSuccessUpdates(String vertexName, TaskStateUpdateList public void unregisterForTaskSuccessUpdates(String vertexName, TaskStateUpdateListener listener) { TezVertexID vertexId = validateAndGetVertexId(vertexName); - Preconditions.checkNotNull(listener, "listener cannot be null"); + Objects.requireNonNull(listener, "listener cannot be null"); taskWriteLock.lock(); try { taskListeners.remove(vertexId, listener); @@ -303,9 +303,9 @@ public void taskSucceeded(String vertexName, TezTaskID taskId, int attemptId) { private TezVertexID validateAndGetVertexId(String vertexName) { - Preconditions.checkNotNull(vertexName, "VertexName cannot be null"); + Objects.requireNonNull(vertexName, "VertexName cannot be null"); Vertex vertex = dag.getVertex(vertexName); - Preconditions.checkNotNull(vertex, "Vertex does not exist: " + vertexName); + Objects.requireNonNull(vertex, "Vertex does not exist: " + vertexName); return vertex.getVertexId(); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGAppMasterEventUserServiceFatalError.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGAppMasterEventUserServiceFatalError.java index 7bc3bd8e9e..728af783d5 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGAppMasterEventUserServiceFatalError.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGAppMasterEventUserServiceFatalError.java @@ -16,7 +16,7 @@ import java.util.EnumSet; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; public class DAGAppMasterEventUserServiceFatalError extends DAGAppMasterEvent implements DiagnosableEvent { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptFailed.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptFailed.java index 299847c31a..d4c7273c97 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptFailed.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptFailed.java @@ -18,7 +18,8 @@ package org.apache.tez.dag.app.dag.event; -import com.google.common.base.Preconditions; +import java.util.Objects; + import org.apache.tez.dag.records.TaskAttemptTerminationCause; import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.runtime.api.TaskFailureType; @@ -44,7 +45,7 @@ public TaskAttemptEventAttemptFailed(TezTaskAttemptID id, TaskAttemptEventType type, TaskFailureType taskFailureType, String diagnostics, TaskAttemptTerminationCause errorCause, boolean isFromRecovery) { super(id, type); - Preconditions.checkNotNull(taskFailureType, "FailureType must be set for a FAILED task attempt"); + Objects.requireNonNull(taskFailureType, "FailureType must be set for a FAILED task attempt"); this.diagnostics = diagnostics; this.errorCause = errorCause; this.taskFailureType = taskFailureType; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventTAFailed.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventTAFailed.java index d6f1526c3a..f68549dd6b 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventTAFailed.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventTAFailed.java @@ -18,7 +18,8 @@ package org.apache.tez.dag.app.dag.event; -import com.google.common.base.Preconditions; +import java.util.Objects; + import org.apache.tez.common.TezAbstractEvent; import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.runtime.api.TaskFailureType; @@ -31,7 +32,7 @@ public class TaskEventTAFailed extends TaskEventTAUpdate { public TaskEventTAFailed(TezTaskAttemptID id, TaskFailureType taskFailureType, TezAbstractEvent causalEvent) { super(id, TaskEventType.T_ATTEMPT_FAILED); - Preconditions.checkNotNull(taskFailureType, "FailureType must be specified for a failed attempt"); + Objects.requireNonNull(taskFailureType, "FailureType must be specified for a failed attempt"); this.taskFailureType = taskFailureType; this.causalEvent = causalEvent; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java index 18b7128e1d..20ecc4f274 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java @@ -137,7 +137,7 @@ import org.apache.tez.runtime.api.OutputCommitter; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java index f78c9a5e23..848b49199d 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java @@ -65,7 +65,7 @@ import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Maps; public class Edge { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexManager.java index 50624ddcd5..fc77e9a04d 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexManager.java @@ -18,7 +18,7 @@ package org.apache.tez.dag.app.dag.impl; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java index dd381801bb..f9bacbf034 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java @@ -28,7 +28,7 @@ import org.apache.tez.runtime.api.events.DataMovementEvent; import org.apache.tez.runtime.api.events.InputReadErrorEvent; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; public class OneToOneEdgeManager extends EdgeManagerPlugin { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManagerOnDemand.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManagerOnDemand.java index 819735a8cd..e2dbf457cf 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManagerOnDemand.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManagerOnDemand.java @@ -30,7 +30,7 @@ import org.apache.tez.runtime.api.events.DataMovementEvent; import org.apache.tez.runtime.api.events.InputReadErrorEvent; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; public class OneToOneEdgeManagerOnDemand extends EdgeManagerPluginOnDemand { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OutputCommitterContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OutputCommitterContextImpl.java index 6eae32d133..dc89514950 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OutputCommitterContextImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OutputCommitterContextImpl.java @@ -18,7 +18,7 @@ package org.apache.tez.dag.app.dag.impl; -import static com.google.common.base.Preconditions.checkNotNull; +import java.util.Objects; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.tez.dag.api.OutputCommitterDescriptor; @@ -42,10 +42,10 @@ public OutputCommitterContextImpl(ApplicationId applicationId, String vertexName, RootInputLeafOutput output, int vertexIdx) { - checkNotNull(applicationId, "applicationId is null"); - checkNotNull(dagName, "dagName is null"); - checkNotNull(vertexName, "vertexName is null"); - checkNotNull(output, "output is null"); + Objects.requireNonNull(applicationId, "applicationId is null"); + Objects.requireNonNull(dagName, "dagName is null"); + Objects.requireNonNull(vertexName, "vertexName is null"); + Objects.requireNonNull(output, "output is null"); this.applicationId = applicationId; this.dagAttemptNumber = dagAttemptNumber; this.dagName = dagName; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/RootInputVertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/RootInputVertexManager.java index 38eba0e34d..afe2606c1b 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/RootInputVertexManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/RootInputVertexManager.java @@ -49,7 +49,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import javax.annotation.Nullable; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java index 4d373cacfe..c5b4e1aa80 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java @@ -31,7 +31,7 @@ import org.apache.tez.runtime.api.events.DataMovementEvent; import org.apache.tez.runtime.api.events.InputReadErrorEvent; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; public class ScatterGatherEdgeManager extends EdgeManagerPluginOnDemand { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java index ade7bc742f..37e818e271 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java @@ -119,7 +119,7 @@ import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java index 39e2b4c1f7..34702163fe 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java @@ -32,7 +32,7 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.base.Predicate; import com.google.common.collect.Maps; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java index f7130545bb..43764878b6 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java @@ -18,9 +18,10 @@ package org.apache.tez.dag.app.dag.impl; -import static com.google.common.base.Preconditions.checkNotNull; + import java.util.Set; +import java.util.Objects; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.Resource; @@ -50,10 +51,10 @@ public TezRootInputInitializerContextImpl( RootInputLeafOutput input, Vertex vertex, AppContext appContext, RootInputInitializerManager manager) { - checkNotNull(input, "input is null"); - checkNotNull(vertex, "vertex is null"); - checkNotNull(appContext, "appContext is null"); - checkNotNull(manager, "initializerManager is null"); + Objects.requireNonNull(input, "input is null"); + Objects.requireNonNull(vertex, "vertex is null"); + Objects.requireNonNull(appContext, "appContext is null"); + Objects.requireNonNull(manager, "initializerManager is null"); this.input = input; this.vertex = vertex; this.appContext = appContext; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index 52fe9325f3..0d34be6d00 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -193,7 +193,7 @@ import org.slf4j.LoggerFactory; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.base.Strings; import com.google.common.collect.HashMultiset; import com.google.common.collect.Lists; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java index 7a1547f0e3..03e03aacee 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java @@ -18,7 +18,7 @@ package org.apache.tez.dag.app.dag.impl; -import static com.google.common.base.Preconditions.checkNotNull; + import java.lang.reflect.UndeclaredThrowableException; import java.security.PrivilegedExceptionAction; @@ -31,6 +31,7 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.Objects; import javax.annotation.Nullable; @@ -77,7 +78,7 @@ import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType; import com.google.common.base.Function; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Collections2; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -302,7 +303,7 @@ public void sendEventToProcessor(Collection events, int ta @Override public synchronized void setVertexLocationHint(VertexLocationHint locationHint) { checkAndThrowIfDone(); - Preconditions.checkNotNull(locationHint, "locationHint is null"); + Objects.requireNonNull(locationHint, "locationHint is null"); managedVertex.setVertexLocationHint(locationHint); } @@ -410,10 +411,10 @@ public void onStateUpdated(VertexStateUpdate event) { public VertexManager(VertexManagerPluginDescriptor pluginDesc, UserGroupInformation dagUgi, Vertex managedVertex, AppContext appContext, StateChangeNotifier stateChangeNotifier) throws TezException { - checkNotNull(pluginDesc, "pluginDesc is null"); - checkNotNull(managedVertex, "managedVertex is null"); - checkNotNull(appContext, "appContext is null"); - checkNotNull(stateChangeNotifier, "notifier is null"); + Objects.requireNonNull(pluginDesc, "pluginDesc is null"); + Objects.requireNonNull(managedVertex, "managedVertex is null"); + Objects.requireNonNull(appContext, "appContext is null"); + Objects.requireNonNull(stateChangeNotifier, "notifier is null"); this.pluginDesc = pluginDesc; this.dagUgi = dagUgi; this.managedVertex = managedVertex; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherManager.java index 58d87c9672..d55787e25a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherManager.java @@ -18,7 +18,7 @@ import java.util.List; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.AbstractService; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerOp.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerOp.java index c62de660ac..df4a9c0b1b 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerOp.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerOp.java @@ -14,7 +14,7 @@ package org.apache.tez.dag.app.launcher; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.tez.serviceplugins.api.ContainerLaunchRequest; import org.apache.tez.serviceplugins.api.ContainerLauncherOperationBase; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java index 13e4115d9b..a63ce3fdfe 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java @@ -34,7 +34,7 @@ import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.atomic.AtomicBoolean; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Maps; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java index 6a78425caa..3ba1bfaf65 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java @@ -19,7 +19,7 @@ package org.apache.tez.dag.app.rm; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.commons.lang.mutable.MutableInt; import org.apache.commons.math3.random.RandomDataGenerator; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java index f273c8b04a..a31b4f1e2d 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java @@ -18,8 +18,8 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.Map; +import java.util.Objects; -import com.google.common.base.Preconditions; import org.apache.hadoop.yarn.api.records.ApplicationAccessType; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.Container; @@ -190,7 +190,7 @@ public int getVertexIndexForTask(Object task) { @Override public void reportError(ServicePluginError servicePluginError, String diagnostics, DagInfo dagInfo) { - Preconditions.checkNotNull(servicePluginError, "ServicePluginError must be specified"); + Objects.requireNonNull(servicePluginError, "ServicePluginError must be specified"); taskSchedulerManager.reportError(schedulerId, servicePluginError, diagnostics, dagInfo); } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java index 61e3702eaf..57eba015d8 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java @@ -28,6 +28,7 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.Objects; import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -92,7 +93,7 @@ import org.apache.tez.hadoop.shim.HadoopShim; import org.apache.tez.hadoop.shim.HadoopShimsLoader; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; public class TaskSchedulerManager extends AbstractService implements @@ -483,7 +484,7 @@ private void handleTaLaunchRequest(AMSchedulerEventTALaunchRequest event) { TaskBasedLocationAffinity taskAffinity = locationHint.getAffinitizedTask(); if (taskAffinity != null) { Vertex vertex = appContext.getCurrentDAG().getVertex(taskAffinity.getVertexName()); - Preconditions.checkNotNull(vertex, "Invalid vertex in task based affinity " + taskAffinity + Objects.requireNonNull(vertex, "Invalid vertex in task based affinity " + taskAffinity + " for attempt: " + taskAttempt.getID()); int taskIndex = taskAffinity.getTaskIndex(); Preconditions.checkState(taskIndex >=0 && taskIndex < vertex.getTotalTasks(), @@ -491,7 +492,8 @@ private void handleTaLaunchRequest(AMSchedulerEventTALaunchRequest event) { + " for attempt: " + taskAttempt.getID()); TaskAttempt affinityAttempt = vertex.getTask(taskIndex).getSuccessfulAttempt(); if (affinityAttempt != null) { - Preconditions.checkNotNull(affinityAttempt.getAssignedContainerID(), affinityAttempt.getID()); + Objects.requireNonNull(affinityAttempt.getAssignedContainerID(), + affinityAttempt.getID() == null ? null : affinityAttempt.getID().toString()); try { taskSchedulers[event.getSchedulerId()].allocateTask(taskAttempt, event.getCapability(), diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java index a3279676b7..c9102e9993 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java @@ -69,7 +69,7 @@ import org.apache.tez.common.ContainerSignatureMatcher; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java index 02243b8aca..6b67eb9da1 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java @@ -73,7 +73,7 @@ import org.apache.tez.dag.records.TezTaskAttemptID; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; @SuppressWarnings("rawtypes") public class AMContainerImpl implements AMContainer { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java index 7b22ba6416..cfc143a402 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java @@ -19,8 +19,8 @@ package org.apache.tez.dag.app.rm.container; import java.util.Map; +import java.util.Objects; -import com.google.common.base.Preconditions; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.tez.runtime.api.impl.TaskSpec; @@ -35,7 +35,7 @@ public class AMContainerTask { public AMContainerTask(TaskSpec tezTask, Map additionalResources, Credentials credentials, boolean credentialsChanged, int priority) { - Preconditions.checkNotNull(tezTask, "TaskSpec cannot be null"); + Objects.requireNonNull(tezTask, "TaskSpec cannot be null"); this.tezTask = tezTask; this.additionalResources = additionalResources; this.credentials = credentials; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/ContainerContextMatcher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/ContainerContextMatcher.java index 436f098427..f9c57c8843 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/ContainerContextMatcher.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/ContainerContextMatcher.java @@ -21,18 +21,19 @@ import java.util.HashMap; import java.util.Map; import java.util.Map.Entry; +import java.util.Objects; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.tez.dag.app.ContainerContext; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import org.apache.tez.common.ContainerSignatureMatcher; public class ContainerContextMatcher implements ContainerSignatureMatcher { private void checkArguments(Object cs1, Object cs2) { - Preconditions.checkNotNull(cs1, "Arguments cannot be null"); - Preconditions.checkNotNull(cs2, "Arguments cannot be null"); + Objects.requireNonNull(cs1, "Arguments cannot be null"); + Objects.requireNonNull(cs2, "Arguments cannot be null"); Preconditions.checkArgument(cs1 instanceof ContainerContext && cs2 instanceof ContainerContext, "Container context can only compare instances of " @@ -62,8 +63,8 @@ public boolean isExactMatch(Object cs1, Object cs2) { @Override public Map getAdditionalResources(Map lr1, Map lr2) { - Preconditions.checkNotNull(lr1); - Preconditions.checkNotNull(lr2); + Objects.requireNonNull(lr1); + Objects.requireNonNull(lr2); Map c2LocalResources = new HashMap(lr2); for (Entry c1LocalResource : lr1.entrySet()) { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java index 1ac178b9f4..b1560a5ead 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java @@ -22,7 +22,7 @@ import java.net.InetSocketAddress; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.inject.name.Names; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/utils/DAGUtils.java b/tez-dag/src/main/java/org/apache/tez/dag/history/utils/DAGUtils.java index 77d3447875..f59e9ace7c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/utils/DAGUtils.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/utils/DAGUtils.java @@ -54,7 +54,7 @@ import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; public class DAGUtils { diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java index 893e03d3a1..638d0844f3 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java @@ -73,7 +73,7 @@ import org.apache.tez.runtime.api.impl.TezEvent; import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.util.concurrent.FutureCallback; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java index 92e43aab20..601aca7b7d 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java @@ -40,7 +40,7 @@ import java.util.List; import java.util.Map; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.BiMap; import com.google.common.collect.HashBiMap; import com.google.common.collect.Lists; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java index 9cd1e231da..b7acc6876c 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java @@ -43,8 +43,8 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.Objects; -import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.hadoop.conf.Configuration; @@ -392,8 +392,8 @@ static class AlwaysMatchesContainerMatcher implements ContainerSignatureMatcher @Override public boolean isSuperSet(Object cs1, Object cs2) { - Preconditions.checkNotNull(cs1, "Arguments cannot be null"); - Preconditions.checkNotNull(cs2, "Arguments cannot be null"); + Objects.requireNonNull(cs1, "Arguments cannot be null"); + Objects.requireNonNull(cs2, "Arguments cannot be null"); return true; } @@ -417,8 +417,8 @@ public Object union(Object cs1, Object cs2) { static class PreemptionMatcher implements ContainerSignatureMatcher { @Override public boolean isSuperSet(Object cs1, Object cs2) { - Preconditions.checkNotNull(cs1, "Arguments cannot be null"); - Preconditions.checkNotNull(cs2, "Arguments cannot be null"); + Objects.requireNonNull(cs1, "Arguments cannot be null"); + Objects.requireNonNull(cs2, "Arguments cannot be null"); return true; } diff --git a/tez-dag/src/test/java/org/apache/tez/test/GraceShuffleVertexManagerForTest.java b/tez-dag/src/test/java/org/apache/tez/test/GraceShuffleVertexManagerForTest.java index 40a6bd3ad1..ff89ef83eb 100644 --- a/tez-dag/src/test/java/org/apache/tez/test/GraceShuffleVertexManagerForTest.java +++ b/tez-dag/src/test/java/org/apache/tez/test/GraceShuffleVertexManagerForTest.java @@ -18,7 +18,7 @@ package org.apache.tez.test; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.protobuf.ByteString; import org.apache.hadoop.conf.Configuration; @@ -33,6 +33,7 @@ import java.io.IOException; import java.util.EnumSet; +import java.util.Objects; /** * A shuffle vertex manager that will set the vertex's parallelism upon @@ -149,7 +150,7 @@ public ByteString toByteString() throws IOException { } private GraceConf build() { - Preconditions.checkNotNull(grandparentVertex, + Objects.requireNonNull(grandparentVertex, "Grandparent vertex is required"); Preconditions.checkArgument(desiredParallelism > 0, "Desired parallelism must be greater than 0"); diff --git a/tez-examples/src/main/java/org/apache/tez/examples/CartesianProduct.java b/tez-examples/src/main/java/org/apache/tez/examples/CartesianProduct.java index 84367f8dfa..a925137f97 100644 --- a/tez-examples/src/main/java/org/apache/tez/examples/CartesianProduct.java +++ b/tez-examples/src/main/java/org/apache/tez/examples/CartesianProduct.java @@ -17,7 +17,7 @@ */ package org.apache.tez.examples; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Text; diff --git a/tez-examples/src/main/java/org/apache/tez/examples/HashJoinExample.java b/tez-examples/src/main/java/org/apache/tez/examples/HashJoinExample.java index e762ac16dc..6eaa03f9d7 100644 --- a/tez-examples/src/main/java/org/apache/tez/examples/HashJoinExample.java +++ b/tez-examples/src/main/java/org/apache/tez/examples/HashJoinExample.java @@ -53,7 +53,7 @@ import org.apache.tez.runtime.library.partitioner.HashPartitioner; import org.apache.tez.runtime.library.processor.SimpleProcessor; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; /** * Simple example of joining 2 data sets using getFsStatistics(Path path, Configuration conf) th } public static Counter getMRCounter(TezCounter tezCounter) { - Preconditions.checkNotNull(tezCounter); + Objects.requireNonNull(tezCounter); return new MRCounters.MRCounter(tezCounter); } diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/MapReduceSplitContainer.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/MapReduceSplitContainer.java index 63e2138ca8..fc6a424bd3 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/MapReduceSplitContainer.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/MapReduceSplitContainer.java @@ -15,8 +15,8 @@ package org.apache.tez.mapreduce.grouper; import java.io.IOException; +import java.util.Objects; -import com.google.common.base.Preconditions; import org.apache.hadoop.mapreduce.InputSplit; public class MapReduceSplitContainer extends SplitContainer { @@ -24,7 +24,7 @@ public class MapReduceSplitContainer extends SplitContainer { private final InputSplit inputSplit; public MapReduceSplitContainer(InputSplit inputSplit) { - Preconditions.checkNotNull(inputSplit); + Objects.requireNonNull(inputSplit); this.inputSplit = inputSplit; } diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/MapredSplitContainer.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/MapredSplitContainer.java index f7dbfda491..ce86ad52ef 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/MapredSplitContainer.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/MapredSplitContainer.java @@ -15,8 +15,8 @@ package org.apache.tez.mapreduce.grouper; import java.io.IOException; +import java.util.Objects; -import com.google.common.base.Preconditions; import org.apache.hadoop.mapred.InputSplit; public class MapredSplitContainer extends SplitContainer { @@ -24,7 +24,7 @@ public class MapredSplitContainer extends SplitContainer { private final InputSplit inputSplit; public MapredSplitContainer(InputSplit inputSplit) { - Preconditions.checkNotNull(inputSplit); + Objects.requireNonNull(inputSplit); this.inputSplit = inputSplit; } diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java index 26e5a9ed37..6f3d9df033 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java @@ -25,7 +25,7 @@ import java.util.Set; import java.util.TreeMap; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.util.RackResolver; import org.apache.tez.dag.api.TezUncheckedException; diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/InputSplitInfoMem.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/InputSplitInfoMem.java index d7873fc0c8..5e1207b7b5 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/InputSplitInfoMem.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/InputSplitInfoMem.java @@ -28,7 +28,7 @@ import org.apache.tez.dag.api.TaskLocationHint; import org.apache.tez.mapreduce.protos.MRRuntimeProtos.MRSplitsProto; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; /** * Represents InputSplitInfo for splits generated to memory.

diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java index 2f3d7ce3ec..9c42cdaf4e 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java @@ -27,9 +27,10 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import com.google.common.base.Function; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.protobuf.ByteString; @@ -165,7 +166,7 @@ public static InputSplit createOldFormatSplitFromUserPayload( throws IOException { // This may not need to use serialization factory, since OldFormat // always uses Writable to write splits. - Preconditions.checkNotNull(splitProto, "splitProto cannot be null"); + Objects.requireNonNull(splitProto, "splitProto cannot be null"); String className = splitProto.getSplitClassName(); Class clazz; @@ -198,7 +199,7 @@ public static InputSplit createOldFormatSplitFromUserPayload( public static org.apache.hadoop.mapreduce.InputSplit createNewFormatSplitFromUserPayload( MRRuntimeProtos.MRSplitProto splitProto, SerializationFactory serializationFactory) throws IOException { - Preconditions.checkNotNull(splitProto, "splitProto must be specified"); + Objects.requireNonNull(splitProto, "splitProto must be specified"); String className = splitProto.getSplitClassName(); Class clazz; @@ -771,8 +772,8 @@ private static UserPayload createMRInputPayload(ByteString bytes, private static String getStringProperty(Configuration conf, String propertyName) { - Preconditions.checkNotNull(conf, "Configuration must be provided"); - Preconditions.checkNotNull(propertyName, "Property name must be provided"); + Objects.requireNonNull(conf, "Configuration must be provided"); + Objects.requireNonNull(propertyName, "Property name must be provided"); return conf.get(propertyName); } diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java index 317f6ebe55..5c8ad4ea68 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java @@ -70,7 +70,7 @@ import org.apache.tez.runtime.library.api.KeyValueReader; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; /** diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java index de54b0d1f4..ee776a59f2 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java @@ -30,7 +30,7 @@ import java.util.concurrent.locks.ReentrantLock; import com.google.common.base.Function; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import com.google.protobuf.ByteString; diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/base/MRInputBase.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/base/MRInputBase.java index 9a26c2b7c5..d8c531ea84 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/base/MRInputBase.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/base/MRInputBase.java @@ -18,7 +18,7 @@ package org.apache.tez.mapreduce.input.base; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapReduce.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapReduce.java index 10b871edcc..ad3d4d67ab 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapReduce.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapReduce.java @@ -19,6 +19,7 @@ package org.apache.tez.mapreduce.lib; import java.io.IOException; +import java.util.Objects; import org.apache.tez.runtime.api.InputContext; import org.apache.tez.runtime.library.api.IOInterruptedException; @@ -34,8 +35,6 @@ import org.apache.hadoop.mapreduce.InputFormat; import org.apache.hadoop.mapreduce.RecordReader; -import com.google.common.base.Preconditions; - public class MRReaderMapReduce extends MRReader { private static final Logger LOG = LoggerFactory.getLogger(MRReader.class); @@ -151,7 +150,7 @@ public Object getCurrentValue() throws IOException { } private void setupNewRecordReader() throws IOException { - Preconditions.checkNotNull(inputSplit, "Input split hasn't yet been setup"); + Objects.requireNonNull(inputSplit, "Input split hasn't yet been setup"); try { recordReader = inputFormat.createRecordReader(inputSplit, taskAttemptContext); recordReader.initialize(inputSplit, taskAttemptContext); diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapred.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapred.java index d81debb732..21a4f96df4 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapred.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapred.java @@ -19,6 +19,7 @@ package org.apache.tez.mapreduce.lib; import java.io.IOException; +import java.util.Objects; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,8 +36,6 @@ import org.apache.tez.mapreduce.input.MRInput; import org.apache.tez.runtime.api.InputContext; -import com.google.common.base.Preconditions; - public class MRReaderMapred extends MRReader { private static final Logger LOG = LoggerFactory.getLogger(MRReaderMapred.class); @@ -153,7 +152,7 @@ public Configuration getConfigUpdates() { } private void setupOldRecordReader() throws IOException { - Preconditions.checkNotNull(inputSplit, "Input split hasn't yet been setup"); + Objects.requireNonNull(inputSplit, "Input split hasn't yet been setup"); recordReader = inputFormat.getRecordReader(inputSplit, this.jobConf, new MRReporter( tezCounters, inputSplit)); setIncrementalConfigParams(inputSplit); diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java index 6ed70c5e73..18047ebf09 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java @@ -29,7 +29,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import org.apache.commons.lang.StringUtils; import org.apache.hadoop.mapreduce.lib.output.LazyOutputFormat; import org.apache.tez.runtime.library.api.IOInterruptedException; diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/ATSImportTool.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/ATSImportTool.java index fee226acf3..f05e946acc 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/ATSImportTool.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/ATSImportTool.java @@ -20,7 +20,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.base.Strings; import com.sun.jersey.api.client.Client; import com.sun.jersey.api.client.ClientHandlerException; diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java index fb42129d21..e4720d41fb 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java @@ -18,7 +18,7 @@ package org.apache.tez.history.parser; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Maps; import org.apache.commons.io.IOUtils; import org.apache.tez.dag.api.TezException; diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java index 989dd5142b..3516de74a8 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java @@ -17,7 +17,7 @@ */ package org.apache.tez.history.parser; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.base.Strings; import com.google.common.collect.Maps; import org.apache.tez.dag.api.TezException; diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseParser.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseParser.java index 362dbd9837..59ec03d972 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseParser.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseParser.java @@ -18,7 +18,7 @@ package org.apache.tez.history.parser.datamodel; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.base.Predicate; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/DagInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/DagInfo.java index 6bd691c8b6..544e86a70d 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/DagInfo.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/DagInfo.java @@ -18,7 +18,7 @@ package org.apache.tez.history.parser.datamodel; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.base.Predicate; import com.google.common.collect.Iterables; import com.google.common.collect.LinkedHashMultimap; diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java index 885d74392a..3f39310d1c 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java @@ -18,7 +18,7 @@ package org.apache.tez.history.parser.datamodel; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.base.Strings; import com.google.common.collect.Lists; import com.google.common.collect.Maps; diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskInfo.java index fb3f232d49..de74ad29ad 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskInfo.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskInfo.java @@ -18,7 +18,7 @@ package org.apache.tez.history.parser.datamodel; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.base.Predicate; import com.google.common.base.Strings; import com.google.common.collect.Iterables; diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java index 038a874de3..efcce3bf76 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java @@ -18,7 +18,7 @@ package org.apache.tez.history.parser.datamodel; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.base.Predicate; import com.google.common.collect.Iterables; import com.google.common.collect.LinkedHashMultimap; diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/InputReadyTracker.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/InputReadyTracker.java index ba4fe1d170..b2b2b58e15 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/InputReadyTracker.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/InputReadyTracker.java @@ -33,7 +33,7 @@ import org.apache.tez.runtime.api.Input; import org.apache.tez.runtime.api.MergedLogicalInput; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java index 87ebb7b9d0..f8a3de2242 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java @@ -98,7 +98,7 @@ import org.apache.tez.runtime.common.resources.MemoryDistributor; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Multimap; import com.google.common.collect.Sets; diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java index 0ee96af27d..1191a9f283 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java @@ -18,11 +18,12 @@ package org.apache.tez.runtime.api.impl; -import static com.google.common.base.Preconditions.checkNotNull; + import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.Objects; import javax.annotation.Nullable; @@ -70,8 +71,8 @@ public EventMetaData() { public EventMetaData(EventProducerConsumerType generator, String taskVertexName, @Nullable String edgeVertexName, @Nullable TezTaskAttemptID taskAttemptID) { - checkNotNull(generator, "generator is null"); - checkNotNull(taskVertexName, "taskVertexName is null"); + Objects.requireNonNull(generator, "generator is null"); + Objects.requireNonNull(taskVertexName, "taskVertexName is null"); this.producerConsumerType = generator; this.taskVertexName = StringInterner.weakIntern(taskVertexName); this.edgeVertexName = StringInterner.weakIntern(edgeVertexName); diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java index 78bb1e9bb7..fe9afc2d0f 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java @@ -17,12 +17,13 @@ package org.apache.tez.runtime.api.impl; -import static com.google.common.base.Preconditions.checkNotNull; + import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Objects; import javax.annotation.Nullable; @@ -73,11 +74,11 @@ public TaskSpec( ProcessorDescriptor processorDescriptor, List inputSpecList, List outputSpecList, @Nullable List groupInputSpecList, Configuration taskConf) { - checkNotNull(dagName, "dagName is null"); - checkNotNull(vertexName, "vertexName is null"); - checkNotNull(processorDescriptor, "processorDescriptor is null"); - checkNotNull(inputSpecList, "inputSpecList is null"); - checkNotNull(outputSpecList, "outputSpecList is null"); + Objects.requireNonNull(dagName, "dagName is null"); + Objects.requireNonNull(vertexName, "vertexName is null"); + Objects.requireNonNull(processorDescriptor, "processorDescriptor is null"); + Objects.requireNonNull(inputSpecList, "inputSpecList is null"); + Objects.requireNonNull(outputSpecList, "outputSpecList is null"); this.taskAttemptId = null; this.dagName = StringInterner.weakIntern(dagName); this.vertexName = StringInterner.weakIntern(vertexName); @@ -105,12 +106,12 @@ public TaskSpec(TezTaskAttemptID taskAttemptID, ProcessorDescriptor processorDescriptor, List inputSpecList, List outputSpecList, @Nullable List groupInputSpecList, Configuration taskConf) { - checkNotNull(taskAttemptID, "taskAttemptID is null"); - checkNotNull(dagName, "dagName is null"); - checkNotNull(vertexName, "vertexName is null"); - checkNotNull(processorDescriptor, "processorDescriptor is null"); - checkNotNull(inputSpecList, "inputSpecList is null"); - checkNotNull(outputSpecList, "outputSpecList is null"); + Objects.requireNonNull(taskAttemptID, "taskAttemptID is null"); + Objects.requireNonNull(dagName, "dagName is null"); + Objects.requireNonNull(vertexName, "vertexName is null"); + Objects.requireNonNull(processorDescriptor, "processorDescriptor is null"); + Objects.requireNonNull(inputSpecList, "inputSpecList is null"); + Objects.requireNonNull(outputSpecList, "outputSpecList is null"); this.taskAttemptId = taskAttemptID; this.dagName = StringInterner.weakIntern(dagName); this.vertexName = StringInterner.weakIntern(vertexName); diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskStatistics.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskStatistics.java index 0b4bef8280..eba5823ebd 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskStatistics.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskStatistics.java @@ -26,7 +26,7 @@ import org.apache.hadoop.io.Writable; import org.apache.hadoop.util.StringInterner; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Maps; public class TaskStatistics implements Writable { diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java index 15a6485791..9ff284d5bf 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java @@ -18,15 +18,12 @@ package org.apache.tez.runtime.api.impl; -import com.google.common.base.Preconditions; - -import static com.google.common.base.Preconditions.checkNotNull; - import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Objects; import javax.annotation.Nullable; @@ -102,10 +99,10 @@ vertexParallelism, taskAttemptID, wrapCounters(runtimeTask, taskVertexName, sourceVertexName, conf), runtimeTask, tezUmbilical, serviceConsumerMetadata, auxServiceEnv, memDist, inputDescriptor, objectRegistry, ExecutionContext, memAvailable, sharedExecutor); - checkNotNull(inputIndex, "inputIndex is null"); - checkNotNull(sourceVertexName, "sourceVertexName is null"); - checkNotNull(inputs, "input map is null"); - checkNotNull(inputReadyTracker, "inputReadyTracker is null"); + Objects.requireNonNull(inputIndex, "inputIndex is null"); + Objects.requireNonNull(sourceVertexName, "sourceVertexName is null"); + Objects.requireNonNull(inputs, "input map is null"); + Objects.requireNonNull(inputReadyTracker, "inputReadyTracker is null"); this.userPayload = userPayload; this.inputIndex = inputIndex; this.sourceVertexName = sourceVertexName; @@ -131,7 +128,7 @@ private static TezCounters wrapCounters(LogicalIOProcessorRuntimeTask task, Stri @Override public void sendEvents(List events) { - Preconditions.checkNotNull(events, "events are null"); + Objects.requireNonNull(events, "events are null"); List tezEvents = new ArrayList(events.size()); for (Event e : events) { TezEvent tEvt = new TezEvent(e, sourceInfo); diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezMergedInputContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezMergedInputContextImpl.java index e35e3325f1..553efcd467 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezMergedInputContextImpl.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezMergedInputContextImpl.java @@ -18,12 +18,13 @@ package org.apache.tez.runtime.api.impl; -import static com.google.common.base.Preconditions.checkNotNull; + import javax.annotation.Nullable; import java.util.Arrays; import java.util.Map; +import java.util.Objects; import org.apache.tez.dag.api.UserPayload; import org.apache.tez.runtime.InputReadyTracker; @@ -45,9 +46,9 @@ public TezMergedInputContextImpl(@Nullable UserPayload userPayload, String group Map groupInputsMap, InputReadyTracker inputReadyTracker, String[] workDirs, LogicalIOProcessorRuntimeTask runtimeTask) { - checkNotNull(groupInputName, "groupInputName is null"); - checkNotNull(groupInputsMap, "input-group map is null"); - checkNotNull(inputReadyTracker, "inputReadyTracker is null"); + Objects.requireNonNull(groupInputName, "groupInputName is null"); + Objects.requireNonNull(groupInputsMap, "input-group map is null"); + Objects.requireNonNull(inputReadyTracker, "inputReadyTracker is null"); this.groupInputName = groupInputName; this.groupInputsMap = groupInputsMap; this.userPayload = userPayload; diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java index 41e8d41fd0..db3212280f 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java @@ -18,15 +18,12 @@ package org.apache.tez.runtime.api.impl; -import com.google.common.base.Preconditions; - -import static com.google.common.base.Preconditions.checkNotNull; - import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Objects; import javax.annotation.Nullable; @@ -97,8 +94,8 @@ public TezOutputContextImpl(Configuration conf, String[] workDirs, int appAttemp runtimeTask, tezUmbilical, serviceConsumerMetadata, auxServiceEnv, memDist, outputDescriptor, objectRegistry, executionContext, memAvailable, sharedExecutor); - checkNotNull(outputIndex, "outputIndex is null"); - checkNotNull(destinationVertexName, "destinationVertexName is null"); + Objects.requireNonNull(outputIndex, "outputIndex is null"); + Objects.requireNonNull(destinationVertexName, "destinationVertexName is null"); this.userPayload = userPayload; this.outputIndex = outputIndex; this.destinationVertexName = destinationVertexName; @@ -121,7 +118,7 @@ private static TezCounters wrapCounters(LogicalIOProcessorRuntimeTask runtimeTas @Override public void sendEvents(List events) { - Preconditions.checkNotNull(events, "events are null"); + Objects.requireNonNull(events, "events are null"); List tezEvents = new ArrayList(events.size()); for (Event e : events) { TezEvent tEvt = new TezEvent(e, sourceInfo); diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java index 54605c8645..71ed077c50 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java @@ -18,16 +18,13 @@ package org.apache.tez.runtime.api.impl; -import com.google.common.base.Preconditions; - -import static com.google.common.base.Preconditions.checkNotNull; - import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.Objects; import javax.annotation.Nullable; @@ -69,7 +66,7 @@ public TezProcessorContextImpl(Configuration conf, String[] workDirs, int appAtt runtimeTask.addAndGetTezCounter(vertexName), runtimeTask, tezUmbilical, serviceConsumerMetadata, auxServiceEnv, memDist, processorDescriptor, objectRegistry, ExecutionContext, memAvailable, sharedExecutor); - checkNotNull(inputReadyTracker, "inputReadyTracker is null"); + Objects.requireNonNull(inputReadyTracker, "inputReadyTracker is null"); this.userPayload = userPayload; this.sourceInfo = new EventMetaData(EventProducerConsumerType.PROCESSOR, taskVertexName, "", taskAttemptID); @@ -78,7 +75,7 @@ public TezProcessorContextImpl(Configuration conf, String[] workDirs, int appAtt @Override public void sendEvents(List events) { - Preconditions.checkNotNull(events, "events are null"); + Objects.requireNonNull(events, "events are null"); List tezEvents = new ArrayList(events.size()); for (Event e : events) { TezEvent tEvt = new TezEvent(e, sourceInfo); diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java index 5a6a405608..dccde823e7 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java @@ -18,7 +18,7 @@ package org.apache.tez.runtime.api.impl; -import static com.google.common.base.Preconditions.checkNotNull; + import java.io.Closeable; import java.io.IOException; @@ -27,6 +27,7 @@ import java.util.Map; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicInteger; +import java.util.Objects; import javax.annotation.Nullable; @@ -46,7 +47,7 @@ import org.apache.tez.runtime.api.TaskContext; import org.apache.tez.runtime.common.resources.MemoryDistributor; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; public abstract class TezTaskContextImpl implements TaskContext, Closeable { @@ -79,16 +80,16 @@ public TezTaskContextImpl(Configuration conf, String[] workDirs, int appAttemptN Map auxServiceEnv, MemoryDistributor memDist, EntityDescriptor descriptor, ObjectRegistry objectRegistry, ExecutionContext ExecutionContext, long memAvailable, TezExecutors sharedExecutor) { - checkNotNull(conf, "conf is null"); - checkNotNull(dagName, "dagName is null"); - checkNotNull(taskVertexName, "taskVertexName is null"); - checkNotNull(taskAttemptID, "taskAttemptId is null"); - checkNotNull(counters, "counters is null"); - checkNotNull(runtimeTask, "runtimeTask is null"); - checkNotNull(auxServiceEnv, "auxServiceEnv is null"); - checkNotNull(memDist, "memDist is null"); - checkNotNull(descriptor, "descriptor is null"); - checkNotNull(sharedExecutor, "sharedExecutor is null"); + Objects.requireNonNull(conf, "conf is null"); + Objects.requireNonNull(dagName, "dagName is null"); + Objects.requireNonNull(taskVertexName, "taskVertexName is null"); + Objects.requireNonNull(taskAttemptID, "taskAttemptId is null"); + Objects.requireNonNull(counters, "counters is null"); + Objects.requireNonNull(runtimeTask, "runtimeTask is null"); + Objects.requireNonNull(auxServiceEnv, "auxServiceEnv is null"); + Objects.requireNonNull(memDist, "memDist is null"); + Objects.requireNonNull(descriptor, "descriptor is null"); + Objects.requireNonNull(sharedExecutor, "sharedExecutor is null"); this.dagName = dagName; this.taskVertexName = taskVertexName; this.taskAttemptID = taskAttemptID; @@ -193,7 +194,7 @@ public ByteBuffer getServiceConsumerMetaData(String serviceName) { @Nullable @Override public ByteBuffer getServiceProviderMetaData(String serviceName) { - Preconditions.checkNotNull(serviceName, "serviceName is null"); + Objects.requireNonNull(serviceName, "serviceName is null"); return AuxiliaryServiceHelper.getServiceDataFromEnv( serviceName, auxServiceEnv); } @@ -225,7 +226,7 @@ protected void signalFatalError(Throwable t, String message, EventMetaData sourc protected void signalFailure(TaskFailureType taskFailureType, Throwable t, String message, EventMetaData sourceInfo) { - Preconditions.checkNotNull(taskFailureType, "TaskFailureType must be specified"); + Objects.requireNonNull(taskFailureType, "TaskFailureType must be specified"); runtimeTask.setFrameworkCounters(); runtimeTask.registerError(); tezUmbilical.signalFailure(taskAttemptID, taskFailureType, t, message, sourceInfo); diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/resources/MemoryDistributor.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/resources/MemoryDistributor.java index e63a414cf7..d8ba1f71ff 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/resources/MemoryDistributor.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/resources/MemoryDistributor.java @@ -25,6 +25,7 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicInteger; +import java.util.Objects; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,7 +44,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Iterables; // Not calling this a MemoryManager explicitly. Not yet anyway. @@ -175,9 +176,9 @@ void setJvmMemory(long size) { private long registerRequest(long requestSize, MemoryUpdateCallback callback, TaskContext entityContext, EntityDescriptor descriptor) { Preconditions.checkArgument(requestSize >= 0); - Preconditions.checkNotNull(callback); - Preconditions.checkNotNull(entityContext); - Preconditions.checkNotNull(descriptor); + Objects.requireNonNull(callback); + Objects.requireNonNull(entityContext); + Objects.requireNonNull(descriptor); if (!dupSet.add(entityContext)) { throw new TezUncheckedException( "A single entity can only make one call to request resources for now"); @@ -207,7 +208,7 @@ private long registerRequest(long requestSize, MemoryUpdateCallback callback, } private void validateAllocations(Iterable allocations, int numRequestors) { - Preconditions.checkNotNull(allocations); + Objects.requireNonNull(allocations); long totalAllocated = 0l; int numAllocations = 0; for (Long l : allocations) { diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/resources/ScalingAllocator.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/resources/ScalingAllocator.java index aebb19be36..872632e1ea 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/resources/ScalingAllocator.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/resources/ScalingAllocator.java @@ -29,7 +29,7 @@ import org.apache.tez.dag.api.TezConfiguration; import com.google.common.base.Function; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java index bae7f522d3..06933170f4 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java @@ -76,7 +76,7 @@ import org.slf4j.LoggerFactory; import com.google.common.base.Function; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.HashMultimap; import com.google.common.collect.Maps; import com.google.common.collect.Multimap; diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java index 306f2a77cd..ae81769aca 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java @@ -29,7 +29,7 @@ import java.util.concurrent.locks.ReentrantLock; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Multimap; import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.hadoop.conf.Configuration; diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution2.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution2.java index 6c25f0a245..810c3ac82b 100644 --- a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution2.java +++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution2.java @@ -38,7 +38,7 @@ import java.util.concurrent.locks.ReentrantLock; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.HashMultimap; import com.google.common.collect.Multimap; import com.google.common.util.concurrent.ListeningExecutorService; diff --git a/tez-runtime-library/src/main/java/org/apache/hadoop/io/FileChunk.java b/tez-runtime-library/src/main/java/org/apache/hadoop/io/FileChunk.java index e7a5c244dc..714bbcd484 100644 --- a/tez-runtime-library/src/main/java/org/apache/hadoop/io/FileChunk.java +++ b/tez-runtime-library/src/main/java/org/apache/hadoop/io/FileChunk.java @@ -18,7 +18,8 @@ package org.apache.hadoop.io; -import com.google.common.base.Preconditions; +import java.util.Objects; + import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.fs.Path; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; @@ -40,7 +41,7 @@ public FileChunk(Path path, long offset, long length, boolean isLocalFile, this.isLocalFile = isLocalFile; this.identifier = identifier; if (isLocalFile) { - Preconditions.checkNotNull(identifier); + Objects.requireNonNull(identifier); } } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/FairShuffleVertexManager.java b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/FairShuffleVertexManager.java index f3971eb998..af4e5b8b26 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/FairShuffleVertexManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/FairShuffleVertexManager.java @@ -18,7 +18,7 @@ package org.apache.tez.dag.library.vertexmanager; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.UnmodifiableIterator; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/InputReadyVertexManager.java b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/InputReadyVertexManager.java index f05cd955e4..33c4a99059 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/InputReadyVertexManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/InputReadyVertexManager.java @@ -40,7 +40,7 @@ import org.apache.tez.runtime.api.TaskAttemptIdentifier; import org.apache.tez.runtime.api.events.VertexManagerEvent; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/ShuffleVertexManager.java b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/ShuffleVertexManager.java index ed27f04b8c..b05c45ad96 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/ShuffleVertexManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/ShuffleVertexManager.java @@ -18,7 +18,7 @@ package org.apache.tez.dag.library.vertexmanager; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import com.google.protobuf.ByteString; import com.google.protobuf.InvalidProtocolBufferException; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/ShuffleVertexManagerBase.java b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/ShuffleVertexManagerBase.java index bb63bd5ce2..05e2d8ce78 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/ShuffleVertexManagerBase.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/ShuffleVertexManagerBase.java @@ -19,7 +19,7 @@ package org.apache.tez.dag.library.vertexmanager; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.base.Predicate; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnection.java b/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnection.java index 9bfe4e7099..3b45cdd709 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnection.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnection.java @@ -19,7 +19,7 @@ package org.apache.tez.http; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.IOUtils; import org.apache.tez.common.security.JobTokenSecretManager; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java b/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java index 9243e97712..ac0a49c401 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java @@ -19,7 +19,7 @@ package org.apache.tez.http.async.netty; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.ning.http.client.AsyncHttpClient; import com.ning.http.client.AsyncHttpClientConfig; import com.ning.http.client.ListenableFuture; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductCombination.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductCombination.java index 8de8a026c0..c1eae7019e 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductCombination.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductCombination.java @@ -17,7 +17,7 @@ */ package org.apache.tez.runtime.library.cartesianproduct; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.primitives.Ints; import java.util.Arrays; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductConfig.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductConfig.java index 7aac1d73ee..e47b083127 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductConfig.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductConfig.java @@ -18,7 +18,7 @@ package org.apache.tez.runtime.library.cartesianproduct; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.primitives.Ints; import com.google.protobuf.ByteString; import com.google.protobuf.InvalidProtocolBufferException; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductEdgeManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductEdgeManager.java index a406c1ba56..0a3346ea16 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductEdgeManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductEdgeManager.java @@ -18,7 +18,7 @@ package org.apache.tez.runtime.library.cartesianproduct; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.protobuf.ByteString; import org.apache.tez.dag.api.EdgeManagerPluginContext; import org.apache.tez.dag.api.EdgeManagerPluginOnDemand; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductVertexManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductVertexManager.java index ff2259302f..d1fcece722 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductVertexManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductVertexManager.java @@ -18,7 +18,7 @@ package org.apache.tez.runtime.library.cartesianproduct; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.protobuf.ByteString; import org.apache.tez.dag.api.EdgeManagerPluginDescriptor; import org.apache.tez.dag.api.EdgeProperty; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java index 6aa797fd0c..24ad0ad1e3 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java @@ -24,7 +24,7 @@ import java.util.Map; import java.util.Set; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/MemoryUpdateCallbackHandler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/MemoryUpdateCallbackHandler.java index 68f754d1e7..ff31b49984 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/MemoryUpdateCallbackHandler.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/MemoryUpdateCallbackHandler.java @@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceStability.Evolving; import org.apache.tez.runtime.api.MemoryUpdateCallback; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; @Public @Evolving diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ValuesIterator.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ValuesIterator.java index 7add8c5ec0..4a75cbdd04 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ValuesIterator.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ValuesIterator.java @@ -33,7 +33,7 @@ import org.apache.tez.common.counters.TezCounter; import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; /** * Iterates values while keys match in sorted input. diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/DiskFetchedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/DiskFetchedInput.java index 5d1c037030..22b2899e1e 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/DiskFetchedInput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/DiskFetchedInput.java @@ -31,7 +31,7 @@ import org.apache.tez.runtime.library.common.InputAttemptIdentifier; import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutputFiles; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; public class DiskFetchedInput extends FetchedInput { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java index a3bbeca365..fa883e4d64 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java @@ -67,7 +67,7 @@ import org.apache.tez.runtime.library.exceptions.FetcherReadTimeoutException; import org.apache.tez.runtime.library.common.shuffle.FetchedInput.Type; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; /** * Responsible for fetching inputs served by the ShuffleHandler for a single diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/LocalDiskFetchedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/LocalDiskFetchedInput.java index 0ae8f08c19..cb6ea0599f 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/LocalDiskFetchedInput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/LocalDiskFetchedInput.java @@ -23,7 +23,7 @@ import java.io.OutputStream; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import org.apache.commons.io.input.BoundedInputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/MemoryFetchedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/MemoryFetchedInput.java index d0c3e771b7..63aefa8d13 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/MemoryFetchedInput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/MemoryFetchedInput.java @@ -25,7 +25,7 @@ import org.apache.tez.common.io.NonSyncByteArrayInputStream; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; public class MemoryFetchedInput extends FetchedInput { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java index 00f37454fd..1482a12b51 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java @@ -35,7 +35,7 @@ import javax.annotation.Nullable; import javax.crypto.SecretKey; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.primitives.Ints; import com.google.protobuf.ByteString; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java index 5e03f08e23..b91a5cafa7 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java @@ -89,7 +89,7 @@ import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils.FetchStatsLogger; import com.google.common.base.Objects; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.util.concurrent.FutureCallback; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java index 6ffdb5657b..2e5cc20096 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java @@ -18,7 +18,7 @@ package org.apache.tez.runtime.library.common.shuffle.orderedgrouped; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import org.apache.commons.io.FilenameUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience.Private; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java index 0089d8c1fe..4b426b75c8 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java @@ -56,7 +56,7 @@ import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator; import org.apache.tez.runtime.library.exceptions.InputAlreadyClosedException; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java index d84793211e..d388b5b1bf 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java @@ -43,7 +43,7 @@ import java.util.concurrent.atomic.AtomicLong; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.LinkedListMultimap; import com.google.common.collect.ListMultimap; import com.google.common.collect.Maps; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java index 9e65862e0c..ae03e3d7a9 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java @@ -64,7 +64,7 @@ import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer; import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; @SuppressWarnings({"unchecked", "rawtypes"}) public abstract class ExternalSorter { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java index 028dd2f4e0..d84164fca6 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java @@ -34,7 +34,7 @@ import java.util.zip.Deflater; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import org.apache.hadoop.classification.InterfaceAudience; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java index 9b5a43c363..6d45ee1560 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java @@ -62,7 +62,7 @@ import org.apache.tez.runtime.library.common.sort.impl.TezMerger.DiskSegment; import org.apache.tez.runtime.library.common.sort.impl.TezMerger.Segment; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import static org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord.SPILL_FILE_PERMS; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutputFiles.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutputFiles.java index 97a250913f..88474f99f4 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutputFiles.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutputFiles.java @@ -20,7 +20,7 @@ import java.io.IOException; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java index db0c30e660..9206b18608 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java @@ -77,7 +77,7 @@ import org.slf4j.LoggerFactory; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedGroupedKVInputConfig.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedGroupedKVInputConfig.java index 11a8d6fd9e..e7b96930a9 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedGroupedKVInputConfig.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedGroupedKVInputConfig.java @@ -24,9 +24,10 @@ import java.io.IOException; import java.util.Map; +import java.util.Objects; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import org.apache.hadoop.classification.InterfaceAudience; @@ -299,8 +300,8 @@ public static class Builder implements SpecificConfigBuilder { @InterfaceAudience.Private Builder(String keyClassName, String valueClassName) { this(); - Preconditions.checkNotNull(keyClassName, "Key class name cannot be null"); - Preconditions.checkNotNull(valueClassName, "Value class name cannot be null"); + Objects.requireNonNull(keyClassName, "Key class name cannot be null"); + Objects.requireNonNull(valueClassName, "Value class name cannot be null"); setKeyClassName(keyClassName); setValueClassName(valueClassName); } @@ -316,14 +317,14 @@ public static class Builder implements SpecificConfigBuilder { @InterfaceAudience.Private Builder setKeyClassName(String keyClassName) { - Preconditions.checkNotNull(keyClassName, "Key class name cannot be null"); + Objects.requireNonNull(keyClassName, "Key class name cannot be null"); this.conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, keyClassName); return this; } @InterfaceAudience.Private Builder setValueClassName(String valueClassName) { - Preconditions.checkNotNull(valueClassName, "Value class name cannot be null"); + Objects.requireNonNull(valueClassName, "Value class name cannot be null"); this.conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS, valueClassName); return this; } @@ -404,7 +405,7 @@ public Builder setKeyComparatorClass(String comparatorClassName) { */ public Builder setKeyComparatorClass(String comparatorClassName, @Nullable Map comparatorConf) { - Preconditions.checkNotNull(comparatorClassName, "Comparator class name cannot be null"); + Objects.requireNonNull(comparatorClassName, "Comparator class name cannot be null"); this.conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_COMPARATOR_CLASS, comparatorClassName); if (comparatorConf != null) { @@ -418,7 +419,7 @@ public Builder setKeyComparatorClass(String comparatorClassName, @SuppressWarnings("unchecked") @Override public Builder setAdditionalConfiguration(String key, String value) { - Preconditions.checkNotNull(key, "Key cannot be null"); + Objects.requireNonNull(key, "Key cannot be null"); if (ConfigUtils.doesKeyQualify(key, Lists.newArrayList(OrderedGroupedKVInput.getConfigurationKeySet(), TezRuntimeConfiguration.getRuntimeAdditionalConfigKeySet()), @@ -435,7 +436,7 @@ public Builder setAdditionalConfiguration(String key, String value) { @SuppressWarnings("unchecked") @Override public Builder setAdditionalConfiguration(Map confMap) { - Preconditions.checkNotNull(confMap, "ConfMap cannot be null"); + Objects.requireNonNull(confMap, "ConfMap cannot be null"); Map map = ConfigUtils.extractConfigurationMap(confMap, Lists.newArrayList(OrderedGroupedKVInput.getConfigurationKeySet(), TezRuntimeConfiguration.getRuntimeAdditionalConfigKeySet()), TezRuntimeConfiguration.getAllowedPrefixes()); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedPartitionedKVEdgeConfig.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedPartitionedKVEdgeConfig.java index 350420390a..e4e9be2de0 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedPartitionedKVEdgeConfig.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedPartitionedKVEdgeConfig.java @@ -21,8 +21,7 @@ import javax.annotation.Nullable; import java.util.Map; - -import com.google.common.base.Preconditions; +import java.util.Objects; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -145,7 +144,7 @@ public EdgeProperty createDefaultEdgeProperty() { * @return an {@link org.apache.tez.dag.api.EdgeProperty} instance */ public EdgeProperty createDefaultCustomEdgeProperty(EdgeManagerPluginDescriptor edgeManagerDescriptor) { - Preconditions.checkNotNull(edgeManagerDescriptor, "EdgeManagerDescriptor cannot be null"); + Objects.requireNonNull(edgeManagerDescriptor, "EdgeManagerDescriptor cannot be null"); EdgeProperty edgeProperty = EdgeProperty.create(edgeManagerDescriptor, EdgeProperty.DataSourceType.PERSISTED, EdgeProperty.SchedulingType.SEQUENTIAL, diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedPartitionedKVOutputConfig.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedPartitionedKVOutputConfig.java index 0f37c66616..7c92f38260 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedPartitionedKVOutputConfig.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedPartitionedKVOutputConfig.java @@ -24,9 +24,10 @@ import java.io.IOException; import java.util.Map; +import java.util.Objects; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import org.apache.hadoop.classification.InterfaceAudience; @@ -261,9 +262,9 @@ public static class Builder implements SpecificConfigBuilder { Builder(String keyClassName, String valueClassName, String partitionerClassName, @Nullable Map partitionerConf) { this(); - Preconditions.checkNotNull(keyClassName, "Key class name cannot be null"); - Preconditions.checkNotNull(valueClassName, "Value class name cannot be null"); - Preconditions.checkNotNull(partitionerClassName, "Partitioner class name cannot be null"); + Objects.requireNonNull(keyClassName, "Key class name cannot be null"); + Objects.requireNonNull(valueClassName, "Value class name cannot be null"); + Objects.requireNonNull(partitionerClassName, "Partitioner class name cannot be null"); setKeyClassName(keyClassName); setValueClassName(valueClassName); setPartitioner(partitionerClassName, partitionerConf); @@ -280,21 +281,21 @@ public static class Builder implements SpecificConfigBuilder { @InterfaceAudience.Private Builder setKeyClassName(String keyClassName) { - Preconditions.checkNotNull(keyClassName, "Key class name cannot be null"); + Objects.requireNonNull(keyClassName, "Key class name cannot be null"); this.conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, keyClassName); return this; } @InterfaceAudience.Private Builder setValueClassName(String valueClassName) { - Preconditions.checkNotNull(valueClassName, "Value class name cannot be null"); + Objects.requireNonNull(valueClassName, "Value class name cannot be null"); this.conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS, valueClassName); return this; } @InterfaceAudience.Private Builder setPartitioner(String partitionerClassName, @Nullable Map partitionerConf) { - Preconditions.checkNotNull(partitionerClassName, "Partitioner class name cannot be null"); + Objects.requireNonNull(partitionerClassName, "Partitioner class name cannot be null"); this.conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_PARTITIONER_CLASS, partitionerClassName); if (partitionerConf != null) { // Merging the confs for now. Change to be specific in the future. @@ -334,7 +335,7 @@ public Builder setSorterNumThreads(int numThreads) { @Override public Builder setSorter(SorterImpl sorterImpl) { - Preconditions.checkNotNull(sorterImpl, "Sorter cannot be null"); + Objects.requireNonNull(sorterImpl, "Sorter cannot be null"); this.conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_SORTER_CLASS, sorterImpl.name()); return this; @@ -344,7 +345,7 @@ public Builder setSorter(SorterImpl sorterImpl) { @SuppressWarnings("unchecked") @Override public Builder setAdditionalConfiguration(String key, String value) { - Preconditions.checkNotNull(key, "Key cannot be null"); + Objects.requireNonNull(key, "Key cannot be null"); if (ConfigUtils.doesKeyQualify(key, Lists.newArrayList(OrderedPartitionedKVOutput.getConfigurationKeySet(), TezRuntimeConfiguration.getRuntimeAdditionalConfigKeySet()), @@ -361,7 +362,7 @@ public Builder setAdditionalConfiguration(String key, String value) { @SuppressWarnings("unchecked") @Override public Builder setAdditionalConfiguration(Map confMap) { - Preconditions.checkNotNull(confMap, "ConfMap cannot be null"); + Objects.requireNonNull(confMap, "ConfMap cannot be null"); Map map = ConfigUtils.extractConfigurationMap(confMap, Lists.newArrayList(OrderedPartitionedKVOutput.getConfigurationKeySet(), TezRuntimeConfiguration.getRuntimeAdditionalConfigKeySet()), TezRuntimeConfiguration.getAllowedPrefixes()); @@ -414,7 +415,7 @@ public Builder setKeyComparatorClass(String comparatorClassName) { */ public Builder setKeyComparatorClass(String comparatorClassName, @Nullable Map comparatorConf) { - Preconditions.checkNotNull(comparatorClassName, "Comparator class name cannot be null"); + Objects.requireNonNull(comparatorClassName, "Comparator class name cannot be null"); this.conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_COMPARATOR_CLASS, comparatorClassName); if (comparatorConf != null) { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVEdgeConfig.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVEdgeConfig.java index 25a48232b0..c4e8694a8a 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVEdgeConfig.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVEdgeConfig.java @@ -23,8 +23,7 @@ import javax.annotation.Nullable; import java.util.Map; - -import com.google.common.base.Preconditions; +import java.util.Objects; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -147,7 +146,7 @@ public EdgeProperty createDefaultOneToOneEdgeProperty() { * @return an {@link org.apache.tez.dag.api.EdgeProperty} instance */ public EdgeProperty createDefaultCustomEdgeProperty(EdgeManagerPluginDescriptor edgeManagerDescriptor) { - Preconditions.checkNotNull(edgeManagerDescriptor, "EdgeManagerDescriptor cannot be null"); + Objects.requireNonNull(edgeManagerDescriptor, "EdgeManagerDescriptor cannot be null"); EdgeProperty edgeProperty = EdgeProperty.create(edgeManagerDescriptor, EdgeProperty.DataSourceType.PERSISTED, EdgeProperty.SchedulingType.SEQUENTIAL, diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVInputConfig.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVInputConfig.java index af7dbf6c22..6ac0dbb3a8 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVInputConfig.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVInputConfig.java @@ -24,9 +24,10 @@ import java.io.IOException; import java.util.Map; +import java.util.Objects; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import org.apache.hadoop.classification.InterfaceAudience; @@ -213,8 +214,8 @@ public static class Builder implements SpecificConfigBuilder { @InterfaceAudience.Private Builder(String keyClassName, String valueClassName) { this(); - Preconditions.checkNotNull(keyClassName, "Key class name cannot be null"); - Preconditions.checkNotNull(valueClassName, "Value class name cannot be null"); + Objects.requireNonNull(keyClassName, "Key class name cannot be null"); + Objects.requireNonNull(valueClassName, "Value class name cannot be null"); setKeyClassName(keyClassName); setValueClassName(valueClassName); } @@ -230,14 +231,14 @@ public static class Builder implements SpecificConfigBuilder { @InterfaceAudience.Private Builder setKeyClassName(String keyClassName) { - Preconditions.checkNotNull(keyClassName, "Key class name cannot be null"); + Objects.requireNonNull(keyClassName, "Key class name cannot be null"); this.conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, keyClassName); return this; } @InterfaceAudience.Private Builder setValueClassName(String valueClassName) { - Preconditions.checkNotNull(valueClassName, "Value class name cannot be null"); + Objects.requireNonNull(valueClassName, "Value class name cannot be null"); this.conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS, valueClassName); return this; } @@ -265,7 +266,7 @@ public Builder setMergeFraction(float mergeFraction) { @SuppressWarnings("unchecked") @Override public Builder setAdditionalConfiguration(String key, String value) { - Preconditions.checkNotNull(key, "Key cannot be null"); + Objects.requireNonNull(key, "Key cannot be null"); if (ConfigUtils.doesKeyQualify(key, Lists.newArrayList(UnorderedKVInput.getConfigurationKeySet(), TezRuntimeConfiguration.getRuntimeAdditionalConfigKeySet()), @@ -282,7 +283,7 @@ public Builder setAdditionalConfiguration(String key, String value) { @SuppressWarnings("unchecked") @Override public Builder setAdditionalConfiguration(Map confMap) { - Preconditions.checkNotNull(confMap, "ConfMap cannot be null"); + Objects.requireNonNull(confMap, "ConfMap cannot be null"); Map map = ConfigUtils.extractConfigurationMap(confMap, Lists.newArrayList(UnorderedKVInput.getConfigurationKeySet(), TezRuntimeConfiguration.getRuntimeAdditionalConfigKeySet()), TezRuntimeConfiguration.getAllowedPrefixes()); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVOutputConfig.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVOutputConfig.java index a9a15a0c15..34513badf4 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVOutputConfig.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVOutputConfig.java @@ -24,9 +24,10 @@ import java.io.IOException; import java.util.Map; +import java.util.Objects; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import org.apache.hadoop.classification.InterfaceAudience; @@ -160,8 +161,8 @@ public static class Builder implements SpecificConfigBuilder { @InterfaceAudience.Private Builder(String keyClassName, String valueClassName) { this(); - Preconditions.checkNotNull(keyClassName, "Key class name cannot be null"); - Preconditions.checkNotNull(valueClassName, "Value class name cannot be null"); + Objects.requireNonNull(keyClassName, "Key class name cannot be null"); + Objects.requireNonNull(valueClassName, "Value class name cannot be null"); setKeyClassName(keyClassName); setValueClassName(valueClassName); } @@ -177,14 +178,14 @@ public static class Builder implements SpecificConfigBuilder { @InterfaceAudience.Private Builder setKeyClassName(String keyClassName) { - Preconditions.checkNotNull(keyClassName, "Key class name cannot be null"); + Objects.requireNonNull(keyClassName, "Key class name cannot be null"); this.conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, keyClassName); return this; } @InterfaceAudience.Private Builder setValueClassName(String valueClassName) { - Preconditions.checkNotNull(valueClassName, "Value class name cannot be null"); + Objects.requireNonNull(valueClassName, "Value class name cannot be null"); this.conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS, valueClassName); return this; } @@ -192,7 +193,7 @@ Builder setValueClassName(String valueClassName) { @SuppressWarnings("unchecked") @Override public Builder setAdditionalConfiguration(String key, String value) { - Preconditions.checkNotNull(key, "Key cannot be null"); + Objects.requireNonNull(key, "Key cannot be null"); if (ConfigUtils.doesKeyQualify(key, Lists.newArrayList(UnorderedKVOutput.getConfigurationKeySet(), TezRuntimeConfiguration.getRuntimeAdditionalConfigKeySet()), @@ -209,7 +210,7 @@ public Builder setAdditionalConfiguration(String key, String value) { @SuppressWarnings("unchecked") @Override public Builder setAdditionalConfiguration(Map confMap) { - Preconditions.checkNotNull(confMap, "ConfMap cannot be null"); + Objects.requireNonNull(confMap, "ConfMap cannot be null"); Map map = ConfigUtils.extractConfigurationMap(confMap, Lists.newArrayList(UnorderedKVOutput.getConfigurationKeySet(), TezRuntimeConfiguration.getRuntimeAdditionalConfigKeySet()), TezRuntimeConfiguration.getAllowedPrefixes()); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedPartitionedKVEdgeConfig.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedPartitionedKVEdgeConfig.java index 52da491517..0d8a5aef75 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedPartitionedKVEdgeConfig.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedPartitionedKVEdgeConfig.java @@ -23,8 +23,7 @@ import javax.annotation.Nullable; import java.util.Map; - -import com.google.common.base.Preconditions; +import java.util.Objects; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -151,7 +150,7 @@ public EdgeProperty createDefaultEdgeProperty() { * @return an {@link org.apache.tez.dag.api.EdgeProperty} instance */ public EdgeProperty createDefaultCustomEdgeProperty(EdgeManagerPluginDescriptor edgeManagerDescriptor) { - Preconditions.checkNotNull(edgeManagerDescriptor, "EdgeManagerDescriptor cannot be null"); + Objects.requireNonNull(edgeManagerDescriptor, "EdgeManagerDescriptor cannot be null"); EdgeProperty edgeProperty = EdgeProperty.create(edgeManagerDescriptor, EdgeProperty.DataSourceType.PERSISTED, EdgeProperty.SchedulingType.SEQUENTIAL, diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedPartitionedKVOutputConfig.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedPartitionedKVOutputConfig.java index 3555e1c3c0..c924e7db66 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedPartitionedKVOutputConfig.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedPartitionedKVOutputConfig.java @@ -24,9 +24,10 @@ import java.io.IOException; import java.util.Map; +import java.util.Objects; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import org.apache.hadoop.classification.InterfaceAudience; @@ -183,9 +184,9 @@ public static class Builder implements SpecificConfigBuilder { Builder(String keyClassName, String valueClassName, String partitionerClassName, Map partitionerConf) { this(); - Preconditions.checkNotNull(keyClassName, "Key class name cannot be null"); - Preconditions.checkNotNull(valueClassName, "Value class name cannot be null"); - Preconditions.checkNotNull(partitionerClassName, "Partitioner class name cannot be null"); + Objects.requireNonNull(keyClassName, "Key class name cannot be null"); + Objects.requireNonNull(valueClassName, "Value class name cannot be null"); + Objects.requireNonNull(partitionerClassName, "Partitioner class name cannot be null"); setKeyClassName(keyClassName); setValueClassName(valueClassName); setPartitioner(partitionerClassName, partitionerConf); @@ -202,21 +203,21 @@ public static class Builder implements SpecificConfigBuilder { @InterfaceAudience.Private Builder setKeyClassName(String keyClassName) { - Preconditions.checkNotNull(keyClassName, "Key class name cannot be null"); + Objects.requireNonNull(keyClassName, "Key class name cannot be null"); this.conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, keyClassName); return this; } @InterfaceAudience.Private Builder setValueClassName(String valueClassName) { - Preconditions.checkNotNull(valueClassName, "Value class name cannot be null"); + Objects.requireNonNull(valueClassName, "Value class name cannot be null"); this.conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS, valueClassName); return this; } @InterfaceAudience.Private Builder setPartitioner(String partitionerClassName, Map partitionerConf) { - Preconditions.checkNotNull(partitionerClassName, "Partitioner class name cannot be null"); + Objects.requireNonNull(partitionerClassName, "Partitioner class name cannot be null"); this.conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_PARTITIONER_CLASS, partitionerClassName); if (partitionerConf != null) { // Merging the confs for now. Change to be specific in the future. @@ -236,7 +237,7 @@ public Builder setAvailableBufferSize(int availableBufferSize) { @SuppressWarnings("unchecked") @Override public Builder setAdditionalConfiguration(String key, String value) { - Preconditions.checkNotNull(key, "Key cannot be null"); + Objects.requireNonNull(key, "Key cannot be null"); if (ConfigUtils.doesKeyQualify(key, Lists.newArrayList(UnorderedPartitionedKVOutput.getConfigurationKeySet(), TezRuntimeConfiguration.getRuntimeAdditionalConfigKeySet()), @@ -253,7 +254,7 @@ public Builder setAdditionalConfiguration(String key, String value) { @SuppressWarnings("unchecked") @Override public Builder setAdditionalConfiguration(Map confMap) { - Preconditions.checkNotNull(confMap, "ConfMap cannot be null"); + Objects.requireNonNull(confMap, "ConfMap cannot be null"); Map map = ConfigUtils.extractConfigurationMap(confMap, Lists.newArrayList(UnorderedPartitionedKVOutput.getConfigurationKeySet(), TezRuntimeConfiguration.getRuntimeAdditionalConfigKeySet()), TezRuntimeConfiguration.getAllowedPrefixes()); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java index 1cc6d4b787..c1879bc364 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java @@ -54,7 +54,7 @@ import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.Shuffle; import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; /** diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java index 6ba8936091..401066dfc4 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java @@ -54,7 +54,7 @@ import org.apache.tez.runtime.library.common.shuffle.impl.ShuffleManager; import org.apache.tez.runtime.library.common.shuffle.impl.SimpleFetchedInputAllocator; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; /** * {@link UnorderedKVInput} provides unordered key value input by diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java index 32a4f4de03..8870e1f0eb 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java @@ -53,7 +53,7 @@ import org.apache.tez.runtime.library.common.sort.impl.dflt.DefaultSorter; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; /** * {@link OrderedPartitionedKVOutput} is an {@link AbstractLogicalOutput} which sorts diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java index 536ee3216f..5e223d6c40 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java @@ -25,7 +25,7 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.slf4j.Logger; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/resources/WeightedScalingMemoryDistributor.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/resources/WeightedScalingMemoryDistributor.java index c5b4fb0f92..b82e6d3d8f 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/resources/WeightedScalingMemoryDistributor.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/resources/WeightedScalingMemoryDistributor.java @@ -41,7 +41,7 @@ import org.apache.tez.runtime.library.output.UnorderedPartitionedKVOutput; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/Grouper.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/Grouper.java index b99f3d4745..84900f8f01 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/Grouper.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/Grouper.java @@ -17,7 +17,7 @@ */ package org.apache.tez.runtime.library.utils; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; /** * This grouper group specified number of items into specified number of groups. diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestTezMerger.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestTezMerger.java index b35c85f574..af10700d89 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestTezMerger.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestTezMerger.java @@ -18,7 +18,7 @@ package org.apache.tez.runtime.library.common.sort.impl; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.LinkedListMultimap; import com.google.common.collect.ListMultimap; import com.google.common.collect.Lists; diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/BroadcastAndOneToOneExample.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/BroadcastAndOneToOneExample.java index 9dd9b59cbb..5c99f3efbf 100644 --- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/BroadcastAndOneToOneExample.java +++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/BroadcastAndOneToOneExample.java @@ -54,7 +54,7 @@ import org.apache.tez.runtime.library.output.UnorderedKVOutput; import org.apache.tez.runtime.library.processor.SimpleProcessor; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; public class BroadcastAndOneToOneExample extends Configured implements Tool { public static class InputProcessor extends SimpleProcessor { diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/BroadcastLoadGen.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/BroadcastLoadGen.java index d9b89c1cf5..03bb0519a0 100644 --- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/BroadcastLoadGen.java +++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/BroadcastLoadGen.java @@ -45,7 +45,7 @@ import org.apache.tez.runtime.library.output.UnorderedKVOutput; import org.apache.tez.runtime.library.processor.SimpleProcessor; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; public class BroadcastLoadGen extends TezExampleBase { diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/CartesianProduct.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/CartesianProduct.java index 6096f969a8..aea662ae04 100644 --- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/CartesianProduct.java +++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/CartesianProduct.java @@ -17,7 +17,7 @@ */ package org.apache.tez.mapreduce.examples; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Text; diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/UnionExample.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/UnionExample.java index f78d162d31..767691d34b 100644 --- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/UnionExample.java +++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/UnionExample.java @@ -64,7 +64,7 @@ import org.apache.tez.runtime.library.input.ConcatenatedMergedKeyValuesInput; import org.apache.tez.runtime.library.partitioner.HashPartitioner; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Maps; public class UnionExample { diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/helpers/SplitsInClientOptionParser.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/helpers/SplitsInClientOptionParser.java index cabc3c4ffd..227e498510 100644 --- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/helpers/SplitsInClientOptionParser.java +++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/helpers/SplitsInClientOptionParser.java @@ -26,7 +26,7 @@ import org.apache.commons.cli.Options; import org.apache.commons.cli.ParseException; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; public class SplitsInClientOptionParser { diff --git a/tez-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java b/tez-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java index 5ce9c5daf9..a756cf9289 100644 --- a/tez-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java +++ b/tez-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java @@ -129,7 +129,7 @@ import org.junit.BeforeClass; import org.junit.Test; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Sets; public class TestMRRJobsDAGApi { diff --git a/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java b/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java index 50c5a667ac..3ed386e346 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java +++ b/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java @@ -50,7 +50,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; /** * Add hook before/after processing RecoveryEvent & SummaryEvent diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestPipelinedShuffle.java b/tez-tests/src/test/java/org/apache/tez/test/TestPipelinedShuffle.java index 36ac488b22..e44b6eb7f5 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestPipelinedShuffle.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestPipelinedShuffle.java @@ -18,7 +18,7 @@ package org.apache.tez.test; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Sets; import org.apache.commons.lang.RandomStringUtils; import org.apache.hadoop.conf.Configuration; diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java b/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java index b66151960d..6de272358b 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java @@ -40,7 +40,7 @@ import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import org.apache.hadoop.io.IntWritable; diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/CSVResult.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/CSVResult.java index 5246c68ee3..1da281c86e 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/CSVResult.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/CSVResult.java @@ -19,7 +19,7 @@ package org.apache.tez.analyzer; import com.google.common.base.Joiner; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.base.Strings; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java index 2edce3ed4e..5944870c77 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java @@ -43,7 +43,7 @@ import org.apache.tez.history.parser.datamodel.TaskInfo; import com.google.common.base.Joiner; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.base.Strings; import com.google.common.collect.Lists; import com.google.common.collect.Maps; diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SkewAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SkewAnalyzer.java index 067d871954..6025541fc9 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SkewAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SkewAnalyzer.java @@ -18,7 +18,7 @@ package org.apache.tez.analyzer.plugins; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import org.apache.hadoop.conf.Configuration; diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java index 1549de9f84..59ae4a3d60 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java @@ -39,7 +39,7 @@ import org.apache.tez.history.parser.SimpleHistoryParser; import org.apache.tez.history.parser.datamodel.DagInfo; -import com.google.common.base.Preconditions; +import org.apache.tez.common.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; From 2736788f49f17bd269b7ea64d959bbaad95421d3 Mon Sep 17 00:00:00 2001 From: Ahmed Hussein Date: Tue, 4 Feb 2020 10:44:29 -0600 Subject: [PATCH 193/512] TEZ-4106. Add Exponential Smooth RuntimeEstimator to the speculator Signed-off-by: Jonathan Eagles --- .../apache/tez/dag/api/TezConfiguration.java | 59 ++- .../speculation/legacy/DataStatistics.java | 25 +- .../speculation/legacy/LegacySpeculator.java | 39 +- ...SimpleExponentialTaskRuntimeEstimator.java | 194 ++++++++++ .../speculation/legacy/StartEndTimesBase.java | 42 +-- .../legacy/TaskRuntimeEstimator.java | 32 +- .../forecast/SimpleExponentialSmoothing.java | 336 ++++++++++++++++++ .../legacy/forecast/package-info.java | 20 ++ .../apache/tez/dag/app/TestSpeculation.java | 262 ++++++++++++-- 9 files changed, 926 insertions(+), 83 deletions(-) create mode 100644 tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/SimpleExponentialTaskRuntimeEstimator.java create mode 100644 tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/forecast/SimpleExponentialSmoothing.java create mode 100644 tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/forecast/package-info.java diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index f087e3a91f..58aecdabb1 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -26,6 +26,7 @@ import java.util.Map; import java.util.Set; +import java.util.concurrent.TimeUnit; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Unstable; @@ -530,14 +531,6 @@ public TezConfiguration(boolean loadDefaults) { public static final String TEZ_AM_SPECULATION_ENABLED = TEZ_AM_PREFIX + "speculation.enabled"; public static final boolean TEZ_AM_SPECULATION_ENABLED_DEFAULT = false; - /** - * Class used to estimate task resource needs. - */ - @ConfigurationScope(Scope.VERTEX) - @ConfigurationProperty - public static final String TEZ_AM_SPECULATION_ESTIMATOR_CLASS = - TEZ_AM_PREFIX + "speculation.estimator.class"; - /** * Float value. Specifies how many standard deviations away from the mean task execution time * should be considered as an outlier/slow task. @@ -559,6 +552,10 @@ public TezConfiguration(boolean loadDefaults) { TEZ_AM_PREFIX + "legacy.speculative.single.task.vertex.timeout"; public static final long TEZ_AM_LEGACY_SPECULATIVE_SINGLE_TASK_VERTEX_TIMEOUT_DEFAULT = -1; + @Private + public static final String TEZ_SPECULATOR_PREFIX = TEZ_AM_PREFIX + "speculator."; + @Private + public static final String TEZ_ESTIMATOR_PREFIX = TEZ_AM_PREFIX + "task.estimator."; /** * Long value. Specifies amount of time (in ms) that needs to elapse to do the next round of * speculation if there is no task speculated in this round. @@ -581,6 +578,52 @@ public TezConfiguration(boolean loadDefaults) { TEZ_AM_PREFIX + "soonest.retry.after.speculate"; public static final long TEZ_AM_SOONEST_RETRY_AFTER_SPECULATE_DEFAULT = 1000L * 15L; + /** The class that should be used for speculative execution calculations. */ + @ConfigurationScope(Scope.VERTEX) + @ConfigurationProperty + public static final String TEZ_AM_SPECULATOR_CLASS = + TEZ_SPECULATOR_PREFIX + "class"; + /** The class that should be used for task runtime estimation. */ + @ConfigurationScope(Scope.VERTEX) + @ConfigurationProperty + public static final String TEZ_AM_TASK_ESTIMATOR_CLASS = + TEZ_ESTIMATOR_PREFIX + "class"; + /** + * Long value. Specifies amount of time (in ms) of the lambda value in the + * smoothing function of the task estimator + */ + @Unstable + @ConfigurationScope(Scope.VERTEX) + @ConfigurationProperty(type="long") + public static final String TEZ_AM_ESTIMATOR_EXPONENTIAL_LAMBDA_MS = + TEZ_ESTIMATOR_PREFIX + "exponential.lambda.ms"; + public static final long TEZ_AM_ESTIMATOR_EXPONENTIAL_LAMBDA_MS_DEFAULT = + TimeUnit.SECONDS.toMillis(120); + + /** + * The window length in the simple exponential smoothing that considers the + * task attempt is stagnated. + */ + @Unstable + @ConfigurationScope(Scope.VERTEX) + @ConfigurationProperty(type="long") + public static final String TEZ_AM_ESTIMATOR_EXPONENTIAL_STAGNATED_MS = + TEZ_ESTIMATOR_PREFIX + "exponential.stagnated.ms"; + public static final long TEZ_AM_ESTIMATOR_EXPONENTIAL_STAGNATED_MS_DEFAULT = + TimeUnit.SECONDS.toMillis(360); + + /** + * The number of initial readings that the estimator ignores before giving a + * prediction. At the beginning the smooth estimator won't be accurate in + * prediction + */ + @Unstable + @ConfigurationScope(Scope.VERTEX) + @ConfigurationProperty(type="integer") + public static final String TEZ_AM_ESTIMATOR_EXPONENTIAL_SKIP_INITIALS = + TEZ_ESTIMATOR_PREFIX + "exponential.skip.initials"; + public static final int TEZ_AM_ESTIMATOR_EXPONENTIAL_SKIP_INITIALS_DEFAULT = 24; + /** * Double value. The max percent (0-1) of running tasks that can be speculatively re-executed at any time. */ diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/DataStatistics.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/DataStatistics.java index 7e6f1c2c82..bbfb950741 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/DataStatistics.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/DataStatistics.java @@ -21,6 +21,11 @@ import com.google.common.annotations.VisibleForTesting; public class DataStatistics { + /** + * factor used to calculate confidence interval within 95%. + */ + private static final double DEFAULT_CI_FACTOR = 1.96; + private int count = 0; private double sum = 0; private double sumSquares = 0; @@ -79,8 +84,24 @@ public synchronized double count() { return count; } + /** + * calculates the mean value within 95% ConfidenceInterval. 1.96 is standard + * for 95%. + * + * @return the mean value adding 95% confidence interval. + */ + public synchronized double meanCI() { + if (count <= 1) { + return 0.0; + } + double currMean = mean(); + double currStd = std(); + return currMean + (DEFAULT_CI_FACTOR * currStd / Math.sqrt(count)); + } + public String toString() { - return "DataStatistics: count is " + count + ", sum is " + sum + - ", sumSquares is " + sumSquares + " mean is " + mean() + " std() is " + std(); + return "DataStatistics: count is " + count + ", sum is " + sum + + ", sumSquares is " + sumSquares + " mean is " + mean() + + " std() is " + std() + ", meanCI() is " + meanCI(); } } \ No newline at end of file diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java index 23b057af2b..f21b8198e2 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java @@ -33,7 +33,6 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.hadoop.service.AbstractService; -import org.apache.hadoop.service.ServiceOperations; import org.apache.tez.common.ProgressHelper; import org.apache.tez.dag.api.TezConfiguration; import org.slf4j.Logger; @@ -100,10 +99,9 @@ public class LegacySpeculator extends AbstractService { private TaskRuntimeEstimator estimator; private final long taskTimeout; private final Clock clock; - private long nextSpeculateTime = Long.MIN_VALUE; private Thread speculationBackgroundThread = null; private volatile boolean stopped = false; - /* Allow the speculator to wait on a blockingQueue in case we use it for event notification */ + /** Allow the speculator to wait on a blockingQueue in case we use it for event notification. */ private BlockingQueue scanControl = new LinkedBlockingQueue(); @VisibleForTesting @@ -132,9 +130,8 @@ public LegacySpeculator(Configuration conf, Clock clock, Vertex vertex) { static private TaskRuntimeEstimator getEstimator (Configuration conf, Vertex vertex) { TaskRuntimeEstimator estimator; - // "tez.am.speculation.estimator.class" Class estimatorClass = - conf.getClass(TezConfiguration.TEZ_AM_SPECULATION_ESTIMATOR_CLASS, + conf.getClass(TezConfiguration.TEZ_AM_TASK_ESTIMATOR_CLASS, LegacyTaskRuntimeEstimator.class, TaskRuntimeEstimator.class); try { @@ -236,6 +233,16 @@ protected void serviceStop() throws Exception { } } + // This interface is intended to be used only for test cases. + public void scanForSpeculationsForTesting() { + if (LOG.isDebugEnabled()) { + LOG.debug("We got asked to run a debug speculation scan."); + LOG.debug("There are {} speculative events stacked already.", scanControl.size()); + } + scanControl.add(new Object()); + Thread.yield(); + } + public Runnable createThread() { return new Runnable() { @Override @@ -267,8 +274,9 @@ public void run() { public void notifyAttemptStarted(TezTaskAttemptID taId, long timestamp) { estimator.enrollAttempt(taId, timestamp); } - - public void notifyAttemptStatusUpdate(TezTaskAttemptID taId, TaskAttemptState reportedState, + + public void notifyAttemptStatusUpdate(TezTaskAttemptID taId, + TaskAttemptState reportedState, long timestamp) { statusUpdate(taId, reportedState, timestamp); } @@ -293,12 +301,10 @@ private void statusUpdate(TezTaskAttemptID attemptID, estimator.updateAttempt(attemptID, reportedState, timestamp); - //if (stateString.equals(TaskAttemptState.RUNNING.name())) { if (reportedState == TaskAttemptState.RUNNING) { runningTasks.putIfAbsent(taskID, Boolean.TRUE); } else { runningTasks.remove(taskID, Boolean.TRUE); - //if (!stateString.equals(TaskAttemptState.STARTING.name())) { if (reportedState == TaskAttemptState.STARTING) { runningTaskAttemptStatistics.remove(attemptID); } @@ -356,7 +362,7 @@ private long speculationValue(Task task, long now, boolean shouldUseTimeout) { } } - TezTaskAttemptID runningTaskAttemptID = null; + TezTaskAttemptID runningTaskAttemptID; int numberRunningAttempts = 0; for (TaskAttempt taskAttempt : attempts.values()) { @@ -387,7 +393,8 @@ private long speculationValue(Task task, long now, boolean shouldUseTimeout) { return ON_SCHEDULE; } } else { - long estimatedRunTime = estimator.estimatedRuntime(runningTaskAttemptID); + long estimatedRunTime = estimator + .estimatedRuntime(runningTaskAttemptID); long estimatedEndTime = estimatedRunTime + taskAttemptStartTime; @@ -399,12 +406,15 @@ private long speculationValue(Task task, long now, boolean shouldUseTimeout) { runningTaskAttemptStatistics.get(runningTaskAttemptID); if (data == null) { runningTaskAttemptStatistics.put(runningTaskAttemptID, - new TaskAttemptHistoryStatistics(estimatedRunTime, progress, now)); + new TaskAttemptHistoryStatistics(estimatedRunTime, progress, + now)); } else { if (estimatedRunTime == data.getEstimatedRunTime() && progress == data.getProgress()) { // Previous stats are same as same stats - if (data.notHeartbeatedInAWhile(now)) { + if (data.notHeartbeatedInAWhile(now) + || estimator + .hasStagnatedProgress(runningTaskAttemptID, now)) { // Stats have stagnated for a while, simulate heart-beat. // Now simulate the heart-beat statusUpdate(taskAttempt.getID(), taskAttempt.getState(), @@ -448,7 +458,8 @@ private long speculationValue(Task task, long now, boolean shouldUseTimeout) { // Add attempt to a given Task. protected void addSpeculativeAttempt(TezTaskID taskID) { - LOG.info("DefaultSpeculator.addSpeculativeAttempt -- we are speculating " + taskID); + LOG.info("DefaultSpeculator.addSpeculativeAttempt -- we are speculating " + + taskID); vertex.scheduleSpeculativeTask(taskID); mayHaveSpeculated.add(taskID); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/SimpleExponentialTaskRuntimeEstimator.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/SimpleExponentialTaskRuntimeEstimator.java new file mode 100644 index 0000000000..b61f153e6b --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/SimpleExponentialTaskRuntimeEstimator.java @@ -0,0 +1,194 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.dag.app.dag.speculation.legacy; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.hadoop.conf.Configuration; +import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.api.oldrecords.TaskAttemptState; +import org.apache.tez.dag.app.dag.Task; +import org.apache.tez.dag.app.dag.TaskAttempt; +import org.apache.tez.dag.app.dag.Vertex; +import org.apache.tez.dag.app.dag.speculation.legacy.forecast.SimpleExponentialSmoothing; +import org.apache.tez.dag.records.TezTaskAttemptID; + +/** + * A task Runtime Estimator based on exponential smoothing. + */ +public class SimpleExponentialTaskRuntimeEstimator extends StartEndTimesBase { + /** + * The default value returned by the estimator when no records exist. + */ + private static final long DEFAULT_ESTIMATE_RUNTIME = -1L; + + /** + * Given a forecast of value 0.0, it is getting replaced by the default value + * to avoid division by 0. + */ + private static final double DEFAULT_PROGRESS_VALUE = 1E-10; + + /** + * Factor used to calculate the confidence interval. + */ + private static final double CONFIDENCE_INTERVAL_FACTOR = 0.25; + /** + * Constant time used to calculate the smoothing exponential factor. + */ + private long constTime; + + /** + * Number of readings before we consider the estimate stable. + * Otherwise, the estimate will be skewed due to the initial estimate + */ + private int skipCount; + + /** + * Time window to automatically update the count of the skipCount. This is + * needed when a task stalls without any progress, causing the estimator to + * return -1 as an estimatedRuntime. + */ + private long stagnatedWindow; + + /** + * A map of TA Id to the statistic model of smooth exponential. + */ + private final ConcurrentMap> + estimates = new ConcurrentHashMap<>(); + + private SimpleExponentialSmoothing getForecastEntry( + final TezTaskAttemptID attemptID) { + AtomicReference entryRef = estimates + .get(attemptID); + if (entryRef == null) { + return null; + } + return entryRef.get(); + } + + private void incorporateReading(final TezTaskAttemptID attemptID, + final float newRawData, final long newTimeStamp) { + SimpleExponentialSmoothing foreCastEntry = getForecastEntry(attemptID); + if (foreCastEntry == null) { + Long tStartTime = startTimes.get(attemptID); + // skip if the startTime is not set yet + if (tStartTime == null) { + return; + } + estimates.putIfAbsent(attemptID, + new AtomicReference<>(SimpleExponentialSmoothing.createForecast( + constTime, skipCount, stagnatedWindow, + tStartTime - 1))); + incorporateReading(attemptID, newRawData, newTimeStamp); + return; + } + foreCastEntry.incorporateReading(newTimeStamp, newRawData); + } + + @Override + public void contextualize(final Configuration conf, final Vertex vertex) { + super.contextualize(conf, vertex); + + constTime + = conf.getLong(TezConfiguration.TEZ_AM_ESTIMATOR_EXPONENTIAL_LAMBDA_MS, + TezConfiguration.TEZ_AM_ESTIMATOR_EXPONENTIAL_LAMBDA_MS_DEFAULT); + + stagnatedWindow = Math.max(2 * constTime, conf.getLong( + TezConfiguration.TEZ_AM_ESTIMATOR_EXPONENTIAL_STAGNATED_MS, + TezConfiguration.TEZ_AM_ESTIMATOR_EXPONENTIAL_STAGNATED_MS_DEFAULT)); + + skipCount = conf + .getInt(TezConfiguration.TEZ_AM_ESTIMATOR_EXPONENTIAL_SKIP_INITIALS, + TezConfiguration + .TEZ_AM_ESTIMATOR_EXPONENTIAL_SKIP_INITIALS_DEFAULT); + } + + @Override + public long estimatedRuntime(final TezTaskAttemptID id) { + SimpleExponentialSmoothing foreCastEntry = getForecastEntry(id); + if (foreCastEntry == null) { + return DEFAULT_ESTIMATE_RUNTIME; + } + double remainingWork = + Math.max(0.0, Math.min(1.0, 1.0 - foreCastEntry.getRawData())); + double forecast = + Math.max(DEFAULT_PROGRESS_VALUE, foreCastEntry.getForecast()); + long remainingTime = (long) (remainingWork / forecast); + long estimatedRuntime = + remainingTime + foreCastEntry.getTimeStamp() - foreCastEntry.getStartTime(); + return estimatedRuntime; + } + + @Override + public long newAttemptEstimatedRuntime() { + if (taskStatistics == null) { + return DEFAULT_ESTIMATE_RUNTIME; + } + + double statsMeanCI = taskStatistics.meanCI(); + double expectedVal = + statsMeanCI + Math.min(statsMeanCI * CONFIDENCE_INTERVAL_FACTOR, + taskStatistics.std() / 2); + return (long) (expectedVal); + } + + @Override + public boolean hasStagnatedProgress(final TezTaskAttemptID id, + final long timeStamp) { + SimpleExponentialSmoothing foreCastEntry = getForecastEntry(id); + if (foreCastEntry == null) { + return false; + } + return foreCastEntry.isDataStagnated(timeStamp); + } + + @Override + public long runtimeEstimateVariance(final TezTaskAttemptID id) { + SimpleExponentialSmoothing forecastEntry = getForecastEntry(id); + if (forecastEntry == null) { + return DEFAULT_ESTIMATE_RUNTIME; + } + double forecast = forecastEntry.getForecast(); + if (forecastEntry.isDefaultForecast(forecast)) { + return DEFAULT_ESTIMATE_RUNTIME; + } + //TODO What is the best way to measure variance in runtime + return 0L; + } + + + @Override + public void updateAttempt(final TezTaskAttemptID attemptID, + final TaskAttemptState state, + final long timestamp) { + super.updateAttempt(attemptID, state, timestamp); + Task task = vertex.getTask(attemptID.getTaskID()); + if (task == null) { + return; + } + TaskAttempt taskAttempt = task.getAttempt(attemptID); + if (taskAttempt == null) { + return; + } + float progress = taskAttempt.getProgress(); + incorporateReading(attemptID, progress, timestamp); + } +} + diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/StartEndTimesBase.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/StartEndTimesBase.java index d4d1a7ff2c..3083986d9c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/StartEndTimesBase.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/StartEndTimesBase.java @@ -35,13 +35,11 @@ /** * Base class that uses the attempt runtime estimations from a derived class * and uses it to determine outliers based on deviating beyond the mean - * estimated runtime by some threshold + * estimated runtime by some threshold. */ abstract class StartEndTimesBase implements TaskRuntimeEstimator { - static final float MINIMUM_COMPLETE_PROPORTION_TO_SPECULATE - = 0.05F; - static final int MINIMUM_COMPLETE_NUMBER_TO_SPECULATE - = 1; + static final float MINIMUM_COMPLETE_PROPORTION_TO_SPECULATE = 0.05F; + static final int MINIMUM_COMPLETE_NUMBER_TO_SPECULATE = 1; protected Vertex vertex; @@ -50,56 +48,58 @@ abstract class StartEndTimesBase implements TaskRuntimeEstimator { protected final DataStatistics taskStatistics = new DataStatistics(); - private float slowTaskRelativeTresholds; + private float slowTaskRelativeThresholds; protected final Set doneTasks = new HashSet(); @Override - public void enrollAttempt(TezTaskAttemptID id, long timestamp) { + public void enrollAttempt(final TezTaskAttemptID id, final long timestamp) { startTimes.put(id, timestamp); } @Override - public long attemptEnrolledTime(TezTaskAttemptID attemptID) { + public long attemptEnrolledTime(final TezTaskAttemptID attemptID) { Long result = startTimes.get(attemptID); return result == null ? Long.MAX_VALUE : result; } @Override - public void contextualize(Configuration conf, Vertex vertex) { - slowTaskRelativeTresholds = conf.getFloat( + public void contextualize(final Configuration conf, final Vertex vertexP) { + slowTaskRelativeThresholds = conf.getFloat( TezConfiguration.TEZ_AM_LEGACY_SPECULATIVE_SLOWTASK_THRESHOLD, 1.0f); - this.vertex = vertex; + this.vertex = vertexP; } - protected DataStatistics dataStatisticsForTask(TezTaskID taskID) { + protected DataStatistics dataStatisticsForTask(final TezTaskID taskID) { return taskStatistics; } @Override - public long thresholdRuntime(TezTaskID taskID) { + public long thresholdRuntime(final TezTaskID taskID) { int completedTasks = vertex.getCompletedTasks(); int totalTasks = vertex.getTotalTasks(); - + if (completedTasks < MINIMUM_COMPLETE_NUMBER_TO_SPECULATE - || (((float)completedTasks) / totalTasks) - < MINIMUM_COMPLETE_PROPORTION_TO_SPECULATE ) { + || (((float) completedTasks) / totalTasks) + < MINIMUM_COMPLETE_PROPORTION_TO_SPECULATE) { return Long.MAX_VALUE; } - - long result = (long)taskStatistics.outlier(slowTaskRelativeTresholds); + + long result = (long) taskStatistics.outlier(slowTaskRelativeThresholds); return result; } @Override public long newAttemptEstimatedRuntime() { - return (long)taskStatistics.mean(); + return (long) taskStatistics.mean(); } @Override - public void updateAttempt(TezTaskAttemptID attemptID, TaskAttemptState state, long timestamp) { + public void updateAttempt(final TezTaskAttemptID attemptID, + final TaskAttemptState state, + final long timestamp) { Task task = vertex.getTask(attemptID.getTaskID()); @@ -109,7 +109,7 @@ public void updateAttempt(TezTaskAttemptID attemptID, TaskAttemptState state, lo Long boxedStart = startTimes.get(attemptID); long start = boxedStart == null ? Long.MIN_VALUE : boxedStart; - + TaskAttempt taskAttempt = task.getAttempt(attemptID); if (taskAttempt.getState() == TaskAttemptState.SUCCEEDED) { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/TaskRuntimeEstimator.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/TaskRuntimeEstimator.java index c8edd1eac4..4f747afc4e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/TaskRuntimeEstimator.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/TaskRuntimeEstimator.java @@ -29,13 +29,14 @@ * */ public interface TaskRuntimeEstimator { - public void enrollAttempt(TezTaskAttemptID id, long timestamp); + void enrollAttempt(TezTaskAttemptID id, long timestamp); - public long attemptEnrolledTime(TezTaskAttemptID attemptID); + long attemptEnrolledTime(TezTaskAttemptID attemptID); - public void updateAttempt(TezTaskAttemptID taId, TaskAttemptState reportedState, long timestamp); + void updateAttempt(TezTaskAttemptID taId, + TaskAttemptState reportedState, long timestamp); - public void contextualize(Configuration conf, Vertex vertex); + void contextualize(Configuration conf, Vertex vertex); /** * @@ -52,7 +53,7 @@ public interface TaskRuntimeEstimator { * however long. * */ - public long thresholdRuntime(TezTaskID id); + long thresholdRuntime(TezTaskID id); /** * @@ -64,7 +65,7 @@ public interface TaskRuntimeEstimator { * we don't have enough information yet to produce an estimate. * */ - public long estimatedRuntime(TezTaskAttemptID id); + long estimatedRuntime(TezTaskAttemptID id); /** * @@ -75,7 +76,7 @@ public interface TaskRuntimeEstimator { * we don't have enough information yet to produce an estimate. * */ - public long newAttemptEstimatedRuntime(); + long newAttemptEstimatedRuntime(); /** * @@ -87,5 +88,20 @@ public interface TaskRuntimeEstimator { * we don't have enough information yet to produce an estimate. * */ - public long runtimeEstimateVariance(TezTaskAttemptID id); + long runtimeEstimateVariance(TezTaskAttemptID id); + + /** + * + * Returns true if the estimator has no updates records for a threshold time + * window. This helps to identify task attempts that are stalled at the + * beginning of execution. + * + * @param id the {@link TezTaskAttemptID} of the attempt we are asking about + * @param timeStamp the time of the report we compare with + * @return true if the task attempt has no progress for a given time window + * + */ + default boolean hasStagnatedProgress(TezTaskAttemptID id, long timeStamp) { + return false; + } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/forecast/SimpleExponentialSmoothing.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/forecast/SimpleExponentialSmoothing.java new file mode 100644 index 0000000000..e7b7dcd57c --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/forecast/SimpleExponentialSmoothing.java @@ -0,0 +1,336 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.app.dag.speculation.legacy.forecast; + +import java.util.concurrent.atomic.AtomicReference; + +/** + * Implementation of the static model for Simple exponential smoothing. + */ +public class SimpleExponentialSmoothing { + private static final double DEFAULT_FORECAST = -1.0d; + private final int kMinimumReads; + private final long kStagnatedWindow; + private final long startTime; + private long timeConstant; + + /** + * Holds reference to the current forecast record. + */ + private AtomicReference forecastRefEntry; + + /** + * Create forecast simple exponential smoothing. + * + * @param timeConstant the time constant + * @param skipCnt the skip cnt + * @param stagnatedWindow the stagnated window + * @param timeStamp the time stamp + * @return the simple exponential smoothing + */ + public static SimpleExponentialSmoothing createForecast( + final long timeConstant, + final int skipCnt, final long stagnatedWindow, final long timeStamp) { + return new SimpleExponentialSmoothing(timeConstant, skipCnt, + stagnatedWindow, timeStamp); + } + + /** + * Instantiates a new Simple exponential smoothing. + * + * @param ktConstant the kt constant + * @param skipCnt the skip cnt + * @param stagnatedWindow the stagnated window + * @param timeStamp the time stamp + */ + SimpleExponentialSmoothing(final long ktConstant, final int skipCnt, + final long stagnatedWindow, final long timeStamp) { + this.kMinimumReads = skipCnt; + this.kStagnatedWindow = stagnatedWindow; + this.timeConstant = ktConstant; + this.startTime = timeStamp; + this.forecastRefEntry = new AtomicReference(null); + } + + private class ForecastRecord { + private final double alpha; + private final long timeStamp; + private final double sample; + private final double rawData; + private double forecast; + private final double sseError; + private final long myIndex; + private ForecastRecord prevRec; + + /** + * Instantiates a new Forecast record. + * + * @param currForecast the curr forecast + * @param currRawData the curr raw data + * @param currTimeStamp the curr time stamp + */ + ForecastRecord(final double currForecast, final double currRawData, + final long currTimeStamp) { + this(0.0, currForecast, currRawData, currForecast, currTimeStamp, 0.0, 0); + } + + /** + * Instantiates a new Forecast record. + * + * @param alphaVal the alpha val + * @param currSample the curr sample + * @param currRawData the curr raw data + * @param currForecast the curr forecast + * @param currTimeStamp the curr time stamp + * @param accError the acc error + * @param index the index + */ + ForecastRecord(final double alphaVal, final double currSample, + final double currRawData, + final double currForecast, final long currTimeStamp, + final double accError, + final long index) { + this.timeStamp = currTimeStamp; + this.alpha = alphaVal; + this.sample = currSample; + this.forecast = currForecast; + this.rawData = currRawData; + this.sseError = accError; + this.myIndex = index; + } + + private ForecastRecord createForecastRecord(final double alphaVal, + final double currSample, + final double currRawData, + final double currForecast, final long currTimeStamp, + final double accError, + final long index, + final ForecastRecord prev) { + ForecastRecord forecastRec = + new ForecastRecord(alphaVal, currSample, currRawData, currForecast, + currTimeStamp, accError, index); + forecastRec.prevRec = prev; + return forecastRec; + } + + private double preProcessRawData(final double rData, final long newTime) { + return processRawData(this.rawData, this.timeStamp, rData, newTime); + } + + /** + * Append forecast record. + * + * @param newTimeStamp the new time stamp + * @param rData the r data + * @return the forecast record + */ + public ForecastRecord append(final long newTimeStamp, final double rData) { + if (this.timeStamp >= newTimeStamp + && Double.compare(this.rawData, rData) >= 0) { + // progress reported twice. Do nothing. + return this; + } + ForecastRecord refRecord = this; + if (newTimeStamp == this.timeStamp) { + // we need to restore old value if possible + if (this.prevRec != null) { + refRecord = this.prevRec; + } + } + double newSample = refRecord.preProcessRawData(rData, newTimeStamp); + long deltaTime = this.timeStamp - newTimeStamp; + if (refRecord.myIndex == kMinimumReads) { + timeConstant = Math.max(timeConstant, newTimeStamp - startTime); + } + double smoothFactor = + 1 - Math.exp(((double) deltaTime) / timeConstant); + double forecastVal = + smoothFactor * newSample + (1.0 - smoothFactor) * refRecord.forecast; + double newSSEError = + refRecord.sseError + Math.pow(newSample - refRecord.forecast, 2); + return refRecord + .createForecastRecord(smoothFactor, newSample, rData, forecastVal, + newTimeStamp, newSSEError, refRecord.myIndex + 1, refRecord); + } + } + + /** + * checks if the task is hanging up. + * + * @param timeStamp current time of the scan. + * @return true if we have number of samples > kMinimumReads and the record + * timestamp has expired. + */ + public boolean isDataStagnated(final long timeStamp) { + ForecastRecord rec = forecastRefEntry.get(); + if (rec != null && rec.myIndex > kMinimumReads) { + return (rec.timeStamp + kStagnatedWindow) > timeStamp; + } + return false; + } + + /** + * Process raw data double. + * + * @param oldRawData the old raw data + * @param oldTime the old time + * @param newRawData the new raw data + * @param newTime the new time + * @return the double + */ + static double processRawData(final double oldRawData, final long oldTime, + final double newRawData, final long newTime) { + double rate = (newRawData - oldRawData) / (newTime - oldTime); + return rate; + } + + /** + * Incorporate reading. + * + * @param timeStamp the time stamp + * @param currRawData the curr raw data + */ + public void incorporateReading(final long timeStamp, + final double currRawData) { + ForecastRecord oldRec = forecastRefEntry.get(); + if (oldRec == null) { + double oldForecast = + processRawData(0, startTime, currRawData, timeStamp); + forecastRefEntry.compareAndSet(null, + new ForecastRecord(oldForecast, 0.0d, startTime)); + incorporateReading(timeStamp, currRawData); + return; + } + while (!forecastRefEntry.compareAndSet(oldRec, oldRec.append(timeStamp, + currRawData))) { + oldRec = forecastRefEntry.get(); + } + } + + /** + * Gets forecast. + * + * @return the forecast + */ + public double getForecast() { + ForecastRecord rec = forecastRefEntry.get(); + if (rec != null && rec.myIndex > kMinimumReads) { + return rec.forecast; + } + return DEFAULT_FORECAST; + } + + /** + * Is default forecast boolean. + * + * @param value the value + * @return the boolean + */ + public boolean isDefaultForecast(final double value) { + return value == DEFAULT_FORECAST; + } + + /** + * Gets sse. + * + * @return the sse + */ + public double getSSE() { + ForecastRecord rec = forecastRefEntry.get(); + if (rec != null) { + return rec.sseError; + } + return DEFAULT_FORECAST; + } + + /** + * Is error within bound boolean. + * + * @param bound the bound + * @return the boolean + */ + public boolean isErrorWithinBound(final double bound) { + double squaredErr = getSSE(); + if (squaredErr < 0) { + return false; + } + return bound > squaredErr; + } + + /** + * Gets raw data. + * + * @return the raw data + */ + public double getRawData() { + ForecastRecord rec = forecastRefEntry.get(); + if (rec != null) { + return rec.rawData; + } + return DEFAULT_FORECAST; + } + + /** + * Gets time stamp. + * + * @return the time stamp + */ + public long getTimeStamp() { + ForecastRecord rec = forecastRefEntry.get(); + if (rec != null) { + return rec.timeStamp; + } + return 0L; + } + + /** + * Gets start time. + * + * @return the start time + */ + public long getStartTime() { + return startTime; + } + + /** + * Gets forecast ref entry. + * + * @return the forecast ref entry + */ + public AtomicReference getForecastRefEntry() { + return forecastRefEntry; + } + + @Override + public String toString() { + String res = "NULL"; + ForecastRecord rec = forecastRefEntry.get(); + if (rec != null) { + StringBuilder strB = new StringBuilder("rec.index = ").append(rec.myIndex) + .append(", timeStamp t: ").append(rec.timeStamp) + .append(", forecast: ").append(rec.forecast).append(", sample: ") + .append(rec.sample).append(", raw: ").append(rec.rawData) + .append(", error: ").append(rec.sseError).append(", alpha: ") + .append(rec.alpha); + res = strB.toString(); + } + return res; + } +} + diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/forecast/package-info.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/forecast/package-info.java new file mode 100644 index 0000000000..3ed8b6accb --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/forecast/package-info.java @@ -0,0 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +@InterfaceAudience.Private +package org.apache.tez.dag.app.dag.speculation.legacy.forecast; +import org.apache.hadoop.classification.InterfaceAudience; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java index a81d4d3b9f..b9a7c5ae17 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java @@ -19,10 +19,15 @@ package org.apache.tez.dag.app; import java.io.IOException; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.util.Arrays; +import java.util.Collection; import java.util.HashMap; import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -45,39 +50,196 @@ import org.apache.tez.dag.app.dag.TaskAttempt; import org.apache.tez.dag.app.dag.impl.DAGImpl; import org.apache.tez.dag.app.dag.speculation.legacy.LegacySpeculator; +import org.apache.tez.dag.app.dag.speculation.legacy.LegacyTaskRuntimeEstimator; +import org.apache.tez.dag.app.dag.speculation.legacy.SimpleExponentialTaskRuntimeEstimator; +import org.apache.tez.dag.app.dag.speculation.legacy.TaskRuntimeEstimator; import org.apache.tez.dag.library.vertexmanager.ShuffleVertexManager; import org.apache.tez.dag.records.TaskAttemptTerminationCause; import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.dag.records.TezTaskID; import org.apache.tez.dag.records.TezVertexID; +import org.junit.After; import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; import org.junit.Test; import com.google.common.base.Joiner; +import org.junit.rules.TestRule; +import org.junit.runner.Description; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.model.Statement; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +/** + * test speculation behavior given the list of estimator classes. + */ +@RunWith(Parameterized.class) public class TestSpeculation { - static Configuration defaultConf; - static FileSystem localFs; - + private final static Logger LOG = LoggerFactory.getLogger(TezConfiguration.class); + + private static final String ASSERT_SPECULATIONS_COUNT_MSG = + "Number of attempts after Speculation should be two"; + private static final String UNIT_EXCEPTION_MESSAGE = + "test timed out after"; + private static final int ASSERT_SPECULATIONS_COUNT_RETRIES = 3; + private Configuration defaultConf; + private FileSystem localFs; + + /** + * The Mock app. + */ MockDAGAppMaster mockApp; + + /** + * The Mock launcher. + */ MockContainerLauncher mockLauncher; - - static { + + /** + * The interface Retry. + */ + @Retention(RetentionPolicy.RUNTIME) + public @interface Retry {} + + /** + * The type Retry rule. + */ + class RetryRule implements TestRule { + + private AtomicInteger retryCount; + + /** + * Instantiates a new Retry rule. + * + * @param retries the retries + */ + RetryRule(int retries) { + super(); + this.retryCount = new AtomicInteger(retries); + } + + @Override + public Statement apply(final Statement base, + final Description description) { + return new Statement() { + @Override + public void evaluate() throws Throwable { + Throwable caughtThrowable = null; + + while (retryCount.getAndDecrement() > 0) { + try { + base.evaluate(); + return; + } catch (Throwable t) { + caughtThrowable = t; + if (retryCount.get() > 0 && + description.getAnnotation(Retry.class) != null) { + if (!((t instanceof AssertionError && t.getMessage() + .contains(ASSERT_SPECULATIONS_COUNT_MSG)) + || (t instanceof Exception && t.getMessage() + .contains(UNIT_EXCEPTION_MESSAGE)))) { + throw caughtThrowable; + } + LOG.warn("{} : Failed. Retries remaining: ", + description.getDisplayName(), + retryCount.toString()); + } else { + throw caughtThrowable; + } + } + } + } + }; + } + } + + /** + * The Rule. + */ + @Rule + public RetryRule rule = new RetryRule(ASSERT_SPECULATIONS_COUNT_RETRIES); + + /** + * Sets default conf. + */ + @Before + public void setDefaultConf() { try { defaultConf = new Configuration(false); defaultConf.set("fs.defaultFS", "file:///"); defaultConf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); defaultConf.setBoolean(TezConfiguration.TEZ_AM_SPECULATION_ENABLED, true); - defaultConf.setFloat(ShuffleVertexManager.TEZ_SHUFFLE_VERTEX_MANAGER_MIN_SRC_FRACTION, 1); - defaultConf.setFloat(ShuffleVertexManager.TEZ_SHUFFLE_VERTEX_MANAGER_MAX_SRC_FRACTION, 1); + defaultConf.setFloat( + ShuffleVertexManager.TEZ_SHUFFLE_VERTEX_MANAGER_MIN_SRC_FRACTION, 1); + defaultConf.setFloat( + ShuffleVertexManager.TEZ_SHUFFLE_VERTEX_MANAGER_MAX_SRC_FRACTION, 1); localFs = FileSystem.getLocal(defaultConf); - String stagingDir = "target" + Path.SEPARATOR + TestSpeculation.class.getName() + "-tmpDir"; + String stagingDir = + "target" + Path.SEPARATOR + TestSpeculation.class.getName() + + "-tmpDir"; defaultConf.set(TezConfiguration.TEZ_AM_STAGING_DIR, stagingDir); + defaultConf.setClass(TezConfiguration.TEZ_AM_TASK_ESTIMATOR_CLASS, + estimatorClass, + TaskRuntimeEstimator.class); + defaultConf.setInt(TezConfiguration.TEZ_AM_MINIMUM_ALLOWED_SPECULATIVE_TASKS, 20); + defaultConf.setDouble(TezConfiguration.TEZ_AM_PROPORTION_TOTAL_TASKS_SPECULATABLE, 0.2); + defaultConf.setDouble(TezConfiguration.TEZ_AM_PROPORTION_RUNNING_TASKS_SPECULATABLE, 0.25); + defaultConf.setLong(TezConfiguration.TEZ_AM_SOONEST_RETRY_AFTER_NO_SPECULATE, 25); + defaultConf.setLong(TezConfiguration.TEZ_AM_SOONEST_RETRY_AFTER_SPECULATE, 50); + defaultConf.setInt(TezConfiguration.TEZ_AM_ESTIMATOR_EXPONENTIAL_SKIP_INITIALS, 2); } catch (IOException e) { throw new RuntimeException("init failure", e); } } + /** + * Tear down. + */ + @After + public void tearDown() { + defaultConf = null; + try { + localFs.close(); + mockLauncher.shutdown(); + mockApp.close(); + } catch (Exception e) { + e.printStackTrace(); + } + } + + /** + * Gets test parameters. + * + * @return the test parameters + */ + @Parameterized.Parameters(name = "{index}: TaskEstimator(EstimatorClass {0})") + public static Collection getTestParameters() { + return Arrays.asList(new Object[][]{ + {SimpleExponentialTaskRuntimeEstimator.class}, + {LegacyTaskRuntimeEstimator.class} + }); + } + + private Class estimatorClass; + + /** + * Instantiates a new Test speculation. + * + * @param estimatorKlass the estimator klass + */ + public TestSpeculation(Class estimatorKlass) { + this.estimatorClass = estimatorKlass; + } + + /** + * Create tez session mock tez client. + * + * @return the mock tez client + * @throws Exception the exception + */ MockTezClient createTezSession() throws Exception { TezConfiguration tezconf = new TezConfiguration(defaultConf); AtomicBoolean mockAppLauncherGoFlag = new AtomicBoolean(false); @@ -87,8 +249,16 @@ MockTezClient createTezSession() throws Exception { syncWithMockAppLauncher(false, mockAppLauncherGoFlag, tezClient); return tezClient; } - - void syncWithMockAppLauncher(boolean allowScheduling, AtomicBoolean mockAppLauncherGoFlag, + + /** + * Sync with mock app launcher. + * + * @param allowScheduling the allow scheduling + * @param mockAppLauncherGoFlag the mock app launcher go flag + * @param tezClient the tez client + * @throws Exception the exception + */ + void syncWithMockAppLauncher(boolean allowScheduling, AtomicBoolean mockAppLauncherGoFlag, MockTezClient tezClient) throws Exception { synchronized (mockAppLauncherGoFlag) { while (!mockAppLauncherGoFlag.get()) { @@ -101,6 +271,12 @@ void syncWithMockAppLauncher(boolean allowScheduling, AtomicBoolean mockAppLaunc } } + /** + * Test single task speculation. + * + * @throws Exception the exception + */ + @Retry @Test (timeout = 10000) public void testSingleTaskSpeculation() throws Exception { // Map @@ -126,9 +302,10 @@ public void testSingleTaskSpeculation() throws Exception { DAGImpl dagImpl = (DAGImpl) mockApp.getContext().getCurrentDAG(); TezVertexID vertexId = TezVertexID.getInstance(dagImpl.getID(), 0); // original attempt is killed and speculative one is successful - TezTaskAttemptID killedTaId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 0); - TezTaskAttemptID successTaId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 1); - + TezTaskAttemptID killedTaId = + TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 0); + TezTaskAttemptID successTaId = + TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 1); Thread.sleep(200); // cause speculation trigger mockLauncher.setStatusUpdatesForTask(killedTaId, 100); @@ -149,16 +326,16 @@ public void testSingleTaskSpeculation() throws Exception { } } + /** + * Test basic speculation. + * + * @param withProgress the with progress + * @throws Exception the exception + */ public void testBasicSpeculation(boolean withProgress) throws Exception { - - defaultConf.setInt(TezConfiguration.TEZ_AM_MINIMUM_ALLOWED_SPECULATIVE_TASKS, 20); - defaultConf.setDouble(TezConfiguration.TEZ_AM_PROPORTION_TOTAL_TASKS_SPECULATABLE, 0.2); - defaultConf.setDouble(TezConfiguration.TEZ_AM_PROPORTION_RUNNING_TASKS_SPECULATABLE, 0.25); - defaultConf.setLong(TezConfiguration.TEZ_AM_SOONEST_RETRY_AFTER_NO_SPECULATE, 25); - defaultConf.setLong(TezConfiguration.TEZ_AM_SOONEST_RETRY_AFTER_SPECULATE, 50); - DAG dag = DAG.create("test"); - Vertex vA = Vertex.create("A", ProcessorDescriptor.create("Proc.class"), 5); + Vertex vA = Vertex.create("A", + ProcessorDescriptor.create("Proc.class"), 5); dag.addVertex(vA); MockTezClient tezClient = createTezSession(); @@ -166,8 +343,10 @@ public void testBasicSpeculation(boolean withProgress) throws Exception { DAGImpl dagImpl = (DAGImpl) mockApp.getContext().getCurrentDAG(); TezVertexID vertexId = TezVertexID.getInstance(dagImpl.getID(), 0); // original attempt is killed and speculative one is successful - TezTaskAttemptID killedTaId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 0); - TezTaskAttemptID successTaId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 1); + TezTaskAttemptID killedTaId = + TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 0); + TezTaskAttemptID successTaId = + TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 1); mockLauncher.updateProgress(withProgress); // cause speculation trigger @@ -175,9 +354,11 @@ public void testBasicSpeculation(boolean withProgress) throws Exception { mockLauncher.startScheduling(true); dagClient.waitForCompletion(); - Assert.assertEquals(DAGStatus.State.SUCCEEDED, dagClient.getDAGStatus(null).getState()); + Assert.assertEquals(DAGStatus.State.SUCCEEDED, + dagClient.getDAGStatus(null).getState()); Task task = dagImpl.getTask(killedTaId.getTaskID()); - Assert.assertEquals(2, task.getAttempts().size()); + Assert.assertEquals(ASSERT_SPECULATIONS_COUNT_MSG, 2, + task.getAttempts().size()); Assert.assertEquals(successTaId, task.getSuccessfulAttempt().getID()); TaskAttempt killedAttempt = task.getAttempt(killedTaId); Joiner.on(",").join(killedAttempt.getDiagnostics()).contains("Killed as speculative attempt"); @@ -204,18 +385,36 @@ public void testBasicSpeculation(boolean withProgress) throws Exception { tezClient.stop(); } - + + /** + * Test basic speculation with progress. + * + * @throws Exception the exception + */ + @Retry @Test (timeout=10000) public void testBasicSpeculationWithProgress() throws Exception { testBasicSpeculation(true); } + /** + * Test basic speculation without progress. + * + * @throws Exception the exception + */ + @Retry @Test (timeout=10000) public void testBasicSpeculationWithoutProgress() throws Exception { testBasicSpeculation(false); } - @Test (timeout=100000) + /** + * Test basic speculation per vertex conf. + * + * @throws Exception the exception + */ + @Retry + @Test (timeout=10000) public void testBasicSpeculationPerVertexConf() throws Exception { DAG dag = DAG.create("test"); String vNameNoSpec = "A"; @@ -224,8 +423,6 @@ public void testBasicSpeculationPerVertexConf() throws Exception { Vertex vA = Vertex.create(vNameNoSpec, ProcessorDescriptor.create("Proc.class"), 5); Vertex vB = Vertex.create(vNameSpec, ProcessorDescriptor.create("Proc.class"), 5); vA.setConf(TezConfiguration.TEZ_AM_SPECULATION_ENABLED, "false"); - vB.setConf(TezConfiguration.TEZ_AM_SOONEST_RETRY_AFTER_NO_SPECULATE, - speculatorSleepTime); dag.addVertex(vA); dag.addVertex(vB); // min/max src fraction is set to 1. So vertices will run sequentially @@ -273,6 +470,12 @@ public void testBasicSpeculationPerVertexConf() throws Exception { tezClient.stop(); } + /** + * Test basic speculation not useful. + * + * @throws Exception the exception + */ + @Retry @Test (timeout=10000) public void testBasicSpeculationNotUseful() throws Exception { DAG dag = DAG.create("test"); @@ -310,5 +513,4 @@ public void testBasicSpeculationNotUseful() throws Exception { .getValue()); tezClient.stop(); } - } From f214eabfb5f1da5ce5970558d97bf15405e3da9b Mon Sep 17 00:00:00 2001 From: Syed Shameerur Rahman Date: Tue, 4 Feb 2020 22:26:11 -0600 Subject: [PATCH 194/512] TEZ-2229. bower ESUDO Cannot be run with sudo -- during build Signed-off-by: Jonathan Eagles --- BUILDING.txt | 1 + tez-ui/pom.xml | 16 +++++++++++++++- 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/BUILDING.txt b/BUILDING.txt index ff7ac51200..f2231ea088 100644 --- a/BUILDING.txt +++ b/BUILDING.txt @@ -52,6 +52,7 @@ Build options: * Use -Dclover.license to specify the path to the clover license file * Use -Dhadoop.version to specify the version of hadoop to build tez against * Use -Dprotoc.path to specify the path to protoc + * Use -Dallow.root.build to root build tez-ui components Tests options: diff --git a/tez-ui/pom.xml b/tez-ui/pom.xml index 365249fbdd..500d21af87 100644 --- a/tez-ui/pom.xml +++ b/tez-ui/pom.xml @@ -34,6 +34,8 @@ node/yarn/dist/bin/yarn.js + --allow-root=false + false @@ -72,6 +74,18 @@ + + allowRootBuild + + + allow.root.build + true + + + + --allow-root=true + + @@ -144,7 +158,7 @@ generate-resources bower install - install + install ${allow-root-build} bower From 7c4c49966cae7fa73e95af30bd539913965998c4 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Wed, 5 Feb 2020 10:52:48 -0600 Subject: [PATCH 195/512] Revert "TEZ-4082. Reduce excessive getFileLinkInfo calls in Tez" This reverts commit 7a3e378b59dad2afe1a068669620846b87d6e732. --- .../org/apache/tez/client/TezClientUtils.java | 57 ++++++++++++------- .../org/apache/tez/common/TezCommonUtils.java | 17 +++++- .../apache/tez/common/TestTezCommonUtils.java | 36 ++++-------- tez-ui/src/main/webapp/bower-shrinkwrap.json | 44 +++++++------- 4 files changed, 84 insertions(+), 70 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index b7d46f8a61..bcb09205a9 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -19,7 +19,6 @@ package org.apache.tez.client; import java.io.File; -import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; import java.net.InetSocketAddress; @@ -122,6 +121,31 @@ public class TezClientUtils { private static Logger LOG = LoggerFactory.getLogger(TezClientUtils.class); private static final int UTF8_CHUNK_SIZE = 16 * 1024; + private static FileStatus[] getLRFileStatus(String fileName, Configuration conf) throws + IOException { + URI uri; + try { + uri = new URI(fileName); + } catch (URISyntaxException e) { + String message = "Invalid URI defined in configuration for" + + " location of TEZ jars. providedURI=" + fileName; + LOG.error(message); + throw new TezUncheckedException(message, e); + } + + Path p = new Path(uri); + FileSystem fs = p.getFileSystem(conf); + p = fs.resolvePath(p.makeQualified(fs.getUri(), + fs.getWorkingDirectory())); + FileSystem targetFS = p.getFileSystem(conf); + if (targetFS.isDirectory(p)) { + return targetFS.listStatus(p); + } else { + FileStatus fStatus = targetFS.getFileStatus(p); + return new FileStatus[]{fStatus}; + } + } + /** * Setup LocalResource map for Tez jars based on provided Configuration * @@ -193,16 +217,8 @@ private static boolean addLocalResources(Configuration conf, } Path p = new Path(u); FileSystem remoteFS = p.getFileSystem(conf); - FileStatus targetStatus = remoteFS.getFileLinkStatus(p); - p = targetStatus.getPath(); - - FileStatus[] fileStatuses; - FileSystem targetFS = p.getFileSystem(conf); - if (targetStatus.isDirectory()) { - fileStatuses = targetFS.listStatus(p); - } else { - fileStatuses = new FileStatus[]{targetStatus}; - } + p = remoteFS.resolvePath(p.makeQualified(remoteFS.getUri(), + remoteFS.getWorkingDirectory())); LocalResourceType type = null; @@ -216,6 +232,8 @@ private static boolean addLocalResources(Configuration conf, type = LocalResourceType.FILE; } + FileStatus [] fileStatuses = getLRFileStatus(configUri, conf); + for (FileStatus fStatus : fileStatuses) { String linkName; if (fStatus.isDirectory()) { @@ -311,16 +329,13 @@ public static FileSystem ensureStagingDirExists(Configuration conf, Path stagingArea) throws IOException { FileSystem fs = stagingArea.getFileSystem(conf); + String realUser; + String currentUser; UserGroupInformation ugi = UserGroupInformation.getLoginUser(); - String realUser = ugi.getShortUserName(); - String currentUser = UserGroupInformation.getCurrentUser().getShortUserName(); - FileStatus fsStatus = null; - try { - fsStatus = fs.getFileStatus(stagingArea); - } catch (FileNotFoundException e) { - TezCommonUtils.mkDirForAM(fs, stagingArea); - } - if (fsStatus != null) { + realUser = ugi.getShortUserName(); + currentUser = UserGroupInformation.getCurrentUser().getShortUserName(); + if (fs.exists(stagingArea)) { + FileStatus fsStatus = fs.getFileStatus(stagingArea); String owner = fsStatus.getOwner(); if (!(owner.equals(currentUser) || owner.equals(realUser))) { throw new IOException("The ownership on the staging directory " @@ -335,6 +350,8 @@ public static FileSystem ensureStagingDirExists(Configuration conf, + TezCommonUtils.TEZ_AM_DIR_PERMISSION); fs.setPermission(stagingArea, TezCommonUtils.TEZ_AM_DIR_PERMISSION); } + } else { + TezCommonUtils.mkDirForAM(fs, stagingArea); } return fs; } diff --git a/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java index fc4789fc7d..16165e2377 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java @@ -79,8 +79,21 @@ public class TezCommonUtils { * @return Fully qualified staging directory */ public static Path getTezBaseStagingPath(Configuration conf) { - return new Path(conf.get(TezConfiguration.TEZ_AM_STAGING_DIR, - TezConfiguration.TEZ_AM_STAGING_DIR_DEFAULT)); + String stagingDirStr = conf.get(TezConfiguration.TEZ_AM_STAGING_DIR, + TezConfiguration.TEZ_AM_STAGING_DIR_DEFAULT); + Path baseStagingDir; + try { + Path p = new Path(stagingDirStr); + FileSystem fs = p.getFileSystem(conf); + if (!fs.exists(p)) { + mkDirForAM(fs, p); + LOG.info("Stage directory " + p + " doesn't exist and is created"); + } + baseStagingDir = fs.resolvePath(p); + } catch (IOException e) { + throw new TezUncheckedException(e); + } + return baseStagingDir; } /** diff --git a/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java b/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java index 52df2c6d01..3929c4bf5a 100644 --- a/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java +++ b/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java @@ -57,7 +57,7 @@ public static void setup() throws Exception { LOG.info("Starting mini clusters"); try { conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, TEST_ROOT_DIR); - dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).format(true).racks(null) + dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).format(true).racks(null) .build(); remoteFs = dfsCluster.getFileSystem(); RESOLVED_STAGE_DIR = remoteFs.getUri() + STAGE_DIR; @@ -80,23 +80,19 @@ public static void afterClass() throws InterruptedException { } // Testing base staging dir - @Test + @Test(timeout = 5000) public void testTezBaseStagingPath() throws Exception { Configuration localConf = new Configuration(); // Check if default works with localFS localConf.unset(TezConfiguration.TEZ_AM_STAGING_DIR); localConf.set("fs.defaultFS", "file:///"); Path stageDir = TezCommonUtils.getTezBaseStagingPath(localConf); - TezCommonUtils.mkDirForAM(stageDir.getFileSystem(localConf), stageDir); - Path resolveStageDir = stageDir.getFileSystem(localConf).resolvePath(stageDir); - Assert.assertEquals(resolveStageDir.toString(), "file:" + TezConfiguration.TEZ_AM_STAGING_DIR_DEFAULT); + Assert.assertEquals(stageDir.toString(), "file:" + TezConfiguration.TEZ_AM_STAGING_DIR_DEFAULT); // check if user set something, indeed works conf.set(TezConfiguration.TEZ_AM_STAGING_DIR, STAGE_DIR); stageDir = TezCommonUtils.getTezBaseStagingPath(conf); - TezCommonUtils.mkDirForAM(stageDir.getFileSystem(conf), stageDir); - resolveStageDir = stageDir.getFileSystem(conf).resolvePath(stageDir); - Assert.assertEquals(resolveStageDir.toString(), RESOLVED_STAGE_DIR); + Assert.assertEquals(stageDir.toString(), RESOLVED_STAGE_DIR); } // Testing System staging dir if createed @@ -115,8 +111,7 @@ public void testCreateTezSysStagingPath() throws Exception { } Assert.assertFalse(fs.exists(stagePath)); Path stageDir = TezCommonUtils.createTezSystemStagingPath(conf, strAppId); - Path resolveStageDir = stageDir.getFileSystem(conf).resolvePath(stageDir); - Assert.assertEquals(resolveStageDir.toString(), expectedStageDir); + Assert.assertEquals(stageDir.toString(), expectedStageDir); Assert.assertTrue(fs.exists(stagePath)); } @@ -127,9 +122,7 @@ public void testTezSysStagingPath() throws Exception { Path stageDir = TezCommonUtils.getTezSystemStagingPath(conf, strAppId); String expectedStageDir = RESOLVED_STAGE_DIR + Path.SEPARATOR + TezCommonUtils.TEZ_SYSTEM_SUB_DIR + Path.SEPARATOR + strAppId; - TezCommonUtils.mkDirForAM(stageDir.getFileSystem(conf), stageDir); - Path resolvedStageDir = stageDir.getFileSystem(conf).resolvePath(stageDir); - Assert.assertEquals(resolvedStageDir.toString(), expectedStageDir); + Assert.assertEquals(stageDir.toString(), expectedStageDir); } // Testing conf staging dir @@ -141,9 +134,7 @@ public void testTezConfStagingPath() throws Exception { String expectedDir = RESOLVED_STAGE_DIR + Path.SEPARATOR + TezCommonUtils.TEZ_SYSTEM_SUB_DIR + Path.SEPARATOR + strAppId + Path.SEPARATOR + TezConstants.TEZ_PB_BINARY_CONF_NAME; - TezCommonUtils.mkDirForAM(confStageDir.getFileSystem(conf), confStageDir); - Path resolvedConfStageDir = confStageDir.getFileSystem(conf).resolvePath(confStageDir); - Assert.assertEquals(resolvedConfStageDir.toString(), expectedDir); + Assert.assertEquals(confStageDir.toString(), expectedDir); } // Testing session jars staging dir @@ -155,9 +146,7 @@ public void testTezSessionJarStagingPath() throws Exception { String expectedDir = RESOLVED_STAGE_DIR + Path.SEPARATOR + TezCommonUtils.TEZ_SYSTEM_SUB_DIR + Path.SEPARATOR + strAppId + Path.SEPARATOR + TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME; - TezCommonUtils.mkDirForAM(confStageDir.getFileSystem(conf), confStageDir); - Path resolvedConfStageDir = confStageDir.getFileSystem(conf).resolvePath(confStageDir); - Assert.assertEquals(resolvedConfStageDir.toString(), expectedDir); + Assert.assertEquals(confStageDir.toString(), expectedDir); } // Testing bin plan staging dir @@ -169,9 +158,7 @@ public void testTezBinPlanStagingPath() throws Exception { String expectedDir = RESOLVED_STAGE_DIR + Path.SEPARATOR + TezCommonUtils.TEZ_SYSTEM_SUB_DIR + Path.SEPARATOR + strAppId + Path.SEPARATOR + TezConstants.TEZ_PB_PLAN_BINARY_NAME; - TezCommonUtils.mkDirForAM(confStageDir.getFileSystem(conf), confStageDir); - Path resolvedConfStageDir = confStageDir.getFileSystem(conf).resolvePath(confStageDir); - Assert.assertEquals(resolvedConfStageDir.toString(), expectedDir); + Assert.assertEquals(confStageDir.toString(), expectedDir); } // Testing text plan staging dir @@ -180,15 +167,12 @@ public void testTezTextPlanStagingPath() throws Exception { String strAppId = "testAppId"; String dagPBName = "testDagPBName"; Path tezSysStagingPath = TezCommonUtils.getTezSystemStagingPath(conf, strAppId); - TezCommonUtils.mkDirForAM(tezSysStagingPath.getFileSystem(conf), tezSysStagingPath); Path confStageDir = TezCommonUtils.getTezTextPlanStagingPath(tezSysStagingPath, strAppId, dagPBName); String expectedDir = RESOLVED_STAGE_DIR + Path.SEPARATOR + TezCommonUtils.TEZ_SYSTEM_SUB_DIR + Path.SEPARATOR + strAppId + Path.SEPARATOR + strAppId + "-" + dagPBName + "-" + TezConstants.TEZ_PB_PLAN_TEXT_NAME; - TezCommonUtils.createFileForAM(confStageDir.getFileSystem(conf), confStageDir); - Path resolvedConfStageDir = confStageDir.getFileSystem(conf).resolvePath(confStageDir); - Assert.assertEquals(resolvedConfStageDir.toString(), expectedDir); + Assert.assertEquals(confStageDir.toString(), expectedDir); } // Testing recovery path staging dir diff --git a/tez-ui/src/main/webapp/bower-shrinkwrap.json b/tez-ui/src/main/webapp/bower-shrinkwrap.json index edcbae33c1..357d57691a 100644 --- a/tez-ui/src/main/webapp/bower-shrinkwrap.json +++ b/tez-ui/src/main/webapp/bower-shrinkwrap.json @@ -1,72 +1,72 @@ { "https://github.com/FortAwesome/Font-Awesome.git": { - "4.5.0": "4.5.0" + "4.5.0": "593ad563a987977f14102be935d0abc2a172903e" }, "https://github.com/Teleborder/FileSaver.js.git": { - "1.20150507.2": "1.20150507.2" + "1.20150507.2": "b7cf622909258086bc63ad764d08fcaed780ab42" }, "https://github.com/adamwdraper/Numeral-js.git": { - "1.5.3": "1.5.3" + "1.5.3": "f97f14bb8bab988f28f1d854525b4cfeff8ec9e1" }, "https://github.com/components/codemirror.git": { - "5.11.0": "5.11.0" + "5.11.0": "7d43f32bb56f83a9c47addb3f91170b3102f3ead" }, "https://github.com/components/ember-data.git": { - "2.1.0": "2.1.0" + "2.1.0": "d8b4d3092f67afe22d9d374c40d719d557915fa3" }, "https://github.com/components/ember.git": { - "2.2.0": "2.2.0" + "2.2.0": "49e042ca89922ed96b27488c2a98add280ae7123" }, "https://github.com/components/jqueryui.git": { - "1.11.4": "1.11.4" + "1.11.4": "c34f8dbf3ba57b3784b93f26119f436c0e8288e1" }, "https://github.com/dockyard/ember-qunit-notifications.git": { - "0.1.0": "0.1.0" + "0.1.0": "a83277aa7a1c0545c66e6d133caebb9a620e71ad" }, "https://github.com/dockyard/qunit-notifications.git": { - "0.1.1": "0.1.1" + "0.1.1": "7a13f6dba5a340e1cb9e0b64c1c711e4d7edaca1" }, "https://github.com/ember-cli/ember-cli-shims.git": { - "0.0.6": "0.0.6" + "0.0.6": "dcab43b58d5698690050bb9a46ead5c8663c7da1" }, "https://github.com/ember-cli/ember-cli-test-loader.git": { - "0.2.1": "0.2.1" + "0.2.1": "3348d801089279296c38f31ae14d9c4d115ce154" }, "https://github.com/ember-cli/ember-load-initializers.git": { - "0.1.7": "0.1.7" + "0.1.7": "7bb21488563bd1bba23e903a812bf5815beddd1a" }, "https://github.com/jquery/jquery-dist.git": { - "2.1.4": "2.1.4" + "2.1.4": "7751e69b615c6eca6f783a81e292a55725af6b85" }, "https://github.com/jquery/jquery-mousewheel.git": { - "3.1.13": "3.1.13" + "3.1.13": "67289b6b2aa0066d7d78a5807f520387135ffb22" }, "https://github.com/jquery/qunit.git": { - "1.19.0": "1.19.0" + "1.19.0": "467e7e34652ad7d5883ce9c568461cf8c5e172a8" }, "https://github.com/moment/moment-timezone.git": { - "0.5.0": "0.5.0" + "0.5.0": "74a2e9378ecf4a31a168f3049f086565c8d66814" }, "https://github.com/moment/moment.git": { - "2.12.0": "2.12.0" + "2.12.0": "d3d7488b4d60632854181cb0a9af325d57fb3d51" }, "https://github.com/rwjblue/ember-qunit-builds.git": { - "0.4.16": "0.4.16" + "0.4.16": "142c4066a5458bef9dfcb92b70152b9c01d79188" }, "https://github.com/sreenaths/more-js.git": { "0.8.6": "f1d9ccdaf7ff74c26b6ee341067a5a5ef33bd64a", "0.8.8": "0.8.8" }, "https://github.com/sreenaths/snippet-ss.git": { - "1.11.0": "1.11.0" + "1.11.0": "c1abc566f4e001b7f1939b6dbdd911eadc969cf9" }, "https://github.com/sreenaths/zip.js.git": { - "1.0.0": "1.0.0" + "1.0.0": "ec67ad22eba116083ea3ef2fe0b40ccc953513ce" }, "https://github.com/stefanpenner/loader.js.git": { - "3.3.0": "3.3.0" + "3.3.0": "ac909550c9544325632542bbea97531cc60bc628" }, "https://github.com/twbs/bootstrap.git": { - "3.3.6": "3.3.6" + "3.3.6": "81df608a40bf0629a1dc08e584849bb1e43e0b7a" } } \ No newline at end of file From 76b96fca8f5a70ecce902740e3673d100a029fca Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Wed, 5 Feb 2020 11:03:15 -0600 Subject: [PATCH 196/512] TEZ-4026. Fetch Download rate shows 0.0 MB per second if duration is 0 millis --- .../library/common/shuffle/ShuffleUtils.java | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java index 1482a12b51..40909d4715 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java @@ -567,15 +567,14 @@ public void logIndividualFetchComplete(long millis, long bytesCompressed, if (activeLogger.isInfoEnabled()) { long wholeMBs = 0; long partialMBs = 0; - if (millis != 0) { - // fast math is done using integer math to avoid double to string conversion - // calculate B/s * 100 to preserve MBs precision to two decimal places - // multiply numerator by 100000 (2^5 * 5^5) and divide denominator by MB (2^20) - // simply fraction to protect ourselves from overflow by factoring out 2^5 - wholeMBs = (bytesCompressed * 3125) / (millis * 32768); - partialMBs = wholeMBs % 100; - wholeMBs /= 100; - } + millis = Math.max(1L, millis); + // fast math is done using integer math to avoid double to string conversion + // calculate B/s * 100 to preserve MBs precision to two decimal places + // multiply numerator by 100000 (2^5 * 5^5) and divide denominator by MB (2^20) + // simply fraction to protect ourselves from overflow by factoring out 2^5 + wholeMBs = (bytesCompressed * 3125) / (millis * 32768); + partialMBs = wholeMBs % 100; + wholeMBs /= 100; StringBuilder sb = new StringBuilder("Completed fetch for attempt: "); toShortString(srcAttemptIdentifier, sb); sb.append(" to "); From b078e3a25f892fcd610a1cc15688e825e41a9bcf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Thu, 6 Feb 2020 09:01:59 -0600 Subject: [PATCH 197/512] TEZ-4081. Container release idle timeout exception for equal min and max values Signed-off-by: Jonathan Eagles --- .../dag/app/rm/DagAwareYarnTaskScheduler.java | 3 ++- .../app/rm/TestDagAwareYarnTaskScheduler.java | 20 +++++++++++++++++++ 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java index 3ba1bfaf65..31a7142987 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java @@ -1543,7 +1543,8 @@ void resetMatchingLevel() { long getIdleExpirationTimestamp(long now) { if (idleExpirationTimestamp == 0) { if (idleContainerTimeoutMin > 0) { - idleExpirationTimestamp = now + random.nextLong(idleContainerTimeoutMin, idleContainerTimeoutMax); + idleExpirationTimestamp = now + (idleContainerTimeoutMin == idleContainerTimeoutMax ? idleContainerTimeoutMin + : random.nextLong(idleContainerTimeoutMin, idleContainerTimeoutMax)); } else { idleExpirationTimestamp = Long.MAX_VALUE; } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java index ad0cf079f2..38088905b3 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java @@ -1530,6 +1530,26 @@ public void testIdleContainerAssignmentReuseNewContainers() throws Exception { verify(mockRMClient).removeContainerRequest(reqv0t0); } + @Test + public void testMinMaxContainerIdleMillisAreEqual() throws Exception { + AMRMClientAsyncWrapperForTest mockRMClient = new AMRMClientAsyncWrapperForTest(); + Configuration conf = new Configuration(); + conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, 10000); + conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, 10000); + + TaskSchedulerContext mockApp = setupMockTaskSchedulerContext("host", 0, "url", conf); + TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp); + MockClock clock = new MockClock(1000); + NewTaskSchedulerForTest scheduler = new NewTaskSchedulerForTest(drainableAppCallback, mockRMClient, clock); + scheduler.initialize(); + + NodeId host1 = NodeId.newInstance("host1", 1); + Container container1 = Container.newInstance(null, host1, null, null, null, null); + HeldContainer heldContainer = scheduler.new HeldContainer(container1); + long now = clock.getTime(); + assertEquals(now + 10000, heldContainer.getIdleExpirationTimestamp(now)); + } + static class AMRMClientAsyncWrapperForTest extends AMRMClientAsyncWrapper { AMRMClientAsyncWrapperForTest() { super(new MockAMRMClient(), 10000, null); From 86182d0355c89bd632d5617ba517cea469128fb3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Mon, 10 Feb 2020 08:56:26 -0600 Subject: [PATCH 198/512] TEZ-4122. TestMRRJobsDAGApi should set TezClassLoader Signed-off-by: Jonathan Eagles --- .../test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tez-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java b/tez-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java index a756cf9289..96b7bbf655 100644 --- a/tez-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java +++ b/tez-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java @@ -71,6 +71,7 @@ import org.apache.tez.client.TezClient; import org.apache.tez.client.TezAppMasterStatus; import org.apache.tez.common.ReflectionUtils; +import org.apache.tez.common.TezClassLoader; import org.apache.tez.common.TezUtils; import org.apache.tez.common.counters.FileSystemCounter; import org.apache.tez.common.counters.TaskCounter; @@ -833,6 +834,7 @@ public List initialize() throws Exception { .getConfigurationBytes()); try { + Thread.currentThread().setContextClassLoader(TezClassLoader.getInstance()); ReflectionUtils.getClazz(RELOCALIZATION_TEST_CLASS_NAME); LOG.info("Class found"); FileSystem fs = FileSystem.get(conf); From 544290e5da84596de5554863737aaee3fffa4d19 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Wed, 12 Feb 2020 09:45:15 -0600 Subject: [PATCH 199/512] TEZ-4124. GuavaShim: introduce an interoperability layer for different guava versions Signed-off-by: Jonathan Eagles --- .../java/org/apache/tez/common/GuavaShim.java | 54 +++++++++++++++++++ .../app/dag/RootInputInitializerManager.java | 3 +- .../apache/tez/dag/app/dag/impl/DAGImpl.java | 6 ++- .../tez/dag/app/dag/impl/VertexImpl.java | 3 +- .../tez/dag/app/dag/impl/VertexManager.java | 3 +- .../apache/tez/dag/app/MockDAGAppMaster.java | 5 +- .../app/rm/TestDagAwareYarnTaskScheduler.java | 12 +++-- .../dag/app/TezTestServiceCommunicator.java | 5 +- .../tez/service/impl/ContainerRunnerImpl.java | 5 +- .../apache/tez/runtime/task/TaskReporter.java | 3 +- .../common/shuffle/impl/ShuffleManager.java | 5 +- .../shuffle/orderedgrouped/Shuffle.java | 3 +- .../orderedgrouped/ShuffleScheduler.java | 3 +- .../writers/UnorderedPartitionedKVWriter.java | 3 +- 14 files changed, 93 insertions(+), 20 deletions(-) create mode 100644 tez-api/src/main/java/org/apache/tez/common/GuavaShim.java diff --git a/tez-api/src/main/java/org/apache/tez/common/GuavaShim.java b/tez-api/src/main/java/org/apache/tez/common/GuavaShim.java new file mode 100644 index 0000000000..d9b8796d61 --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/common/GuavaShim.java @@ -0,0 +1,54 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.common; + +import com.google.common.util.concurrent.MoreExecutors; + +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.concurrent.Executor; + +/** + * A interoperability layer to work with multiple versions of guava. + */ +public final class GuavaShim { + + static { + try { + executorMethod = MoreExecutors.class.getDeclaredMethod("directExecutor"); + } catch (NoSuchMethodException nsme) { + try { + executorMethod = MoreExecutors.class.getDeclaredMethod("sameThreadExecutor"); + } catch (NoSuchMethodException nsmeSame) { + } + } + } + + private GuavaShim() { + } + + private static Method executorMethod; + + public static Executor directExecutor() { + try { + return (Executor) executorMethod.invoke(null); + } catch (IllegalAccessException | IllegalArgumentException | InvocationTargetException e) { + throw new RuntimeException(e); + } + } +} \ No newline at end of file diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java index b2a0c0bb59..7ff9fa9c0d 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java @@ -46,6 +46,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tez.common.GuavaShim; import org.apache.tez.common.ReflectionUtils; import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.dag.api.InputDescriptor; @@ -138,7 +139,7 @@ public void runInputInitializers(List> future = executor .submit(new InputInitializerCallable(initializerWrapper, dagUgi, appContext)); - Futures.addCallback(future, createInputInitializerCallback(initializerWrapper)); + Futures.addCallback(future, createInputInitializerCallback(initializerWrapper), GuavaShim.directExecutor()); } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java index 20ecc4f274..3cde8e7812 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java @@ -43,6 +43,7 @@ import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.tez.Utils; +import org.apache.tez.common.GuavaShim; import org.apache.tez.common.ProgressHelper; import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.common.counters.LimitExceededException; @@ -1142,7 +1143,7 @@ public Void call() throws Exception { } for (Map.Entry entry : commitEvents.entrySet()) { ListenableFuture commitFuture = appContext.getExecService().submit(entry.getValue()); - Futures.addCallback(commitFuture, entry.getValue().getCallback()); + Futures.addCallback(commitFuture, entry.getValue().getCallback(), GuavaShim.directExecutor()); commitFutures.put(entry.getKey(), commitFuture); } } @@ -2156,7 +2157,8 @@ public Void call() throws Exception { }; }; ListenableFuture groupCommitFuture = appContext.getExecService().submit(groupCommitCallableEvent); - Futures.addCallback(groupCommitFuture, groupCommitCallableEvent.getCallback()); + Futures.addCallback(groupCommitFuture, groupCommitCallableEvent.getCallback(), + GuavaShim.directExecutor()); commitFutures.put(outputKey, groupCommitFuture); } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index 0d34be6d00..f0a8642c1c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -64,6 +64,7 @@ import org.apache.hadoop.yarn.util.Clock; import org.apache.tez.client.TezClientUtils; import org.apache.tez.common.ATSConstants; +import org.apache.tez.common.GuavaShim; import org.apache.tez.common.ProgressHelper; import org.apache.tez.common.ReflectionUtils; import org.apache.tez.common.TezUtilsInternal; @@ -2259,7 +2260,7 @@ public Void run() throws Exception { }; ListenableFuture commitFuture = vertex.getAppContext().getExecService().submit(commitCallableEvent); - Futures.addCallback(commitFuture, commitCallableEvent.getCallback()); + Futures.addCallback(commitFuture, commitCallableEvent.getCallback(), GuavaShim.directExecutor()); vertex.commitFutures.put(outputName, commitFuture); } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java index 03e03aacee..292742530a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java @@ -78,6 +78,7 @@ import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType; import com.google.common.base.Function; +import org.apache.tez.common.GuavaShim; import org.apache.tez.common.Preconditions; import com.google.common.collect.Collections2; import com.google.common.collect.Lists; @@ -488,7 +489,7 @@ private void tryScheduleNextEvent() { VertexManagerEvent e = eventQueue.poll(); if (e != null) { ListenableFuture future = execService.submit(e); - Futures.addCallback(future, e.getCallback()); + Futures.addCallback(future, e.getCallback(), GuavaShim.directExecutor()); } else { // This may happen. Lets say Callback succeeded on threadA. It set eventInFlight to false // and called tryScheduleNextEvent() and found queue not empty but got paused before it diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java index 638d0844f3..9bceaece93 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java @@ -57,6 +57,8 @@ import org.apache.tez.client.TezApiVersionInfo; import org.apache.tez.common.ContainerContext; import org.apache.tez.common.ContainerTask; +import org.apache.tez.common.GuavaShim; +import org.apache.tez.common.Preconditions; import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.TezUncheckedException; @@ -73,7 +75,6 @@ import org.apache.tez.runtime.api.impl.TezEvent; import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType; -import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.util.concurrent.FutureCallback; @@ -323,7 +324,7 @@ public void run() { Worker worker = workers.remove(); worker.setContainerData(cData); ListenableFuture future = executorService.submit(worker); - Futures.addCallback(future, worker.getCallback()); + Futures.addCallback(future, worker.getCallback(), GuavaShim.directExecutor()); } else { containers.remove(cData.cId); } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java index 38088905b3..48dd93825b 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java @@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl; +import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes; import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.tez.common.MockDNSToSwitchMapping; @@ -1559,6 +1560,12 @@ RegisterApplicationMasterResponse getRegistrationResponse() { return ((MockAMRMClient) client).getRegistrationResponse(); } + @Override + public RegisterApplicationMasterResponse registerApplicationMaster(String appHostName, int appHostPort, + String appTrackingUrl) throws YarnException, IOException { + return client.registerApplicationMaster(appHostName, appHostPort, appTrackingUrl); + } + @Override protected void serviceStart() { } @@ -1585,10 +1592,9 @@ protected void serviceStart() { protected void serviceStop() { } - @SuppressWarnings("unchecked") @Override - public RegisterApplicationMasterResponse registerApplicationMaster( - String appHostName, int appHostPort, String appTrackingUrl) { + public RegisterApplicationMasterResponse registerApplicationMaster(String appHostName, int appHostPort, + String appTrackingUrl) { mockRegResponse = mock(RegisterApplicationMasterResponse.class); Resource mockMaxResource = Resources.createResource(1024*1024, 1024); Map mockAcls = Collections.emptyMap(); diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/TezTestServiceCommunicator.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/TezTestServiceCommunicator.java index ac50878cf5..713a3d3fcd 100644 --- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/TezTestServiceCommunicator.java +++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/TezTestServiceCommunicator.java @@ -29,6 +29,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.protobuf.Message; import org.apache.hadoop.service.AbstractService; +import org.apache.tez.common.GuavaShim; import org.apache.tez.service.TezTestServiceProtocolBlockingPB; import org.apache.tez.service.impl.TezTestServiceProtocolClientImpl; import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto; @@ -69,7 +70,7 @@ public void onSuccess(RunContainerResponseProto result) { public void onFailure(Throwable t) { callback.indicateError(t); } - }); + }, GuavaShim.directExecutor()); } @@ -86,7 +87,7 @@ public void onSuccess(SubmitWorkResponseProto result) { public void onFailure(Throwable t) { callback.indicateError(t); } - }); + }, GuavaShim.directExecutor()); } diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java index d440d1f17f..e7777e2590 100644 --- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java +++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java @@ -30,6 +30,7 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import org.apache.tez.common.GuavaShim; import org.apache.tez.common.Preconditions; import com.google.common.collect.HashMultimap; import com.google.common.collect.Multimap; @@ -210,7 +211,7 @@ public void queueContainer(RunContainerRequestProto request) throws TezException workingDir, credentials, memoryPerExecutor); ListenableFuture future = executorService .submit(callable); - Futures.addCallback(future, new ContainerRunnerCallback(request, callable)); + Futures.addCallback(future, new ContainerRunnerCallback(request, callable), GuavaShim.directExecutor()); } /** @@ -269,7 +270,7 @@ public void submitWork(SubmitWorkRequestProto request) throws TezException { new ExecutionContextImpl(localAddress.get().getHostName()), env, localDirs, workingDir, credentials, memoryPerExecutor, sharedExecutor); ListenableFuture future = executorService.submit(callable); - Futures.addCallback(future, new TaskRunnerCallback(request, callable)); + Futures.addCallback(future, new TaskRunnerCallback(request, callable), GuavaShim.directExecutor()); } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskReporter.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskReporter.java index 809ce325ec..fb066fd2bd 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskReporter.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskReporter.java @@ -35,6 +35,7 @@ import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.hadoop.util.ShutdownHookManager; +import org.apache.tez.common.GuavaShim; import org.apache.tez.common.TezTaskUmbilicalProtocol; import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.TezException; @@ -108,7 +109,7 @@ public synchronized void registerTask(RuntimeTask task, currentCallable = new HeartbeatCallable(task, umbilical, pollInterval, sendCounterInterval, maxEventsToGet, requestCounter, containerIdStr); ListenableFuture future = heartbeatExecutor.submit(currentCallable); - Futures.addCallback(future, new HeartbeatCallback(errorReporter)); + Futures.addCallback(future, new HeartbeatCallback(errorReporter), GuavaShim.directExecutor()); } /** diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java index b91a5cafa7..8ae4f60667 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java @@ -63,6 +63,7 @@ import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.tez.common.CallableWithNdc; +import org.apache.tez.common.GuavaShim; import org.apache.tez.common.TezRuntimeFrameworkConfigs; import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.common.counters.TaskCounter; @@ -346,7 +347,7 @@ public void run() throws IOException { } ListenableFuture runShuffleFuture = schedulerExecutor.submit(schedulerCallable); - Futures.addCallback(runShuffleFuture, new SchedulerFutureCallback()); + Futures.addCallback(runShuffleFuture, new SchedulerFutureCallback(), GuavaShim.directExecutor()); // Shutdown this executor once this task, and the callback complete. schedulerExecutor.shutdown(); } @@ -460,7 +461,7 @@ protected Void callInternal() throws Exception { } ListenableFuture future = fetcherExecutor .submit(fetcher); - Futures.addCallback(future, new FetchFutureCallback(fetcher)); + Futures.addCallback(future, new FetchFutureCallback(fetcher), GuavaShim.directExecutor()); if (++count >= maxFetchersToRun) { break; } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java index 4b426b75c8..38f079a20b 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java @@ -42,6 +42,7 @@ import org.apache.hadoop.io.compress.DefaultCodec; import org.apache.hadoop.util.ReflectionUtils; import org.apache.tez.common.CallableWithNdc; +import org.apache.tez.common.GuavaShim; import org.apache.tez.common.TezRuntimeFrameworkConfigs; import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.common.counters.TaskCounter; @@ -270,7 +271,7 @@ public TezRawKeyValueIterator waitForInput() throws IOException, InterruptedExce public void run() throws IOException { merger.configureAndStart(); runShuffleFuture = executor.submit(runShuffleCallable); - Futures.addCallback(runShuffleFuture, new ShuffleRunnerFutureCallback()); + Futures.addCallback(runShuffleFuture, new ShuffleRunnerFutureCallback(), GuavaShim.directExecutor()); executor.shutdown(); } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java index d388b5b1bf..707f920fd7 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java @@ -43,6 +43,7 @@ import java.util.concurrent.atomic.AtomicLong; import com.google.common.annotations.VisibleForTesting; +import org.apache.tez.common.GuavaShim; import org.apache.tez.common.Preconditions; import com.google.common.collect.LinkedListMultimap; import com.google.common.collect.ListMultimap; @@ -1440,7 +1441,7 @@ protected Void callInternal() throws InterruptedException { FetcherOrderedGrouped fetcherOrderedGrouped = constructFetcherForHost(mapHost); runningFetchers.add(fetcherOrderedGrouped); ListenableFuture future = fetcherExecutor.submit(fetcherOrderedGrouped); - Futures.addCallback(future, new FetchFutureCallback(fetcherOrderedGrouped)); + Futures.addCallback(future, new FetchFutureCallback(fetcherOrderedGrouped), GuavaShim.directExecutor()); } } } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java index 9206b18608..c823b6518b 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java @@ -52,6 +52,7 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.tez.common.CallableWithNdc; +import org.apache.tez.common.GuavaShim; import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.common.counters.TaskCounter; @@ -542,7 +543,7 @@ private boolean scheduleSpill(boolean block) throws IOException { new ArrayList(filledBuffers), codec, spilledRecordsCounter, spillNumber)); filledBuffers.clear(); - Futures.addCallback(future, new SpillCallback(spillNumber)); + Futures.addCallback(future, new SpillCallback(spillNumber), GuavaShim.directExecutor()); // Update once per buffer (instead of every record) updateTezCountersAndNotify(); return true; From 1fe978438e6cb213ac7fb70e0dde72c6e290dcae Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Wed, 12 Feb 2020 10:06:01 -0600 Subject: [PATCH 200/512] TEZ-4123. TestMRRJobsDAGApi flaky timeout - unhealthy node Signed-off-by: Jonathan Eagles --- .../test/java/org/apache/tez/test/MiniTezCluster.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java b/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java index bac0e8c5cc..17c688590c 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java +++ b/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java @@ -88,6 +88,15 @@ public MiniTezCluster(String testName, int noOfNMs, super(testName, noOfNMs, numLocalDirs, numLogDirs); } + @Override + public void init(Configuration conf) { + if (conf.getFloat(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, + YarnConfiguration.DEFAULT_NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE) == YarnConfiguration.DEFAULT_NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE) { + conf.setFloat(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, 99.0f); + } + super.init(conf); + } + @Override public void serviceInit(Configuration conf) throws Exception { conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.YARN_TEZ_FRAMEWORK_NAME); From 47eed6474ccc5540291953e3ff0f78fd5894897a Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Wed, 12 Feb 2020 10:07:46 -0600 Subject: [PATCH 201/512] Revert "TEZ-4123. TestMRRJobsDAGApi flaky timeout - unhealthy node" This reverts commit 1fe978438e6cb213ac7fb70e0dde72c6e290dcae. --- .../test/java/org/apache/tez/test/MiniTezCluster.java | 9 --------- 1 file changed, 9 deletions(-) diff --git a/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java b/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java index 17c688590c..bac0e8c5cc 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java +++ b/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java @@ -88,15 +88,6 @@ public MiniTezCluster(String testName, int noOfNMs, super(testName, noOfNMs, numLocalDirs, numLogDirs); } - @Override - public void init(Configuration conf) { - if (conf.getFloat(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, - YarnConfiguration.DEFAULT_NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE) == YarnConfiguration.DEFAULT_NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE) { - conf.setFloat(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, 99.0f); - } - super.init(conf); - } - @Override public void serviceInit(Configuration conf) throws Exception { conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.YARN_TEZ_FRAMEWORK_NAME); From fd19ce6c93bc1f899ccca7161b0c0407f850bd77 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Wed, 12 Feb 2020 10:09:26 -0600 Subject: [PATCH 202/512] TEZ-4123. TestMRRJobsDAGApi flaky timeout - unhealthy node Signed-off-by: Jonathan Eagles --- .../org/apache/tez/test/MiniTezCluster.java | 9 ++++ .../apache/tez/test/TestMiniTezCluster.java | 53 +++++++++++++++++++ 2 files changed, 62 insertions(+) create mode 100644 tez-tests/src/test/java/org/apache/tez/test/TestMiniTezCluster.java diff --git a/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java b/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java index bac0e8c5cc..17c688590c 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java +++ b/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java @@ -88,6 +88,15 @@ public MiniTezCluster(String testName, int noOfNMs, super(testName, noOfNMs, numLocalDirs, numLogDirs); } + @Override + public void init(Configuration conf) { + if (conf.getFloat(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, + YarnConfiguration.DEFAULT_NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE) == YarnConfiguration.DEFAULT_NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE) { + conf.setFloat(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, 99.0f); + } + super.init(conf); + } + @Override public void serviceInit(Configuration conf) throws Exception { conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.YARN_TEZ_FRAMEWORK_NAME); diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestMiniTezCluster.java b/tez-tests/src/test/java/org/apache/tez/test/TestMiniTezCluster.java new file mode 100644 index 0000000000..4fda977a4c --- /dev/null +++ b/tez-tests/src/test/java/org/apache/tez/test/TestMiniTezCluster.java @@ -0,0 +1,53 @@ +/** +* Licensed to the Apache Software Foundation (ASF) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. The ASF licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ +package org.apache.tez.test; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.junit.Assert; +import org.junit.Test; + +public class TestMiniTezCluster { + + @Test + public void testOverrideYarnDiskHealthCheck() throws IOException { + MiniTezCluster tezMiniCluster = new MiniTezCluster(TestMiniTezCluster.class.getName(), 1, 1, 1); + tezMiniCluster.init(new Configuration()); + tezMiniCluster.start(); + + // overrides if not set + Assert.assertEquals(99.0, tezMiniCluster.getConfig() + .getFloat(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, -1), 0.00001); + + tezMiniCluster.close(); + + tezMiniCluster = new MiniTezCluster(TestMiniTezCluster.class.getName(), 1, 1, 1); + Configuration conf = new Configuration(); + conf.setFloat(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, 50); + tezMiniCluster.init(conf); + tezMiniCluster.start(); + + // respects provided non-default value + Assert.assertEquals(50.0, tezMiniCluster.getConfig() + .getFloat(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, -1), 0.00001); + + tezMiniCluster.close(); + } +} From 6a21d48daa2703b5f19dab49785332e0344e45a3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Thu, 13 Feb 2020 09:02:51 -0600 Subject: [PATCH 203/512] TEZ-4100. Upgrade to Hadoop 3.1.3 and Guava 27 Signed-off-by: Jonathan Eagles --- pom.xml | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 1a5148ac64..8b81f23003 100644 --- a/pom.xml +++ b/pom.xml @@ -37,7 +37,8 @@ true ${user.home}/clover.license - 3.0.3 + 27.0-jre + 3.1.3 9.3.28.v20191105 3.10.5.Final 0.13.0 @@ -717,7 +718,7 @@ com.google.guava guava - 11.0.2 + ${guava.version} org.codehaus.jettison From 9be1b1cbc6e859ae815e4f0629e9103d67a1b76b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Mon, 17 Feb 2020 09:01:57 +0100 Subject: [PATCH 204/512] =?UTF-8?q?TEZ-4109:=20Improve=20TezCommonUtils.ge?= =?UTF-8?q?tCredentialsInfo=20and=20use=20it=20from=20more=20contexts=20(L?= =?UTF-8?q?=C3=A1szl=C3=B3=20Bodor=20reviewed=20by=20Ashutosh=20Chauhan)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../java/org/apache/tez/client/TezClient.java | 3 +-- .../org/apache/tez/client/TezClientUtils.java | 3 ++- .../org/apache/tez/common/TezCommonUtils.java | 21 ++++++++++--------- .../org/apache/tez/dag/app/DAGAppMaster.java | 4 ++-- .../tez/service/impl/ContainerRunnerImpl.java | 2 ++ .../tez/runtime/task/TaskRunner2Callable.java | 2 ++ .../org/apache/tez/runtime/task/TezChild.java | 3 +++ 7 files changed, 23 insertions(+), 15 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java index a615179e85..60c0e5e16f 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java @@ -22,7 +22,6 @@ import java.net.InetAddress; import java.net.UnknownHostException; import java.text.NumberFormat; -import java.util.TimerTask; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicBoolean; @@ -1127,7 +1126,7 @@ DAGClient submitDAGApplication(ApplicationId appId, DAG dag) + ", applicationId=" + appId + ", dagName=" + dag.getName() + callerContextStr); - + TezCommonUtils.logCredentials(LOG, credentials, "appContext"); frameworkClient.submitApplication(appContext); ApplicationReport appReport = frameworkClient.getApplicationReport(appId); LOG.info("The url to track the Tez AM: " + appReport.getTrackingUrl()); diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index bcb09205a9..db1bc0c415 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -398,7 +398,6 @@ static Credentials setupDAGCredentials(DAG dag, Credentials sessionCredentials, Objects.requireNonNull(sessionCredentials); TezCommonUtils.logCredentials(LOG, sessionCredentials, "session"); - Credentials dagCredentials = new Credentials(); // All session creds are required for the DAG. dagCredentials.mergeAll(sessionCredentials); @@ -725,6 +724,7 @@ static Credentials prepareAmLaunchCredentials(AMConfiguration amConfig, Credenti if (amConfig.getCredentials() != null) { amLaunchCredentials.mergeAll(amConfig.getCredentials()); } + TezCommonUtils.logCredentials(LOG, amLaunchCredentials, "amLaunch"); return amLaunchCredentials; } @@ -750,6 +750,7 @@ static DAGPlan prepareAndCreateDAGPlan(DAG dag, AMConfiguration amConfig, JavaOptsChecker javaOptsChecker) throws IOException { Credentials dagCredentials = setupDAGCredentials(dag, credentials, amConfig.getTezConfiguration()); + TezCommonUtils.logCredentials(LOG, dagCredentials, "dagPlan"); return dag.createDag(amConfig.getTezConfiguration(), dagCredentials, tezJarResources, amConfig.getBinaryConfLR(), tezLrsAsArchive, servicePluginsDescriptor, javaOptsChecker); } diff --git a/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java index 16165e2377..27259580d4 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java @@ -22,10 +22,10 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.StringTokenizer; +import java.util.stream.Collectors; import java.util.zip.Deflater; import java.util.zip.DeflaterOutputStream; import java.util.zip.Inflater; @@ -405,20 +405,21 @@ public static byte[] decompressByteStringToByteArray(ByteString byteString, Infl } public static String getCredentialsInfo(Credentials credentials, String identifier) { + if (credentials == null) { + return "Credentials: #" + identifier + "Tokens=null"; + } + StringBuilder sb = new StringBuilder(); sb.append("Credentials: #" + identifier + "Tokens=").append(credentials.numberOfTokens()); if (credentials.numberOfTokens() > 0) { sb.append(", Services="); - Iterator> tokenItr = credentials.getAllTokens().iterator(); - if (tokenItr.hasNext()) { - Token token = tokenItr.next(); - sb.append(token.getService()).append("(").append(token.getKind()).append(")"); + sb.append(credentials.getAllTokens().stream() + .map(t -> String.format("%s(%s)", t.getService(), t.getKind())) + .collect(Collectors.joining(","))); - } - while(tokenItr.hasNext()) { - Token token = tokenItr.next(); - sb.append(",").append(token.getService()).append("(").append(token.getKind()).append(")"); - } + sb.append(", TokenDetails="); + sb.append(credentials.getAllTokens().stream().map(Token::toString) + .collect(Collectors.joining(","))); } return sb.toString(); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index 99b01d6eae..18cffadb5f 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -374,7 +374,7 @@ public DAGAppMaster(ApplicationAttemptId applicationAttemptId, LOG.info("Created DAGAppMaster for application " + applicationAttemptId + ", versionInfo=" + dagVersionInfo.toString()); - + TezCommonUtils.logCredentials(LOG, this.appMasterUgi.getCredentials(), "am"); } // Pull this WebAppUtils function into Tez until YARN-4186 @@ -1020,7 +1020,7 @@ DAGImpl createDAG(DAGPlan dagPB, TezDAGID dagId) { // TODO Does this move to the client in case of work-preserving recovery. TokenCache.setSessionToken(sessionToken, dagCredentials); - + TezCommonUtils.logCredentials(LOG, dagCredentials, "newDag"); // create single dag DAGImpl newDag = new DAGImpl(dagId, amConf, dagPB, dispatcher.getEventHandler(), diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java index e7777e2590..eef73a0d0a 100644 --- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java +++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java @@ -266,6 +266,7 @@ public void submitWork(SubmitWorkRequestProto request) throws TezException { // TODO Unregistering does not happen at the moment, since there's no signals on when an app completes. LOG.info("Registering request with the ShuffleHandler for containerId {}", request.getContainerIdString()); ShuffleHandler.get().registerApplication(request.getApplicationIdString(), jobToken, request.getUser()); + TezCommonUtils.logCredentials(LOG, credentials, "taskCallable"); TaskRunnerCallable callable = new TaskRunnerCallable(request, new Configuration(getConfig()), new ExecutionContextImpl(localAddress.get().getHostName()), env, localDirs, workingDir, credentials, memoryPerExecutor, sharedExecutor); @@ -457,6 +458,7 @@ public TezTaskUmbilicalProtocol run() throws Exception { new AtomicLong(0), request.getContainerIdString()); + TezCommonUtils.logCredentials(LOG, taskUgi.getCredentials(), "taskUgi"); taskRunner = new TezTaskRunner2(conf, taskUgi, localDirs, ProtoConverters.getTaskSpecfromProto(request.getTaskSpec()), request.getAppAttemptNumber(), diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java index b39af69c40..0e6dfda080 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java @@ -20,6 +20,7 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.tez.common.CallableWithNdc; +import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.common.counters.TezCounters; import org.apache.tez.runtime.LogicalIOProcessorRuntimeTask; @@ -66,6 +67,7 @@ public TaskRunner2CallableResult run() throws Exception { } LOG.info("Initializing task" + ", taskAttemptId={}", task.getTaskAttemptID()); TezUtilsInternal.setHadoopCallerContext(task.getHadoopShim(), task.getTaskAttemptID()); + TezCommonUtils.logCredentials(LOG, ugi.getCredentials(), "taskInit"); task.initialize(); if (!stopRequested.get() && !Thread.currentThread().isInterrupted()) { diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java index 06933170f4..6dd146a8fb 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java @@ -198,6 +198,7 @@ public TezTaskUmbilicalProtocol run() throws Exception { this.umbilical = umbilical; ownUmbilical = false; } + TezCommonUtils.logCredentials(LOG, credentials, "tezChildInit"); } public ContainerExecutionResult run() throws IOException, InterruptedException, TezException { @@ -237,6 +238,7 @@ public ContainerExecutionResult run() throws IOException, InterruptedException, shutdown(); } } + TezCommonUtils.logCredentials(LOG, containerTask.getCredentials(), "containerTask"); if (containerTask.shouldDie()) { LOG.info("ContainerTask returned shouldDie=true for container {}, Exiting", containerIdString); shutdown(); @@ -256,6 +258,7 @@ public ContainerExecutionResult run() throws IOException, InterruptedException, FileSystem.clearStatistics(); childUGI = handleNewTaskCredentials(containerTask, childUGI); + TezCommonUtils.logCredentials(LOG, childUGI.getCredentials(), "taskChildUGI"); handleNewTaskLocalResources(containerTask, childUGI); cleanupOnTaskChanged(containerTask); From 66a9dc8df3245350c09ee4b035b9398bdde3c853 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Mon, 24 Feb 2020 16:20:36 +0100 Subject: [PATCH 205/512] =?UTF-8?q?TEZ-4126:=20Shell=20scripts=20under=20t?= =?UTF-8?q?ez-tools=20should=20be=20runnable=20(L=C3=A1szl=C3=B3=20Bodor?= =?UTF-8?q?=20reviewed=20by=20Jonathan=20Turner=20Eagles)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- tez-tools/swimlanes/yarn-swimlanes.sh | 0 tez-tools/tez-log-split/README.md | 4 ++-- tez-tools/tez-log-split/tez-log-splitter.sh | 0 3 files changed, 2 insertions(+), 2 deletions(-) mode change 100644 => 100755 tez-tools/swimlanes/yarn-swimlanes.sh mode change 100644 => 100755 tez-tools/tez-log-split/tez-log-splitter.sh diff --git a/tez-tools/swimlanes/yarn-swimlanes.sh b/tez-tools/swimlanes/yarn-swimlanes.sh old mode 100644 new mode 100755 diff --git a/tez-tools/tez-log-split/README.md b/tez-tools/tez-log-split/README.md index a7341a757d..9ca48aa72e 100644 --- a/tez-tools/tez-log-split/README.md +++ b/tez-tools/tez-log-split/README.md @@ -72,6 +72,6 @@ an aggregated yarn log file to separate files into a hierarchical folder structu To use the tool, run e.g. -`tez-log-splitter.sh application_1576254620247_0010` (app log is fetched from yarn) -`tez-log-splitter.sh ~/path/to/application_1576254620247_0010.log` (...when app log is already on your computer) +`tez-log-splitter.sh application_1576254620247_0010` (app log is fetched from yarn) +`tez-log-splitter.sh ~/path/to/application_1576254620247_0010.log` (...when app log is already on your computer) `tez-log-splitter.sh ~/path/to/application_1576254620247_0010.log.gz` (...when app log is already on your computer in gz) diff --git a/tez-tools/tez-log-split/tez-log-splitter.sh b/tez-tools/tez-log-split/tez-log-splitter.sh old mode 100644 new mode 100755 From 4c5db43044be74052e113314a6dc4c3ab9b1c72d Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Mon, 24 Feb 2020 16:25:00 -0600 Subject: [PATCH 206/512] TEZ-3664. Flaky tests due to writing to /tmp directory --- pom.xml | 2 ++ .../org/apache/tez/client/TestTezClient.java | 4 ++++ .../apache/tez/client/TestTezClientUtils.java | 6 ++++++ .../apache/tez/common/TestTezCommonUtils.java | 8 ++++++-- .../apache/tez/dag/app/TestDAGAppMaster.java | 17 ++++++++--------- .../tez/tests/TestExtServicesWithLocalMode.java | 4 ++++ .../mapreduce/output/TestMROutputLegacy.java | 11 +++++++---- .../tez/mapreduce/output/TestMultiMROutput.java | 5 ++++- .../tez/auxservices/TestShuffleHandler.java | 15 +++++++++++++++ .../dag/history/ats/acls/TestATSHistoryV15.java | 1 + .../impl/TestSimpleFetchedInputAllocator.java | 5 +++-- .../java/org/apache/tez/test/TestLocalMode.java | 10 +++++----- .../tez/test/TestTaskErrorsUsingLocalMode.java | 4 ++++ 13 files changed, 69 insertions(+), 23 deletions(-) diff --git a/pom.xml b/pom.xml index 8b81f23003..4375b61545 100644 --- a/pom.xml +++ b/pom.xml @@ -65,6 +65,7 @@ 3.0.0 8.16 1.3.6 + ${project.build.directory}/tmp ${scm.url} @@ -909,6 +910,7 @@ 4 + ${test.build.data} true ${hadoop.version} diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java index 0c297d3fae..91dce5ed5f 100644 --- a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java +++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java @@ -18,6 +18,7 @@ package org.apache.tez.client; +import java.io.File; import java.io.IOException; import java.net.InetAddress; import java.nio.ByteBuffer; @@ -108,6 +109,8 @@ public class TestTezClient { static final long HARD_KILL_TIMEOUT = 1500L; + private static final File STAGING_DIR = new File(System.getProperty("test.build.data"), + TestTezClient.class.getName()).getAbsoluteFile(); class TezClientForTest extends TezClient { TezYarnClient mockTezYarnClient; @@ -163,6 +166,7 @@ TezClientForTest configureAndCreateTezClient(Map lrs, boo } conf.setBoolean(TezConfiguration.TEZ_IGNORE_LIB_URIS, true); conf.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, isSession); + conf.set(TezConfiguration.TEZ_AM_STAGING_DIR, STAGING_DIR.getAbsolutePath()); TezClientForTest client = new TezClientForTest("test", conf, lrs, null); ApplicationId appId1 = ApplicationId.newInstance(0, 1); diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java index edcec49ff2..adcc65c3bf 100644 --- a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java +++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java @@ -83,6 +83,8 @@ public class TestTezClientUtils { private static String TEST_ROOT_DIR = "target" + Path.SEPARATOR + TestTezClientUtils.class.getName() + "-tmpDir"; + private static final File STAGING_DIR = new File(System.getProperty("test.build.data"), + TestTezClientUtils.class.getName()).getAbsoluteFile(); /** * */ @@ -330,6 +332,7 @@ public void validateSetTezJarLocalResourcesMixTarballAndJar() throws Exception { // ApplicationSubmissionContext public void testAppSubmissionContextForPriority() throws Exception { TezConfiguration tezConf = new TezConfiguration(); + tezConf.set(TezConfiguration.TEZ_AM_STAGING_DIR, STAGING_DIR.getAbsolutePath()); int testpriority = 999; ApplicationId appId = ApplicationId.newInstance(1000, 1); Credentials credentials = new Credentials(); @@ -380,6 +383,7 @@ public void testSetApplicationTags() { public void testSessionTokenInAmClc() throws IOException, YarnException { TezConfiguration tezConf = new TezConfiguration(); + tezConf.set(TezConfiguration.TEZ_AM_STAGING_DIR, STAGING_DIR.getAbsolutePath()); ApplicationId appId = ApplicationId.newInstance(1000, 1); DAG dag = DAG.create("testdag"); @@ -417,6 +421,7 @@ public void testAMLoggingOptsSimple() throws IOException, YarnException { TezConfiguration tezConf = new TezConfiguration(); tezConf.set(TezConfiguration.TEZ_AM_LOG_LEVEL, "WARN"); + tezConf.set(TezConfiguration.TEZ_AM_STAGING_DIR, STAGING_DIR.getAbsolutePath()); ApplicationId appId = ApplicationId.newInstance(1000, 1); Credentials credentials = new Credentials(); @@ -457,6 +462,7 @@ public void testAMLoggingOptsPerLogger() throws IOException, YarnException { TezConfiguration tezConf = new TezConfiguration(); tezConf.set(TezConfiguration.TEZ_AM_LOG_LEVEL, "WARN;org.apache.hadoop.ipc=DEBUG;org.apache.hadoop.security=DEBUG"); + tezConf.set(TezConfiguration.TEZ_AM_STAGING_DIR, STAGING_DIR.getAbsolutePath()); ApplicationId appId = ApplicationId.newInstance(1000, 1); Credentials credentials = new Credentials(); diff --git a/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java b/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java index 3929c4bf5a..d7bd397386 100644 --- a/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java +++ b/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java @@ -18,6 +18,7 @@ package org.apache.tez.common; +import java.io.File; import java.io.IOException; import java.util.Map; @@ -43,6 +44,9 @@ public class TestTezCommonUtils { private static final String STAGE_DIR = "/tmp/mystage"; + + private static final File LOCAL_STAGING_DIR = new File(System.getProperty("test.build.data"), + TestTezCommonUtils.class.getSimpleName()).getAbsoluteFile(); private static String RESOLVED_STAGE_DIR; private static Configuration conf = new Configuration();; private static String TEST_ROOT_DIR = "target" + Path.SEPARATOR @@ -84,10 +88,10 @@ public static void afterClass() throws InterruptedException { public void testTezBaseStagingPath() throws Exception { Configuration localConf = new Configuration(); // Check if default works with localFS - localConf.unset(TezConfiguration.TEZ_AM_STAGING_DIR); + localConf.set(TezConfiguration.TEZ_AM_STAGING_DIR, LOCAL_STAGING_DIR.getAbsolutePath()); localConf.set("fs.defaultFS", "file:///"); Path stageDir = TezCommonUtils.getTezBaseStagingPath(localConf); - Assert.assertEquals(stageDir.toString(), "file:" + TezConfiguration.TEZ_AM_STAGING_DIR_DEFAULT); + Assert.assertEquals("file:" + LOCAL_STAGING_DIR, stageDir.toString()); // check if user set something, indeed works conf.set(TezConfiguration.TEZ_AM_STAGING_DIR, STAGE_DIR); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java index 601aca7b7d..4adf310e82 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java @@ -94,15 +94,13 @@ public class TestDAGAppMaster { private static final String CL_NAME = "CL"; private static final String TC_NAME = "TC"; private static final String CLASS_SUFFIX = "_CLASS"; - private static final File TEST_DIR = new File( - System.getProperty("test.build.data", - System.getProperty("java.io.tmpdir")), - TestDAGAppMaster.class.getSimpleName()).getAbsoluteFile(); + private static final File TEST_DIR = new File(System.getProperty("test.build.data"), + TestDAGAppMaster.class.getName()).getAbsoluteFile(); @Before public void setup() { FileUtil.fullyDelete(TEST_DIR); - TEST_DIR.mkdir(); + TEST_DIR.mkdirs(); } @After @@ -649,9 +647,10 @@ private static Credentials createCredentials() { return creds; } - private static void stubSessionResources() throws IOException { - FileOutputStream out = new FileOutputStream( - new File(TEST_DIR, TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME)); + private static void stubSessionResources(Configuration conf) throws IOException { + File file = new File(TEST_DIR, TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME); + conf.set(TezConfiguration.TEZ_AM_STAGING_DIR, TEST_DIR.getAbsolutePath()); + FileOutputStream out = new FileOutputStream(file); PlanLocalResourcesProto planProto = PlanLocalResourcesProto.getDefaultInstance(); planProto.writeDelimitedTo(out); out.close(); @@ -659,7 +658,7 @@ private static void stubSessionResources() throws IOException { @Override public synchronized void serviceInit(Configuration conf) throws Exception { - stubSessionResources(); + stubSessionResources(conf); conf.setBoolean(TezConfiguration.TEZ_AM_WEBSERVICE_ENABLE, false); super.serviceInit(conf); } diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExtServicesWithLocalMode.java b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExtServicesWithLocalMode.java index 3d8c08781b..c0bfe76bd4 100644 --- a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExtServicesWithLocalMode.java +++ b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExtServicesWithLocalMode.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertEquals; +import java.io.File; import java.io.IOException; import java.util.Map; @@ -59,6 +60,8 @@ public class TestExtServicesWithLocalMode { private static String TEST_ROOT_DIR = "target" + Path.SEPARATOR + TestExtServicesWithLocalMode.class.getName() + "-tmpDir"; + private static final String STAGING_DIR = new File(System.getProperty("test.build.data"), + TestExtServicesWithLocalMode.class.getName()).getAbsolutePath(); private static final Path SRC_DATA_DIR = new Path(TEST_ROOT_DIR + Path.SEPARATOR + "data"); private static final Path HASH_JOIN_EXPECTED_RESULT_PATH = @@ -92,6 +95,7 @@ public static void setup() throws Exception { confForJobs.set(entry.getKey(), entry.getValue()); } confForJobs.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); + confForJobs.set(TezConfiguration.TEZ_AM_STAGING_DIR, STAGING_DIR.toString()); } @AfterClass diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/output/TestMROutputLegacy.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/output/TestMROutputLegacy.java index e4fa0ea9dc..01b5c84e70 100644 --- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/output/TestMROutputLegacy.java +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/output/TestMROutputLegacy.java @@ -42,13 +42,16 @@ import org.apache.tez.runtime.api.OutputContext; import org.junit.Test; +import java.io.File; public class TestMROutputLegacy { + private static final File TEST_DIR = new File(System.getProperty("test.build.data"), + TestMROutputLegacy.class.getName()).getAbsoluteFile(); // simulate the behavior of translating MR to DAG using MR old API @Test (timeout = 5000) public void testOldAPI_MR() throws Exception { - String outputPath = "/tmp/output"; + String outputPath = TEST_DIR.getAbsolutePath(); JobConf conf = new JobConf(); conf.setOutputKeyClass(NullWritable.class); conf.setOutputValueClass(Text.class); @@ -79,7 +82,7 @@ public void testOldAPI_MR() throws Exception { // simulate the behavior of translating MR to DAG using MR new API @Test (timeout = 5000) public void testNewAPI_MR() throws Exception { - String outputPath = "/tmp/output"; + String outputPath = TEST_DIR.getAbsolutePath(); Job job = Job.getInstance(); job.setOutputKeyClass(NullWritable.class); job.setOutputValueClass(Text.class); @@ -111,7 +114,7 @@ public void testNewAPI_MR() throws Exception { // simulate the behavior of translating Mapper-only job to DAG using MR old API @Test (timeout = 5000) public void testOldAPI_MapperOnly() throws Exception { - String outputPath = "/tmp/output"; + String outputPath = TEST_DIR.getAbsolutePath(); JobConf conf = new JobConf(); conf.setOutputKeyClass(NullWritable.class); conf.setOutputValueClass(Text.class); @@ -142,7 +145,7 @@ public void testOldAPI_MapperOnly() throws Exception { //simulate the behavior of translating mapper-only job to DAG using MR new API @Test (timeout = 5000) public void testNewAPI_MapperOnly() throws Exception { - String outputPath = "/tmp/output"; + String outputPath = TEST_DIR.getAbsolutePath(); Job job = Job.getInstance(); job.setOutputKeyClass(NullWritable.class); job.setOutputValueClass(Text.class); diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/output/TestMultiMROutput.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/output/TestMultiMROutput.java index 3618e408c0..c8eca16027 100644 --- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/output/TestMultiMROutput.java +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/output/TestMultiMROutput.java @@ -39,10 +39,13 @@ import org.junit.Assert; import org.junit.Test; +import java.io.File; import java.io.IOException; public class TestMultiMROutput { + private static final File TEST_DIR = new File(System.getProperty("test.build.data"), + TestMultiMROutput.class.getName()).getAbsoluteFile(); @Test(timeout = 5000) public void testNewAPI_TextOutputFormat() throws Exception { @@ -176,7 +179,7 @@ private void validate(boolean expectedUseNewAPIValue, Class outputFormat, private MultiMROutput createMROutputs(Class outputFormat, boolean isMapper, boolean useLazyOutputFormat) throws InterruptedException, IOException { - String outputPath = "/tmp/output"; + String outputPath = TEST_DIR.getAbsolutePath(); JobConf conf = new JobConf(); conf.setBoolean(MRConfig.IS_MAP_PROCESSOR, isMapper); conf.setOutputKeyClass(Text.class); diff --git a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java index 7c421a9c17..0d542cfee1 100644 --- a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java @@ -103,6 +103,9 @@ public class TestShuffleHandler { static final long MiB = 1024 * 1024; private static final Logger LOG = LoggerFactory.getLogger(TestShuffleHandler.class); + private static final File TEST_DIR = new File(System.getProperty("test.build.data"), + TestShuffleHandler.class.getName()).getAbsoluteFile(); + private static final String HADOOP_TMP_DIR = "hadoop.tmp.dir"; class MockShuffleHandler extends org.apache.tez.auxservices.ShuffleHandler { @Override protected Shuffle getShuffle(final Configuration conf) { @@ -231,6 +234,7 @@ static void checkShuffleMetrics(MetricsSystem ms, long bytes, int failed, public void testClientClosesConnection() throws Exception { final ArrayList failures = new ArrayList(1); Configuration conf = new Configuration(); + conf.set(HADOOP_TMP_DIR, TEST_DIR.getAbsolutePath()); conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); ShuffleHandler shuffleHandler = new ShuffleHandler() { @Override @@ -336,6 +340,7 @@ SocketAddress getSocketAddres() { public void testKeepAlive() throws Exception { final ArrayList failures = new ArrayList(1); Configuration conf = new Configuration(); + conf.set(HADOOP_TMP_DIR, TEST_DIR.getAbsolutePath()); conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); conf.setBoolean(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, true); // try setting to -ve keep alive timeout. @@ -485,6 +490,7 @@ protected void sendError(ChannelHandlerContext ctx, String message, @Test public void testSocketKeepAlive() throws Exception { Configuration conf = new Configuration(); + conf.set(HADOOP_TMP_DIR, TEST_DIR.getAbsolutePath()); conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); conf.setBoolean(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, true); // try setting to -ve keep alive timeout. @@ -528,6 +534,7 @@ public void testSocketKeepAlive() throws Exception { public void testIncompatibleShuffleVersion() throws Exception { final int failureNum = 3; Configuration conf = new Configuration(); + conf.set(HADOOP_TMP_DIR, TEST_DIR.getAbsolutePath()); conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); ShuffleHandler shuffleHandler = new ShuffleHandler(); shuffleHandler.init(conf); @@ -562,6 +569,7 @@ public void testIncompatibleShuffleVersion() throws Exception { public void testMaxConnections() throws Exception { Configuration conf = new Configuration(); + conf.set(HADOOP_TMP_DIR, TEST_DIR.getAbsolutePath()); conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, 3); ShuffleHandler shuffleHandler = new ShuffleHandler() { @@ -666,6 +674,7 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, @Test(timeout = 10000) public void testRangedFetch() throws IOException { Configuration conf = new Configuration(); + conf.set(HADOOP_TMP_DIR, TEST_DIR.getAbsolutePath()); conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, 3); conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, @@ -767,6 +776,7 @@ public void testMapFileAccess() throws IOException { // This will run only in NativeIO is enabled as SecureIOUtils need it assumeTrue(NativeIO.isAvailable()); Configuration conf = new Configuration(); + conf.set(HADOOP_TMP_DIR, TEST_DIR.getAbsolutePath()); conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, 3); conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, @@ -900,6 +910,7 @@ public void testRecovery() throws IOException { System.getProperty("java.io.tmpdir")), TestShuffleHandler.class.getName()); Configuration conf = new Configuration(); + conf.set(HADOOP_TMP_DIR, TEST_DIR.getAbsolutePath()); conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, 3); ShuffleHandler shuffle = new ShuffleHandler(); @@ -967,6 +978,7 @@ public void testRecoveryFromOtherVersions() throws IOException { System.getProperty("java.io.tmpdir")), TestShuffleHandler.class.getName()); Configuration conf = new Configuration(); + conf.set(HADOOP_TMP_DIR, TEST_DIR.getAbsolutePath()); conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, 3); ShuffleHandler shuffle = new ShuffleHandler(); @@ -1073,6 +1085,7 @@ private static int getShuffleResponseCode(ShuffleHandler shuffle, public void testGetMapOutputInfo() throws Exception { final ArrayList failures = new ArrayList(1); Configuration conf = new Configuration(); + conf.set(HADOOP_TMP_DIR, TEST_DIR.getAbsolutePath()); conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, 3); conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, @@ -1177,6 +1190,7 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, public void testDagDelete() throws Exception { final ArrayList failures = new ArrayList(1); Configuration conf = new Configuration(); + conf.set(HADOOP_TMP_DIR, TEST_DIR.getAbsolutePath()); conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, 3); conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, @@ -1287,6 +1301,7 @@ public void testSendMapCount() throws Exception { final ShuffleHandler sh = new MockShuffleHandler(); Configuration conf = new Configuration(); + conf.set(HADOOP_TMP_DIR, TEST_DIR.getAbsolutePath()); // The Shuffle handler port associated with the service is bound to but not used. conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); sh.init(conf); diff --git a/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryV15.java b/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryV15.java index 54abd4465b..f49e588afe 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryV15.java +++ b/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryV15.java @@ -98,6 +98,7 @@ public static void setup() throws IOException { 1, 1, 1, true); Configuration conf = new Configuration(); conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); + conf.set(YarnConfiguration.TIMELINE_SERVICE_LEVELDB_PATH, TEST_ROOT_DIR); conf.set("fs.defaultFS", remoteFs.getUri().toString()); // use HDFS conf.setInt("yarn.nodemanager.delete.debug-delay-sec", 20000); atsActivePath = new Path("/tmp/ats/active/" + random.nextInt(100000)); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestSimpleFetchedInputAllocator.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestSimpleFetchedInputAllocator.java index 1b63b17149..01faa5df7a 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestSimpleFetchedInputAllocator.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestSimpleFetchedInputAllocator.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertEquals; +import java.io.File; import java.io.IOException; import java.util.UUID; @@ -38,7 +39,7 @@ public class TestSimpleFetchedInputAllocator { @Test(timeout = 5000) public void testInMemAllocation() throws IOException { - String localDirs = "/tmp/" + this.getClass().getName(); + File localDirs = new File(System.getProperty("test.build.data", "/tmp"), this.getClass().getName()); Configuration conf = new Configuration(); long jvmMax = Runtime.getRuntime().maxMemory(); @@ -47,7 +48,7 @@ public void testInMemAllocation() throws IOException { float bufferPercent = 0.1f; conf.setFloat(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_BUFFER_PERCENT, bufferPercent); conf.setFloat(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT, 1.0f); - conf.setStrings(TezRuntimeFrameworkConfigs.LOCAL_DIRS, localDirs); + conf.setStrings(TezRuntimeFrameworkConfigs.LOCAL_DIRS, localDirs.getAbsolutePath()); long inMemThreshold = (long) (bufferPercent * jvmMax); LOG.info("InMemThreshold: " + inMemThreshold); diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java b/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java index ffc67fe51e..e6ef8c9cb7 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java @@ -61,9 +61,8 @@ @RunWith(Parameterized.class) public class TestLocalMode { - private static final File TEST_DIR = new File( - System.getProperty("test.build.data", - System.getProperty("java.io.tmpdir")), "TestLocalMode-tez-localmode"); + private static final File STAGING_DIR = new File(System.getProperty("test.build.data"), + TestLocalMode.class.getName()); private static MiniDFSCluster dfsCluster; private static FileSystem remoteFs; @@ -111,6 +110,7 @@ private TezConfiguration createConf() { } else { conf.set("fs.defaultFS", "file:///"); } + conf.set(TezConfiguration.TEZ_AM_STAGING_DIR, STAGING_DIR.getAbsolutePath()); conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); return conf; } @@ -261,9 +261,9 @@ public void testMultiDAGsOnSession() throws IOException, TezException, Interrupt //create inputs and outputs FileSystem fs = FileSystem.get(tezConf); for(int i = 0; i < dags; i++) { - inputPaths[i] = new Path(TEST_DIR.getAbsolutePath(),"in-"+i).toString(); + inputPaths[i] = new Path(STAGING_DIR.getAbsolutePath(), "in-" + i).toString(); createInputFile(fs, inputPaths[i]); - outputPaths[i] = new Path(TEST_DIR.getAbsolutePath(),"out-"+i).toString(); + outputPaths[i] = new Path(STAGING_DIR.getAbsolutePath(), "out-" + i).toString(); } //start testing diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestTaskErrorsUsingLocalMode.java b/tez-tests/src/test/java/org/apache/tez/test/TestTaskErrorsUsingLocalMode.java index d622698f29..d7d2dd925f 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestTaskErrorsUsingLocalMode.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestTaskErrorsUsingLocalMode.java @@ -16,6 +16,7 @@ import static org.junit.Assert.assertEquals; +import java.io.File; import java.io.IOException; import java.util.List; import java.util.Map; @@ -44,6 +45,8 @@ public class TestTaskErrorsUsingLocalMode { private static final Logger LOG = LoggerFactory.getLogger(TestTaskErrorsUsingLocalMode.class); private static final String VERTEX_NAME = "vertex1"; + private static final File STAGING_DIR = new File(System.getProperty("test.build.data"), + TestTaskErrorsUsingLocalMode.class.getName()).getAbsoluteFile(); @Test(timeout = 20000) @@ -123,6 +126,7 @@ private TezClient getTezClient(String name) throws IOException, TezException { TezConfiguration tezConf1 = new TezConfiguration(); tezConf1.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); tezConf1.set("fs.defaultFS", "file:///"); + tezConf1.set(TezConfiguration.TEZ_AM_STAGING_DIR, STAGING_DIR.getAbsolutePath()); tezConf1.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); tezConf1.setLong(TezConfiguration.TEZ_AM_SLEEP_TIME_BEFORE_EXIT_MILLIS, 500); TezClient tezClient1 = TezClient.create(name, tezConf1, true); From 7e65a9eafd775574db740075968bdce82ea0b7ca Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Tue, 25 Feb 2020 11:21:21 +0100 Subject: [PATCH 207/512] =?UTF-8?q?TEZ-3727:=20When=20using=20HDFS=20feder?= =?UTF-8?q?ation,=20token=20of=20tez.simple.history.logging.dir=20is=20not?= =?UTF-8?q?=20added,=20causing=20AM=20to=20fail=20(contributed=20by=20Xi?= =?UTF-8?q?=20Chen,=20reviewed=20by=20L=C3=A1szl=C3=B3=20Bodor,=20Jonathan?= =?UTF-8?q?=20Turner=20Eagles)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../main/java/org/apache/tez/client/TezClientUtils.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index db1bc0c415..8850ca74ca 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -713,6 +713,15 @@ static Credentials prepareAmLaunchCredentials(AMConfiguration amConfig, Credenti // Setup security tokens Credentials amLaunchCredentials = new Credentials(); + // Add SimpleHistoryLoggingService logDir creds to the list of session credentials + // If it is on HDFS + String simpleHistoryLogDir = conf.get(TezConfiguration.TEZ_SIMPLE_HISTORY_LOGGING_DIR); + if (simpleHistoryLogDir != null && !simpleHistoryLogDir.isEmpty()) { + Path simpleHistoryLogDirPath = new Path(simpleHistoryLogDir); + TokenCache.obtainTokensForFileSystems(sessionCreds, new Path[] { simpleHistoryLogDirPath }, + conf); + } + // Add Staging dir creds to the list of session credentials. TokenCache.obtainTokensForFileSystems(sessionCreds, new Path[] {binaryConfPath }, conf); From 5755f70f6d1dc0de769e5de9adf41cc7afa1ce73 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Tue, 25 Feb 2020 11:28:24 +0100 Subject: [PATCH 208/512] =?UTF-8?q?TEZ-4099:=20Add=20details=20on=20whethe?= =?UTF-8?q?r=20SSL=20is=20enabled=20or=20not=20in=20HttpConnectionParams?= =?UTF-8?q?=20(L=C3=A1szl=C3=B3=20Bodor=20reviewed=20by=20Jonathan=20Turne?= =?UTF-8?q?r=20Eagles)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../src/main/java/org/apache/tez/http/HttpConnectionParams.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnectionParams.java b/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnectionParams.java index aac4bb3861..9f1c002a20 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnectionParams.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnectionParams.java @@ -76,7 +76,7 @@ public String toString() { sb.append("connectionTimeout=").append(connectionTimeout).append(", "); sb.append("readTimeout=").append(readTimeout).append(", "); sb.append("bufferSize=").append(bufferSize).append(", "); - sb.append("bufferSize=").append(bufferSize); + sb.append("sslShuffle=").append(sslShuffle); return sb.toString(); } } From f12fd5caf965680aa0c56adbe963dad3dca8bd3f Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Thu, 27 Feb 2020 09:04:15 -0600 Subject: [PATCH 209/512] TEZ-4127. TestMergeManager#testOnDiskMergerFilenames fails with dot directory name --- .../common/shuffle/orderedgrouped/TestMergeManager.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java index 5737578c70..92d9125dab 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java @@ -757,9 +757,9 @@ public void testOnDiskMergerFilenames() throws IOException, InterruptedException assertEquals(m2Path.toString().length(), m3Path.toString().length()); // Ensure the filenames are used correctly - based on the first file given to the merger. - String m1Prefix = m1Path.toString().substring(0, m1Path.toString().indexOf(".")); - String m2Prefix = m2Path.toString().substring(0, m2Path.toString().indexOf(".")); - String m3Prefix = m3Path.toString().substring(0, m3Path.toString().indexOf(".")); + String m1Prefix = m1Path.toString().substring(0, m1Path.toString().lastIndexOf('.')); + String m2Prefix = m2Path.toString().substring(0, m2Path.toString().lastIndexOf('.')); + String m3Prefix = m3Path.toString().substring(0, m3Path.toString().lastIndexOf('.')); assertEquals(m1Prefix, m2Prefix); assertNotEquals(m1Prefix, m3Prefix); From ba441c1ca9dac51e20ba8561ed3c4b7b620d9505 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Sun, 1 Mar 2020 16:27:04 +0100 Subject: [PATCH 210/512] =?UTF-8?q?TEZ-4097:=20Report=20localHostname=20in?= =?UTF-8?q?=20Fetcher=20and=20FetcherOrderedGrouped=20failure=20log=20mess?= =?UTF-8?q?ages=20(L=C3=A1szl=C3=B3=20Bodor=20reviewed=20by=20Ashutosh=20C?= =?UTF-8?q?hauhan)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../tez/runtime/library/common/shuffle/Fetcher.java | 11 ++++++----- .../orderedgrouped/FetcherOrderedGrouped.java | 12 ++++++------ 2 files changed, 12 insertions(+), 11 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java index fa883e4d64..5da12de0ab 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java @@ -542,8 +542,9 @@ private HostFetchResult setupConnection(Collection attem } } else { InputAttemptIdentifier firstAttempt = attempts.iterator().next(); - LOG.warn("Fetch Failure from host while connecting: " + host + ", attempt: " + firstAttempt - + " Informing ShuffleManager: ", e); + LOG.warn(String.format( + "Fetch Failure while connecting from %s to: %s:%d, attempt: %s Informing ShuffleManager: ", + localHostname, host, port, firstAttempt), e); return new HostFetchResult(new FetchResult(host, port, partition, partitionCount, srcAttemptsRemaining.values()), new InputAttemptIdentifier[] { firstAttempt }, false); } @@ -1008,8 +1009,8 @@ private InputAttemptIdentifier[] fetchInputs(DataInputStream input, return new InputAttemptIdentifier[] { srcAttemptId }; } } - LOG.warn("Failed to shuffle output of " + srcAttemptId + " from " + host, - ioe); + LOG.warn("Failed to shuffle output of " + srcAttemptId + " from " + host + " (to " + + localHostname + ")", ioe); // Cleanup the fetchedInput cleanupFetchedInput(fetchedInput); @@ -1049,7 +1050,7 @@ private boolean shouldRetry(InputAttemptIdentifier srcAttemptId, Throwable ioe) if (currentTime - retryStartTime < httpConnectionParams.getReadTimeout()) { LOG.warn("Shuffle output from " + srcAttemptId + - " failed, retry it."); + " failed (to "+ localHostname +"), retry it."); //retry connecting to the host return true; } else { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java index af9b929643..cb8e751620 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java @@ -67,7 +67,6 @@ class FetcherOrderedGrouped extends CallableWithNdc { private final TezCounter ioErrs; private final TezCounter wrongLengthErrs; private final TezCounter badIdErrs; - private final TezCounter wrongMapErrs; private final TezCounter wrongReduceErrs; private final FetchedInputAllocatorOrderedGrouped allocator; private final ShuffleScheduler scheduler; @@ -77,7 +76,7 @@ class FetcherOrderedGrouped extends CallableWithNdc { private final String localShuffleHost; private final int localShufflePort; private final String applicationId; - private final int dagId; + private final int dagId; private final MapHost mapHost; private final int minPartition; @@ -143,7 +142,6 @@ public FetcherOrderedGrouped(HttpConnectionParams httpConnectionParams, this.ioErrs = ioErrsCounter; this.wrongLengthErrs = wrongLengthErrsCounter; this.badIdErrs = badIdErrsCounter; - this.wrongMapErrs = wrongMapErrsCounter; this.connectionErrs = connectionErrsCounter; this.wrongReduceErrs = wrongReduceErrsCounter; this.applicationId = applicationId; @@ -363,11 +361,13 @@ boolean setupConnection(MapHost host, Collection attempt } ioErrs.increment(1); if (!connectSucceeded) { - LOG.warn("Failed to connect to " + host + " with " + remaining.size() + " inputs", ie); + LOG.warn(String.format("Failed to connect from %s to %s with %d inputs", localShuffleHost, + host, remaining.size()), ie); connectionErrs.increment(1); } else { - LOG.warn("Failed to verify reply after connecting to " + host + " with " + remaining.size() - + " inputs pending", ie); + LOG.warn(String.format( + "Failed to verify reply after connecting from %s to %s with %d inputs pending", + localShuffleHost, host, remaining.size()), ie); } // At this point, either the connection failed, or the initial header verification failed. From 240f80bbe1024c911b179bc25b9a99849dee74d0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Sat, 7 Mar 2020 12:27:43 +0100 Subject: [PATCH 211/512] =?UTF-8?q?TEZ-4114:=20Remove=20direct=20jetty=20d?= =?UTF-8?q?ependency=20from=20tez=20(L=C3=A1szl=C3=B3=20Bodor=20reviewed?= =?UTF-8?q?=20by=20Ashutosh=20Chauhan,=20Jonathan=20Turner=20Eagles)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- pom.xml | 66 ++++++++++++------------------------------------- tez-dag/pom.xml | 16 +++++------- 2 files changed, 22 insertions(+), 60 deletions(-) diff --git a/pom.xml b/pom.xml index 4375b61545..d34c6dcda3 100644 --- a/pom.xml +++ b/pom.xml @@ -39,7 +39,6 @@ ${user.home}/clover.license 27.0-jre 3.1.3 - 9.3.28.v20191105 3.10.5.Final 0.13.0 1.19 @@ -248,24 +247,12 @@ commons-lang 2.6 - - org.eclipse.jetty - jetty-server - compile - ${jetty.version} - io.netty netty compile ${netty.version} - - org.eclipse.jetty - jetty-util - compile - ${jetty.version} - javax.servlet javax.servlet-api @@ -296,27 +283,7 @@ org.eclipse.jetty - jetty-server - - - org.eclipse.jetty - jetty-util - - - org.eclipse.jetty - servlet-api-2.5 - - - org.mortbay.jetty - jetty - - - org.mortbay.jetty - jetty-util - - - org.mortbay.jetty - servlet-api-2.5 + * com.sun.jersey @@ -388,18 +355,6 @@ commons-logging commons-logging-api - - org.mortbay.jetty - jetty - - - org.mortbay.jetty - jetty-util - - - org.mortbay.jetty - servlet-api-2.5 - com.sun.jersey jersey-core @@ -494,7 +449,7 @@ ${hadoop.version} - org.mortbay.jetty + org.eclipse.jetty jetty-util @@ -512,6 +467,10 @@ com.sun.jersey jersey-server + + org.eclipse.jetty + * + @@ -525,6 +484,12 @@ org.apache.hadoop hadoop-yarn-server-web-proxy ${hadoop.version} + + + org.eclipse.jetty + * + + org.apache.hadoop @@ -654,8 +619,8 @@ avro - org.mortbay.jetty - jetty + org.eclipse.jetty + * com.sun.jersey @@ -682,8 +647,9 @@ org.apache.hadoop hadoop-hdfs - test-jar ${hadoop.version} + test-jar + test org.mockito diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml index 01f5623e4e..89033859c6 100644 --- a/tez-dag/pom.xml +++ b/tez-dag/pom.xml @@ -142,21 +142,17 @@ org.codehaus.jettison jettison - - org.eclipse.jetty - jetty-server - compile - - - org.eclipse.jetty - jetty-util - compile - javax.servlet javax.servlet-api compile + + org.apache.hadoop + hadoop-common + test-jar + test + From 9df9f100e5f4e17f4fa67b69623a7bc8f6fd454c Mon Sep 17 00:00:00 2001 From: Zhang Butao Date: Mon, 16 Mar 2020 16:27:22 +0000 Subject: [PATCH 212/512] Reduce apache commons collections direct dependencies Signed-off-by: Jonathan Eagles --- pom.xml | 5 ----- tez-dist/dist-files/minimal/LICENSE | 1 - tez-mapreduce/pom.xml | 4 ---- .../apache/tez/mapreduce/hadoop/mapred/MRCounters.java | 2 +- .../apache/tez/history/parser/datamodel/DagInfo.java | 10 +++++----- 5 files changed, 6 insertions(+), 16 deletions(-) diff --git a/pom.xml b/pom.xml index d34c6dcda3..0cbdfd6380 100644 --- a/pom.xml +++ b/pom.xml @@ -263,11 +263,6 @@ commons-codec 1.11 - - commons-collections - commons-collections - 3.2.2 - org.apache.hadoop hadoop-common diff --git a/tez-dist/dist-files/minimal/LICENSE b/tez-dist/dist-files/minimal/LICENSE index 11d7d0c210..7083a9f8c9 100644 --- a/tez-dist/dist-files/minimal/LICENSE +++ b/tez-dist/dist-files/minimal/LICENSE @@ -234,7 +234,6 @@ license: - hadoop-shim-*.jar - async-http-client-*.jar - commons-cli-*.jar - - commons-collections-*.jar - commons-io-*.jar - commons-math3-*.jar - commons-codec-*.jar diff --git a/tez-mapreduce/pom.xml b/tez-mapreduce/pom.xml index a263d83b54..b1822cd425 100644 --- a/tez-mapreduce/pom.xml +++ b/tez-mapreduce/pom.xml @@ -84,10 +84,6 @@ commons-lang commons-lang - - commons-collections - commons-collections - org.apache.tez tez-runtime-internals diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRCounters.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRCounters.java index cc9b6baf92..b6b8e150ea 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRCounters.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRCounters.java @@ -26,7 +26,7 @@ import java.util.Collection; import java.util.Iterator; -import org.apache.commons.collections.IteratorUtils; +import org.apache.commons.collections4.IteratorUtils; public class MRCounters extends org.apache.hadoop.mapred.Counters { private final org.apache.tez.common.counters.TezCounters raw; diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/DagInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/DagInfo.java index 544e86a70d..64ddcf862e 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/DagInfo.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/DagInfo.java @@ -28,8 +28,8 @@ import com.google.common.collect.Multimap; import com.google.common.collect.Multimaps; import com.google.common.collect.Ordering; -import org.apache.commons.collections.BidiMap; -import org.apache.commons.collections.bidimap.DualHashBidiMap; +import org.apache.commons.collections4.BidiMap; +import org.apache.commons.collections4.bidimap.DualHashBidiMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.util.StringInterner; @@ -71,7 +71,7 @@ public class DagInfo extends BaseInfo { private CallerContext callerContext; //VertexID --> VertexName & vice versa - private final BidiMap vertexNameIDMapping; + private final BidiMap vertexNameIDMapping; //edgeId to EdgeInfo mapping private final Map edgeInfoMap; @@ -89,7 +89,7 @@ public class DagInfo extends BaseInfo { super(jsonObject); vertexNameMap = Maps.newHashMap(); - vertexNameIDMapping = new DualHashBidiMap(); + vertexNameIDMapping = new DualHashBidiMap<>(); edgeInfoMap = Maps.newHashMap(); basicVertexInfoMap = Maps.newHashMap(); containerMapping = LinkedHashMultimap.create(); @@ -607,7 +607,7 @@ public final Multimap getContainersToTaskAttemptMapp return Multimaps.unmodifiableMultimap(containerMapping); } - public final Map getVertexNameIDMapping() { + public final Map getVertexNameIDMapping() { return vertexNameIDMapping; } From b5b432bf05d5b902d66c19d5b693bb620c577390 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Thu, 19 Mar 2020 15:43:37 +0000 Subject: [PATCH 213/512] TEZ-4134. Upgrade maven surefire plugin to 3.0.0-M4 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 0cbdfd6380..cd4cf7ccd3 100644 --- a/pom.xml +++ b/pom.xml @@ -860,7 +860,7 @@ org.apache.maven.plugins maven-surefire-plugin - 3.0.0-M1 + 3.0.0-M4 1 false From 0eeef27413db97b52242878301788ac5fd8def16 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Mon, 13 Apr 2020 09:20:10 -0500 Subject: [PATCH 214/512] TEZ-4142. TezUtils.createConfFromByteString on Configuration larger than 32MB throws com.google.protobuf.CodedInputStream exception --- .../java/org/apache/tez/common/TezUtils.java | 5 +++- .../org/apache/tez/common/TestTezUtils.java | 24 +++++++++++++++++++ 2 files changed, 28 insertions(+), 1 deletion(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java index 536eda47db..51311ffd80 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java @@ -29,6 +29,7 @@ import com.google.protobuf.ByteString; +import com.google.protobuf.CodedInputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience; @@ -110,7 +111,9 @@ public static UserPayload createUserPayloadFromConf(Configuration conf) throws I public static Configuration createConfFromByteString(ByteString byteString) throws IOException { Objects.requireNonNull(byteString, "ByteString must be specified"); try(SnappyInputStream uncompressIs = new SnappyInputStream(byteString.newInput());) { - DAGProtos.ConfigurationProto confProto = DAGProtos.ConfigurationProto.parseFrom(uncompressIs); + CodedInputStream in = CodedInputStream.newInstance(uncompressIs); + in.setSizeLimit(Integer.MAX_VALUE); + DAGProtos.ConfigurationProto confProto = DAGProtos.ConfigurationProto.parseFrom(in); Configuration conf = new Configuration(false); readConfFromPB(confProto, conf); return conf; diff --git a/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java b/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java index 04eb2c0d9e..c0c3e5d3a8 100644 --- a/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java +++ b/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java @@ -54,6 +54,30 @@ public void testByteStringToAndFromConf() throws IOException { checkConf(conf); } + @Test (timeout=20000) + public void testByteStringToAndFromLargeConf() throws IOException { + Configuration conf = getConf(); + int largeSize = 64 * 1024 * 1024; + final String alphaString = "ABCDEFGHIJKLMNOPQRSTUVWXYZ"; + StringBuilder sb = new StringBuilder(largeSize); + + for (int i = 0; i < largeSize; i++) { + int index = (int)(alphaString.length() * Math.random()); + sb.append(alphaString.charAt(index)); + } + + String largeValue = sb.toString(); + conf.set("testLargeValue", largeValue); + Assert.assertEquals(conf.size(), 7); + ByteString bsConf = TezUtils.createByteStringFromConf(conf); + conf.clear(); + Assert.assertEquals(conf.size(), 0); + conf = TezUtils.createConfFromByteString(bsConf); + Assert.assertEquals(conf.size(), 7); + checkConf(conf); + Assert.assertEquals(conf.get("testLargeValue"), largeValue); + } + @Test (timeout=2000) public void testPayloadToAndFromConf() throws IOException { Configuration conf = getConf(); From 7dbec63e1f97eea95ab998e16ffcd592ff6be332 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Tue, 14 Apr 2020 23:02:48 +0200 Subject: [PATCH 215/512] =?UTF-8?q?TEZ-4145:=20Reduce=20lock=20contention?= =?UTF-8?q?=20in=20TezSpillRecord=20(L=C3=A1szl=C3=B3=20Bodor=20reviewed?= =?UTF-8?q?=20by=20Ashutosh=20Chauhan,=20Jonathan=20Turner=20Eagles,=20Raj?= =?UTF-8?q?esh=20Balamohan)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../apache/tez/shufflehandler/IndexCache.java | 13 ++- .../apache/tez/auxservices/IndexCache.java | 13 ++- .../tez/auxservices/TestShuffleHandler.java | 3 +- .../library/common/shuffle/Fetcher.java | 5 +- .../orderedgrouped/FetcherOrderedGrouped.java | 6 +- .../orderedgrouped/ShuffleScheduler.java | 6 +- .../common/sort/impl/ExternalSorter.java | 2 + .../common/sort/impl/PipelinedSorter.java | 8 +- .../common/sort/impl/TezSpillRecord.java | 23 ++-- .../common/sort/impl/dflt/DefaultSorter.java | 20 ++-- .../BaseUnorderedPartitionedKVWriter.java | 8 ++ .../writers/UnorderedPartitionedKVWriter.java | 6 +- .../output/OrderedPartitionedKVOutput.java | 7 +- .../common/shuffle/TestShuffleUtils.java | 2 +- .../shuffle/orderedgrouped/TestFetcher.java | 108 +++++++++--------- 15 files changed, 140 insertions(+), 90 deletions(-) diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/IndexCache.java b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/IndexCache.java index e358fcc61e..51224cd8cf 100644 --- a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/IndexCache.java +++ b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/IndexCache.java @@ -19,6 +19,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.tez.runtime.library.common.Constants; import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord; @@ -38,11 +39,21 @@ class IndexCache { private final LinkedBlockingQueue queue = new LinkedBlockingQueue(); + private FileSystem fs; public IndexCache(Configuration conf) { this.conf = conf; totalMemoryAllowed = 10 * 1024 * 1024; LOG.info("IndexCache created with max memory = " + totalMemoryAllowed); + initLocalFs(); + } + + private void initLocalFs() { + try { + this.fs = FileSystem.getLocal(conf).getRaw(); + } catch (IOException e) { + throw new RuntimeException(e); + } } /** @@ -114,7 +125,7 @@ private IndexInformation readIndexFileToCache(Path indexFileName, LOG.debug("IndexCache MISS: MapId " + mapId + " not found") ; TezSpillRecord tmp = null; try { - tmp = new TezSpillRecord(indexFileName, conf, expectedIndexOwner); + tmp = new TezSpillRecord(indexFileName, fs, expectedIndexOwner); } catch (Throwable e) { tmp = new TezSpillRecord(0); cache.remove(mapId); diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/IndexCache.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/IndexCache.java index 1a9cfb2b12..625f7ab8ba 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/IndexCache.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/IndexCache.java @@ -19,6 +19,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.tez.runtime.library.common.Constants; import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord; @@ -38,12 +39,22 @@ class IndexCache { private final LinkedBlockingQueue queue = new LinkedBlockingQueue(); + private FileSystem fs; public static final String INDEX_CACHE_MB = "tez.shuffle.indexcache.mb"; public IndexCache(Configuration conf) { this.conf = conf; totalMemoryAllowed = conf.getInt(INDEX_CACHE_MB, 10) * 1024 * 1024; LOG.info("IndexCache created with max memory = " + totalMemoryAllowed); + initLocalFs(); + } + + private void initLocalFs() { + try { + this.fs = FileSystem.getLocal(conf).getRaw(); + } catch (IOException e) { + throw new RuntimeException(e); + } } /** @@ -160,7 +171,7 @@ private IndexInformation readIndexFileToCache(Path indexFileName, } TezSpillRecord tmp = null; try { - tmp = new TezSpillRecord(indexFileName, conf, expectedIndexOwner); + tmp = new TezSpillRecord(indexFileName, fs, expectedIndexOwner); } catch (Throwable e) { tmp = new TezSpillRecord(0); cache.remove(mapId); diff --git a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java index 0d542cfee1..a7f444680b 100644 --- a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java @@ -898,7 +898,8 @@ private static void createIndexFile(File indexFile, Configuration conf) TezSpillRecord tezSpillRecord = new TezSpillRecord(2); tezSpillRecord.putIndex(new TezIndexRecord(0, 10, 10), 0); tezSpillRecord.putIndex(new TezIndexRecord(10, 10, 10), 1); - tezSpillRecord.writeToFile(new Path(indexFile.getAbsolutePath()), conf, crc); + tezSpillRecord.writeToFile(new Path(indexFile.getAbsolutePath()), conf, + FileSystem.getLocal(conf).getRaw(), crc); } @Test diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java index 5da12de0ab..f9899f1903 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java @@ -235,6 +235,7 @@ private Fetcher(FetcherCallback fetcherCallback, HttpConnectionParams params, } } + // helper method to populate the remaining map void populateRemainingMap(List origlist) { if (srcAttemptsRemaining == null) { @@ -357,7 +358,7 @@ public void cache(String host, } spillRec.putIndex(indexRec, 0); - spillRec.writeToFile(tmpIndex, conf); + spillRec.writeToFile(tmpIndex, conf, localFs); // everything went well so far - rename it boolean renamed = localFs.rename(tmpIndex, outputPath .suffix(Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING)); @@ -736,7 +737,7 @@ protected TezIndexRecord getTezIndexRecord(InputAttemptIdentifier srcAttemptId, Path indexFile = getShuffleInputFileName(srcAttemptId.getPathComponent(), Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING); - TezSpillRecord spillRecord = new TezSpillRecord(indexFile, conf); + TezSpillRecord spillRecord = new TezSpillRecord(indexFile, localFs); idxRecord = spillRecord.getIndex(partition); return idxRecord; } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java index cb8e751620..e732ab0063 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java @@ -39,6 +39,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.tez.common.TezRuntimeFrameworkConfigs; import org.apache.tez.common.counters.TezCounter; @@ -73,6 +74,7 @@ class FetcherOrderedGrouped extends CallableWithNdc { private final ExceptionReporter exceptionReporter; private final int id; private final String logIdentifier; + private final RawLocalFileSystem localFs; private final String localShuffleHost; private final int localShufflePort; private final String applicationId; @@ -113,6 +115,7 @@ public FetcherOrderedGrouped(HttpConnectionParams httpConnectionParams, boolean ifileReadAhead, int ifileReadAheadLength, CompressionCodec codec, Configuration conf, + RawLocalFileSystem localFs, boolean localDiskFetchEnabled, String localHostname, int shufflePort, @@ -157,6 +160,7 @@ public FetcherOrderedGrouped(HttpConnectionParams httpConnectionParams, this.codec = null; } this.conf = conf; + this.localFs = localFs; this.localShuffleHost = localHostname; this.localShufflePort = shufflePort; @@ -772,7 +776,7 @@ protected TezIndexRecord getIndexRecord(String pathComponent, int partitionId) throws IOException { Path indexFile = getShuffleInputFileName(pathComponent, Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING); - TezSpillRecord spillRecord = new TezSpillRecord(indexFile, conf); + TezSpillRecord spillRecord = new TezSpillRecord(indexFile, localFs); return spillRecord.getIndex(partitionId); } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java index 707f920fd7..ff07e913fb 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java @@ -65,6 +65,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.hadoop.io.IntWritable; import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.common.counters.TaskCounter; @@ -214,6 +216,7 @@ enum ShuffleErrors { private final int ifileReadAheadLength; private final CompressionCodec codec; private final Configuration conf; + private final RawLocalFileSystem localFs; private final boolean localDiskFetchEnabled; private final String localHostname; private final int shufflePort; @@ -263,6 +266,7 @@ public ShuffleScheduler(InputContext inputContext, String srcNameTrimmed) throws IOException { this.inputContext = inputContext; this.conf = conf; + this.localFs = (RawLocalFileSystem) FileSystem.getLocal(conf).getRaw(); this.exceptionReporter = exceptionReporter; this.allocator = allocator; this.mergeManager = mergeManager; @@ -1464,7 +1468,7 @@ private synchronized void waitAndNotifyProgress() throws InterruptedException { FetcherOrderedGrouped constructFetcherForHost(MapHost mapHost) { return new FetcherOrderedGrouped(httpConnectionParams, ShuffleScheduler.this, allocator, exceptionReporter, jobTokenSecretManager, ifileReadAhead, ifileReadAheadLength, - codec, conf, localDiskFetchEnabled, localHostname, shufflePort, srcNameTrimmed, mapHost, + codec, conf, localFs, localDiskFetchEnabled, localHostname, shufflePort, srcNameTrimmed, mapHost, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, applicationId, dagId, asyncHttp, sslShuffle, verifyDiskChecksum, compositeFetch); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java index ae03e3d7a9..16d58492fb 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java @@ -102,6 +102,7 @@ public void progress() { protected final Combiner combiner; protected final Partitioner partitioner; protected final Configuration conf; + protected final RawLocalFileSystem localFs; protected final FileSystem rfs; protected final TezTaskOutput mapOutputFile; protected final int partitions; @@ -171,6 +172,7 @@ public ExternalSorter(OutputContext outputContext, Configuration conf, int numOu long initialMemoryAvailable) throws IOException { this.outputContext = outputContext; this.conf = conf; + this.localFs = (RawLocalFileSystem) FileSystem.getLocal(conf).getRaw(); this.partitions = numOutputs; reportPartitionStats = ReportPartitionStats.fromString( conf.get(TezRuntimeConfiguration.TEZ_RUNTIME_REPORT_PARTITION_STATS, diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java index d84164fca6..610cae9de3 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java @@ -535,7 +535,7 @@ private void spillSingleRecord(final Object key, final Object value, } spillFileIndexPaths.put(numSpills, indexFilename); - spillRec.writeToFile(indexFilename, conf); + spillRec.writeToFile(indexFilename, conf, localFs); //TODO: honor cache limits indexCacheList.add(spillRec); ++numSpills; @@ -625,7 +625,7 @@ public boolean spill(boolean ignoreEmptySpills) throws IOException { mapOutputFile.getSpillIndexFileForWrite(numSpills, partitions * MAP_OUTPUT_INDEX_RECORD_LENGTH); spillFileIndexPaths.put(numSpills, indexFilename); - spillRec.writeToFile(indexFilename, conf); + spillRec.writeToFile(indexFilename, conf, localFs); //TODO: honor cache limits indexCacheList.add(spillRec); ++numSpills; @@ -737,7 +737,7 @@ public void flush() throws IOException { + "finalIndexFile=" + finalIndexFile + ", filename=" + filename + ", indexFilename=" + indexFilename); } - TezSpillRecord spillRecord = new TezSpillRecord(finalIndexFile, conf); + TezSpillRecord spillRecord = new TezSpillRecord(finalIndexFile, localFs); if (reportPartitionStats()) { for (int i = 0; i < spillRecord.size(); i++) { partitionStats[i] += spillRecord.getIndex(i).getPartLength(); @@ -833,7 +833,7 @@ public void flush() throws IOException { numShuffleChunks.setValue(1); //final merge has happened. fileOutputByteCounter.increment(rfs.getFileStatus(finalOutputFile).getLen()); - spillRec.writeToFile(finalIndexFile, conf); + spillRec.writeToFile(finalIndexFile, conf, localFs); finalOut.close(); for (int i = 0; i < numSpills; i++) { Path indexFilename = spillFileIndexPaths.get(i); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.java index 48bd211dfe..e16b7a06b7 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.java @@ -49,20 +49,23 @@ public TezSpillRecord(int numPartitions) { entries = buf.asLongBuffer(); } - public TezSpillRecord(Path indexFileName, Configuration job) throws IOException { - this(indexFileName, job, null); + public TezSpillRecord(Path indexFileName, Configuration conf) throws IOException { + this(indexFileName, FileSystem.getLocal(conf).getRaw()); } - public TezSpillRecord(Path indexFileName, Configuration job, String expectedIndexOwner) + public TezSpillRecord(Path indexFileName, FileSystem fs) throws IOException { + this(indexFileName, fs, null); + } + + public TezSpillRecord(Path indexFileName, FileSystem fs, String expectedIndexOwner) throws IOException { - this(indexFileName, job, new PureJavaCrc32(), expectedIndexOwner); + this(indexFileName, fs, new PureJavaCrc32(), expectedIndexOwner); } - public TezSpillRecord(Path indexFileName, Configuration job, Checksum crc, + public TezSpillRecord(Path indexFileName, FileSystem rfs, Checksum crc, String expectedIndexOwner) throws IOException { - final FileSystem rfs = FileSystem.getLocal(job).getRaw(); final FSDataInputStream in = rfs.open(indexFileName); try { final long length = rfs.getFileStatus(indexFileName).getLen(); @@ -117,14 +120,12 @@ public void putIndex(TezIndexRecord rec, int partition) { /** * Write this spill record to the location provided. */ - public void writeToFile(Path loc, Configuration job) - throws IOException { - writeToFile(loc, job, new PureJavaCrc32()); + public void writeToFile(Path loc, Configuration job, FileSystem fs) throws IOException { + writeToFile(loc, job, fs, new PureJavaCrc32()); } - public void writeToFile(Path loc, Configuration job, Checksum crc) + public void writeToFile(Path loc, Configuration job, FileSystem rfs, Checksum crc) throws IOException { - final FileSystem rfs = FileSystem.getLocal(job).getRaw(); CheckedOutputStream chk = null; final FSDataOutputStream out = rfs.create(loc); try { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java index 6d45ee1560..124f0784a8 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java @@ -971,7 +971,7 @@ protected void spill(int mstart, int mend, long sameKeyCount, long totalKeysCoun mapOutputFile.getSpillIndexFileForWrite(numSpills, partitions * MAP_OUTPUT_INDEX_RECORD_LENGTH); spillFileIndexPaths.put(numSpills, indexFilename); - spillRec.writeToFile(indexFilename, conf); + spillRec.writeToFile(indexFilename, conf, localFs); } else { indexCacheList.add(spillRec); totalIndexCacheMemory += @@ -1053,7 +1053,7 @@ private void spillSingleRecord(final Object key, final Object value, mapOutputFile.getSpillIndexFileForWrite(numSpills, partitions * MAP_OUTPUT_INDEX_RECORD_LENGTH); spillFileIndexPaths.put(numSpills, indexFilename); - spillRec.writeToFile(indexFilename, conf); + spillRec.writeToFile(indexFilename, conf, localFs); } else { indexCacheList.add(spillRec); totalIndexCacheMemory += @@ -1193,13 +1193,13 @@ private void maybeAddEventsForSpills() throws IOException { TezSpillRecord spillRecord = indexCacheList.get(i); if (spillRecord == null) { //File was already written and location is stored in spillFileIndexPaths - spillRecord = new TezSpillRecord(spillFileIndexPaths.get(i), conf); + spillRecord = new TezSpillRecord(spillFileIndexPaths.get(i), localFs); } else { //Double check if this file has to be written if (spillFileIndexPaths.get(i) == null) { Path indexPath = mapOutputFile.getSpillIndexFileForWrite(i, partitions * MAP_OUTPUT_INDEX_RECORD_LENGTH); - spillRecord.writeToFile(indexPath, conf); + spillRecord.writeToFile(indexPath, conf, localFs); } } @@ -1228,10 +1228,10 @@ private void mergeParts() throws IOException, InterruptedException { sameVolRename(filename[0], finalOutputFile); if (indexCacheList.size() == 0) { sameVolRename(spillFileIndexPaths.get(0), finalIndexFile); - spillRecord = new TezSpillRecord(finalIndexFile, conf); + spillRecord = new TezSpillRecord(finalIndexFile, localFs); } else { spillRecord = indexCacheList.get(0); - spillRecord.writeToFile(finalIndexFile, conf); + spillRecord.writeToFile(finalIndexFile, conf, localFs); } } else { List events = Lists.newLinkedList(); @@ -1239,7 +1239,7 @@ private void mergeParts() throws IOException, InterruptedException { spillRecord = indexCacheList.get(0); Path indexPath = mapOutputFile.getSpillIndexFileForWrite(numSpills-1, partitions * MAP_OUTPUT_INDEX_RECORD_LENGTH); - spillRecord.writeToFile(indexPath, conf); + spillRecord.writeToFile(indexPath, conf, localFs); maybeSendEventForSpill(events, true, spillRecord, 0, true); fileOutputByteCounter.increment(rfs.getFileStatus(spillFilePaths.get(0)).getLen()); //No need to populate finalIndexFile, finalOutputFile etc when finalMerge is disabled @@ -1256,7 +1256,7 @@ private void mergeParts() throws IOException, InterruptedException { // read in paged indices for (int i = indexCacheList.size(); i < numSpills; ++i) { Path indexFileName = spillFileIndexPaths.get(i); - indexCacheList.add(new TezSpillRecord(indexFileName, conf)); + indexCacheList.add(new TezSpillRecord(indexFileName, localFs)); } //Check if it is needed to do final merge. Or else, exit early. @@ -1309,7 +1309,7 @@ private void mergeParts() throws IOException, InterruptedException { outputBytesWithOverheadCounter.increment(rawLength); sr.putIndex(rec, i); } - sr.writeToFile(finalIndexFile, conf); + sr.writeToFile(finalIndexFile, conf, localFs); } finally { finalOut.close(); } @@ -1392,7 +1392,7 @@ private void mergeParts() throws IOException, InterruptedException { } } numShuffleChunks.setValue(1); //final merge has happened - spillRec.writeToFile(finalIndexFile, conf); + spillRec.writeToFile(finalIndexFile, conf, localFs); finalOut.close(); for(int i = 0; i < numSpills; i++) { rfs.delete(filename[i],true); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/BaseUnorderedPartitionedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/BaseUnorderedPartitionedKVWriter.java index 3467c82bb3..26da98f378 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/BaseUnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/BaseUnorderedPartitionedKVWriter.java @@ -25,6 +25,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.DefaultCodec; import org.apache.hadoop.io.serializer.SerializationFactory; @@ -48,6 +50,7 @@ public abstract class BaseUnorderedPartitionedKVWriter extends KeyValuesWriter { protected final OutputContext outputContext; protected final Configuration conf; + protected final RawLocalFileSystem localFs; protected final Partitioner partitioner; protected final Class keyClass; protected final Class valClass; @@ -110,6 +113,11 @@ public abstract class BaseUnorderedPartitionedKVWriter extends KeyValuesWriter { public BaseUnorderedPartitionedKVWriter(OutputContext outputContext, Configuration conf, int numOutputs) { this.outputContext = outputContext; this.conf = conf; + try { + this.localFs = (RawLocalFileSystem) FileSystem.getLocal(conf).getRaw(); + } catch (IOException e) { + throw new RuntimeException(e); + } this.numPartitions = numOutputs; // k/v serialization diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java index c823b6518b..62170d95a3 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java @@ -813,7 +813,7 @@ public List close() throws IOException, InterruptedException { TezSpillRecord sr = new TezSpillRecord(1); sr.putIndex(rec, 0); finalIndexPath = outputFileHandler.getOutputIndexFileForWrite(indexFileSizeEstimate); - sr.writeToFile(finalIndexPath, conf); + sr.writeToFile(finalIndexPath, conf, localFs); } eventList.add(generateDMEvent(false, -1, false, outputContext .getUniqueIdentifier(), emptyPartitions)); @@ -1140,7 +1140,7 @@ private void mergeAll() throws IOException { } deleteIntermediateSpills(); } - finalSpillRecord.writeToFile(finalIndexPath, conf); + finalSpillRecord.writeToFile(finalIndexPath, conf, localFs); fileOutputBytesCounter.increment(indexFileSizeEstimate); LOG.info(destNameTrimmed + ": " + "Finished final spill after merging : " + numSpills.get() + " spills"); } @@ -1233,7 +1233,7 @@ private void handleSpillIndex(SpillPathDetails spillPathDetails, TezSpillRecord throws IOException { if (spillPathDetails.indexFilePath != null) { //write the index record - spillRecord.writeToFile(spillPathDetails.indexFilePath, conf); + spillRecord.writeToFile(spillPathDetails.indexFilePath, conf, localFs); } else { //add to cache SpillInfo spillInfo = new SpillInfo(spillRecord, spillPathDetails.outputFilePath); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java index 8870e1f0eb..86c20dd9e3 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java @@ -36,6 +36,8 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.TezRuntimeFrameworkConfigs; import org.apache.tez.common.TezUtils; @@ -67,6 +69,7 @@ public class OrderedPartitionedKVOutput extends AbstractLogicalOutput { protected ExternalSorter sorter; protected Configuration conf; + private RawLocalFileSystem localFs; protected MemoryUpdateCallbackHandler memoryUpdateCallbackHandler; private long startTime; private long endTime; @@ -88,6 +91,8 @@ public OrderedPartitionedKVOutput(OutputContext outputContext, int numPhysicalOu public synchronized List initialize() throws IOException { this.startTime = System.nanoTime(); this.conf = TezUtils.createConfFromUserPayload(getContext().getUserPayload()); + this.localFs = (RawLocalFileSystem) FileSystem.getLocal(conf).getRaw(); + // Initializing this parametr in this conf since it is used in multiple // places (wherever LocalDirAllocator is used) - TezTaskOutputFiles, // TezMerger, etc. @@ -205,7 +210,7 @@ private List generateEvents() throws IOException { String auxiliaryService = conf.get(TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID, TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID_DEFAULT); ShuffleUtils.generateEventOnSpill(eventList, finalMergeEnabled, isLastEvent, - getContext(), 0, new TezSpillRecord(sorter.getFinalIndexFile(), conf), + getContext(), 0, new TezSpillRecord(sorter.getFinalIndexFile(), localFs), getNumPhysicalOutputs(), sendEmptyPartitionDetails, getContext().getUniqueIdentifier(), sorter.getPartitionStats(), sorter.reportDetailedPartitionStats(), auxiliaryService, deflater); } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java index f61c7e5f66..cc918fa7ae 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java @@ -149,7 +149,7 @@ private Path createIndexFile(int numPartitions, boolean allEmptyPartitions) thro startOffset += partLen; spillRecord.putIndex(indexRecord, i); } - spillRecord.writeToFile(path, conf); + spillRecord.writeToFile(path, conf, FileSystem.getLocal(conf).getRaw()); return path; } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java index 6d304480b8..ec0eeee37a 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java @@ -59,7 +59,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.tez.common.counters.TezCounters; import org.apache.tez.common.security.JobTokenSecretManager; @@ -126,11 +128,10 @@ public void testInputsReturnedOnConnectionException() throws Exception { doReturn(mapsForHost).when(scheduler).getMapsForHost(mapHost); FetcherOrderedGrouped fetcher = - new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, - null, conf, false, HOST, PORT, "src vertex", mapHost, ioErrsCounter, - wrongLengthErrsCounter, badIdErrsCounter, - wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, - false, false, true, false); + new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, null, conf, + getRawFs(conf), false, HOST, PORT, "src vertex", mapHost, ioErrsCounter, + wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, connectionErrsCounter, + wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); fetcher.call(); verify(scheduler).getMapsForHost(mapHost); @@ -154,11 +155,10 @@ public void testLocalFetchModeSetting1() throws Exception { final boolean DISABLE_LOCAL_FETCH = false; MapHost mapHost = new MapHost(HOST, PORT, 0, 1); FetcherOrderedGrouped fetcher = - new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, - null, conf, ENABLE_LOCAL_FETCH, HOST, PORT, "src vertex", mapHost, ioErrsCounter, - wrongLengthErrsCounter, badIdErrsCounter, - wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, - false, false, true, false); + new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, null, conf, + getRawFs(conf), ENABLE_LOCAL_FETCH, HOST, PORT, "src vertex", mapHost, ioErrsCounter, + wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, connectionErrsCounter, + wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); // when local mode is enabled and host and port matches use local fetch FetcherOrderedGrouped spyFetcher = spy(fetcher); @@ -172,11 +172,10 @@ public void testLocalFetchModeSetting1() throws Exception { // if hostname does not match use http mapHost = new MapHost(HOST + "_OTHER", PORT, 0, 1); fetcher = - new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, - null, conf, ENABLE_LOCAL_FETCH, HOST, PORT, "src vertex", mapHost, ioErrsCounter, - wrongLengthErrsCounter, badIdErrsCounter, - wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, - false, false, true, false); + new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, null, conf, + getRawFs(conf), ENABLE_LOCAL_FETCH, HOST, PORT, "src vertex", mapHost, ioErrsCounter, + wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, connectionErrsCounter, + wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); spyFetcher = spy(fetcher); doNothing().when(spyFetcher).setupLocalDiskFetch(mapHost); @@ -188,11 +187,10 @@ public void testLocalFetchModeSetting1() throws Exception { // if port does not match use http mapHost = new MapHost(HOST, PORT + 1, 0, 1); fetcher = - new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, - null, conf, ENABLE_LOCAL_FETCH, HOST, PORT, "src vertex", mapHost, ioErrsCounter, - wrongLengthErrsCounter, badIdErrsCounter, - wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, - false, false, true, false); + new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, null, conf, + getRawFs(conf), ENABLE_LOCAL_FETCH, HOST, PORT, "src vertex", mapHost, ioErrsCounter, + wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, connectionErrsCounter, + wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); spyFetcher = spy(fetcher); doNothing().when(spyFetcher).setupLocalDiskFetch(mapHost); @@ -203,11 +201,10 @@ public void testLocalFetchModeSetting1() throws Exception { //if local fetch is not enabled mapHost = new MapHost(HOST, PORT, 0, 1); - fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, - null, conf, DISABLE_LOCAL_FETCH, HOST, PORT, "src vertex", mapHost, ioErrsCounter, - wrongLengthErrsCounter, badIdErrsCounter, - wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, - false, false, true, false); + fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, null, + conf, getRawFs(conf), DISABLE_LOCAL_FETCH, HOST, PORT, "src vertex", mapHost, ioErrsCounter, + wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, connectionErrsCounter, + wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); spyFetcher = spy(fetcher); doNothing().when(spyFetcher).setupLocalDiskFetch(mapHost); @@ -228,10 +225,10 @@ public void testSetupLocalDiskFetch() throws Exception { when(inputContext.getSourceVertexName()).thenReturn(""); MapHost host = new MapHost(HOST, PORT, 1, 1); - FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, - null, conf, true, HOST, PORT, "src vertex", host, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, - wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, - false, false, true, false); + FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, + null, false, 0, null, conf, getRawFs(conf), true, HOST, PORT, "src vertex", host, + ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, + connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); FetcherOrderedGrouped spyFetcher = spy(fetcher); @@ -338,10 +335,10 @@ public void testSetupLocalDiskFetchEmptyPartitions() throws Exception { when(inputContext.getSourceVertexName()).thenReturn(""); MapHost host = new MapHost(HOST, PORT, 1, 1); - FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, - null, conf, true, HOST, PORT, "src vertex", host, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, - wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, - false, false, true, false); + FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, + null, false, 0, null, conf, getRawFs(conf), true, HOST, PORT, "src vertex", host, + ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, + connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); FetcherOrderedGrouped spyFetcher = spy(fetcher); final List srcAttempts = Arrays.asList( @@ -413,10 +410,10 @@ public void testSetupLocalDiskFetchAutoReduce() throws Exception { when(inputContext.getSourceVertexName()).thenReturn(""); MapHost host = new MapHost(HOST, PORT, 1, 2); - FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, - null, conf, true, HOST, PORT, "src vertex", host, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, - wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, - false, false, true, false); + FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, + null, false, 0, null, conf, getRawFs(conf), true, HOST, PORT, "src vertex", host, + ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, + connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); FetcherOrderedGrouped spyFetcher = spy(fetcher); @@ -587,10 +584,10 @@ public void testWithRetry() throws Exception { HttpConnectionParams httpConnectionParams = ShuffleUtils.getHttpConnectionParams(conf); final MapHost host = new MapHost(HOST, PORT, 1, 1); - FetcherOrderedGrouped mockFetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, - null, conf, false, HOST, PORT, "src vertex", host, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, - wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, - false, false, true, false); + FetcherOrderedGrouped mockFetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, + null, false, 0, null, conf, getRawFs(conf), false, HOST, PORT, "src vertex", host, + ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, + connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); final FetcherOrderedGrouped fetcher = spy(mockFetcher); @@ -676,12 +673,10 @@ public void testAsyncWithException() throws Exception { HttpConnectionParams httpConnectionParams = ShuffleUtils.getHttpConnectionParams(conf); final MapHost host = new MapHost(HOST, PORT, 1, 1); FetcherOrderedGrouped mockFetcher = - new FetcherOrderedGrouped(httpConnectionParams, scheduler, merger, shuffle, jobMgr, - false, 0, - null, conf, false, HOST, PORT, "src vertex", host, ioErrsCounter, - wrongLengthErrsCounter, badIdErrsCounter, - wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, - true, false, true, false); + new FetcherOrderedGrouped(httpConnectionParams, scheduler, merger, shuffle, jobMgr, false, + 0, null, conf, getRawFs(conf), false, HOST, PORT, "src vertex", host, ioErrsCounter, + wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, connectionErrsCounter, + wrongReduceErrsCounter, APP_ID, DAG_ID, true, false, true, false); final FetcherOrderedGrouped fetcher = spy(mockFetcher); fetcher.remaining = new LinkedHashMap(); final List srcAttempts = Arrays.asList( @@ -743,12 +738,10 @@ public void testInputAttemptIdentifierMap() { MergeManager merger = mock(MergeManager.class); Shuffle shuffle = mock(Shuffle.class); MapHost mapHost = new MapHost(HOST, PORT, 0, 1); - FetcherOrderedGrouped fetcher = - new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, - null, conf, false, HOST, PORT, "src vertex", mapHost, ioErrsCounter, - wrongLengthErrsCounter, badIdErrsCounter, - wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, - false, false, true, false); + FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, + null, false, 0, null, conf, getRawFs(conf), false, HOST, PORT, "src vertex", mapHost, + ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, + connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); fetcher.populateRemainingMap(new LinkedList(Arrays.asList(srcAttempts))); Assert.assertEquals(expectedSrcAttempts.length, fetcher.remaining.size()); Iterator> iterator = fetcher.remaining.entrySet().iterator(); @@ -758,4 +751,13 @@ public void testInputAttemptIdentifierMap() { Assert.assertTrue(expectedSrcAttempts[count++].toString().compareTo(key) == 0); } } + + private RawLocalFileSystem getRawFs(Configuration conf) { + try { + return (RawLocalFileSystem) FileSystem.getLocal(conf).getRaw(); + } catch (IOException e) { + // TODO Auto-generated catch block + throw new RuntimeException(e); + } + } } From f14baf95f35a66566652a17e592bb02b26beba7d Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Wed, 15 Apr 2020 09:25:21 +0530 Subject: [PATCH 216/512] TEZ-4146: Register RUNNING state in DAG's state change callback (Rajesh Balamohan, reviewed by Gopal V) --- .../org/apache/tez/dag/app/dag/impl/DAGImpl.java | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java index 3cde8e7812..b8bdcc9a35 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java @@ -164,7 +164,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG, // TODO Recovery //private final List amInfos; private final Lock dagStatusLock = new ReentrantLock(); - private final Condition dagCompletionCondition = dagStatusLock.newCondition(); + private final Condition dagStateChangedCondition = dagStatusLock.newCondition(); private final AtomicBoolean isFinalState = new AtomicBoolean(false); private final Lock readLock; private final Lock writeLock; @@ -569,6 +569,8 @@ public DAGImpl(TezDAGID dagId, private void augmentStateMachine() { stateMachine + .registerStateEnteredCallback(DAGState.RUNNING, + STATE_CHANGED_CALLBACK) .registerStateEnteredCallback(DAGState.SUCCEEDED, STATE_CHANGED_CALLBACK) .registerStateEnteredCallback(DAGState.FAILED, @@ -583,10 +585,12 @@ private static class DagStateChangedCallback implements OnStateChangedCallback { @Override public void onStateChanged(DAGImpl dag, DAGState dagState) { - dag.isFinalState.set(true); + if (dagState != DAGState.RUNNING) { + dag.isFinalState.set(true); + } dag.dagStatusLock.lock(); try { - dag.dagCompletionCondition.signal(); + dag.dagStateChangedCondition.signal(); } finally { dag.dagStatusLock.unlock(); } @@ -946,7 +950,7 @@ public DAGStatusBuilder getDAGStatus(Set statusOptions, if (isFinalState.get()) { break; } - nanosLeft = dagCompletionCondition.awaitNanos(timeoutNanos); + nanosLeft = dagStateChangedCondition.awaitNanos(timeoutNanos); } catch (InterruptedException e) { throw new TezException("Interrupted while waiting for dag to complete", e); } finally { From 214c544472f80997755b62c382f016fbe8373177 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Wed, 15 Apr 2020 08:18:34 +0200 Subject: [PATCH 217/512] =?UTF-8?q?TEZ-4143:=20Provide=20an=20option=20to?= =?UTF-8?q?=20disable=20DAG=20graph=20(.dot)=20generation=20for=20latency?= =?UTF-8?q?=20sensitive=20jobs=20(L=C3=A1szl=C3=B3=20Bodor=20reviewed=20by?= =?UTF-8?q?=20Rajesh=20Balamohan)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../org/apache/tez/dag/app/DAGAppMaster.java | 36 ++++++++++--------- .../apache/tez/dag/app/dag/impl/DAGImpl.java | 5 ++- 2 files changed, 24 insertions(+), 17 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index 18cffadb5f..7e5a7a9d67 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -1038,28 +1038,32 @@ DAGImpl createDAG(DAGPlan dagPB, TezDAGID dagId) { LOG.warn("Failed to generate json for DAG", e); } - Utils.generateDAGVizFile(newDag, dagPB, logDirs, newDag.getDAGScheduler()); - writePBTextFile(newDag); + writeDebugArtifacts(dagPB, newDag); return newDag; } // end createDag() + private void writeDebugArtifacts(DAGPlan dagPB, DAGImpl newDag) { + boolean debugArtifacts = + newDag.getConf().getBoolean(TezConfiguration.TEZ_GENERATE_DEBUG_ARTIFACTS, + TezConfiguration.TEZ_GENERATE_DEBUG_ARTIFACTS_DEFAULT); + if (debugArtifacts) { + Utils.generateDAGVizFile(newDag, dagPB, logDirs, newDag.getDAGScheduler()); + writePBTextFile(newDag); + } + } private void writePBTextFile(DAG dag) { - if (dag.getConf().getBoolean(TezConfiguration.TEZ_GENERATE_DEBUG_ARTIFACTS, - TezConfiguration.TEZ_GENERATE_DEBUG_ARTIFACTS_DEFAULT)) { + String logFile = logDirs[new Random().nextInt(logDirs.length)] + File.separatorChar + + dag.getID().toString() + "-" + TezConstants.TEZ_PB_PLAN_TEXT_NAME; - String logFile = logDirs[new Random().nextInt(logDirs.length)] + File.separatorChar + - dag.getID().toString() + "-" + TezConstants.TEZ_PB_PLAN_TEXT_NAME; - - LOG.info("Writing DAG plan to: " + logFile); - File outFile = new File(logFile); - try { - PrintWriter printWriter = new PrintWriter(outFile, "UTF-8"); - printWriter.println(TezUtilsInternal.convertDagPlanToString(dag.getJobPlan())); - printWriter.close(); - } catch (IOException e) { - LOG.warn("Failed to write TEZ_PLAN to " + outFile.toString(), e); - } + LOG.info("Writing DAG plan to: " + logFile); + File outFile = new File(logFile); + try { + PrintWriter printWriter = new PrintWriter(outFile, "UTF-8"); + printWriter.println(TezUtilsInternal.convertDagPlanToString(dag.getJobPlan())); + printWriter.close(); + } catch (IOException e) { + LOG.warn("Failed to write TEZ_PLAN to " + outFile.toString(), e); } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java index b8bdcc9a35..b20a4f6fc0 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java @@ -1648,7 +1648,10 @@ DAGState initializeDAG() { // This is going to override the previously generated file // which didn't have the priorities - Utils.generateDAGVizFile(this, jobPlan, dagScheduler); + if (getConf().getBoolean(TezConfiguration.TEZ_GENERATE_DEBUG_ARTIFACTS, + TezConfiguration.TEZ_GENERATE_DEBUG_ARTIFACTS_DEFAULT)) { + Utils.generateDAGVizFile(this, jobPlan, dagScheduler); + } return DAGState.INITED; } From c383ff5e7ceca9a1794072c35f0d602d0157ab5f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Fri, 17 Apr 2020 08:42:45 -0500 Subject: [PATCH 218/512] Checkstyle: '{' is followed by whitespace Signed-off-by: Jonathan Eagles --- tez-build-tools/src/main/resources/checkstyle/checkstyle.xml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tez-build-tools/src/main/resources/checkstyle/checkstyle.xml b/tez-build-tools/src/main/resources/checkstyle/checkstyle.xml index ef661c7a60..fd002ded72 100644 --- a/tez-build-tools/src/main/resources/checkstyle/checkstyle.xml +++ b/tez-build-tools/src/main/resources/checkstyle/checkstyle.xml @@ -135,7 +135,9 @@ - + + + From 049511708a6735201e19e3af4575dba3a51743eb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Fri, 17 Apr 2020 14:41:29 -0500 Subject: [PATCH 219/512] TEZ-4147. Reduce NN calls in RecoveryService::handleRecoveryEvent Signed-off-by: Jonathan Eagles --- .../dag/history/recovery/RecoveryService.java | 23 +++++++++---------- 1 file changed, 11 insertions(+), 12 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java index d874e0a6d1..a0a152c81c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java @@ -415,10 +415,10 @@ protected void handleSummaryEvent(TezDAGID dagID, if (LOG.isDebugEnabled()) { LOG.debug("AppId :" + appContext.getApplicationID() + " summaryPath " + summaryPath); } - if (!recoveryDirFS.exists(summaryPath)) { - summaryStream = recoveryDirFS.create(summaryPath, false, - bufferSize); - } else { + try { + summaryStream = recoveryDirFS.create(summaryPath, false, bufferSize); + } catch (IOException e) { + LOG.error("Error handling summary event, eventType=" + eventType, e); createFatalErrorFlagDir(); return; } @@ -456,16 +456,15 @@ protected void handleRecoveryEvent(DAGHistoryEvent event) throws IOException { RecoveryStream recoveryStream = outputStreamMap.get(dagID); if (recoveryStream == null) { Path dagFilePath = TezCommonUtils.getDAGRecoveryPath(recoveryPath, dagID.toString()); - if (recoveryDirFS.exists(dagFilePath)) { - createFatalErrorFlagDir(); - return; - } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Opening DAG recovery file in create mode" - + ", filePath=" + dagFilePath); - } + + try { FSDataOutputStream outputStream = recoveryDirFS.create(dagFilePath, false, bufferSize); + LOG.debug("Opened DAG recovery file in create mode, filePath={}", dagFilePath); recoveryStream = new RecoveryStream(outputStream); + } catch (IOException ioe) { + LOG.error("Error handling history event, eventType=" + eventType, ioe); + createFatalErrorFlagDir(); + return; } outputStreamMap.put(dagID, recoveryStream); } From e7c24f06e220cb707f114b4f5cc7210d27cce72d Mon Sep 17 00:00:00 2001 From: Syed Shameerur Rahman Date: Sun, 19 Apr 2020 10:12:09 +0200 Subject: [PATCH 220/512] =?UTF-8?q?TEZ-4140:=20Tez=20DAG=20Recovery:=20Dis?= =?UTF-8?q?crepancy=20In=20Scheduling=20Vertices=20During=20Vertex=20Recov?= =?UTF-8?q?ery=20(Syed=20Shameerur=20Rahman=20via=20L=C3=A1szl=C3=B3=20Bod?= =?UTF-8?q?or)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../apache/tez/dag/app/RecoveryParser.java | 5 +- .../tez/dag/app/dag/impl/VertexImpl.java | 22 +- .../tez/dag/app/dag/impl/TestDAGRecovery.java | 259 +++++++++++++++++- 3 files changed, 268 insertions(+), 18 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java index bab614259d..dfb7f61e6e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java @@ -74,7 +74,6 @@ import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.dag.recovery.records.RecoveryProtos; import org.apache.tez.dag.recovery.records.RecoveryProtos.SummaryEventProto; -import org.apache.tez.runtime.api.impl.TezEvent; import com.google.common.annotations.VisibleForTesting; import org.apache.tez.common.Preconditions; @@ -971,6 +970,10 @@ public boolean shouldSkipInit() { return vertexInitedEvent != null && vertexConfigurationDoneEvent != null; } + public boolean isVertexTasksStarted() { + return taskRecoveryDataMap != null && !taskRecoveryDataMap.isEmpty(); + } + public boolean isVertexStarted() { return vertexStartedEvent != null; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index f0a8642c1c..b67809e476 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -231,6 +231,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl private final AppContext appContext; private final DAG dag; private final VertexRecoveryData recoveryData; + private boolean isVertexInitSkipped = false; private List initGeneratedEvents = new ArrayList(); // set it to be true when setParallelism is called(used for recovery) private boolean setParallelismCalledFlag = false; @@ -2804,6 +2805,15 @@ private VertexState setupVertex() { return VertexState.INITED; } + private boolean isVertexInitSkippedInParentVertices() { + for (Map.Entry entry : sourceVertices.entrySet()) { + if(!(((VertexImpl) entry.getKey()).isVertexInitSkipped())) { + return false; + } + } + return true; + } + private void assignVertexManager() throws TezException { // condition for skip initializing stage // - VertexInputInitializerEvent is seen @@ -2816,8 +2826,10 @@ private void assignVertexManager() throws TezException { // - Why using VertexReconfigureDoneEvent // - VertexReconfigureDoneEvent represent the case that user use API reconfigureVertex // VertexReconfigureDoneEvent will be logged - if (recoveryData != null - && recoveryData.shouldSkipInit()) { + // - TaskStartEvent is seen in that vertex + // - All the parent vertices have skipped initializing stage while recovering + if (recoveryData != null && recoveryData.shouldSkipInit() + && recoveryData.isVertexTasksStarted() && isVertexInitSkippedInParentVertices()) { // Replace the original VertexManager with NoOpVertexManager if the reconfiguration is done in the last AM attempt VertexConfigurationDoneEvent reconfigureDoneEvent = recoveryData.getVertexConfigurationDoneEvent(); if (LOG.isInfoEnabled()) { @@ -2837,6 +2849,7 @@ private void assignVertexManager() throws TezException { VertexManagerPluginDescriptor.create(NoOpVertexManager.class.getName()) .setUserPayload(UserPayload.create(ByteBuffer.wrap(out.toByteArray()))), dagUgi, this, appContext, stateChangeNotifier); + isVertexInitSkipped = true; return; } @@ -4666,6 +4679,11 @@ VertexManager getVertexManager() { return this.vertexManager; } + public boolean isVertexInitSkipped() { + return isVertexInitSkipped; + } + + private static void logLocationHints(String vertexName, VertexLocationHint locationHint) { if (locationHint == null) { diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java index 260bd42f2c..fcf6db8d42 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java @@ -176,14 +176,23 @@ public class TestDAGRecovery { private TezVertexID v2Id; private TezTaskID t1v2Id; private TezTaskAttemptID ta1t1v2Id; + private TezVertexID v3Id; + private TezTaskID t1v3Id; + private TezTaskAttemptID ta1t1v3Id; //////////////////////// private Random rand = new Random(); private long dagInitedTime = System.currentTimeMillis() + rand.nextInt(100); private long dagStartedTime = dagInitedTime + rand.nextInt(100); private long v1InitedTime = dagStartedTime + rand.nextInt(100); + private long v2InitedTime = dagStartedTime + rand.nextInt(100); + private long v3InitedTime = Math.max(v1InitedTime, v2InitedTime) + rand.nextInt(100); private long v1StartedTime = v1InitedTime + rand.nextInt(100); + private long v2StartedTime = v2InitedTime + rand.nextInt(100); + private long v3StartedTime = v3InitedTime + rand.nextInt(100); private int v1NumTask = 10; + private int v2NumTask = 5; + private int v3NumTask = 2; private long t1StartedTime = v1StartedTime + rand.nextInt(100); private long t1FinishedTime = t1StartedTime + rand.nextInt(100); private long ta1LaunchTime = t1StartedTime + rand.nextInt(100); @@ -354,6 +363,9 @@ public ListenableFuture answer(InvocationOnMock invocation) { v2Id = TezVertexID.getInstance(dagId, 1); t1v2Id = TezTaskID.getInstance(v2Id, 0); ta1t1v2Id = TezTaskAttemptID.getInstance(t1v2Id, 0); + v3Id = TezVertexID.getInstance(dagId, 2); + t1v3Id = TezTaskID.getInstance(v3Id, 0); + ta1t1v3Id = TezTaskAttemptID.getInstance(t1v3Id, 0); dispatcher.register(CallableEventType.class, new CallableEventDispatcher()); taskEventDispatcher = new TaskEventDispatcher(); @@ -724,7 +736,7 @@ public void testVertexRecoverFromInited() { * DAG: DAGInitedEvent -> DAGStartedEvent * V1: VertexReconfigrationDoneEvent -> VertexInitializedEvent * - * V1 skip initialization. + * Reinitialize V1 again. */ @Test//(timeout=5000) public void testVertexRecoverFromInitedAndReconfigureDone() { @@ -751,35 +763,75 @@ public void testVertexRecoverFromInitedAndReconfigureDone() { VertexImpl v2 = (VertexImpl)dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl)dag.getVertex("vertex3"); assertEquals(DAGState.RUNNING, dag.getState()); - // v1 skip initialization - assertEquals(VertexState.RUNNING, v1.getState()); - assertEquals(v1InitedTime, v1.initedTime); - assertEquals(v1NumTask, v1.getTotalTasks()); + // reinitialize v1 + assertEquals(VertexState.INITIALIZING, v1.getState()); assertEquals(VertexState.RUNNING, v2.getState()); - assertEquals(VertexState.RUNNING, v3.getState()); + assertEquals(VertexState.INITED, v3.getState()); } - + /** * RecoveryEvents: * DAG: DAGInitedEvent -> DAGStartedEvent * V1: VertexReconfigrationDoneEvent -> VertexInitializedEvent -> VertexStartedEvent - * - * V1 skip initialization. + * + * Reinitialize V1 again. */ @Test(timeout=5000) public void testVertexRecoverFromStart() { - initMockDAGRecoveryDataForVertex(); + initMockDAGRecoveryDataForVertex(); List inputGeneratedTezEvents = new ArrayList(); - VertexInitializedEvent v1InitedEvent = new VertexInitializedEvent(v1Id, - "vertex1", 0L, v1InitedTime, + VertexInitializedEvent v1InitedEvent = new VertexInitializedEvent(v1Id, + "vertex1", 0L, v1InitedTime, v1NumTask, "", null, inputGeneratedTezEvents, null); - VertexConfigurationDoneEvent v1ReconfigureDoneEvent = new VertexConfigurationDoneEvent(v1Id, + VertexConfigurationDoneEvent v1ReconfigureDoneEvent = new VertexConfigurationDoneEvent(v1Id, 0L, v1NumTask, null, null, null, true); VertexStartedEvent v1StartedEvent = new VertexStartedEvent(v1Id, 0L, v1StartedTime); VertexRecoveryData vertexRecoveryData = new VertexRecoveryData(v1InitedEvent, v1ReconfigureDoneEvent, v1StartedEvent, null, new HashMap(), false); doReturn(vertexRecoveryData).when(dagRecoveryData).getVertexRecoveryData(v1Id); - + + DAGEventRecoverEvent recoveryEvent = new DAGEventRecoverEvent(dagId, dagRecoveryData); + dag.handle(recoveryEvent); + dispatcher.await(); + + VertexImpl v1 = (VertexImpl)dag.getVertex("vertex1"); + VertexImpl v2 = (VertexImpl)dag.getVertex("vertex2"); + VertexImpl v3 = (VertexImpl)dag.getVertex("vertex3"); + assertEquals(DAGState.RUNNING, dag.getState()); + // reinitialize v1 + assertEquals(VertexState.INITIALIZING, v1.getState()); + assertEquals(VertexState.RUNNING, v2.getState()); + assertEquals(VertexState.INITED, v3.getState()); + } + + /** + * RecoveryEvents: + * DAG: DAGInitedEvent -> DAGStartedEvent + * V1: VertexReconfigrationDoneEvent -> VertexInitializedEvent -> VertexStartedEvent -> VertexTaskStart + * + * V1 skip initialization. + */ + @Test(timeout=5000) + public void testVertexRecoverFromVertexTaskStart() { + initMockDAGRecoveryDataForVertex(); + List inputGeneratedTezEvents = new ArrayList(); + VertexInitializedEvent v1InitedEvent = new VertexInitializedEvent(v1Id, + "vertex1", 0L, v1InitedTime, + v1NumTask, "", null, inputGeneratedTezEvents, null); + VertexConfigurationDoneEvent v1ReconfigureDoneEvent = new VertexConfigurationDoneEvent(v1Id, + 0L, v1NumTask, null, null, null, true); + VertexStartedEvent v1StartedEvent = new VertexStartedEvent(v1Id, 0L, v1StartedTime); + + TaskStartedEvent taskStartedEvent = new TaskStartedEvent(t1v1Id, "v1", 0L, 0L); + TaskRecoveryData taskRecoveryData = new TaskRecoveryData(taskStartedEvent, null, null); + Map taskRecoveryDataMap = new HashMap<>(); + // put dummy tasks + taskRecoveryDataMap.put(t1v2Id, taskRecoveryData); + + VertexRecoveryData vertexRecoveryData = new VertexRecoveryData(v1InitedEvent, + v1ReconfigureDoneEvent, v1StartedEvent, null, taskRecoveryDataMap, false); + doReturn(vertexRecoveryData).when(dagRecoveryData).getVertexRecoveryData(v1Id); + DAGEventRecoverEvent recoveryEvent = new DAGEventRecoverEvent(dagId, dagRecoveryData); dag.handle(recoveryEvent); dispatcher.await(); @@ -796,6 +848,178 @@ public void testVertexRecoverFromStart() { assertEquals(VertexState.RUNNING, v2.getState()); assertEquals(VertexState.RUNNING, v3.getState()); } + + /** + * RecoveryEvents: + * DAG: DAGInitedEvent -> DAGStartedEvent + * V1: VertexReconfigrationDoneEvent -> VertexInitializedEvent -> VertexStartedEvent -> VertexTaskStart + * V2: VertexReconfigrationDoneEvent -> VertexInitializedEvent -> VertexStartedEvent -> VertexTaskStart + * V3: VertexReconfigrationDoneEvent -> VertexInitializedEvent -> VertexStartedEvent -> VertexTaskStart + * + * V1 skip initialization. + * V2 skip initialization. + * V3 skip initialization. + */ + @Test(timeout=5000) + public void testMultipleVertexRecoverFromVertexTaskStart() { + initMockDAGRecoveryDataForVertex(); + List inputGeneratedTezEvents = new ArrayList(); + VertexInitializedEvent v1InitedEvent = new VertexInitializedEvent(v1Id, + "vertex1", 0L, v1InitedTime, + v1NumTask, "", null, inputGeneratedTezEvents, null); + VertexInitializedEvent v2InitedEvent = new VertexInitializedEvent(v2Id, + "vertex2", 0L, v2InitedTime, + v2NumTask, "", null, inputGeneratedTezEvents, null); + VertexInitializedEvent v3InitedEvent = new VertexInitializedEvent(v3Id, + "vertex3", 0L, v3InitedTime, + v3NumTask, "", null, inputGeneratedTezEvents, null); + + VertexConfigurationDoneEvent v1ReconfigureDoneEvent = new VertexConfigurationDoneEvent(v1Id, + 0L, v1NumTask, null, null, null, true); + VertexConfigurationDoneEvent v2ReconfigureDoneEvent = new VertexConfigurationDoneEvent(v2Id, + 0L, v2NumTask, null, null, null, true); + VertexConfigurationDoneEvent v3ReconfigureDoneEvent = new VertexConfigurationDoneEvent(v3Id, + 0L, v3NumTask, null, null, null, true); + + VertexStartedEvent v1StartedEvent = new VertexStartedEvent(v1Id, 0L, v1StartedTime); + VertexStartedEvent v2StartedEvent = new VertexStartedEvent(v2Id, 0L, v2StartedTime); + VertexStartedEvent v3StartedEvent = new VertexStartedEvent(v3Id, 0L, v3StartedTime); + + TaskStartedEvent v1taskStartedEvent = new TaskStartedEvent(t1v1Id, "vertex1", 0L, 0L); + TaskRecoveryData v1taskRecoveryData = new TaskRecoveryData(v1taskStartedEvent, null, null); + Map v1taskRecoveryDataMap = new HashMap<>(); + // put dummy tasks + v1taskRecoveryDataMap.put(t1v1Id, v1taskRecoveryData); + + TaskStartedEvent v2taskStartedEvent = new TaskStartedEvent(t1v2Id, "vertex2", 0L, 0L); + TaskRecoveryData v2taskRecoveryData = new TaskRecoveryData(v2taskStartedEvent, null, null); + Map v2taskRecoveryDataMap = new HashMap<>(); + // put dummy tasks + v2taskRecoveryDataMap.put(t1v2Id, v2taskRecoveryData); + + TaskStartedEvent v3taskStartedEvent = new TaskStartedEvent(t1v3Id, "vertex3", 0L, 0L); + TaskRecoveryData v3taskRecoveryData = new TaskRecoveryData(v3taskStartedEvent, null, null); + Map v3taskRecoveryDataMap = new HashMap<>(); + // put dummy tasks + v3taskRecoveryDataMap.put(t1v3Id, v3taskRecoveryData); + + VertexRecoveryData vertex1RecoveryData = new VertexRecoveryData(v1InitedEvent, + v1ReconfigureDoneEvent, v1StartedEvent, null, v1taskRecoveryDataMap, false); + VertexRecoveryData vertex2RecoveryData = new VertexRecoveryData(v2InitedEvent, + v2ReconfigureDoneEvent, v2StartedEvent, null, v2taskRecoveryDataMap, false); + VertexRecoveryData vertex3RecoveryData = new VertexRecoveryData(v3InitedEvent, + v3ReconfigureDoneEvent, v3StartedEvent, null, v3taskRecoveryDataMap, false); + + doReturn(vertex1RecoveryData).when(dagRecoveryData).getVertexRecoveryData(v1Id); + doReturn(vertex2RecoveryData).when(dagRecoveryData).getVertexRecoveryData(v2Id); + doReturn(vertex3RecoveryData).when(dagRecoveryData).getVertexRecoveryData(v3Id); + + DAGEventRecoverEvent recoveryEvent = new DAGEventRecoverEvent(dagId, dagRecoveryData); + dag.handle(recoveryEvent); + dispatcher.await(); + + VertexImpl v1 = (VertexImpl)dag.getVertex("vertex1"); + VertexImpl v2 = (VertexImpl)dag.getVertex("vertex2"); + VertexImpl v3 = (VertexImpl)dag.getVertex("vertex3"); + + assertEquals(DAGState.RUNNING, dag.getState()); + + // v1 skip initialization + assertEquals(VertexState.RUNNING, v1.getState()); + assertEquals(v1InitedTime, v1.initedTime); + assertEquals(v1StartedTime, v1.startedTime); + assertEquals(v1NumTask, v1.getTotalTasks()); + + // v2 skip initialization + assertEquals(VertexState.RUNNING, v2.getState()); + assertEquals(v2InitedTime, v2.initedTime); + assertEquals(v2StartedTime, v2.startedTime); + assertEquals(v2NumTask, v2.getTotalTasks()); + + // v3 skip initialization + assertEquals(VertexState.RUNNING, v3.getState()); + assertEquals(v3InitedTime, v3.initedTime); + assertEquals(v3StartedTime, v3.startedTime); + assertEquals(v3NumTask, v3.getTotalTasks()); + } + + /** + * RecoveryEvents: + * DAG: DAGInitedEvent -> DAGStartedEvent + * V1: VertexReconfigrationDoneEvent -> VertexInitializedEvent + * V2: VertexReconfigrationDoneEvent -> VertexInitializedEvent -> VertexStartedEvent -> VertexTaskStart + * V3: VertexReconfigrationDoneEvent -> VertexInitializedEvent -> VertexStartedEvent -> VertexTaskStart + * + * Reinitialize V1 again. + * V2 skip initialization. + * Reinitialize V3 again. Since V3 is dependent on V1 + */ + @Test(timeout=5000) + public void testMultipleVertexRecoverFromVertex() { + initMockDAGRecoveryDataForVertex(); + List inputGeneratedTezEvents = new ArrayList(); + VertexInitializedEvent v1InitedEvent = new VertexInitializedEvent(v1Id, + "vertex1", 0L, v1InitedTime, + v1NumTask, "", null, inputGeneratedTezEvents, null); + VertexInitializedEvent v2InitedEvent = new VertexInitializedEvent(v2Id, + "vertex2", 0L, v2InitedTime, + v2NumTask, "", null, inputGeneratedTezEvents, null); + VertexInitializedEvent v3InitedEvent = new VertexInitializedEvent(v3Id, + "vertex3", 0L, v3InitedTime, + v3NumTask, "", null, inputGeneratedTezEvents, null); + + VertexConfigurationDoneEvent v2ReconfigureDoneEvent = new VertexConfigurationDoneEvent(v2Id, + 0L, v2NumTask, null, null, null, true); + VertexConfigurationDoneEvent v3ReconfigureDoneEvent = new VertexConfigurationDoneEvent(v3Id, + 0L, v3NumTask, null, null, null, true); + + VertexStartedEvent v2StartedEvent = new VertexStartedEvent(v2Id, 0L, v2StartedTime); + VertexStartedEvent v3StartedEvent = new VertexStartedEvent(v3Id, 0L, v3StartedTime); + + TaskStartedEvent v2taskStartedEvent = new TaskStartedEvent(t1v2Id, "vertex2", 0L, 0L); + TaskRecoveryData v2taskRecoveryData = new TaskRecoveryData(v2taskStartedEvent, null, null); + Map v2taskRecoveryDataMap = new HashMap<>(); + // put dummy tasks + v2taskRecoveryDataMap.put(t1v2Id, v2taskRecoveryData); + + TaskStartedEvent v3taskStartedEvent = new TaskStartedEvent(t1v3Id, "vertex3", 0L, 0L); + TaskRecoveryData v3taskRecoveryData = new TaskRecoveryData(v3taskStartedEvent, null, null); + Map v3taskRecoveryDataMap = new HashMap<>(); + // put dummy tasks + v3taskRecoveryDataMap.put(t1v3Id, v3taskRecoveryData); + + VertexRecoveryData vertex1RecoveryData = new VertexRecoveryData(v1InitedEvent, + null, null, null, null, false); + VertexRecoveryData vertex2RecoveryData = new VertexRecoveryData(v2InitedEvent, + v2ReconfigureDoneEvent, v2StartedEvent, null, v2taskRecoveryDataMap, false); + VertexRecoveryData vertex3RecoveryData = new VertexRecoveryData(v3InitedEvent, + v3ReconfigureDoneEvent, v3StartedEvent, null, v3taskRecoveryDataMap, false); + + doReturn(vertex1RecoveryData).when(dagRecoveryData).getVertexRecoveryData(v1Id); + doReturn(vertex2RecoveryData).when(dagRecoveryData).getVertexRecoveryData(v2Id); + doReturn(vertex3RecoveryData).when(dagRecoveryData).getVertexRecoveryData(v3Id); + + DAGEventRecoverEvent recoveryEvent = new DAGEventRecoverEvent(dagId, dagRecoveryData); + dag.handle(recoveryEvent); + dispatcher.await(); + + VertexImpl v1 = (VertexImpl)dag.getVertex("vertex1"); + VertexImpl v2 = (VertexImpl)dag.getVertex("vertex2"); + VertexImpl v3 = (VertexImpl)dag.getVertex("vertex3"); + assertEquals(DAGState.RUNNING, dag.getState()); + + // reinitialize v1 + assertEquals(VertexState.INITIALIZING, v1.getState()); + + // v2 skip initialization + assertEquals(VertexState.RUNNING, v2.getState()); + assertEquals(v2InitedTime, v2.initedTime); + assertEquals(v2StartedTime, v2.startedTime); + assertEquals(v2NumTask, v2.getTotalTasks()); + + // reinitialize v3 + assertEquals(VertexState.INITED, v3.getState()); + } /////////////////////////////// Task //////////////////////////////////////////////////////////// @@ -808,8 +1032,13 @@ private void initMockDAGRecoveryDataForTask() { VertexConfigurationDoneEvent v1ReconfigureDoneEvent = new VertexConfigurationDoneEvent(v1Id, 0L, v1NumTask, null, null, rootInputSpecs, true); VertexStartedEvent v1StartedEvent = new VertexStartedEvent(v1Id, 0L, v1StartedTime); + TaskStartedEvent v1taskStartedEvent = new TaskStartedEvent(t1v1Id, "vertex1", 0L, 0L); + TaskRecoveryData v1taskRecoveryData = new TaskRecoveryData(v1taskStartedEvent, null, null); + Map v1taskRecoveryDataMap = new HashMap<>(); + // put dummy tasks + v1taskRecoveryDataMap.put(t1v1Id, v1taskRecoveryData); VertexRecoveryData v1RecoveryData = new VertexRecoveryData(v1InitedEvent, - v1ReconfigureDoneEvent, v1StartedEvent, null, new HashMap(), false); + v1ReconfigureDoneEvent, v1StartedEvent, null, v1taskRecoveryDataMap, false); DAGInitializedEvent dagInitedEvent = new DAGInitializedEvent(dagId, dagInitedTime, "user", "dagName", null); From 24217e1432cafc9278863326f53901e982146227 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Tue, 21 Apr 2020 08:40:09 -0500 Subject: [PATCH 221/512] TEZ-4151. Missing apache commons collections4 dependency on tez-mapreduce and tez-plugins/tez-history-parser (jeagles) --- tez-mapreduce/pom.xml | 4 ++++ tez-plugins/tez-history-parser/pom.xml | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/tez-mapreduce/pom.xml b/tez-mapreduce/pom.xml index b1822cd425..a01d028039 100644 --- a/tez-mapreduce/pom.xml +++ b/tez-mapreduce/pom.xml @@ -116,6 +116,10 @@ junit test + + org.apache.commons + commons-collections4 + com.google.protobuf protobuf-java diff --git a/tez-plugins/tez-history-parser/pom.xml b/tez-plugins/tez-history-parser/pom.xml index 3804253857..7937fa4e50 100644 --- a/tez-plugins/tez-history-parser/pom.xml +++ b/tez-plugins/tez-history-parser/pom.xml @@ -131,6 +131,10 @@ mockito-all test + + org.apache.commons + commons-collections4 + junit junit From 3f8e753778bddfcb78e57011342cfa4b79741019 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Wed, 22 Apr 2020 09:27:06 +0200 Subject: [PATCH 222/512] =?UTF-8?q?TEZ-4136:=20String=20representation=20f?= =?UTF-8?q?or=20tez=20counters=20(L=C3=A1szl=C3=B3=20Bodor=20reviewed=20by?= =?UTF-8?q?=20Jonathan=20Turner=20Eagles)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../java/org/apache/tez/common/counters/AbstractCounter.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounter.java index 7bc1109518..befc7c023e 100644 --- a/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounter.java +++ b/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounter.java @@ -49,4 +49,9 @@ public synchronized boolean equals(Object genericRight) { public synchronized int hashCode() { return Objects.hashCode(getName(), getDisplayName(), getValue()); } + + @Override + public String toString() { + return "[" + getClass().getSimpleName() + "]: " + getDisplayName() + "=" + getValue(); + } } From 117515c5c8e85455130d9ed803c3b8f56e57571f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Tue, 28 Apr 2020 17:32:04 +0200 Subject: [PATCH 223/512] =?UTF-8?q?TEZ-4155:=20Remove=20sync=20bottleneck?= =?UTF-8?q?=20in=20counters=20(L=C3=A1szl=C3=B3=20Bodor=20reviewed=20by=20?= =?UTF-8?q?Rajesh=20Balamohan)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../apache/tez/common/counters/Limits.java | 52 +++++++------------ 1 file changed, 18 insertions(+), 34 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/Limits.java b/tez-api/src/main/java/org/apache/tez/common/counters/Limits.java index 332df8e1e0..13cc7fbdbf 100644 --- a/tez-api/src/main/java/org/apache/tez/common/counters/Limits.java +++ b/tez-api/src/main/java/org/apache/tez/common/counters/Limits.java @@ -35,28 +35,27 @@ public class Limits { private static int COUNTER_NAME_MAX; private static int GROUPS_MAX; private static int COUNTERS_MAX; - private static boolean initialized = false; - private static synchronized void ensureInitialized() { - if (initialized) { - return; - } - if (conf == null) { - conf = new TezConfiguration(); + static { + init(new TezConfiguration()); + } + + public synchronized static void setConfiguration(Configuration conf) { + if (Limits.conf == null && conf != null) { + init(conf); } - GROUP_NAME_MAX = - conf.getInt(TezConfiguration.TEZ_COUNTERS_GROUP_NAME_MAX_LENGTH, - TezConfiguration.TEZ_COUNTERS_GROUP_NAME_MAX_LENGTH_DEFAULT); - COUNTER_NAME_MAX = - conf.getInt(TezConfiguration.TEZ_COUNTERS_COUNTER_NAME_MAX_LENGTH, - TezConfiguration.TEZ_COUNTERS_COUNTER_NAME_MAX_LENGTH_DEFAULT); - GROUPS_MAX = - conf.getInt(TezConfiguration.TEZ_COUNTERS_MAX_GROUPS, - TezConfiguration.TEZ_COUNTERS_MAX_GROUPS_DEFAULT); + } + + private static void init(Configuration conf) { + Limits.conf = conf; + GROUP_NAME_MAX = conf.getInt(TezConfiguration.TEZ_COUNTERS_GROUP_NAME_MAX_LENGTH, + TezConfiguration.TEZ_COUNTERS_GROUP_NAME_MAX_LENGTH_DEFAULT); + COUNTER_NAME_MAX = conf.getInt(TezConfiguration.TEZ_COUNTERS_COUNTER_NAME_MAX_LENGTH, + TezConfiguration.TEZ_COUNTERS_COUNTER_NAME_MAX_LENGTH_DEFAULT); + GROUPS_MAX = conf.getInt(TezConfiguration.TEZ_COUNTERS_MAX_GROUPS, + TezConfiguration.TEZ_COUNTERS_MAX_GROUPS_DEFAULT); COUNTERS_MAX = - conf.getInt(TezConfiguration.TEZ_COUNTERS_MAX, - TezConfiguration.TEZ_COUNTERS_MAX_DEFAULT); - initialized = true; + conf.getInt(TezConfiguration.TEZ_COUNTERS_MAX, TezConfiguration.TEZ_COUNTERS_MAX_DEFAULT); LOG.info("Counter limits initialized with parameters: " + " GROUP_NAME_MAX=" + GROUP_NAME_MAX + ", MAX_GROUPS=" + GROUPS_MAX + ", COUNTER_NAME_MAX=" + COUNTER_NAME_MAX + ", MAX_COUNTERS=" + COUNTERS_MAX); @@ -70,17 +69,14 @@ public static String filterName(String name, int maxLen) { } public static String filterCounterName(String name) { - ensureInitialized(); return filterName(name, COUNTER_NAME_MAX); } public static String filterGroupName(String name) { - ensureInitialized(); return filterName(name, GROUP_NAME_MAX); } public synchronized void checkCounters(int size) { - ensureInitialized(); if (firstViolation != null) { throw new LimitExceededException(firstViolation); } @@ -97,7 +93,6 @@ public synchronized void incrCounters() { } public synchronized void checkGroups(int size) { - ensureInitialized(); if (firstViolation != null) { throw new LimitExceededException(firstViolation); } @@ -107,21 +102,10 @@ public synchronized void checkGroups(int size) { } } - public synchronized LimitExceededException violation() { - return firstViolation; - } - - public synchronized static void setConfiguration(Configuration conf) { - if (Limits.conf == null && conf != null) { - Limits.conf = conf; - } - } - @VisibleForTesting @InterfaceAudience.Private public synchronized static void reset() { conf = null; - initialized = false; } } From 184c90c0bec9c8b788be717fe09952c9d574014e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Tue, 28 Apr 2020 17:39:49 +0200 Subject: [PATCH 224/512] =?UTF-8?q?TEZ-4158:=20Change=20to=20a=20maintaine?= =?UTF-8?q?d=20bouncy=20castle=20version=20(L=C3=A1szl=C3=B3=20Bodor=20rev?= =?UTF-8?q?iewed=20by=20Ashutosh=20Chauhan,=20Jonathan=20Turner=20Eagles)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- pom.xml | 4 ++-- tez-tests/pom.xml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index cd4cf7ccd3..7de532221e 100644 --- a/pom.xml +++ b/pom.xml @@ -704,8 +704,8 @@ org.bouncycastle - bcprov-jdk16 - 1.46 + bcprov-jdk15on + 1.65 test diff --git a/tez-tests/pom.xml b/tez-tests/pom.xml index c744e6892d..d76f27a1a9 100644 --- a/tez-tests/pom.xml +++ b/tez-tests/pom.xml @@ -126,7 +126,7 @@ org.bouncycastle - bcprov-jdk16 + bcprov-jdk15on test From af3a9692ee4a4689464b82db3a78c70293631dce Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Wed, 29 Apr 2020 03:43:49 +0530 Subject: [PATCH 225/512] =?UTF-8?q?TEZ-4156:=20Fix=20Tez=20to=20reuse=20IP?= =?UTF-8?q?C=20connections=20(Rajesh=20Balamohan,=20reviewed=20by=20Siddha?= =?UTF-8?q?rth=20Seth,=20L=C3=A1szl=C3=B3=20Bodor,=20Jonathan=20Turner=20E?= =?UTF-8?q?agles)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/org/apache/tez/client/TezClient.java | 24 ++++++++++++++----- .../org/apache/tez/client/TezClientUtils.java | 9 ++++--- .../tez/dag/api/client/DAGClientImpl.java | 6 +++-- .../dag/api/client/rpc/DAGClientRPCImpl.java | 8 +++++-- .../tez/dag/api/client/rpc/TestDAGClient.java | 16 +++++++------ 5 files changed, 41 insertions(+), 22 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java index 60c0e5e16f..4fb37dce9a 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java @@ -37,6 +37,7 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.tez.common.JavaOptsChecker; import org.apache.tez.common.RPCUtil; import org.apache.tez.common.TezCommonUtils; @@ -154,6 +155,8 @@ public class TezClient { private ScheduledExecutorService amKeepAliveService; + private final Map ugiMap; + private TezClient(String name, TezConfiguration tezConf) { this(name, tezConf, tezConf.getBoolean( TezConfiguration.TEZ_AM_SESSION_MODE, TezConfiguration.TEZ_AM_SESSION_MODE_DEFAULT)); @@ -197,6 +200,7 @@ protected TezClient(String name, TezConfiguration tezConf, boolean isSession, LOG.warn("The host name of the client the tez application was submitted from was unable to be retrieved", e); } + this.ugiMap = new HashMap<>(); this.amConfig = new AMConfiguration(tezConf, localResources, credentials); this.apiVersionInfo = new TezApiVersionInfo(); this.servicePluginsDescriptor = servicePluginsDescriptor; @@ -713,7 +717,13 @@ private DAGClient submitDAGSession(DAG dag) throws TezException, IOException { return new DAGClientImpl(sessionAppId, dagId, amConfig.getTezConfiguration(), amConfig.getYarnConfiguration(), - frameworkClient); + frameworkClient, getUgi()); + } + + private UserGroupInformation getUgi() throws IOException { + String userName = UserGroupInformation.getCurrentUser().getUserName(); + return ugiMap.computeIfAbsent(userName, + v -> UserGroupInformation.createRemoteUser(userName)); } @VisibleForTesting @@ -1058,7 +1068,7 @@ protected FrameworkClient createFrameworkClient() { protected DAGClientAMProtocolBlockingPB getAMProxy(ApplicationId appId) throws TezException, IOException { return TezClientUtils.getAMProxy( - frameworkClient, amConfig.getYarnConfiguration(), appId); + frameworkClient, amConfig.getYarnConfiguration(), appId, getUgi()); } private DAGClientAMProtocolBlockingPB waitForProxy() @@ -1137,7 +1147,7 @@ DAGClient submitDAGApplication(ApplicationId appId, DAG dag) // wait for dag in non-session mode to start running, so that we can start to getDAGStatus waitNonSessionTillReady(); return getDAGClient(appId, amConfig.getTezConfiguration(), amConfig.getYarnConfiguration(), - frameworkClient); + frameworkClient, getUgi()); } private ApplicationId createApplication() throws TezException, IOException { @@ -1161,17 +1171,19 @@ private synchronized Map getTezJarResources(Credentials c @Private static DAGClient getDAGClient(ApplicationId appId, TezConfiguration tezConf, YarnConfiguration - yarnConf, FrameworkClient frameworkClient) + yarnConf, FrameworkClient frameworkClient, UserGroupInformation ugi) throws IOException, TezException { return new DAGClientImpl(appId, getDefaultTezDAGID(appId), tezConf, - yarnConf, frameworkClient); + yarnConf, frameworkClient, ugi); } @Private // Used only for MapReduce compatibility code static DAGClient getDAGClient(ApplicationId appId, TezConfiguration tezConf, FrameworkClient frameworkClient) throws IOException, TezException { - return getDAGClient(appId, tezConf, new YarnConfiguration(tezConf), frameworkClient); + UserGroupInformation ugi = + UserGroupInformation.createRemoteUser(UserGroupInformation.getCurrentUser().getUserName()); + return getDAGClient(appId, tezConf, new YarnConfiguration(tezConf), frameworkClient, ugi); } // DO NOT CHANGE THIS. This code is replicated from TezDAGID.java diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index 8850ca74ca..d34d31e4cf 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -919,7 +919,7 @@ private static Path localizeDagPlanAsText(DAGPlan dagPB, FileSystem fs, AMConfig static DAGClientAMProtocolBlockingPB getAMProxy(FrameworkClient yarnClient, Configuration conf, - ApplicationId applicationId) throws TezException, IOException { + ApplicationId applicationId, UserGroupInformation ugi) throws TezException, IOException { ApplicationReport appReport; try { appReport = yarnClient.getApplicationReport( @@ -954,16 +954,15 @@ static DAGClientAMProtocolBlockingPB getAMProxy(FrameworkClient yarnClient, throw new TezException(e); } return getAMProxy(conf, appReport.getHost(), - appReport.getRpcPort(), appReport.getClientToAMToken()); + appReport.getRpcPort(), appReport.getClientToAMToken(), ugi); } @Private public static DAGClientAMProtocolBlockingPB getAMProxy(final Configuration conf, String amHost, - int amRpcPort, org.apache.hadoop.yarn.api.records.Token clientToAMToken) throws IOException { + int amRpcPort, org.apache.hadoop.yarn.api.records.Token clientToAMToken, + UserGroupInformation userUgi) throws IOException { final InetSocketAddress serviceAddr = NetUtils.createSocketAddrForHost(amHost, amRpcPort); - UserGroupInformation userUgi = UserGroupInformation.createRemoteUser(UserGroupInformation - .getCurrentUser().getUserName()); if (clientToAMToken != null) { Token token = ConverterUtils.convertFromYarn(clientToAMToken, serviceAddr); diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java index 52f66e3c63..0b899fd27d 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java @@ -29,6 +29,7 @@ import java.util.Set; import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.tez.common.Preconditions; import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException; @@ -79,7 +80,8 @@ public class DAGClientImpl extends DAGClient { private boolean cleanupFrameworkClient; public DAGClientImpl(ApplicationId appId, String dagId, TezConfiguration conf, - YarnConfiguration yarnConf, @Nullable FrameworkClient frameworkClient) { + YarnConfiguration yarnConf, @Nullable FrameworkClient frameworkClient, + UserGroupInformation ugi) { this.appId = appId; this.dagId = dagId; this.conf = conf; @@ -99,7 +101,7 @@ public DAGClientImpl(ApplicationId appId, String dagId, TezConfiguration conf, TezConfiguration.TEZ_AM_HISTORY_LOGGING_ENABLED_DEFAULT) && DAGClientTimelineImpl.isSupported(); - realClient = new DAGClientRPCImpl(appId, dagId, conf, this.frameworkClient); + realClient = new DAGClientRPCImpl(appId, dagId, conf, this.frameworkClient, ugi); statusPollInterval = conf.getLong( TezConfiguration.TEZ_DAG_STATUS_POLLINTERVAL_MS, TezConfiguration.TEZ_DAG_STATUS_POLLINTERVAL_MS_DEFAULT); diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java index 02935dfd20..c54058be8a 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java @@ -23,6 +23,7 @@ import javax.annotation.Nullable; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException; import org.apache.tez.common.RPCUtil; import org.apache.tez.dag.api.SessionNotRunning; @@ -68,13 +69,16 @@ public class DAGClientRPCImpl extends DAGClientInternal { @VisibleForTesting DAGClientAMProtocolBlockingPB proxy = null; + private UserGroupInformation ugi; + public DAGClientRPCImpl(ApplicationId appId, String dagId, - TezConfiguration conf, @Nullable FrameworkClient frameworkClient) { + TezConfiguration conf, @Nullable FrameworkClient frameworkClient, UserGroupInformation ugi) { this.appId = appId; this.dagId = dagId; this.conf = conf; this.frameworkClient = frameworkClient; appReport = null; + this.ugi = ugi; } @Override @@ -286,7 +290,7 @@ boolean createAMProxyIfNeeded() throws IOException, TezException, } proxy = TezClientUtils.getAMProxy(conf, appReport.getHost(), appReport.getRpcPort(), - appReport.getClientToAMToken()); + appReport.getClientToAMToken(), ugi); return true; } diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java index 70ee1d495d..57087cb7b4 100644 --- a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java +++ b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java @@ -32,6 +32,7 @@ import java.util.EnumSet; import java.util.Set; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.conf.YarnConfiguration; @@ -204,7 +205,8 @@ public void setUp() throws YarnException, IOException, TezException, ServiceExce TezConfiguration tezConf = new TezConfiguration(); YarnConfiguration yarnConf = new YarnConfiguration(tezConf); - dagClient = new DAGClientImpl(mockAppId, dagIdStr, tezConf, yarnConf, null); + dagClient = new DAGClientImpl(mockAppId, dagIdStr, tezConf, yarnConf, null, + UserGroupInformation.getCurrentUser()); DAGClientRPCImpl realClient = (DAGClientRPCImpl)((DAGClientImpl)dagClient).getRealClient(); realClient.appReport = mockAppReport; realClient.proxy = mockProxy; @@ -410,7 +412,7 @@ public void testGetDagStatusWithTimeout() throws Exception { } @Test(timeout = 5000) - public void testDagClientTimelineEnabledCondition() { + public void testDagClientTimelineEnabledCondition() throws IOException { String historyLoggingClass = "org.apache.tez.dag.history.logging.ats.ATSHistoryLoggingService"; testAtsEnabled(mockAppId, dagIdStr, false, "", true, true); @@ -422,7 +424,7 @@ public void testDagClientTimelineEnabledCondition() { private static void testAtsEnabled(ApplicationId appId, String dagIdStr, boolean expected, String loggingClass, boolean amHistoryLoggingEnabled, - boolean dagHistoryLoggingEnabled) { + boolean dagHistoryLoggingEnabled) throws IOException { TezConfiguration tezConf = new TezConfiguration(); YarnConfiguration yarnConf = new YarnConfiguration(tezConf); @@ -441,8 +443,8 @@ private static class DAGClientRPCImplForTest extends DAGClientRPCImpl { public DAGClientRPCImplForTest(ApplicationId appId, String dagId, TezConfiguration conf, - @Nullable FrameworkClient frameworkClient) { - super(appId, dagId, conf, frameworkClient); + @Nullable FrameworkClient frameworkClient) throws IOException { + super(appId, dagId, conf, frameworkClient, UserGroupInformation.getCurrentUser()); } void setAMProxy(DAGClientAMProtocolBlockingPB proxy) { @@ -477,8 +479,8 @@ private static class DAGClientImplForTest extends DAGClientImpl { public DAGClientImplForTest(ApplicationId appId, String dagId, TezConfiguration conf, YarnConfiguration yarnConf, - @Nullable FrameworkClient frameworkClient) { - super(appId, dagId, conf, yarnConf, frameworkClient); + @Nullable FrameworkClient frameworkClient) throws IOException { + super(appId, dagId, conf, yarnConf, frameworkClient, UserGroupInformation.getCurrentUser()); } private void setRealClient(DAGClientRPCImplForTest dagClientRpcImplForTest) { From 41f6f3b587b00815c9abe016e617ef194326e58d Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Thu, 30 Apr 2020 04:44:23 -0500 Subject: [PATCH 226/512] TEZ-4161. Speed up TestTezUtils --- .../java/org/apache/tez/common/TestTezUtils.java | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java b/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java index c0c3e5d3a8..f9008b9947 100644 --- a/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java +++ b/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java @@ -57,16 +57,20 @@ public void testByteStringToAndFromConf() throws IOException { @Test (timeout=20000) public void testByteStringToAndFromLargeConf() throws IOException { Configuration conf = getConf(); - int largeSize = 64 * 1024 * 1024; + int largeSizeMinimum = 64 * 1024 * 1024; final String alphaString = "ABCDEFGHIJKLMNOPQRSTUVWXYZ"; + int largeSize = (largeSizeMinimum + alphaString.length() - 1) / alphaString.length(); + + largeSize *= alphaString.length(); + assertTrue(largeSize >= alphaString.length()); StringBuilder sb = new StringBuilder(largeSize); - for (int i = 0; i < largeSize; i++) { - int index = (int)(alphaString.length() * Math.random()); - sb.append(alphaString.charAt(index)); + while (sb.length() < largeSize) { + sb.append(alphaString); } String largeValue = sb.toString(); + Assert.assertEquals(largeSize, largeValue.length()); conf.set("testLargeValue", largeValue); Assert.assertEquals(conf.size(), 7); ByteString bsConf = TezUtils.createByteStringFromConf(conf); From 5dafc70530155453701fdfe5d8d8f65687accc62 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Thu, 30 Apr 2020 04:49:46 -0500 Subject: [PATCH 227/512] TEZ-4162. Speed up TestInputReadyTracker --- .../test/java/org/apache/tez/runtime/TestInputReadyTracker.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/TestInputReadyTracker.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/TestInputReadyTracker.java index 18463541fa..e10c83f9a5 100644 --- a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/TestInputReadyTracker.java +++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/TestInputReadyTracker.java @@ -44,7 +44,7 @@ public class TestInputReadyTracker { - private static final long SLEEP_TIME = 2000l; + private static final long SLEEP_TIME = 200l; @Test(timeout = 20000) public void testWithoutGrouping1() throws InterruptedException { From 5373ca8ff06cc315c63ac66448daaa31c9b5882e Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Thu, 30 Apr 2020 04:52:36 -0500 Subject: [PATCH 228/512] TEZ-4163. Speed up TestTaskReporter --- .../org/apache/tez/runtime/task/TestTaskReporter.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskReporter.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskReporter.java index 04c467a8fc..9f514f405b 100644 --- a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskReporter.java +++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskReporter.java @@ -60,6 +60,7 @@ public void testContinuousHeartbeatsOnMaxEvents() throws Exception { final Object lock = new Object(); final AtomicBoolean hb2Done = new AtomicBoolean(false); + final int maxEvents = 5; TezTaskUmbilicalProtocol mockUmbilical = mock(TezTaskUmbilicalProtocol.class); doAnswer(new Answer() { @@ -68,7 +69,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { Object[] args = invocation.getArguments(); TezHeartbeatRequest request = (TezHeartbeatRequest) args[0]; if (request.getRequestId() == 1 || request.getRequestId() == 2) { - TezHeartbeatResponse response = new TezHeartbeatResponse(createEvents(5)); + TezHeartbeatResponse response = new TezHeartbeatResponse(createEvents(maxEvents)); response.setLastRequestId(request.getRequestId()); return response; } else if (request.getRequestId() == 3) { @@ -92,7 +93,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { // Setup the sleep time to be way higher than the test timeout TaskReporter.HeartbeatCallable heartbeatCallable = - new TaskReporter.HeartbeatCallable(mockTask, mockUmbilical, 100000, 100000, 5, + new TaskReporter.HeartbeatCallable(mockTask, mockUmbilical, 100000, 100000, maxEvents, new AtomicLong(0), "containerIdStr"); @@ -105,8 +106,8 @@ public Object answer(InvocationOnMock invocation) throws Throwable { } } verify(mockUmbilical, times(3)).heartbeat(any(TezHeartbeatRequest.class)); - Thread.sleep(2000l); - // Sleep for 2 seconds, less than the callable sleep time. No more invocations. + Thread.sleep(200l); + // Sleep for less than the callable sleep time. No more invocations. verify(mockUmbilical, times(3)).heartbeat(any(TezHeartbeatRequest.class)); } finally { executor.shutdownNow(); From 33980b80ddad271112de66482435b2f5d5097b6f Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Thu, 30 Apr 2020 04:55:17 -0500 Subject: [PATCH 229/512] TEZ-4164. Speed up TestFetcher --- .../library/common/shuffle/orderedgrouped/TestFetcher.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java index ec0eeee37a..5f7fe4ba7e 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java @@ -616,10 +616,8 @@ public MapOutput answer(InvocationOnMock invocation) throws Throwable { //Create read timeout when reading data doAnswer(new Answer() { @Override public Void answer(InvocationOnMock invocation) throws Throwable { - // Emulate host down for 4 seconds. - Thread.sleep(4000); doReturn(false).when(fetcher).setupConnection(any(MapHost.class), any(Collection.class)); - // Throw IOException when fetcher tries to connect again to the same node + // Simulate read timeout by throwing proper exception throw new FetcherReadTimeoutException("creating fetcher socket read timeout exception"); } }).when(fetcher).copyMapOutput(any(MapHost.class), any(DataInputStream.class), any(InputAttemptIdentifier.class)); From 4e4b8e5c9f373cff182272249b10a0d101254080 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Thu, 30 Apr 2020 04:58:33 -0500 Subject: [PATCH 230/512] TEZ-4165. Speed up TestShuffleScheduler#testNumParallelScheduledFetchers Signed-off-by: Jonathan Eagles --- .../orderedgrouped/TestShuffleScheduler.java | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java index 7a7b1ee369..fabfa270d7 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java @@ -31,11 +31,11 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; @@ -90,7 +90,6 @@ public void testNumParallelScheduledFetchers() throws IOException, InterruptedEx new ShuffleSchedulerForTest(inputContext, conf, numInputs, shuffle, mergeManager, mergeManager, System.currentTimeMillis(), null, false, 0, "srcName", true); - Future executorFuture = null; ExecutorService executor = Executors.newFixedThreadPool(1); try { @@ -111,10 +110,9 @@ public Void call() throws Exception { scheduler.addKnownMapOutput("host" + i, 10000, 1, inputAttemptIdentifier); identifiers[i] = inputAttemptIdentifier; } - - // Sleep for a bit to allow the copies to be scheduled. - Thread.sleep(2000l); - assertEquals(10, scheduler.numFetchersCreated.get()); + // wait for all the copies to be scheduled with timeout + scheduler.latch.await(2000, TimeUnit.MILLISECONDS); + assertEquals(0, scheduler.latch.getCount()); } finally { scheduler.close(); @@ -1033,7 +1031,7 @@ public ExecutorService answer(InvocationOnMock invocation) throws Throwable { private static class ShuffleSchedulerForTest extends ShuffleScheduler { - private final AtomicInteger numFetchersCreated = new AtomicInteger(0); + private CountDownLatch latch = new CountDownLatch(10); private final boolean fetcherShouldWait; private final ExceptionReporter reporter; private final InputContext inputContext; @@ -1067,7 +1065,7 @@ public ShuffleSchedulerForTest(InputContext inputContext, Configuration conf, @Override FetcherOrderedGrouped constructFetcherForHost(MapHost mapHost) { - numFetchersCreated.incrementAndGet(); + latch.countDown(); FetcherOrderedGrouped mockFetcher = mock(FetcherOrderedGrouped.class); doAnswer(new Answer() { @Override From bc6a961229a9e9ab8c67bb978e5da745de32b1e1 Mon Sep 17 00:00:00 2001 From: Syed Shameerur Rahman Date: Tue, 12 May 2020 10:30:18 +0200 Subject: [PATCH 231/512] =?UTF-8?q?TEZ-4173:=20isSetParallelismCalled=20sh?= =?UTF-8?q?ould=20be=20checked=20before=20skipping=20vertex=20reinit=20(Sy?= =?UTF-8?q?ed=20Shameerur=20Rahman=20via=20L=C3=A1szl=C3=B3=20Bodor)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../tez/dag/app/dag/impl/VertexImpl.java | 6 ++- .../tez/dag/app/dag/impl/TestDAGRecovery.java | 42 ++++++++++++++++++- 2 files changed, 44 insertions(+), 4 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index b67809e476..85ae38da47 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -2826,10 +2826,12 @@ private void assignVertexManager() throws TezException { // - Why using VertexReconfigureDoneEvent // - VertexReconfigureDoneEvent represent the case that user use API reconfigureVertex // VertexReconfigureDoneEvent will be logged - // - TaskStartEvent is seen in that vertex + // - TaskStartEvent is seen in that vertex or setVertexParallelism is called // - All the parent vertices have skipped initializing stage while recovering if (recoveryData != null && recoveryData.shouldSkipInit() - && recoveryData.isVertexTasksStarted() && isVertexInitSkippedInParentVertices()) { + && (recoveryData.isVertexTasksStarted() || + recoveryData.getVertexConfigurationDoneEvent().isSetParallelismCalled()) + && isVertexInitSkippedInParentVertices()) { // Replace the original VertexManager with NoOpVertexManager if the reconfiguration is done in the last AM attempt VertexConfigurationDoneEvent reconfigureDoneEvent = recoveryData.getVertexConfigurationDoneEvent(); if (LOG.isInfoEnabled()) { diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java index fcf6db8d42..95ea8a030e 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java @@ -746,7 +746,7 @@ public void testVertexRecoverFromInitedAndReconfigureDone() { "vertex1", 0L, v1InitedTime, v1NumTask, "", null, inputGeneratedTezEvents, null); VertexConfigurationDoneEvent v1ReconfigureDoneEvent = new VertexConfigurationDoneEvent(v1Id, - 0L, v1NumTask, null, null, null, true); + 0L, v1NumTask, null, null, null, false); VertexRecoveryData vertexRecoveryData = new VertexRecoveryData(v1InitedEvent, v1ReconfigureDoneEvent, null, null, new HashMap(), false); doReturn(vertexRecoveryData).when(dagRecoveryData).getVertexRecoveryData(v1Id); @@ -784,7 +784,7 @@ public void testVertexRecoverFromStart() { "vertex1", 0L, v1InitedTime, v1NumTask, "", null, inputGeneratedTezEvents, null); VertexConfigurationDoneEvent v1ReconfigureDoneEvent = new VertexConfigurationDoneEvent(v1Id, - 0L, v1NumTask, null, null, null, true); + 0L, v1NumTask, null, null, null, false); VertexStartedEvent v1StartedEvent = new VertexStartedEvent(v1Id, 0L, v1StartedTime); VertexRecoveryData vertexRecoveryData = new VertexRecoveryData(v1InitedEvent, v1ReconfigureDoneEvent, v1StartedEvent, null, new HashMap(), false); @@ -804,6 +804,44 @@ public void testVertexRecoverFromStart() { assertEquals(VertexState.INITED, v3.getState()); } + /** + * RecoveryEvents: + * DAG: DAGInitedEvent -> DAGStartedEvent + * V1: VertexReconfigrationDoneEvent -> VertexInitializedEvent -> VertexStartedEvent -> setParallelismCalledFlag + * + * V1 skip initialization. + */ + @Test(timeout=5000) + public void testVertexRecoverWithSetParallelismCalledFlag() { + initMockDAGRecoveryDataForVertex(); + List inputGeneratedTezEvents = new ArrayList(); + VertexInitializedEvent v1InitedEvent = new VertexInitializedEvent(v1Id, + "vertex1", 0L, v1InitedTime, + v1NumTask, "", null, inputGeneratedTezEvents, null); + VertexConfigurationDoneEvent v1ReconfigureDoneEvent = new VertexConfigurationDoneEvent(v1Id, + 0L, v1NumTask, null, null, null, true); + VertexStartedEvent v1StartedEvent = new VertexStartedEvent(v1Id, 0L, v1StartedTime); + VertexRecoveryData vertexRecoveryData = new VertexRecoveryData(v1InitedEvent, + v1ReconfigureDoneEvent, v1StartedEvent, null, new HashMap(), false); + doReturn(vertexRecoveryData).when(dagRecoveryData).getVertexRecoveryData(v1Id); + + DAGEventRecoverEvent recoveryEvent = new DAGEventRecoverEvent(dagId, dagRecoveryData); + dag.handle(recoveryEvent); + dispatcher.await(); + + VertexImpl v1 = (VertexImpl)dag.getVertex("vertex1"); + VertexImpl v2 = (VertexImpl)dag.getVertex("vertex2"); + VertexImpl v3 = (VertexImpl)dag.getVertex("vertex3"); + assertEquals(DAGState.RUNNING, dag.getState()); + // v1 skip initialization + assertEquals(VertexState.RUNNING, v1.getState()); + assertEquals(v1InitedTime, v1.initedTime); + assertEquals(v1StartedTime, v1.startedTime); + assertEquals(v1NumTask, v1.getTotalTasks()); + assertEquals(VertexState.RUNNING, v2.getState()); + assertEquals(VertexState.RUNNING, v3.getState()); + } + /** * RecoveryEvents: * DAG: DAGInitedEvent -> DAGStartedEvent From 4a97643cdc021b71e2a8bc60ac4fa0f7f9940d0f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Tue, 12 May 2020 10:36:38 +0200 Subject: [PATCH 232/512] =?UTF-8?q?TEZ-2672:=20Allow=20specifying=20a=20ne?= =?UTF-8?q?w=20payload=20for=20plugins=20when=20a=20new=20DAG=20starts=20(?= =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor=20reviewed=20by=20Rajesh=20Balamohan)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../tez/serviceplugins/api/DagInfo.java | 3 +++ .../org/apache/tez/dag/app/AppContext.java | 1 - .../dag/app/TaskCommunicatorContextImpl.java | 4 ++- .../java/org/apache/tez/dag/app/dag/DAG.java | 4 --- .../api/TaskCommunicatorContext.java | 1 - .../app/TestTaskCommunicatorContextImpl.java | 26 +++++++++++++++++++ .../tez/dag/helpers/DagInfoImplForTest.java | 6 +++++ 7 files changed, 38 insertions(+), 7 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/DagInfo.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/DagInfo.java index 328cb62c71..b05fa8bbb7 100644 --- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/DagInfo.java +++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/DagInfo.java @@ -14,6 +14,7 @@ package org.apache.tez.serviceplugins.api; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.Credentials; import java.util.BitSet; @@ -41,4 +42,6 @@ public interface DagInfo { int getTotalVertices(); BitSet getVertexDescendants(int vertexIndex); + + Configuration getConf(); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java index b3d561aa31..4eb2ae2842 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java @@ -25,7 +25,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.security.Credentials; -import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; import org.apache.hadoop.yarn.api.records.ApplicationAccessType; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java index b09eac7663..faa6fe1967 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java @@ -21,6 +21,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.Objects; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import org.apache.tez.common.Preconditions; import com.google.common.collect.Iterables; @@ -58,7 +59,8 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver private final ReentrantReadWriteLock.WriteLock dagChangedWriteLock; private final UserPayload userPayload; - private DAG dag; + @VisibleForTesting + DAG dag; public TaskCommunicatorContextImpl(AppContext appContext, TaskCommunicatorManager taskCommunicatorManager, diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java index 5c2eba122e..280966ef78 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java @@ -22,8 +22,6 @@ import java.util.Map; import java.util.Set; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.event.EventHandler; @@ -76,8 +74,6 @@ public interface DAG extends DagInfo { boolean isUber(); String getUserName(); - Configuration getConf(); - DAGPlan getJobPlan(); DAGStatusBuilder getDAGStatus(Set statusOptions); DAGStatusBuilder getDAGStatus(Set statusOptions, long timeout) diff --git a/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicatorContext.java b/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicatorContext.java index b6f3a549b6..6741a361c9 100644 --- a/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicatorContext.java +++ b/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicatorContext.java @@ -228,5 +228,4 @@ void taskFailed(TezTaskAttemptID taskAttemptId, TaskFailureType taskFailureType, * @return time when the current dag started executing */ long getDagStartTime(); - } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorContextImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorContextImpl.java index 9f9150f758..e73ccf02fe 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorContextImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorContextImpl.java @@ -28,7 +28,9 @@ import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.tez.common.ContainerSignatureMatcher; import org.apache.tez.serviceplugins.api.TaskCommunicatorContext; +import org.apache.tez.dag.app.dag.DAG; import org.apache.tez.dag.app.rm.container.AMContainerMap; +import org.junit.Assert; import org.junit.Test; public class TestTaskCommunicatorContextImpl { @@ -83,6 +85,30 @@ public void testIsKnownContainer() { taskCommContext1.containerAlive(containerId01); verify(tal, never()).containerAlive(containerId01); reset(tal); + } + + @Test + public void testTaskCommContextReachesDAGConf() { + Configuration conf = new Configuration(); + conf.set("dagkey", "dagvalue"); + + DAG dag = mock(DAG.class); + when(dag.getConf()).thenReturn(conf); + + // TaskCommunicatorContextImpl.dag is present + TaskCommunicatorContextImpl commContext = new TaskCommunicatorContextImpl(null, null, null, 0); + commContext.dag = dag; + + Assert.assertEquals("DAG config should be exposed via context.dag.getConf()", + commContext.getCurrentDagInfo().getConf().get("dagkey"), "dagvalue"); + + // TaskCommunicatorContextImpl.appContext.getCurrentDAG() is present + AppContext appContext = mock(AppContext.class); + when(appContext.getCurrentDAG()).thenReturn(dag); + commContext = new TaskCommunicatorContextImpl(appContext, null, null, 0); + Assert.assertEquals( + "DAG config should be exposed via context.appContext.getCurrentDAG().getConf()", + commContext.getCurrentDagInfo().getConf().get("dagkey"), "dagvalue"); } } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/helpers/DagInfoImplForTest.java b/tez-dag/src/test/java/org/apache/tez/dag/helpers/DagInfoImplForTest.java index ab446ac087..26a1a0be7b 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/helpers/DagInfoImplForTest.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/helpers/DagInfoImplForTest.java @@ -14,6 +14,7 @@ package org.apache.tez.dag.helpers; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.Credentials; import org.apache.tez.serviceplugins.api.DagInfo; @@ -53,4 +54,9 @@ public int getTotalVertices() { public BitSet getVertexDescendants(int vertexIndex) { return null; } + + @Override + public Configuration getConf() { + return null; + } } From 354c2a4177fe8c3cf6b8a4c6009d4068a19d81f1 Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Thu, 14 May 2020 03:53:07 +0530 Subject: [PATCH 233/512] TEZ-4174: [Kubernetes] Fetcher should connection failure on SocketException (Prasanth Jayachandran reviewed by Rajesh Balamohan) --- .../org/apache/tez/runtime/library/common/shuffle/Fetcher.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java index f9899f1903..18f66cc8ef 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java @@ -547,7 +547,7 @@ private HostFetchResult setupConnection(Collection attem "Fetch Failure while connecting from %s to: %s:%d, attempt: %s Informing ShuffleManager: ", localHostname, host, port, firstAttempt), e); return new HostFetchResult(new FetchResult(host, port, partition, partitionCount, srcAttemptsRemaining.values()), - new InputAttemptIdentifier[] { firstAttempt }, false); + new InputAttemptIdentifier[] { firstAttempt }, true); } } catch (InterruptedException e) { Thread.currentThread().interrupt(); //reset status From cea0c797b2f08b1ad122f9293fc476f97d8e489b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Fri, 15 May 2020 09:14:38 -0500 Subject: [PATCH 234/512] TEZ-4182. Expose build user and java version in version-info.properties Signed-off-by: Jonathan Eagles --- .../org/apache/tez/common/VersionInfo.java | 18 ++++++++++++++++-- .../resources/tez-api-version-info.properties | 2 ++ .../resources/tez-dag-version-info.properties | 2 ++ 3 files changed, 20 insertions(+), 2 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/VersionInfo.java b/tez-api/src/main/java/org/apache/tez/common/VersionInfo.java index 9f98974314..1afde0d3f4 100644 --- a/tez-api/src/main/java/org/apache/tez/common/VersionInfo.java +++ b/tez-api/src/main/java/org/apache/tez/common/VersionInfo.java @@ -42,6 +42,8 @@ public class VersionInfo { private static final String VERSION = "version"; private static final String REVISION = "revision"; private static final String BUILD_TIME = "buildtime"; + private static final String BUILD_USER = "builduser"; + private static final String BUILD_JAVA_VERSION = "buildjavaversion"; private static final String SCM_URL = "scmurl"; public static final String UNKNOWN = "Unknown"; @@ -66,13 +68,15 @@ protected VersionInfo(String component) { @VisibleForTesting @Private - protected VersionInfo(String component, String version, String revision, - String buildTime, String scmUrl) { + protected VersionInfo(String component, String version, String revision, String buildTime, + String scmUrl) { this.info = new Properties(); this.component = component; info.setProperty(VERSION, version); info.setProperty(REVISION, revision); info.setProperty(BUILD_TIME, buildTime); + info.setProperty(BUILD_USER, System.getProperty("user.name")); + info.setProperty(BUILD_JAVA_VERSION, System.getProperty("java.version")); info.setProperty(SCM_URL, scmUrl); } @@ -84,6 +88,14 @@ public String getBuildTime() { return info.getProperty(BUILD_TIME, UNKNOWN); } + public String getBuildUser() { + return info.getProperty(BUILD_USER, UNKNOWN); + } + + public String getBuildJavaVersion() { + return info.getProperty(BUILD_JAVA_VERSION, UNKNOWN); + } + public String getRevision() { return info.getProperty(REVISION, UNKNOWN); } @@ -99,6 +111,8 @@ public String toString() { + ", revision=" + getRevision() + ", SCM-URL=" + getSCMURL() + ", buildTime=" + getBuildTime() + + ", buildUser=" + getBuildUser() + + ", buildJavaVersion=" + getBuildJavaVersion() + " ]"; } diff --git a/tez-api/src/main/resources/tez-api-version-info.properties b/tez-api/src/main/resources/tez-api-version-info.properties index 4bb7d40b8b..0bc30c4bb0 100644 --- a/tez-api/src/main/resources/tez-api-version-info.properties +++ b/tez-api/src/main/resources/tez-api-version-info.properties @@ -19,4 +19,6 @@ version=${pom.version} revision=${buildNumber} buildtime=${build.time} +builduser=${user.name} +buildjavaversion=${java.version} scmurl=${scm.url} diff --git a/tez-dag/src/main/resources/tez-dag-version-info.properties b/tez-dag/src/main/resources/tez-dag-version-info.properties index 4bb7d40b8b..0bc30c4bb0 100644 --- a/tez-dag/src/main/resources/tez-dag-version-info.properties +++ b/tez-dag/src/main/resources/tez-dag-version-info.properties @@ -19,4 +19,6 @@ version=${pom.version} revision=${buildNumber} buildtime=${build.time} +builduser=${user.name} +buildjavaversion=${java.version} scmurl=${scm.url} From 07c807b2d474eada6afc55f623f2b8613bb891a0 Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Sat, 16 May 2020 10:32:07 -0500 Subject: [PATCH 235/512] TEZ-4171. DAGImp::getDAGStatus should try to report RUNNING state information correctly Signed-off-by: Jonathan Eagles --- .../apache/tez/dag/app/dag/impl/DAGImpl.java | 17 ++++++++++++++++- .../tez/dag/app/dag/impl/TestDAGImpl.java | 8 ++++++-- 2 files changed, 22 insertions(+), 3 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java index b20a4f6fc0..bb5c1aa966 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java @@ -166,6 +166,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG, private final Lock dagStatusLock = new ReentrantLock(); private final Condition dagStateChangedCondition = dagStatusLock.newCondition(); private final AtomicBoolean isFinalState = new AtomicBoolean(false); + private final AtomicBoolean runningStatusYetToBeConsumed = new AtomicBoolean(false); private final Lock readLock; private final Lock writeLock; private final String dagName; @@ -585,8 +586,18 @@ private static class DagStateChangedCallback implements OnStateChangedCallback { @Override public void onStateChanged(DAGImpl dag, DAGState dagState) { - if (dagState != DAGState.RUNNING) { + switch(dagState) { + case RUNNING: + dag.runningStatusYetToBeConsumed.set(true); + break; + case SUCCEEDED: + case FAILED: + case KILLED: + case ERROR: dag.isFinalState.set(true); + break; + default: + break; } dag.dagStatusLock.lock(); try { @@ -950,6 +961,10 @@ public DAGStatusBuilder getDAGStatus(Set statusOptions, if (isFinalState.get()) { break; } + if (runningStatusYetToBeConsumed.compareAndSet(true, false)) { + // No need to wait further, as state just got changed to RUNNING + break; + } nanosLeft = dagStateChangedCondition.awaitNanos(timeoutNanos); } catch (InterruptedException e) { throw new TezException("Interrupted while waiting for dag to complete", e); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java index c0506de68d..2f2b3b8b8f 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java @@ -1762,7 +1762,7 @@ public void testGetDAGStatusWithWait() throws TezException { DAGStatusBuilder dagStatus = dag.getDAGStatus(EnumSet.noneOf(StatusGetOpts.class), 2000l); long dagStatusEndTime = System.currentTimeMillis(); long diff = dagStatusEndTime - dagStatusStartTime; - Assert.assertTrue(diff > 1500 && diff < 2500); + Assert.assertTrue(diff >= 0 && diff < 2500); Assert.assertEquals(DAGStatusBuilder.State.RUNNING, dagStatus.getState()); } @@ -1805,6 +1805,9 @@ public void runTestGetDAGStatusReturnOnDagFinished(DAGStatusBuilder.State testSt dispatcher.await(); Assert.assertEquals(DAGState.RUNNING, dag.getState()); Assert.assertEquals(5, dag.getSuccessfulVertices()); + // Verify that dagStatus is running state + Assert.assertEquals(DAGStatus.State.RUNNING, dag.getDAGStatus(EnumSet.noneOf(StatusGetOpts.class), + 10000L).getState()); ReentrantLock lock = new ReentrantLock(); Condition startCondition = lock.newCondition(); @@ -1851,7 +1854,8 @@ public void runTestGetDAGStatusReturnOnDagFinished(DAGStatusBuilder.State testSt long diff = statusCheckRunnable.dagStatusEndTime - statusCheckRunnable.dagStatusStartTime; Assert.assertNotNull(statusCheckRunnable.dagStatus); - Assert.assertTrue(diff > 1000 && diff < 3500); + Assert.assertTrue("Status: " + statusCheckRunnable.dagStatus.getState() + + ", Diff:" + diff, diff >= 0 && diff < 3500); Assert.assertEquals(testState, statusCheckRunnable.dagStatus.getState()); t1.join(); } From 9d6b11ca3ef2d51b0a5fd19d979a4a862f4aaa69 Mon Sep 17 00:00:00 2001 From: Gopal V Date: Thu, 21 May 2020 14:29:42 -0700 Subject: [PATCH 236/512] TEZ-4186: Limits: Fix init order regression from TEZ-4155 (Gopal V, reviewed by Rajesh Balamohan) Signed-off-by: Gopal V --- .../main/java/org/apache/tez/common/counters/Limits.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/Limits.java b/tez-api/src/main/java/org/apache/tez/common/counters/Limits.java index 13cc7fbdbf..3e926c693b 100644 --- a/tez-api/src/main/java/org/apache/tez/common/counters/Limits.java +++ b/tez-api/src/main/java/org/apache/tez/common/counters/Limits.java @@ -30,6 +30,7 @@ public class Limits { private static final Logger LOG = LoggerFactory.getLogger(Limits.class); + private static final Configuration DEFAULT_CONFIGURATION = new TezConfiguration(); private static Configuration conf = null; private static int GROUP_NAME_MAX; private static int COUNTER_NAME_MAX; @@ -37,11 +38,12 @@ public class Limits { private static int COUNTERS_MAX; static { - init(new TezConfiguration()); + init(DEFAULT_CONFIGURATION); } public synchronized static void setConfiguration(Configuration conf) { - if (Limits.conf == null && conf != null) { + // see change to reset() + if (Limits.conf == DEFAULT_CONFIGURATION && conf != null) { init(conf); } } @@ -105,7 +107,7 @@ public synchronized void checkGroups(int size) { @VisibleForTesting @InterfaceAudience.Private public synchronized static void reset() { - conf = null; + conf = DEFAULT_CONFIGURATION; } } From f047d4a712b7da6985561d2fa99de734002d71d9 Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Fri, 22 May 2020 11:30:05 +0530 Subject: [PATCH 237/512] TEZ-4179: [Kubernetes] Extend NodeId in tez to support unique worker identity (Prasanth Jayachandran, Attila Magyar, reviewed by Rajesh Balamohan) --- .../tez/dag/app/rm/node/AMNodeEvent.java | 11 ++- .../tez/dag/app/rm/node/ExtendedNodeId.java | 97 +++++++++++++++++++ .../dag/app/rm/node/TestAMNodeTracker.java | 54 ++++++++++- 3 files changed, 156 insertions(+), 6 deletions(-) create mode 100644 tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/ExtendedNodeId.java diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEvent.java index 1a975b042c..d9e249af4e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEvent.java @@ -25,15 +25,24 @@ public class AMNodeEvent extends AbstractEvent { private final NodeId nodeId; private final int schedulerId; + private final ExtendedNodeId amNodeId; public AMNodeEvent(NodeId nodeId, int schedulerId, AMNodeEventType type) { super(type); this.nodeId = nodeId; this.schedulerId = schedulerId; + this.amNodeId = null; + } + + public AMNodeEvent(ExtendedNodeId amNodeId, int schedulerId, AMNodeEventType type) { + super(type); + this.nodeId = null; + this.schedulerId = schedulerId; + this.amNodeId = amNodeId; } public NodeId getNodeId() { - return this.nodeId; + return amNodeId == null ? this.nodeId : this.amNodeId; } public int getSchedulerId() { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/ExtendedNodeId.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/ExtendedNodeId.java new file mode 100644 index 0000000000..ea58d8624c --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/ExtendedNodeId.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.app.rm.node; + +import java.util.Objects; + +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.tez.common.Preconditions; + +/** + * ExtendedNodeId extends NodeId with unique identifier in addition to hostname and port. + */ +public class ExtendedNodeId extends NodeId { + private NodeId nodeId; + private String host; + private int port; + private final String uniqueIdentifier; + + public ExtendedNodeId(NodeId nodeId, String uniqueIdentifier) { + Preconditions.checkArgument(nodeId != null); + this.nodeId = nodeId; + this.uniqueIdentifier = uniqueIdentifier == null ? "" : uniqueIdentifier.trim(); + } + + @Override + public String getHost() { + return nodeId.getHost(); + } + + @Override + protected void setHost(final String host) { + this.host = host; + build(); + } + + @Override + public int getPort() { + return nodeId.getPort(); + } + + @Override + protected void setPort(final int port) { + this.port = port; + build(); + } + + @Override + protected void build() { + this.nodeId = NodeId.newInstance(host, port); + } + + @Override + public String toString() { + if (!uniqueIdentifier.isEmpty()) { + return super.toString() + ":" + uniqueIdentifier; + } + return super.toString(); + } + + @Override + public int hashCode() { + if (!uniqueIdentifier.isEmpty()) { + return super.hashCode() + 31 * uniqueIdentifier.hashCode(); + } + return super.hashCode(); + } + + @Override + public boolean equals(final Object obj) { + if (this == obj) { + return true; + } else if (obj == null) { + return false; + } else if (this.getClass() != obj.getClass()) { + return false; + } else { + ExtendedNodeId amNodeId = (ExtendedNodeId) obj; + return super.equals(obj) && Objects.equals(uniqueIdentifier, amNodeId.uniqueIdentifier); + } + } +} diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/node/TestAMNodeTracker.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/node/TestAMNodeTracker.java index 11d3b7ac21..060cdc4dcd 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/node/TestAMNodeTracker.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/node/TestAMNodeTracker.java @@ -327,6 +327,33 @@ public void testNodeSelfBlacklistAlternateScheduler2() { } } + @Test(timeout=10000) + public void testMultipleAMNodeIDs() { + AppContext appContext = mock(AppContext.class); + Configuration conf = new Configuration(false); + conf.setInt(TezConfiguration.TEZ_AM_MAX_TASK_FAILURES_PER_NODE, 2); + TestEventHandler handler = new TestEventHandler(); + AMNodeTracker amNodeTracker = new AMNodeTracker(handler, appContext); + doReturn(amNodeTracker).when(appContext).getNodeTracker(); + AMContainerMap amContainerMap = mock(AMContainerMap.class); + TaskSchedulerManager taskSchedulerManager = + mock(TaskSchedulerManager.class); + dispatcher.register(AMNodeEventType.class, amNodeTracker); + dispatcher.register(AMContainerEventType.class, amContainerMap); + dispatcher.register(AMSchedulerEventType.class, taskSchedulerManager); + amNodeTracker.init(conf); + amNodeTracker.start(); + try { + amNodeTracker.nodeSeen(new ExtendedNodeId(NodeId.newInstance("host", 2222), "uuid1"), 0); + amNodeTracker.nodeSeen(new ExtendedNodeId(NodeId.newInstance("host", 2222), "uuid1"), 0); + amNodeTracker.nodeSeen(new ExtendedNodeId(NodeId.newInstance("host", 2222), "uuid2"), 0); + amNodeTracker.nodeSeen(new ExtendedNodeId(NodeId.newInstance("host", 2222), "uuid2"), 0); + assertEquals(2, amNodeTracker.getNumNodes(0)); + } finally { + amNodeTracker.stop(); + } + } + @Test(timeout = 10000L) public void testNodeCompletedAndCleanup() { AppContext appContext = mock(AppContext.class); @@ -401,15 +428,26 @@ public void testNodeCompletedAndCleanup() { @Test(timeout=10000) public void testNodeUnhealthyRescheduleTasksEnabled() throws Exception { - _testNodeUnhealthyRescheduleTasks(true); + _testNodeUnhealthyRescheduleTasks(true, false); } @Test(timeout=10000) public void testNodeUnhealthyRescheduleTasksDisabled() throws Exception { - _testNodeUnhealthyRescheduleTasks(false); + _testNodeUnhealthyRescheduleTasks(false, false); + } + + + @Test(timeout=10000) + public void testNodeUnhealthyRescheduleTasksEnabledAMNode() throws Exception { + _testNodeUnhealthyRescheduleTasks(true, true); + } + + @Test(timeout=10000) + public void testNodeUnhealthyRescheduleTasksDisabledAMNode() throws Exception { + _testNodeUnhealthyRescheduleTasks(false, true); } - private void _testNodeUnhealthyRescheduleTasks(boolean rescheduleTasks) { + private void _testNodeUnhealthyRescheduleTasks(boolean rescheduleTasks, boolean useExtendedNodeId) { AppContext appContext = mock(AppContext.class); Configuration conf = new Configuration(false); conf.setBoolean(TezConfiguration.TEZ_AM_NODE_UNHEALTHY_RESCHEDULE_TASKS, @@ -422,8 +460,14 @@ private void _testNodeUnhealthyRescheduleTasks(boolean rescheduleTasks) { // add a node amNodeTracker.handle(new AMNodeEventNodeCountUpdated(1, 0)); - NodeId nodeId = NodeId.newInstance("host1", 1234); - amNodeTracker.nodeSeen(nodeId, 0); + NodeId nodeId; + if (useExtendedNodeId) { + nodeId = new ExtendedNodeId(NodeId.newInstance("host1", 1234), "uuid2"); + amNodeTracker.nodeSeen(nodeId, 0); + } else { + nodeId = NodeId.newInstance("host1", 1234); + amNodeTracker.nodeSeen(nodeId, 0); + } AMNodeImpl node = (AMNodeImpl) amNodeTracker.get(nodeId, 0); // simulate task starting on node From 7659726a1ed877d1f5303fc3673e4399bab33b65 Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Tue, 26 May 2020 17:13:00 -0700 Subject: [PATCH 238/512] TEZ-4087 : Shuffle: Fix shuffle cleanup to prevent thread leaks (Rajesh Balamohan via Prasanth J, Ashutosh Chauhan) Signed-off-by: Ashutosh Chauhan --- .../shuffle/orderedgrouped/MergeManager.java | 23 +++++++++++++++---- .../shuffle/orderedgrouped/MergeThread.java | 18 +++++++++++++-- .../orderedgrouped/ShuffleScheduler.java | 1 + 3 files changed, 35 insertions(+), 7 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java index 2e5cc20096..70f9e55f70 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java @@ -314,6 +314,16 @@ public MergeManager(Configuration conf, this.onDiskMerger = new OnDiskMerger(this); } + void setupParentThread(Thread shuffleSchedulerThread) { + LOG.info("Setting merger's parent thread to " + + shuffleSchedulerThread.getName()); + if (this.memToMemMerger != null) { + memToMemMerger.setParentThread(shuffleSchedulerThread); + } + this.inMemoryMerger.setParentThread(shuffleSchedulerThread);; + this.onDiskMerger.setParentThread(shuffleSchedulerThread); + } + @Private void configureAndStart() { if (this.memToMemMerger != null) { @@ -714,7 +724,8 @@ public IntermediateMemoryToMemoryMerger(MergeManager manager, int mergeFactor) { super(manager, mergeFactor, exceptionReporter); setName("MemToMemMerger [" + TezUtilsInternal - .cleanVertexName(inputContext.getSourceVertexName()) + "]"); + .cleanVertexName(inputContext.getSourceVertexName()) + + "_" + inputContext.getUniqueIdentifier() + "]"); setDaemon(true); } @@ -831,8 +842,9 @@ private class InMemoryMerger extends MergeThread { public InMemoryMerger(MergeManager manager) { super(manager, Integer.MAX_VALUE, exceptionReporter); - setName("MemtoDiskMerger [" + TezUtilsInternal - .cleanVertexName(inputContext.getSourceVertexName()) + "]"); + setName("MemtoDiskMerger [" + TezUtilsInternal + .cleanVertexName(inputContext.getSourceVertexName()) + + "_" + inputContext.getUniqueIdentifier() + "]"); setDaemon(true); } @@ -952,8 +964,9 @@ class OnDiskMerger extends MergeThread { public OnDiskMerger(MergeManager manager) { super(manager, ioSortFactor, exceptionReporter); - setName("DiskToDiskMerger [" + TezUtilsInternal - .cleanVertexName(inputContext.getSourceVertexName()) + "]"); + setName("DiskToDiskMerger [" + TezUtilsInternal + .cleanVertexName(inputContext.getSourceVertexName()) + + "_" + inputContext.getUniqueIdentifier() + "]"); setDaemon(true); } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeThread.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeThread.java index 52b4c5bbe9..c0af90fde0 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeThread.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeThread.java @@ -36,6 +36,8 @@ abstract class MergeThread extends Thread { private final ExceptionReporter reporter; private boolean closed = false; private final int mergeFactor; + + private Thread shuffleSchedulerThread; public MergeThread(MergeManager manager, int mergeFactor, ExceptionReporter reporter) { @@ -60,6 +62,10 @@ public synchronized void close() throws InterruptedException { } } + public void setParentThread(Thread shuffleSchedulerThread) { + this.shuffleSchedulerThread = shuffleSchedulerThread; + } + public synchronized boolean isInProgress() { return inProgress; } @@ -81,7 +87,11 @@ public synchronized void startMerge(Set inputs) { public synchronized void waitForMerge() throws InterruptedException { while (inProgress) { - wait(); + if (shuffleSchedulerThread != null + && !shuffleSchedulerThread.isAlive()) { + return; + } + wait(5000); } } @@ -91,7 +101,11 @@ public void run() { // Wait for notification to start the merge... synchronized (this) { while (!inProgress) { - wait(); + if (shuffleSchedulerThread != null + && !shuffleSchedulerThread.isAlive()) { + return; + } + wait(5000); } } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java index ff07e913fb..0954a76ec9 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java @@ -445,6 +445,7 @@ public ShuffleScheduler(InputContext inputContext, public void start() throws Exception { shuffleSchedulerThread = Thread.currentThread(); + mergeManager.setupParentThread(shuffleSchedulerThread); ShuffleSchedulerCallable schedulerCallable = new ShuffleSchedulerCallable(); schedulerCallable.call(); } From 0411a2e0244dc4e222e63b1e6f56ca8524e2d3aa Mon Sep 17 00:00:00 2001 From: Mustafa Iman Date: Tue, 26 May 2020 22:39:21 -0700 Subject: [PATCH 239/512] TEZ-4137 : Input/Output/Processor should merge payload to local conf (Mustafa Iman via Laszlo Bodor, Ashutosh Chauhan) Signed-off-by: Ashutosh Chauhan --- .../java/org/apache/tez/common/TezUtils.java | 22 ++++++++++ .../runtime/api/InputInitializerContext.java | 9 +++- .../apache/tez/runtime/api/TaskContext.java | 7 ++++ .../TezRootInputInitializerContextImpl.java | 8 +++- .../common/MRInputAMSplitGenerator.java | 5 +-- .../tez/mapreduce/input/base/MRInputBase.java | 5 ++- .../apache/tez/mapreduce/output/MROutput.java | 6 ++- .../apache/tez/mapreduce/TezTestUtils.java | 10 ++++- .../common/TestMRInputAMSplitGenerator.java | 2 +- .../common/TestMRInputSplitDistributor.java | 6 ++- .../tez/mapreduce/input/MRInputForTest.java | 35 ++++++++++++++++ .../mapreduce/input/MultiMRInputForTest.java | 35 ++++++++++++++++ .../tez/mapreduce/input/TestMRInput.java | 39 +++++++++++++++++ .../tez/mapreduce/input/TestMultiMRInput.java | 34 +++++++++++---- .../tez/mapreduce/output/TestMROutput.java | 42 +++++++++++++++---- .../mapreduce/output/TestMROutputLegacy.java | 2 + .../mapreduce/output/TestMultiMROutput.java | 30 +++++++++++++ .../runtime/api/impl/TezTaskContextImpl.java | 7 ++++ .../library/input/OrderedGroupedKVInput.java | 4 +- .../library/input/UnorderedKVInput.java | 4 +- .../output/OrderedPartitionedKVOutput.java | 4 +- .../library/output/UnorderedKVOutput.java | 9 ++-- .../output/UnorderedPartitionedKVOutput.java | 8 ++-- .../input/TestOrderedGroupedKVInput.java | 30 +++++++++++++ .../library/output/OutputTestHelpers.java | 7 +++- .../output/TestOnFileSortedOutput.java | 3 +- .../output/TestOnFileUnorderedKVOutput.java | 36 ++++++++++++---- .../TestOrderedPartitionedKVOutput2.java | 19 ++++++++- .../output/TestUnorderedKVOutput2.java | 17 +++++++- .../TestUnorderedPartitionedKVOutput2.java | 19 +++++++++ 30 files changed, 410 insertions(+), 54 deletions(-) create mode 100644 tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/MRInputForTest.java create mode 100644 tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/MultiMRInputForTest.java diff --git a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java index 51311ffd80..23811aa7f1 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java @@ -30,6 +30,7 @@ import com.google.protobuf.ByteString; import com.google.protobuf.CodedInputStream; +import org.apache.tez.runtime.api.TaskContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience; @@ -120,6 +121,27 @@ public static Configuration createConfFromByteString(ByteString byteString) thro } } + public static Configuration createConfFromBaseConfAndPayload(TaskContext context) + throws IOException { + Configuration baseConf = context.getContainerConfiguration(); + Configuration configuration = new Configuration(baseConf); + UserPayload payload = context.getUserPayload(); + ByteString byteString = ByteString.copyFrom(payload.getPayload()); + try(SnappyInputStream uncompressIs = new SnappyInputStream(byteString.newInput())) { + DAGProtos.ConfigurationProto confProto = DAGProtos.ConfigurationProto.parseFrom(uncompressIs); + readConfFromPB(confProto, configuration); + return configuration; + } + } + + public static void addToConfFromByteString(Configuration configuration, ByteString byteString) + throws IOException { + try(SnappyInputStream uncompressIs = new SnappyInputStream(byteString.newInput())) { + DAGProtos.ConfigurationProto confProto = DAGProtos.ConfigurationProto.parseFrom(uncompressIs); + readConfFromPB(confProto, configuration); + } + } + /** * Convert an instance of {@link org.apache.tez.dag.api.UserPayload} to {@link * org.apache.hadoop.conf.Configuration} diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializerContext.java b/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializerContext.java index ccfac46e21..7c9562e7c9 100644 --- a/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializerContext.java +++ b/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializerContext.java @@ -24,6 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.tez.common.counters.TezCounters; @@ -49,7 +50,13 @@ public interface InputInitializerContext { * @return DAG name */ String getDAGName(); - + + /** + * Get vertex configuration + * @return Vertex configuration + */ + Configuration getVertexConfiguration(); + /** * Get the name of the input * @return Input name diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/TaskContext.java b/tez-api/src/main/java/org/apache/tez/runtime/api/TaskContext.java index dd2951a382..1ba1a90e3e 100644 --- a/tez-api/src/main/java/org/apache/tez/runtime/api/TaskContext.java +++ b/tez-api/src/main/java/org/apache/tez/runtime/api/TaskContext.java @@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.UserPayload; @@ -62,6 +63,12 @@ public interface TaskContext { */ public int getTaskAttemptNumber(); + /** + * Get container configuration + * @return Container configuration + */ + public Configuration getContainerConfiguration(); + /** * Get the name of the DAG * @return the DAG name diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java index 43764878b6..a994359354 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java @@ -23,6 +23,7 @@ import java.util.Set; import java.util.Objects; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.tez.common.counters.TezCounters; @@ -85,7 +86,12 @@ public UserPayload getInputUserPayload() { public UserPayload getUserPayload() { return this.input.getControllerDescriptor().getUserPayload(); } - + + @Override + public Configuration getVertexConfiguration() { + return vertex.getConf(); + } + @Override public int getNumTasks() { return vertex.getTotalTasks(); diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/common/MRInputAMSplitGenerator.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/common/MRInputAMSplitGenerator.java index dbfdcb3843..d06a5f46a0 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/common/MRInputAMSplitGenerator.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/common/MRInputAMSplitGenerator.java @@ -30,7 +30,6 @@ import org.apache.hadoop.classification.InterfaceStability.Evolving; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapreduce.split.TezMapReduceSplitsGrouper; import org.apache.hadoop.security.UserGroupInformation; import org.apache.tez.common.TezUtils; import org.apache.tez.dag.api.VertexLocationHint; @@ -80,8 +79,8 @@ public List initialize() throws Exception { + sw.now(TimeUnit.MILLISECONDS)); } sw.reset().start(); - Configuration conf = TezUtils.createConfFromByteString(userPayloadProto - .getConfigurationBytes()); + Configuration conf = new JobConf(getContext().getVertexConfiguration()); + TezUtils.addToConfFromByteString(conf, userPayloadProto.getConfigurationBytes()); sendSerializedEvents = conf.getBoolean( MRJobConfig.MR_TEZ_INPUT_INITIALIZER_SERIALIZE_EVENT_PAYLOAD, diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/base/MRInputBase.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/base/MRInputBase.java index d8c531ea84..ccae0b1964 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/base/MRInputBase.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/base/MRInputBase.java @@ -72,8 +72,9 @@ public List initialize() throws IOException { boolean isGrouped = mrUserPayload.getGroupingEnabled(); Preconditions.checkArgument(mrUserPayload.hasSplits() == false, "Split information not expected in " + this.getClass().getName()); - Configuration conf = TezUtils - .createConfFromByteString(mrUserPayload.getConfigurationBytes()); + + Configuration conf = new JobConf(getContext().getContainerConfiguration()); + TezUtils.addToConfFromByteString(conf, mrUserPayload.getConfigurationBytes()); this.jobConf = new JobConf(conf); useNewApi = this.jobConf.getUseNewMapper(); if (isGrouped) { diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java index 18047ebf09..950e629907 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java @@ -29,6 +29,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import com.google.common.annotations.VisibleForTesting; +import com.google.protobuf.ByteString; import org.apache.tez.common.Preconditions; import org.apache.commons.lang.StringUtils; import org.apache.hadoop.mapreduce.lib.output.LazyOutputFormat; @@ -398,8 +399,9 @@ protected List initializeBase() throws IOException, InterruptedException taskNumberFormat.setGroupingUsed(false); nonTaskNumberFormat.setMinimumIntegerDigits(3); nonTaskNumberFormat.setGroupingUsed(false); - Configuration conf = TezUtils.createConfFromUserPayload(getContext().getUserPayload()); - this.jobConf = new JobConf(conf); + UserPayload userPayload = getContext().getUserPayload(); + this.jobConf = new JobConf(getContext().getContainerConfiguration()); + TezUtils.addToConfFromByteString(this.jobConf, ByteString.copyFrom(userPayload.getPayload())); // Add tokens to the jobConf - in case they are accessed within the RW / OF jobConf.getCredentials().mergeAll(UserGroupInformation.getCurrentUser().getCredentials()); this.isMapperOutput = jobConf.getBoolean(MRConfig.IS_MAP_PROCESSOR, diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TezTestUtils.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TezTestUtils.java index 369afbe6b3..83c28dd7bb 100644 --- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TezTestUtils.java +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TezTestUtils.java @@ -17,6 +17,7 @@ */ package org.apache.tez.mapreduce; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.tez.common.counters.TezCounters; @@ -59,10 +60,12 @@ public static class TezRootInputInitializerContextForTest implements private final ApplicationId appId; private final UserPayload payload; + private final Configuration vertexConfig; - public TezRootInputInitializerContextForTest(UserPayload payload) throws IOException { + public TezRootInputInitializerContextForTest(UserPayload payload, Configuration vertexConfig) throws IOException { appId = ApplicationId.newInstance(1000, 200); this.payload = payload == null ? UserPayload.create(null) : payload; + this.vertexConfig = vertexConfig; } @Override @@ -75,6 +78,11 @@ public String getDAGName() { return "FakeDAG"; } + @Override + public Configuration getVertexConfiguration() { + return vertexConfig; + } + @Override public String getInputName() { return "MRInput"; diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/common/TestMRInputAMSplitGenerator.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/common/TestMRInputAMSplitGenerator.java index 6cf2700564..9f6ac3b74f 100644 --- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/common/TestMRInputAMSplitGenerator.java +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/common/TestMRInputAMSplitGenerator.java @@ -96,7 +96,7 @@ private void testGroupSplitsAndSortSplits(boolean groupSplitsEnabled, UserPayload userPayload = dataSource.getInputDescriptor().getUserPayload(); InputInitializerContext context = - new TezTestUtils.TezRootInputInitializerContextForTest(userPayload); + new TezTestUtils.TezRootInputInitializerContextForTest(userPayload, new Configuration(false)); MRInputAMSplitGenerator splitGenerator = new MRInputAMSplitGenerator(context); diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/common/TestMRInputSplitDistributor.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/common/TestMRInputSplitDistributor.java index 3772cde946..4aaa7e2e76 100644 --- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/common/TestMRInputSplitDistributor.java +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/common/TestMRInputSplitDistributor.java @@ -70,7 +70,8 @@ public void testSerializedPayload() throws IOException { UserPayload userPayload = UserPayload.create(payloadProto.build().toByteString().asReadOnlyByteBuffer()); - InputInitializerContext context = new TezTestUtils.TezRootInputInitializerContextForTest(userPayload); + InputInitializerContext context = new TezTestUtils.TezRootInputInitializerContextForTest(userPayload, + new Configuration(false)); MRInputSplitDistributor splitDist = new MRInputSplitDistributor(context); List events = splitDist.initialize(); @@ -119,7 +120,8 @@ public void testDeserializedPayload() throws IOException { UserPayload userPayload = UserPayload.create(payloadProto.build().toByteString().asReadOnlyByteBuffer()); - InputInitializerContext context = new TezTestUtils.TezRootInputInitializerContextForTest(userPayload); + InputInitializerContext context = new TezTestUtils.TezRootInputInitializerContextForTest(userPayload, + new Configuration(false)); MRInputSplitDistributor splitDist = new MRInputSplitDistributor(context); List events = splitDist.initialize(); diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/MRInputForTest.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/MRInputForTest.java new file mode 100644 index 0000000000..0d1d24ff6f --- /dev/null +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/MRInputForTest.java @@ -0,0 +1,35 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.mapreduce.input; + +import org.apache.hadoop.conf.Configuration; +import org.apache.tez.runtime.api.InputContext; + +/** + * This is used for inspecting jobConf in test. + */ +public class MRInputForTest extends MRInput { + public MRInputForTest(InputContext inputContext, int numPhysicalInputs) { + super(inputContext, numPhysicalInputs); + } + + public Configuration getConfiguration() { + return jobConf; + } +} diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/MultiMRInputForTest.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/MultiMRInputForTest.java new file mode 100644 index 0000000000..f0f0a77aa7 --- /dev/null +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/MultiMRInputForTest.java @@ -0,0 +1,35 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.mapreduce.input; + +import org.apache.hadoop.conf.Configuration; +import org.apache.tez.runtime.api.InputContext; + +/** + * This is used for inspecting jobConf in test. + */ +public class MultiMRInputForTest extends MultiMRInput { + public MultiMRInputForTest(InputContext inputContext, int numPhysicalInputs) { + super(inputContext, numPhysicalInputs); + } + + public Configuration getConfiguration() { + return jobConf; + } +} diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/TestMRInput.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/TestMRInput.java index 9109cd9c47..5ca5c26619 100644 --- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/TestMRInput.java +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/TestMRInput.java @@ -69,6 +69,7 @@ public void test0PhysicalInputs() throws IOException { doReturn(1).when(inputContext).getTaskIndex(); doReturn(1).when(inputContext).getTaskAttemptNumber(); doReturn(new TezCounters()).when(inputContext).getCounters(); + doReturn(new JobConf(false)).when(inputContext).getContainerConfiguration(); MRInput mrInput = new MRInput(inputContext, 0); @@ -120,6 +121,7 @@ public void testAttributesInJobConf() throws Exception { doReturn(TEST_ATTRIBUTES_INPUT_NAME).when(inputContext).getSourceVertexName(); doReturn(TEST_ATTRIBUTES_APPLICATION_ID).when(inputContext).getApplicationId(); doReturn(TEST_ATTRIBUTES_UNIQUE_IDENTIFIER).when(inputContext).getUniqueIdentifier(); + doReturn(new Configuration(false)).when(inputContext).getContainerConfiguration(); DataSourceDescriptor dsd = MRInput.createConfigBuilder(new Configuration(false), @@ -147,6 +149,43 @@ public void testAttributesInJobConf() throws Exception { assertTrue(TestInputFormat.invoked.get()); } + @Test(timeout = 5000) + public void testConfigMerge() throws Exception { + JobConf jobConf = new JobConf(false); + jobConf.set("payload-key", "payload-value"); + + Configuration localConfig = new Configuration(false); + localConfig.set("local-key", "local-value"); + + InputContext inputContext = mock(InputContext.class); + + DataSourceDescriptor dsd = MRInput.createConfigBuilder(jobConf, + TestInputFormat.class).groupSplits(false).build(); + + doReturn(dsd.getInputDescriptor().getUserPayload()).when(inputContext).getUserPayload(); + doReturn(TEST_ATTRIBUTES_DAG_INDEX).when(inputContext).getDagIdentifier(); + doReturn(TEST_ATTRIBUTES_VERTEX_INDEX).when(inputContext).getTaskVertexIndex(); + doReturn(TEST_ATTRIBUTES_TASK_INDEX).when(inputContext).getTaskIndex(); + doReturn(TEST_ATTRIBUTES_TASK_ATTEMPT_INDEX).when(inputContext).getTaskAttemptNumber(); + doReturn(TEST_ATTRIBUTES_INPUT_INDEX).when(inputContext).getInputIndex(); + doReturn(TEST_ATTRIBUTES_DAG_ATTEMPT_NUMBER).when(inputContext).getDAGAttemptNumber(); + doReturn(TEST_ATTRIBUTES_DAG_NAME).when(inputContext).getDAGName(); + doReturn(TEST_ATTRIBUTES_VERTEX_NAME).when(inputContext).getTaskVertexName(); + doReturn(TEST_ATTRIBUTES_INPUT_NAME).when(inputContext).getSourceVertexName(); + doReturn(TEST_ATTRIBUTES_APPLICATION_ID).when(inputContext).getApplicationId(); + doReturn(TEST_ATTRIBUTES_UNIQUE_IDENTIFIER).when(inputContext).getUniqueIdentifier(); + doReturn(localConfig).when(inputContext).getContainerConfiguration(); + doReturn(new TezCounters()).when(inputContext).getCounters(); + + MRInputForTest input = new MRInputForTest(inputContext, 1); + input.initialize(); + + Configuration mergedConfig = input.getConfiguration(); + + assertEquals("local-value", mergedConfig.get("local-key")); + assertEquals("payload-value", mergedConfig.get("payload-key")); + } + /** * Test class to verify */ diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/TestMultiMRInput.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/TestMultiMRInput.java index 8d77a0539b..bd6e891bd2 100644 --- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/TestMultiMRInput.java +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/TestMultiMRInput.java @@ -102,7 +102,7 @@ public void test0PhysicalInputs() throws Exception { jobConf.setInputFormat(org.apache.hadoop.mapred.SequenceFileInputFormat.class); FileInputFormat.setInputPaths(jobConf, workDir); - InputContext inputContext = createTezInputContext(jobConf); + InputContext inputContext = createTezInputContext(jobConf, new Configuration(false)); MultiMRInput mMrInput = new MultiMRInput(inputContext, 0); @@ -121,6 +121,25 @@ public void test0PhysicalInputs() throws Exception { } } + @Test(timeout = 5000) + public void testConfigMerge() throws Exception { + JobConf jobConf = new JobConf(false); + jobConf.set("payload-key", "payload-value"); + + Configuration localConfig = new Configuration(false); + localConfig.set("local-key", "local-value"); + + InputContext inputContext = createTezInputContext(jobConf, localConfig); + + MultiMRInputForTest input = new MultiMRInputForTest(inputContext, 1); + input.initialize(); + + Configuration mergedConfig = input.getConfiguration(); + + assertEquals("local-value", mergedConfig.get("local-key")); + assertEquals("payload-value", mergedConfig.get("payload-key")); + } + @Test(timeout = 5000) public void testSingleSplit() throws Exception { @@ -129,7 +148,7 @@ public void testSingleSplit() throws Exception { jobConf.setInputFormat(org.apache.hadoop.mapred.SequenceFileInputFormat.class); FileInputFormat.setInputPaths(jobConf, workDir); - InputContext inputContext = createTezInputContext(jobConf); + InputContext inputContext = createTezInputContext(jobConf, new Configuration(false)); MultiMRInput input = new MultiMRInput(inputContext, 1); input.initialize(); @@ -180,7 +199,7 @@ public void testNewFormatSplits() throws Exception { splitProto.toByteString().asReadOnlyByteBuffer()); // Create input context. - InputContext inputContext = createTezInputContext(conf); + InputContext inputContext = createTezInputContext(conf, new Configuration(false)); // Create the MR input object and process the event MultiMRInput input = new MultiMRInput(inputContext, 1); @@ -198,7 +217,7 @@ public void testMultipleSplits() throws Exception { jobConf.setInputFormat(org.apache.hadoop.mapred.SequenceFileInputFormat.class); FileInputFormat.setInputPaths(jobConf, workDir); - InputContext inputContext = createTezInputContext(jobConf); + InputContext inputContext = createTezInputContext(jobConf, new Configuration(false)); MultiMRInput input = new MultiMRInput(inputContext, 2); input.initialize(); @@ -265,7 +284,7 @@ public void testExtraEvents() throws Exception { jobConf.setInputFormat(org.apache.hadoop.mapred.SequenceFileInputFormat.class); FileInputFormat.setInputPaths(jobConf, workDir); - InputContext inputContext = createTezInputContext(jobConf); + InputContext inputContext = createTezInputContext(jobConf, new Configuration(false)); MultiMRInput input = new MultiMRInput(inputContext, 1); input.initialize(); @@ -308,10 +327,10 @@ private LinkedHashMap createSplits(int splitCount, Path work return data; } - private InputContext createTezInputContext(Configuration conf) throws Exception { + private InputContext createTezInputContext(Configuration payloadConf, Configuration baseConf) throws Exception { MRInputUserPayloadProto.Builder builder = MRInputUserPayloadProto.newBuilder(); builder.setGroupingEnabled(false); - builder.setConfigurationBytes(TezUtils.createByteStringFromConf(conf)); + builder.setConfigurationBytes(TezUtils.createByteStringFromConf(payloadConf)); byte[] payload = builder.build().toByteArray(); ApplicationId applicationId = ApplicationId.newInstance(10000, 1); @@ -330,6 +349,7 @@ private InputContext createTezInputContext(Configuration conf) throws Exception doReturn(UUID.randomUUID().toString()).when(inputContext).getUniqueIdentifier(); doReturn("taskVertexName").when(inputContext).getTaskVertexName(); doReturn(UserPayload.create(ByteBuffer.wrap(payload))).when(inputContext).getUserPayload(); + doReturn(baseConf).when(inputContext).getContainerConfiguration(); return inputContext; } diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/output/TestMROutput.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/output/TestMROutput.java index c60ca228b3..bfc09dc9b8 100644 --- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/output/TestMROutput.java +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/output/TestMROutput.java @@ -94,7 +94,8 @@ public void testNewAPI_TextOutputFormat() throws Exception { tmpDir.getPath()) .build(); - OutputContext outputContext = createMockOutputContext(dataSink.getOutputDescriptor().getUserPayload()); + OutputContext outputContext = createMockOutputContext(dataSink.getOutputDescriptor().getUserPayload(), + new Configuration(false)); MROutput output = new MROutput(outputContext, 2); output.initialize(); @@ -109,6 +110,27 @@ public void testNewAPI_TextOutputFormat() throws Exception { assertEquals(FileOutputCommitter.class, output.committer.getClass()); } + @Test + public void testMergeConfig() throws Exception { + String outputPath = "/tmp/output"; + Configuration localConf = new Configuration(false); + localConf.set("local-key", "local-value"); + DataSinkDescriptor dataSink = MROutput + .createConfigBuilder(localConf, org.apache.hadoop.mapred.TextOutputFormat.class, outputPath) + .build(); + + Configuration baseConf = new Configuration(false); + baseConf.set("base-key", "base-value"); + + OutputContext outputContext = createMockOutputContext(dataSink.getOutputDescriptor().getUserPayload(), baseConf); + MROutput output = new MROutput(outputContext, 2); + output.initialize(); + + Configuration mergedConf = output.jobConf; + assertEquals("local-value", mergedConf.get("local-key")); + assertEquals("base-value", mergedConf.get("base-key")); + } + @Test(timeout = 5000) public void testOldAPI_TextOutputFormat() throws Exception { Configuration conf = new Configuration(); @@ -119,7 +141,8 @@ public void testOldAPI_TextOutputFormat() throws Exception { tmpDir.getPath()) .build(); - OutputContext outputContext = createMockOutputContext(dataSink.getOutputDescriptor().getUserPayload()); + OutputContext outputContext = createMockOutputContext(dataSink.getOutputDescriptor().getUserPayload(), + new Configuration(false)); MROutput output = new MROutput(outputContext, 2); output.initialize(); @@ -144,7 +167,8 @@ public void testNewAPI_SequenceFileOutputFormat() throws Exception { tmpDir.getPath()) .build(); - OutputContext outputContext = createMockOutputContext(dataSink.getOutputDescriptor().getUserPayload()); + OutputContext outputContext = createMockOutputContext(dataSink.getOutputDescriptor().getUserPayload(), + new Configuration(false)); MROutput output = new MROutput(outputContext, 2); output.initialize(); assertEquals(true, output.useNewApi); @@ -169,7 +193,8 @@ public void testOldAPI_SequenceFileOutputFormat() throws Exception { tmpDir.getPath()) .build(); - OutputContext outputContext = createMockOutputContext(dataSink.getOutputDescriptor().getUserPayload()); + OutputContext outputContext = createMockOutputContext(dataSink.getOutputDescriptor().getUserPayload(), + new Configuration(false)); MROutput output = new MROutput(outputContext, 2); output.initialize(); assertEquals(false, output.useNewApi); @@ -194,7 +219,8 @@ public void testNewAPI_WorkOutputPathOutputFormat() throws Exception { tmpDir.getPath()) .build(); - OutputContext outputContext = createMockOutputContext(dataSink.getOutputDescriptor().getUserPayload()); + OutputContext outputContext = createMockOutputContext(dataSink.getOutputDescriptor().getUserPayload(), + new Configuration(false)); MROutput output = new MROutput(outputContext, 2); output.initialize(); @@ -220,7 +246,8 @@ public void testOldAPI_WorkOutputPathOutputFormat() throws Exception { tmpDir.getPath()) .build(); - OutputContext outputContext = createMockOutputContext(dataSink.getOutputDescriptor().getUserPayload()); + OutputContext outputContext = createMockOutputContext(dataSink.getOutputDescriptor().getUserPayload(), + new Configuration(false)); MROutput output = new MROutput(outputContext, 2); output.initialize(); @@ -235,7 +262,7 @@ public void testOldAPI_WorkOutputPathOutputFormat() throws Exception { assertEquals(org.apache.hadoop.mapred.FileOutputCommitter.class, output.committer.getClass()); } - private OutputContext createMockOutputContext(UserPayload payload) { + private OutputContext createMockOutputContext(UserPayload payload, Configuration baseConf) { OutputContext outputContext = mock(OutputContext.class); ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1); when(outputContext.getUserPayload()).thenReturn(payload); @@ -243,6 +270,7 @@ private OutputContext createMockOutputContext(UserPayload payload) { when(outputContext.getTaskVertexIndex()).thenReturn(1); when(outputContext.getTaskAttemptNumber()).thenReturn(1); when(outputContext.getCounters()).thenReturn(new TezCounters()); + when(outputContext.getContainerConfiguration()).thenReturn(baseConf); return outputContext; } diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/output/TestMROutputLegacy.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/output/TestMROutputLegacy.java index 01b5c84e70..60596be89d 100644 --- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/output/TestMROutputLegacy.java +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/output/TestMROutputLegacy.java @@ -23,6 +23,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Text; @@ -182,6 +183,7 @@ private OutputContext createMockOutputContext(UserPayload payload) { when(outputContext.getTaskVertexIndex()).thenReturn(1); when(outputContext.getTaskAttemptNumber()).thenReturn(1); when(outputContext.getCounters()).thenReturn(new TezCounters()); + when(outputContext.getContainerConfiguration()).thenReturn(new Configuration(false)); return outputContext; } } diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/output/TestMultiMROutput.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/output/TestMultiMROutput.java index c8eca16027..2662827678 100644 --- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/output/TestMultiMROutput.java +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/output/TestMultiMROutput.java @@ -23,6 +23,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter; @@ -107,6 +108,34 @@ public void testInvalidBasePath() throws Exception { } } + @Test + public void testMergeConf() throws Exception { + JobConf payloadConf = new JobConf(); + payloadConf.set("local-key", "local-value"); + DataSinkDescriptor dataSink = MultiMROutput.createConfigBuilder( + payloadConf, SequenceFileOutputFormat.class, "/output", false).build(); + + Configuration baseConf = new Configuration(false); + baseConf.set("base-key", "base-value"); + + OutputContext outputContext = mock(OutputContext.class); + ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1); + when(outputContext.getUserPayload()).thenReturn(dataSink.getOutputDescriptor().getUserPayload()); + when(outputContext.getApplicationId()).thenReturn(appId); + when(outputContext.getTaskVertexIndex()).thenReturn(1); + when(outputContext.getTaskAttemptNumber()).thenReturn(1); + when(outputContext.getCounters()).thenReturn(new TezCounters()); + when(outputContext.getStatisticsReporter()).thenReturn(mock(OutputStatisticsReporter.class)); + when(outputContext.getContainerConfiguration()).thenReturn(baseConf); + + MultiMROutput output = new MultiMROutput(outputContext, 2); + output.initialize(); + + Configuration mergedConf = output.jobConf; + assertEquals("base-value", mergedConf.get("base-key")); + assertEquals("local-value", mergedConf.get("local-key")); + } + private OutputContext createMockOutputContext(UserPayload payload) { OutputContext outputContext = mock(OutputContext.class); ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1); @@ -117,6 +146,7 @@ private OutputContext createMockOutputContext(UserPayload payload) { when(outputContext.getCounters()).thenReturn(new TezCounters()); when(outputContext.getStatisticsReporter()).thenReturn( mock(OutputStatisticsReporter.class)); + when(outputContext.getContainerConfiguration()).thenReturn(new Configuration(false)); return outputContext; } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java index dccde823e7..a47dac1e0a 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java @@ -56,6 +56,7 @@ public abstract class TezTaskContextImpl implements TaskContext, Closeable { protected final String taskVertexName; protected final TezTaskAttemptID taskAttemptID; private final TezCounters counters; + private Configuration configuration; private String[] workDirs; private String uniqueIdentifier; protected final LogicalIOProcessorRuntimeTask runtimeTask; @@ -91,6 +92,7 @@ public TezTaskContextImpl(Configuration conf, String[] workDirs, int appAttemptN Objects.requireNonNull(descriptor, "descriptor is null"); Objects.requireNonNull(sharedExecutor, "sharedExecutor is null"); this.dagName = dagName; + this.configuration = conf; this.taskVertexName = taskVertexName; this.taskAttemptID = taskAttemptID; this.counters = counters; @@ -135,6 +137,11 @@ public int getTaskAttemptNumber() { return taskAttemptID.getId(); } + @Override + public Configuration getContainerConfiguration() { + return configuration; + } + @Override public String getDAGName() { return dagName; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java index c1879bc364..2b405bb343 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java @@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import com.google.common.annotations.VisibleForTesting; +import org.apache.tez.common.TezUtils; import org.apache.tez.runtime.api.ProgressFailedException; import org.apache.tez.runtime.library.api.IOInterruptedException; import org.apache.tez.runtime.library.common.Constants; @@ -37,7 +38,6 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.RawComparator; -import org.apache.tez.common.TezUtils; import org.apache.tez.common.TezRuntimeFrameworkConfigs; import org.apache.tez.common.counters.TaskCounter; import org.apache.tez.common.counters.TezCounter; @@ -97,7 +97,7 @@ public OrderedGroupedKVInput(InputContext inputContext, int numPhysicalInputs) { @Override public synchronized List initialize() throws IOException { - this.conf = TezUtils.createConfFromUserPayload(getContext().getUserPayload()); + this.conf = TezUtils.createConfFromBaseConfAndPayload(getContext()); if (this.getNumPhysicalInputs() == 0) { getContext().requestInitialMemory(0l, null); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java index 401066dfc4..1db786995a 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java @@ -24,6 +24,7 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.tez.common.TezUtils; import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.runtime.api.ProgressFailedException; import org.apache.tez.runtime.library.common.Constants; @@ -36,7 +37,6 @@ import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.DefaultCodec; import org.apache.hadoop.util.ReflectionUtils; -import org.apache.tez.common.TezUtils; import org.apache.tez.common.TezRuntimeFrameworkConfigs; import org.apache.tez.common.counters.TaskCounter; import org.apache.tez.common.counters.TezCounter; @@ -88,7 +88,7 @@ public UnorderedKVInput(InputContext inputContext, int numPhysicalInputs) { @Override public synchronized List initialize() throws Exception { Preconditions.checkArgument(getNumPhysicalInputs() != -1, "Number of Inputs has not been set"); - this.conf = TezUtils.createConfFromUserPayload(getContext().getUserPayload()); + this.conf = TezUtils.createConfFromBaseConfAndPayload(getContext()); if (getNumPhysicalInputs() == 0) { getContext().requestInitialMemory(0l, null); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java index 86c20dd9e3..676fe17a5f 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java @@ -30,6 +30,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; +import org.apache.tez.common.TezUtils; import org.apache.tez.runtime.library.conf.OrderedPartitionedKVOutputConfig.SorterImpl; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,7 +41,6 @@ import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.TezRuntimeFrameworkConfigs; -import org.apache.tez.common.TezUtils; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.runtime.api.AbstractLogicalOutput; import org.apache.tez.runtime.api.Event; @@ -90,7 +90,7 @@ public OrderedPartitionedKVOutput(OutputContext outputContext, int numPhysicalOu @Override public synchronized List initialize() throws IOException { this.startTime = System.nanoTime(); - this.conf = TezUtils.createConfFromUserPayload(getContext().getUserPayload()); + this.conf = TezUtils.createConfFromBaseConfAndPayload(getContext()); this.localFs = (RawLocalFileSystem) FileSystem.getLocal(conf).getRaw(); // Initializing this parametr in this conf since it is used in multiple diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java index 85368f6ea9..e7a4429d95 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java @@ -25,6 +25,7 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.tez.common.TezUtils; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,7 +33,6 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.conf.Configuration; -import org.apache.tez.common.TezUtils; import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.TezRuntimeFrameworkConfigs; import org.apache.tez.common.counters.TaskCounter; @@ -62,8 +62,9 @@ public class UnorderedKVOutput extends AbstractLogicalOutput { @VisibleForTesting UnorderedPartitionedKVWriter kvWriter; - - private Configuration conf; + + @VisibleForTesting + Configuration conf; private MemoryUpdateCallbackHandler memoryUpdateCallbackHandler; private final AtomicBoolean isStarted = new AtomicBoolean(false); @@ -76,7 +77,7 @@ public UnorderedKVOutput(OutputContext outputContext, int numPhysicalOutputs) { @Override public synchronized List initialize() throws Exception { - this.conf = TezUtils.createConfFromUserPayload(getContext().getUserPayload()); + this.conf = TezUtils.createConfFromBaseConfAndPayload(getContext()); this.conf.setStrings(TezRuntimeFrameworkConfigs.LOCAL_DIRS, getContext().getWorkDirs()); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java index 5e223d6c40..439b732db5 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java @@ -26,14 +26,15 @@ import java.util.concurrent.atomic.AtomicBoolean; import org.apache.tez.common.Preconditions; +import com.google.common.annotations.VisibleForTesting; +import org.apache.tez.common.TezUtils; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.conf.Configuration; -import org.apache.tez.common.TezUtils; import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.TezRuntimeFrameworkConfigs; import org.apache.tez.common.counters.TaskCounter; @@ -57,7 +58,8 @@ public class UnorderedPartitionedKVOutput extends AbstractLogicalOutput { private static final Logger LOG = LoggerFactory.getLogger(UnorderedPartitionedKVOutput.class); - private Configuration conf; + @VisibleForTesting + Configuration conf; private MemoryUpdateCallbackHandler memoryUpdateCallbackHandler; private UnorderedPartitionedKVWriter kvWriter; private final AtomicBoolean isStarted = new AtomicBoolean(false); @@ -68,7 +70,7 @@ public UnorderedPartitionedKVOutput(OutputContext outputContext, int numPhysical @Override public synchronized List initialize() throws Exception { - this.conf = TezUtils.createConfFromUserPayload(getContext().getUserPayload()); + this.conf = TezUtils.createConfFromBaseConfAndPayload(getContext()); this.conf.setStrings(TezRuntimeFrameworkConfigs.LOCAL_DIRS, getContext().getWorkDirs()); this.conf.setInt(TezRuntimeFrameworkConfigs.TEZ_RUNTIME_NUM_EXPECTED_PARTITIONS, getNumPhysicalOutputs()); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/input/TestOrderedGroupedKVInput.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/input/TestOrderedGroupedKVInput.java index d4be80211a..56b6805a63 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/input/TestOrderedGroupedKVInput.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/input/TestOrderedGroupedKVInput.java @@ -14,6 +14,7 @@ package org.apache.tez.runtime.library.input; +import static org.junit.Assert.assertEquals; import static org.mockito.Matchers.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; @@ -57,6 +58,34 @@ public void testInterruptWhileAwaitingInput() throws IOException, TezException { } + @Test + public void testMergeConfig() throws IOException, TezException { + Configuration baseConf = new Configuration(false); + baseConf.set("base-key", "base-value"); + + Configuration payloadConf = new Configuration(false); + payloadConf.set("local-key", "local-value"); + + InputContext inputContext = mock(InputContext.class); + + UserPayload payLoad = TezUtils.createUserPayloadFromConf(payloadConf); + String[] workingDirs = new String[]{"workDir1"}; + TezCounters counters = new TezCounters(); + + + doReturn(payLoad).when(inputContext).getUserPayload(); + doReturn(workingDirs).when(inputContext).getWorkDirs(); + doReturn(counters).when(inputContext).getCounters(); + doReturn(baseConf).when(inputContext).getContainerConfiguration(); + + OrderedGroupedKVInput input = new OrderedGroupedKVInput(inputContext, 1); + input.initialize(); + + Configuration mergedConf = input.conf; + assertEquals("base-value", mergedConf.get("base-key")); + assertEquals("local-value", mergedConf.get("local-key")); + } + private InputContext createMockInputContext() throws IOException { InputContext inputContext = mock(InputContext.class); @@ -70,6 +99,7 @@ private InputContext createMockInputContext() throws IOException { doReturn(workingDirs).when(inputContext).getWorkDirs(); doReturn(200 * 1024 * 1024l).when(inputContext).getTotalMemoryAvailableToTask(); doReturn(counters).when(inputContext).getCounters(); + doReturn(new Configuration(false)).when(inputContext).getContainerConfiguration(); doAnswer(new Answer() { @Override diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java index 573d53e7b1..b81c2bd036 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java @@ -51,10 +51,12 @@ static OutputContext createOutputContext() throws IOException { doReturn(200 * 1024 * 1024l).when(outputContext).getTotalMemoryAvailableToTask(); doReturn(counters).when(outputContext).getCounters(); doReturn(statsReporter).when(outputContext).getStatisticsReporter(); + doReturn(new Configuration(false)).when(outputContext).getContainerConfiguration(); return outputContext; } - static OutputContext createOutputContext(Configuration conf, Path workingDir) throws IOException { + static OutputContext createOutputContext(Configuration conf, Configuration userPayloadConf, Path workingDir) + throws IOException { OutputContext ctx = mock(OutputContext.class); doAnswer(new Answer() { @Override public Void answer(InvocationOnMock invocation) throws Throwable { @@ -65,7 +67,8 @@ static OutputContext createOutputContext(Configuration conf, Path workingDir) th return null; } }).when(ctx).requestInitialMemory(anyLong(), any(MemoryUpdateCallback.class)); - doReturn(TezUtils.createUserPayloadFromConf(conf)).when(ctx).getUserPayload(); + doReturn(conf).when(ctx).getContainerConfiguration(); + doReturn(TezUtils.createUserPayloadFromConf(userPayloadConf)).when(ctx).getUserPayload(); doReturn("destinationVertex").when(ctx).getDestinationVertexName(); doReturn("UUID").when(ctx).getUniqueIdentifier(); doReturn(new String[] { workingDir.toString() }).when(ctx).getWorkDirs(); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileSortedOutput.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileSortedOutput.java index 77620258dc..2c9c3b2ace 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileSortedOutput.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileSortedOutput.java @@ -44,7 +44,6 @@ import org.apache.tez.runtime.library.common.sort.impl.dflt.DefaultSorter; import org.apache.tez.runtime.library.conf.OrderedPartitionedKVOutputConfig.SorterImpl; import org.apache.tez.runtime.library.partitioner.HashPartitioner; -import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads; import org.junit.After; import org.junit.Assert; @@ -378,6 +377,7 @@ public void testAllEmptyPartition() throws Exception { private OutputContext createTezOutputContext() throws IOException { String[] workingDirs = { workingDir.toString() }; + Configuration localConf = new Configuration(false); UserPayload payLoad = TezUtils.createUserPayloadFromConf(conf); DataOutputBuffer serviceProviderMetaData = new DataOutputBuffer(); serviceProviderMetaData.writeInt(PORT); @@ -400,6 +400,7 @@ private OutputContext createTezOutputContext() throws IOException { OutputContext context = mock(OutputContext.class); + doReturn(localConf).when(context).getContainerConfiguration(); doReturn(counters).when(context).getCounters(); doReturn(workingDirs).when(context).getWorkDirs(); doReturn(payLoad).when(context).getUserPayload(); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileUnorderedKVOutput.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileUnorderedKVOutput.java index 393ac2e71d..963300cd40 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileUnorderedKVOutput.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileUnorderedKVOutput.java @@ -128,7 +128,7 @@ public void testGeneratedDataMovementEvent() throws Exception { conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS, IntWritable.class.getName()); TezSharedExecutor sharedExecutor = new TezSharedExecutor(conf); - OutputContext outputContext = createOutputContext(conf, sharedExecutor); + OutputContext outputContext = createOutputContext(conf, new Configuration(false), sharedExecutor); UnorderedKVOutput kvOutput = new UnorderedKVOutput(outputContext, 1); @@ -161,6 +161,26 @@ public void testGeneratedDataMovementEvent() throws Exception { sharedExecutor.shutdownNow(); } + @Test + public void testMergeConfig() throws Exception { + Configuration baseConf = new Configuration(false); + baseConf.set("local-key", "local-value"); + + Configuration payloadConf = new Configuration(false); + payloadConf.set("base-key", "base-value"); + + TezSharedExecutor sharedExecutor = new TezSharedExecutor(baseConf); + OutputContext outputContext = createOutputContext(payloadConf, baseConf, sharedExecutor); + + UnorderedKVOutput kvOutput = new UnorderedKVOutput(outputContext, 1); + + kvOutput.initialize(); + + Configuration mergedConf = kvOutput.conf; + assertEquals("local-value", mergedConf.get("local-key")); + assertEquals("base-value", mergedConf.get("base-key")); + } + @Test(timeout = 30000) @SuppressWarnings("unchecked") public void testWithPipelinedShuffle() throws Exception { @@ -173,7 +193,7 @@ public void testWithPipelinedShuffle() throws Exception { conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_UNORDERED_OUTPUT_BUFFER_SIZE_MB, 1); TezSharedExecutor sharedExecutor = new TezSharedExecutor(conf); - OutputContext outputContext = createOutputContext(conf, sharedExecutor); + OutputContext outputContext = createOutputContext(conf, new Configuration(false), sharedExecutor); UnorderedKVOutput kvOutput = new UnorderedKVOutput(outputContext, 1); @@ -211,8 +231,8 @@ public void testWithPipelinedShuffle() throws Exception { sharedExecutor.shutdownNow(); } - private OutputContext createOutputContext(Configuration conf, TezSharedExecutor sharedExecutor) - throws IOException { + private OutputContext createOutputContext(Configuration payloadConf, Configuration baseConf, + TezSharedExecutor sharedExecutor) throws IOException { int appAttemptNumber = 1; TezUmbilical tezUmbilical = mock(TezUmbilical.class); String dagName = "currentDAG"; @@ -222,7 +242,7 @@ private OutputContext createOutputContext(Configuration conf, TezSharedExecutor TezVertexID vertexID = TezVertexID.getInstance(dagID, 1); TezTaskID taskID = TezTaskID.getInstance(vertexID, 1); TezTaskAttemptID taskAttemptID = TezTaskAttemptID.getInstance(taskID, 1); - UserPayload userPayload = TezUtils.createUserPayloadFromConf(conf); + UserPayload userPayload = TezUtils.createUserPayloadFromConf(payloadConf); TaskSpec mockSpec = mock(TaskSpec.class); when(mockSpec.getInputs()).thenReturn(Collections.singletonList(mock(InputSpec.class))); @@ -237,17 +257,17 @@ private OutputContext createOutputContext(Configuration conf, TezSharedExecutor ByteBuffer bb = ByteBuffer.allocate(4); bb.putInt(shufflePort); bb.position(0); - AuxiliaryServiceHelper.setServiceDataIntoEnv(conf.get(TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID, + AuxiliaryServiceHelper.setServiceDataIntoEnv(payloadConf.get(TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID, TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID_DEFAULT), bb, auxEnv); OutputDescriptor outputDescriptor = mock(OutputDescriptor.class); when(outputDescriptor.getClassName()).thenReturn("OutputDescriptor"); - OutputContext realOutputContext = new TezOutputContextImpl(conf, new String[] {workDir.toString()}, + OutputContext realOutputContext = new TezOutputContextImpl(baseConf, new String[] {workDir.toString()}, appAttemptNumber, tezUmbilical, dagName, taskVertexName, destinationVertexName, -1, taskAttemptID, 0, userPayload, runtimeTask, - null, auxEnv, new MemoryDistributor(1, 1, conf) , outputDescriptor, null, + null, auxEnv, new MemoryDistributor(1, 1, payloadConf), outputDescriptor, null, new ExecutionContextImpl("localhost"), 2048, new TezSharedExecutor(defaultConf)); verify(runtimeTask, times(1)).addAndGetTezCounter(destinationVertexName); verify(runtimeTask, times(1)).getTaskStatistics(); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOrderedPartitionedKVOutput2.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOrderedPartitionedKVOutput2.java index f226b7c385..29ce890309 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOrderedPartitionedKVOutput2.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOrderedPartitionedKVOutput2.java @@ -69,7 +69,7 @@ public void cleanup() throws IOException { @Test(timeout = 5000) public void testNonStartedOutput() throws IOException { - OutputContext outputContext = OutputTestHelpers.createOutputContext(conf, workingDir); + OutputContext outputContext = OutputTestHelpers.createOutputContext(conf, conf, workingDir); int numPartitions = 10; OrderedPartitionedKVOutput output = new OrderedPartitionedKVOutput(outputContext, numPartitions); output.initialize(); @@ -94,9 +94,24 @@ public void testNonStartedOutput() throws IOException { } } + @Test(timeout = 5000) + public void testConfigMerge() throws IOException { + Configuration localConf = new Configuration(conf); + localConf.set("config-from-local", "config-from-local-value"); + Configuration payload = new Configuration(false); + payload.set("config-from-payload", "config-from-payload-value"); + OutputContext outputContext = OutputTestHelpers.createOutputContext(localConf, payload, workingDir); + int numPartitions = 10; + OrderedPartitionedKVOutput output = new OrderedPartitionedKVOutput(outputContext, numPartitions); + output.initialize(); + Configuration configAfterMerge = output.conf; + assertEquals("config-from-local-value", configAfterMerge.get("config-from-local")); + assertEquals("config-from-payload-value", configAfterMerge.get("config-from-payload")); + } + @Test(timeout = 10000) public void testClose() throws Exception { - OutputContext outputContext = OutputTestHelpers.createOutputContext(conf, workingDir); + OutputContext outputContext = OutputTestHelpers.createOutputContext(conf, conf, workingDir); int numPartitions = 10; OrderedPartitionedKVOutput output = new OrderedPartitionedKVOutput(outputContext, numPartitions); output.initialize(); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestUnorderedKVOutput2.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestUnorderedKVOutput2.java index 792b03f572..a52788e716 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestUnorderedKVOutput2.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestUnorderedKVOutput2.java @@ -93,9 +93,24 @@ public void testNonStartedOutput() throws Exception { } } + @Test(timeout = 5000) + public void testConfigMerge() throws Exception { + Configuration localConf = new Configuration(conf); + localConf.set("config-from-local", "config-from-local-value"); + Configuration payload = new Configuration(false); + payload.set("config-from-payload", "config-from-payload-value"); + OutputContext outputContext = OutputTestHelpers.createOutputContext(localConf, payload, workingDir); + int numPartitions = 10; + UnorderedKVOutput output = new UnorderedKVOutput(outputContext, numPartitions); + output.initialize(); + Configuration configAfterMerge = output.conf; + assertEquals("config-from-local-value", configAfterMerge.get("config-from-local")); + assertEquals("config-from-payload-value", configAfterMerge.get("config-from-payload")); + } + @Test(timeout = 10000) public void testClose() throws Exception { - OutputContext outputContext = OutputTestHelpers.createOutputContext(conf, workingDir); + OutputContext outputContext = OutputTestHelpers.createOutputContext(conf, conf, workingDir); int numPartitions = 1; UnorderedKVOutput output = new UnorderedKVOutput(outputContext, numPartitions); output.initialize(); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestUnorderedPartitionedKVOutput2.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestUnorderedPartitionedKVOutput2.java index eec4bf59e3..52e06300dd 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestUnorderedPartitionedKVOutput2.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestUnorderedPartitionedKVOutput2.java @@ -22,6 +22,8 @@ import java.util.List; import com.google.protobuf.ByteString; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.runtime.api.Event; @@ -59,4 +61,21 @@ public void testNonStartedOutput() throws Exception { assertTrue(emptyPartionsBitSet.get(i)); } } + + @Test + public void testConfigMerge() throws Exception { + Configuration userPayloadConf = new Configuration(false); + Configuration baseConf = new Configuration(false); + + userPayloadConf.set("local-key", "local-value"); + baseConf.set("base-key", "base-value"); + OutputContext outputContext = OutputTestHelpers.createOutputContext( + userPayloadConf, baseConf, new Path("/")); + UnorderedPartitionedKVOutput output = + new UnorderedPartitionedKVOutput(outputContext, 1); + output.initialize(); + Configuration mergedConf = output.conf; + assertEquals("base-value", mergedConf.get("base-key")); + assertEquals("local-value", mergedConf.get("local-key")); + } } From 28cf7dcd3b5cd422755e61b21848c31397c157d7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Thu, 28 May 2020 17:16:40 +0200 Subject: [PATCH 240/512] =?UTF-8?q?TEZ-4135:=20Improve=20memory=20allocati?= =?UTF-8?q?on=20when=20executing=20in-memory=20reads=20(L=C3=A1szl=C3=B3?= =?UTF-8?q?=20Bodor=20reviewed=20by=20Ashutosh=20Chauhan)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../library/common/TezRuntimeUtils.java | 19 +++++ .../library/common/sort/impl/IFile.java | 22 +++++- .../common/shuffle/TestShuffleUtils.java | 69 ++++++++++++++++++- .../library/common/sort/impl/TestIFile.java | 44 +++++++++--- .../runtime/library/testutils/KVDataGen.java | 20 ++++-- 5 files changed, 159 insertions(+), 15 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java index 8e13c130f7..8be8fa20d1 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java @@ -26,6 +26,7 @@ import java.nio.ByteBuffer; import org.apache.hadoop.io.DataInputByteBuffer; +import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.http.BaseHttpConnection; import org.apache.tez.http.HttpConnection; @@ -36,6 +37,7 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.tez.dag.api.TezUncheckedException; import org.apache.tez.runtime.api.OutputContext; import org.apache.tez.runtime.api.TaskContext; @@ -261,4 +263,21 @@ public static int deserializeShuffleProviderMetaData(ByteBuffer meta) in.close(); } } + + public static String getBufferSizeProperty(CompressionCodec codec) { + switch (codec.getClass().getSimpleName().toString()) { + case "DefaultCodec": + return "io.file.buffer.size"; + case "SnappyCodec": + return CommonConfigurationKeys.IO_COMPRESSION_CODEC_SNAPPY_BUFFERSIZE_KEY; + case "ZStandardCodec": + return CommonConfigurationKeys.IO_COMPRESSION_CODEC_ZSTD_BUFFER_SIZE_KEY; + case "LzoCodec": + return CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY; + case "Lz4Codec": + return CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZ4_BUFFERSIZE_KEY; + default: + return null; + } + } } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java index 09abf27bc8..5d8e944424 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java @@ -30,11 +30,13 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.io.BoundedByteArrayOutputStream; +import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -824,7 +826,7 @@ public static void readToMemory(byte[] buffer, InputStream in, int compressedLen decompressor = CodecPool.getDecompressor(codec); if (decompressor != null) { decompressor.reset(); - in = codec.createInputStream(checksumIn, decompressor); + in = getDecompressedInputStreamWithBufferSize(codec, checksumIn, decompressor, compressedLength); } else { LOG.warn("Could not obtain decompressor from CodecPool"); in = checksumIn; @@ -860,6 +862,24 @@ public static void readToMemory(byte[] buffer, InputStream in, int compressedLen } } + private static InputStream getDecompressedInputStreamWithBufferSize(CompressionCodec codec, + IFileInputStream checksumIn, Decompressor decompressor, int compressedLength) + throws IOException { + String bufferSizeProp = TezRuntimeUtils.getBufferSizeProperty(codec); + + if (bufferSizeProp != null) { + Configurable configurableCodec = (Configurable) codec; + Configuration conf = configurableCodec.getConf(); + + int bufSize = Math.min(compressedLength, DEFAULT_BUFFER_SIZE); + LOG.trace("buffer size was set according to min(compressedLength, {}): {}={}", + DEFAULT_BUFFER_SIZE, bufferSizeProp, bufSize); + conf.setInt(bufferSizeProp, bufSize); + } + + return codec.createInputStream(checksumIn, decompressor); + } + /** * Read entire IFile content to disk. * diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java index cc918fa7ae..520dec7e70 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java @@ -3,6 +3,7 @@ import com.google.common.collect.Lists; import com.google.protobuf.ByteString; +import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -10,6 +11,8 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.CompressionInputStream; +import org.apache.hadoop.io.compress.CompressionOutputStream; +import org.apache.hadoop.io.compress.Compressor; import org.apache.hadoop.io.compress.Decompressor; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.tez.common.TezCommonUtils; @@ -286,7 +289,7 @@ public void testInternalErrorTranslation() throws Exception { when(mockCodecStream.read(any(byte[].class), anyInt(), anyInt())) .thenThrow(new InternalError(codecErrorMsg)); Decompressor mockDecoder = mock(Decompressor.class); - CompressionCodec mockCodec = mock(CompressionCodec.class); + CompressionCodec mockCodec = mock(ConfigurableCodecForTest.class); when(mockCodec.createDecompressor()).thenReturn(mockDecoder); when(mockCodec.createInputStream(any(InputStream.class), any(Decompressor.class))) .thenReturn(mockCodecStream); @@ -308,7 +311,7 @@ public void testExceptionTranslation() throws Exception { when(mockCodecStream.read(any(byte[].class), anyInt(), anyInt())) .thenThrow(new IllegalArgumentException(codecErrorMsg)); Decompressor mockDecoder = mock(Decompressor.class); - CompressionCodec mockCodec = mock(CompressionCodec.class); + CompressionCodec mockCodec = mock(ConfigurableCodecForTest.class); when(mockCodec.createDecompressor()).thenReturn(mockDecoder); when(mockCodec.createInputStream(any(InputStream.class), any(Decompressor.class))) .thenReturn(mockCodecStream); @@ -396,4 +399,66 @@ public void testFetchStatsLogger() throws Exception { verify(activeLogger, times(1000)).info(anyString()); verify(aggregateLogger, times(1)).info(anyString(), Matchers.anyVararg()); } + + /** + * A codec class which implements CompressionCodec, Configurable for testing purposes. + */ + public static class ConfigurableCodecForTest implements CompressionCodec, Configurable { + + @Override + public Compressor createCompressor() { + return null; + } + + @Override + public Decompressor createDecompressor() { + return null; + } + + @Override + public CompressionInputStream createInputStream(InputStream arg0) throws IOException { + return null; + } + + @Override + public CompressionInputStream createInputStream(InputStream arg0, Decompressor arg1) + throws IOException { + return null; + } + + @Override + public CompressionOutputStream createOutputStream(OutputStream arg0) throws IOException { + return null; + } + + @Override + public CompressionOutputStream createOutputStream(OutputStream arg0, Compressor arg1) + throws IOException { + return null; + } + + @Override + public Class getCompressorType() { + return null; + } + + @Override + public Class getDecompressorType() { + return null; + } + + @Override + public String getDefaultExtension() { + return null; + } + + @Override + public Configuration getConf() { + return null; + } + + @Override + public void setConf(Configuration arg0) { + } + } } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java index 518f733802..b9c556df7f 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java @@ -18,6 +18,13 @@ package org.apache.tez.runtime.library.common.sort.impl; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -29,12 +36,7 @@ import java.util.List; import java.util.Random; -import com.google.protobuf.ByteString; -import org.apache.tez.common.TezRuntimeFrameworkConfigs; -import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutputFiles; -import org.junit.Assert; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.ChecksumException; import org.apache.hadoop.fs.FSDataInputStream; @@ -42,7 +44,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.BoundedByteArrayOutputStream; -import org.apache.tez.runtime.library.utils.BufferUtils; import org.apache.hadoop.io.DataInputBuffer; import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.IntWritable; @@ -51,18 +52,25 @@ import org.apache.hadoop.io.compress.CompressionCodecFactory; import org.apache.hadoop.io.serializer.Deserializer; import org.apache.hadoop.io.serializer.SerializationFactory; +import org.apache.tez.common.TezRuntimeFrameworkConfigs; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; +import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.InMemoryReader; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.InMemoryWriter; import org.apache.tez.runtime.library.common.sort.impl.IFile.Reader; import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer; +import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutputFiles; import org.apache.tez.runtime.library.testutils.KVDataGen; import org.apache.tez.runtime.library.testutils.KVDataGen.KVPair; +import org.apache.tez.runtime.library.utils.BufferUtils; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import static org.junit.Assert.*; +import com.google.protobuf.ByteString; public class TestIFile { @@ -714,6 +722,26 @@ public void testReadToDisk() throws IOException { reader.close(); } + @Test + public void testInMemoryBufferSize() throws IOException { + // for smaller amount of data, codec buffer should be sized according to compressed data length + List data = KVDataGen.generateTestData(false, rnd.nextInt(100)); + Writer writer = writeTestFile(false, false, data, codec); + readAndVerifyData(writer.getRawLength(), writer.getCompressedLength(), data, codec); + + Configurable configurableCodec = (Configurable) codec; + Assert.assertEquals(writer.getCompressedLength(), + configurableCodec.getConf().getInt(TezRuntimeUtils.getBufferSizeProperty(codec), 0)); + + // buffer size cannot grow infinitely with compressed data size + data = KVDataGen.generateTestDataOfKeySize(false, 20000, rnd.nextInt(100)); + writer = writeTestFile(false, false, data, codec); + readAndVerifyData(writer.getRawLength(), writer.getCompressedLength(), data, codec); + + Assert.assertEquals(128*1024, + configurableCodec.getConf().getInt(TezRuntimeUtils.getBufferSizeProperty(codec), 0)); + } + /** * Test different options (RLE, repeat keys, compression) on reader/writer * diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/testutils/KVDataGen.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/testutils/KVDataGen.java index 318cfc5020..960d371078 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/testutils/KVDataGen.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/testutils/KVDataGen.java @@ -34,17 +34,29 @@ public static List generateTestData(boolean repeatKeys) { } /** - * Generate key value pair + * Generate key value pair. * * @param sorted whether data should be sorted by key * @param repeatCount number of keys to be repeated * @return */ public static List generateTestData(boolean sorted, int repeatCount) { + return generateTestDataOfKeySize(sorted, 5, repeatCount); + } + + /** + * Generate key value pair of given amount of keys. + * + * @param sorted whether data should be sorted by key + * @param keys number of keys + * @param repeatCount number of keys to be repeated + * @return + */ + public static List generateTestDataOfKeySize(boolean sorted, int keys, int repeatCount) { List data = new LinkedList(); Random rnd = new Random(); KVPair kvp = null; - for (int i = 0; i < 5; i++) { + for (int i = 0; i < keys; i++) { String keyStr = (sorted) ? ("key" + i) : (rnd.nextLong() + "key" + i); Text key = new Text(keyStr); IntWritable value = new IntWritable(i + repeatCount); @@ -52,7 +64,7 @@ public static List generateTestData(boolean sorted, int repeatCount) { data.add(kvp); if ((repeatCount > 0) && (i % 2 == 0)) { // Repeat this key for random number of times int count = rnd.nextInt(5); - for(int j = 0; j < count; j++) { + for (int j = 0; j < count; j++) { repeatCount++; value.set(i + rnd.nextInt()); kvp = new KVPair(key, value); @@ -60,7 +72,7 @@ public static List generateTestData(boolean sorted, int repeatCount) { } } } - //If we need to generated repeated keys, try to add some repeated keys to the end of file also. + // If we need to generated repeated keys, try to add some repeated keys to the end of file also. if (repeatCount > 0 && kvp != null) { data.add(kvp); data.add(kvp); From 6487834281f8ef5f8afd9f5ba9fba68736d5b44a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Wed, 3 Jun 2020 13:30:01 +0200 Subject: [PATCH 241/512] =?UTF-8?q?TEZ-4105:=20Tez=20job-analyzer=20tool?= =?UTF-8?q?=20to=20support=20proto=20logging=20history=20(L=C3=A1szl=C3=B3?= =?UTF-8?q?=20Bodor=20reviewed=20by=20Jonathan=20Turner=20Eagles)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- pom.xml | 5 + tez-plugins/tez-history-parser/pom.xml | 4 + .../tez/history/parser/ATSFileParser.java | 13 +- .../history/parser/ProtoHistoryParser.java | 128 +++ .../history/parser/SimpleHistoryParser.java | 244 +++--- .../history/parser/datamodel/BaseParser.java | 21 + .../tez/history/parser/datamodel/DagInfo.java | 23 +- .../parser/datamodel/TaskAttemptInfo.java | 2 +- .../apache/tez/history/TestHistoryParser.java | 5 +- .../proto/HistoryEventProtoConverter.java | 4 + .../HistoryEventProtoJsonConversion.java | 766 ++++++++++++++++++ .../logging/proto/ProtoMessageReader.java | 2 +- .../proto/TestHistoryEventProtoConverter.java | 4 +- .../org/apache/tez/analyzer/CSVResult.java | 6 + .../tez/analyzer/plugins/AnalyzerDriver.java | 2 + .../TaskAttemptResultStatisticsAnalyzer.java | 125 +++ .../tez/analyzer/plugins/TezAnalyzerBase.java | 95 ++- .../org/apache/tez/analyzer/TestAnalyzer.java | 5 +- 18 files changed, 1323 insertions(+), 131 deletions(-) create mode 100644 tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ProtoHistoryParser.java create mode 100644 tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoJsonConversion.java create mode 100644 tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskAttemptResultStatisticsAnalyzer.java diff --git a/pom.xml b/pom.xml index 7de532221e..fe67b50c93 100644 --- a/pom.xml +++ b/pom.xml @@ -196,6 +196,11 @@ tez-yarn-timeline-history ${project.version} + + org.apache.tez + tez-protobuf-history-plugin + ${project.version} + org.apache.tez tez-yarn-timeline-history diff --git a/tez-plugins/tez-history-parser/pom.xml b/tez-plugins/tez-history-parser/pom.xml index 7937fa4e50..1b1d4e8285 100644 --- a/tez-plugins/tez-history-parser/pom.xml +++ b/tez-plugins/tez-history-parser/pom.xml @@ -53,6 +53,10 @@ org.apache.tez tez-yarn-timeline-history + + org.apache.tez + tez-protobuf-history-plugin + org.apache.tez tez-yarn-timeline-history diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java index e4720d41fb..e64fb43f94 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java @@ -41,6 +41,7 @@ import java.io.InputStream; import java.util.Enumeration; import java.util.Iterator; +import java.util.List; import java.util.zip.ZipEntry; import java.util.zip.ZipFile; @@ -60,10 +61,10 @@ public class ATSFileParser extends BaseParser implements ATSData { private final File atsZipFile; - public ATSFileParser(File atsZipFile) throws TezException { + public ATSFileParser(List files) throws TezException { super(); - Preconditions.checkArgument(atsZipFile.exists(), "Zipfile " + atsZipFile + " does not exist"); - this.atsZipFile = atsZipFile; + Preconditions.checkArgument(checkFiles(files), "Zipfile " + files + " are empty or they don't exist"); + this.atsZipFile = files.get(0); //doesn't support multiple files at the moment } @Override @@ -72,14 +73,12 @@ public DagInfo getDAGData(String dagId) throws TezException { parseATSZipFile(atsZipFile); linkParsedContents(); + addRawDataToDagInfo(); return dagInfo; - } catch (IOException e) { + } catch (IOException | JSONException e) { LOG.error("Error in reading DAG ", e); throw new TezException(e); - } catch (JSONException e) { - LOG.error("Error in parsing DAG ", e); - throw new TezException(e); } catch (InterruptedException e) { throw new TezException(e); } diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ProtoHistoryParser.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ProtoHistoryParser.java new file mode 100644 index 0000000000..d28fd67d4f --- /dev/null +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ProtoHistoryParser.java @@ -0,0 +1,128 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.history.parser; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Iterator; +import java.util.List; + +import org.apache.hadoop.fs.Path; +import org.apache.tez.common.Preconditions; +import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.api.TezException; +import org.apache.tez.dag.history.logging.proto.HistoryEventProtoJsonConversion; +import org.apache.tez.dag.history.logging.proto.HistoryLoggerProtos.HistoryEventProto; +import org.apache.tez.dag.history.logging.proto.ProtoMessageReader; +import org.apache.tez.history.parser.datamodel.DagInfo; +import org.codehaus.jettison.json.JSONException; +import org.codehaus.jettison.json.JSONObject; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Strings; + +/** + * Parser utility to parse data generated by ProtoHistoryLoggingService. + */ +public class ProtoHistoryParser extends SimpleHistoryParser { + private static final Logger LOG = LoggerFactory.getLogger(ProtoHistoryParser.class); + private List protoFiles; + + public ProtoHistoryParser(List files) { + super(files); + this.protoFiles = files; + } + + /** + * Get in-memory representation of DagInfo. + * + * @return DagInfo + * @throws TezException + */ + public DagInfo getDAGData(String dagId) throws TezException { + try { + Preconditions.checkArgument(!Strings.isNullOrEmpty(dagId), "Please provide valid dagId"); + dagId = dagId.trim(); + parseContents(dagId); + linkParsedContents(); + addRawDataToDagInfo(); + return dagInfo; + } catch (IOException | JSONException e) { + LOG.error("Error in reading DAG ", e); + throw new TezException(e); + } + } + + private void parseContents(String dagId) + throws JSONException, FileNotFoundException, TezException, IOException { + JSONObjectSource source = getJsonSource(); + parse(dagId, source); + } + + private JSONObjectSource getJsonSource() throws IOException { + final TezConfiguration conf = new TezConfiguration(); + + Iterator fileIt = protoFiles.iterator(); + + JSONObjectSource source = new JSONObjectSource() { + private HistoryEventProto message = null; + private ProtoMessageReader reader = new ProtoMessageReader<>(conf, + new Path(fileIt.next().getPath()), HistoryEventProto.PARSER); + + @Override + public JSONObject next() throws JSONException { + return HistoryEventProtoJsonConversion.convertToJson(message); + } + + @Override + public boolean hasNext() throws IOException { + try { + message = (HistoryEventProto) reader.readEvent(); + return message != null; + } catch (java.io.EOFException e) { + reader.close(); + + if (!fileIt.hasNext()) { + return false; + } else { + reader = new ProtoMessageReader<>(conf, new Path(fileIt.next().getPath()), + HistoryEventProto.PARSER); + try { + message = (HistoryEventProto) reader.readEvent(); + return message != null; + } catch (java.io.EOFException e2) { + return false; + } + } + } + } + + @Override + public void close() { + try { + reader.close(); + } catch (IOException e) { + LOG.warn("error while closing ProtoMessageReader", e); + } + } + }; + return source; + } +} \ No newline at end of file diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java index 3516de74a8..db3f6488e0 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java @@ -17,9 +17,16 @@ */ package org.apache.tez.history.parser; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Scanner; + +import org.apache.tez.common.ATSConstants; import org.apache.tez.common.Preconditions; -import com.google.common.base.Strings; -import com.google.common.collect.Maps; import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.history.logging.impl.SimpleHistoryLoggingService; import org.apache.tez.dag.records.TezDAGID; @@ -38,33 +45,34 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.util.Iterator; -import java.util.Map; -import java.util.Scanner; +import com.google.common.base.Strings; +import com.google.common.collect.Maps; /** - * Parser utility to parse data generated by SimpleHistoryLogging to in-memory datamodel provided - * in org.apache.tez.history.parser.datamodel + * Parser utility to parse data generated by SimpleHistoryLogging to in-memory datamodel provided in + * org.apache.tez.history.parser.datamodel. *

*

- * Most of the information should be available. Minor info like VersionInfo may not be available, - * as it is not captured in SimpleHistoryLogging. + * Most of the information should be available. Minor info like VersionInfo may not be available, as + * it is not captured in SimpleHistoryLogging. */ public class SimpleHistoryParser extends BaseParser { private static final Logger LOG = LoggerFactory.getLogger(SimpleHistoryParser.class); - private static final String UTF8 = "UTF-8"; + protected static final String UTF8 = "UTF-8"; private final File historyFile; - - public SimpleHistoryParser(File historyFile) { + public SimpleHistoryParser(List files) { super(); - Preconditions.checkArgument(historyFile.exists(), historyFile + " does not exist"); - this.historyFile = historyFile; + Preconditions.checkArgument(checkFiles(files), files + " are empty or they don't exist"); + this.historyFile = files.get(0); //doesn't support multiple files at the moment } + protected interface JSONObjectSource { + boolean hasNext() throws IOException; + JSONObject next() throws JSONException; + void close(); + }; + /** * Get in-memory representation of DagInfo * @@ -77,13 +85,11 @@ public DagInfo getDAGData(String dagId) throws TezException { dagId = dagId.trim(); parseContents(historyFile, dagId); linkParsedContents(); + addRawDataToDagInfo(); return dagInfo; - } catch (IOException e) { + } catch (IOException | JSONException e) { LOG.error("Error in reading DAG ", e); throw new TezException(e); - } catch (JSONException e) { - LOG.error("Error in parsing DAG ", e); - throw new TezException(e); } } @@ -106,18 +112,117 @@ private void populateOtherInfo(JSONObject source, String entityName, } private void parseContents(File historyFile, String dagId) - throws JSONException, FileNotFoundException, TezException { - Scanner scanner = new Scanner(historyFile, UTF8); + throws JSONException, FileNotFoundException, TezException, IOException { + JSONObjectSource source = getJsonSource(); + + parse(dagId, source); + } + + private JSONObjectSource getJsonSource() throws FileNotFoundException { + final Scanner scanner = new Scanner(historyFile, UTF8); scanner.useDelimiter(SimpleHistoryLoggingService.RECORD_SEPARATOR); - JSONObject dagJson = null; + + JSONObjectSource source = new JSONObjectSource() { + @Override + public JSONObject next() throws JSONException { + String line = scanner.next(); + return new JSONObject(line); + } + + @Override + public boolean hasNext() throws IOException { + return scanner.hasNext(); + } + + @Override + public void close() { + scanner.close(); + } + }; + return source; + } + + protected void parse(String dagId, JSONObjectSource source) + throws JSONException, TezException, IOException { Map vertexJsonMap = Maps.newHashMap(); Map taskJsonMap = Maps.newHashMap(); Map attemptJsonMap = Maps.newHashMap(); + + readEventsFromSource(dagId, source, vertexJsonMap, taskJsonMap, attemptJsonMap); + postProcessMaps(vertexJsonMap, taskJsonMap, attemptJsonMap); + } + + protected void postProcessMaps(Map vertexJsonMap, + Map taskJsonMap, Map attemptJsonMap) + throws JSONException { + for (JSONObject jsonObject : vertexJsonMap.values()) { + VertexInfo vertexInfo = VertexInfo.create(jsonObject); + this.vertexList.add(vertexInfo); + LOG.debug("Parsed vertex {}", vertexInfo.getVertexName()); + } + for (JSONObject jsonObject : taskJsonMap.values()) { + TaskInfo taskInfo = TaskInfo.create(jsonObject); + this.taskList.add(taskInfo); + LOG.debug("Parsed task {}", taskInfo.getTaskId()); + } + for (JSONObject jsonObject : attemptJsonMap.values()) { + /** + * For converting SimpleHistoryLogging to in-memory representation + * + * We need to get "relatedEntities":[{"entity":"cn055-10.l42scl.hortonworks.com:58690", + * "entitytype":"nodeId"},{"entity":"container_1438652049951_0008_01_000152", + * "entitytype":"containerId"} and populate it in otherInfo object so that in-memory + * representation can parse it correctly + */ + JSONArray relatedEntities = jsonObject.optJSONArray(Constants.RELATED_ENTITIES); + if (relatedEntities == null) { + //This can happen when CONTAINER_EXITED abruptly. (e.g Container failed, exitCode=1) + LOG.debug("entity {} did not have related entities", + jsonObject.optJSONObject(Constants.ENTITY)); + } else { + JSONObject subJsonObject = relatedEntities.optJSONObject(0); + if (subJsonObject != null) { + String nodeId = subJsonObject.optString(Constants.ENTITY_TYPE); + if (!Strings.isNullOrEmpty(nodeId) && nodeId.equalsIgnoreCase(Constants.NODE_ID)) { + //populate it in otherInfo + JSONObject otherInfo = jsonObject.optJSONObject(Constants.OTHER_INFO); + String nodeIdVal = subJsonObject.optString(Constants.ENTITY); + if (otherInfo != null && nodeIdVal != null) { + otherInfo.put(Constants.NODE_ID, nodeIdVal); + } + } + } + + subJsonObject = relatedEntities.optJSONObject(1); + if (subJsonObject != null) { + String containerId = subJsonObject.optString(Constants.ENTITY_TYPE); + if (!Strings.isNullOrEmpty(containerId) && containerId + .equalsIgnoreCase(Constants.CONTAINER_ID)) { + //populate it in otherInfo + JSONObject otherInfo = jsonObject.optJSONObject(Constants.OTHER_INFO); + String containerIdVal = subJsonObject.optString(Constants.ENTITY); + if (otherInfo != null && containerIdVal != null) { + otherInfo.put(Constants.CONTAINER_ID, containerIdVal); + } + } + } + } + TaskAttemptInfo attemptInfo = TaskAttemptInfo.create(jsonObject); + this.attemptList.add(attemptInfo); + LOG.debug("Parsed task attempt {}", attemptInfo.getTaskAttemptId()); + } + } + + protected void readEventsFromSource(String dagId, JSONObjectSource source, + Map vertexJsonMap, Map taskJsonMap, + Map attemptJsonMap) throws JSONException, TezException, IOException{ + JSONObject dagJson = null; TezDAGID tezDAGID = TezDAGID.fromString(dagId); String userName = null; - while (scanner.hasNext()) { - String line = scanner.next(); - JSONObject jsonObject = new JSONObject(line); + + while (source.hasNext()) { + JSONObject jsonObject = source.next(); + String entity = jsonObject.getString(Constants.ENTITY); String entityType = jsonObject.getString(Constants.ENTITY_TYPE); switch (entityType) { @@ -131,9 +236,12 @@ private void parseContents(File historyFile, String dagId) // time etc). if (dagJson == null) { dagJson = jsonObject; + } else if (dagJson.optJSONObject(ATSConstants.OTHER_INFO) + .optJSONObject(ATSConstants.DAG_PLAN) == null) { + // if DAG_PLAN is not filled already, let's try to fetch it from other + dagJson.getJSONObject(ATSConstants.OTHER_INFO).put(ATSConstants.DAG_PLAN, jsonObject + .getJSONObject(ATSConstants.OTHER_INFO).getJSONObject(ATSConstants.DAG_PLAN)); } - JSONObject otherInfo = jsonObject.optJSONObject(Constants.OTHER_INFO); - JSONObject dagOtherInfo = dagJson.getJSONObject(Constants.OTHER_INFO); JSONArray relatedEntities = dagJson.optJSONArray(Constants .RELATED_ENTITIES); //UserName is present in related entities @@ -148,52 +256,52 @@ private void parseContents(File historyFile, String dagId) } } } - populateOtherInfo(otherInfo, dagOtherInfo); + populateOtherInfo(jsonObject.optJSONObject(Constants.OTHER_INFO), + dagJson.getJSONObject(Constants.OTHER_INFO)); break; case Constants.TEZ_VERTEX_ID: String vertexName = entity; TezVertexID tezVertexID = TezVertexID.fromString(vertexName); if (!tezDAGID.equals(tezVertexID.getDAGId())) { - LOG.warn(vertexName + " does not belong to " + tezDAGID); + LOG.warn("{} does not belong to {} ('{}' != '{}')}", vertexName, tezDAGID, tezDAGID, tezVertexID.getDAGId()); continue; } if (!vertexJsonMap.containsKey(vertexName)) { vertexJsonMap.put(vertexName, jsonObject); } - otherInfo = jsonObject.optJSONObject(Constants.OTHER_INFO); - populateOtherInfo(otherInfo, vertexName, vertexJsonMap); + populateOtherInfo(jsonObject.optJSONObject(Constants.OTHER_INFO), vertexName, vertexJsonMap); break; case Constants.TEZ_TASK_ID: String taskName = entity; TezTaskID tezTaskID = TezTaskID.fromString(taskName); if (!tezDAGID.equals(tezTaskID.getVertexID().getDAGId())) { - LOG.warn(taskName + " does not belong to " + tezDAGID); + LOG.warn("{} does not belong to {} ('{}' != '{}')}", taskName, tezDAGID, tezDAGID, + tezTaskID.getVertexID().getDAGId()); continue; } if (!taskJsonMap.containsKey(taskName)) { taskJsonMap.put(taskName, jsonObject); } - otherInfo = jsonObject.optJSONObject(Constants.OTHER_INFO); - populateOtherInfo(otherInfo, taskName, taskJsonMap); + populateOtherInfo(jsonObject.optJSONObject(Constants.OTHER_INFO), taskName, taskJsonMap); break; case Constants.TEZ_TASK_ATTEMPT_ID: String taskAttemptName = entity; TezTaskAttemptID tezAttemptId = TezTaskAttemptID.fromString(taskAttemptName); if (!tezDAGID.equals(tezAttemptId.getTaskID().getVertexID().getDAGId())) { - LOG.warn(taskAttemptName + " does not belong to " + tezDAGID); + LOG.warn("{} does not belong to {} ('{}' != '{}')}", taskAttemptName, tezDAGID, tezDAGID, + tezAttemptId.getTaskID().getVertexID().getDAGId()); continue; } if (!attemptJsonMap.containsKey(taskAttemptName)) { attemptJsonMap.put(taskAttemptName, jsonObject); } - otherInfo = jsonObject.optJSONObject(Constants.OTHER_INFO); - populateOtherInfo(otherInfo, taskAttemptName, attemptJsonMap); + populateOtherInfo(jsonObject.optJSONObject(Constants.OTHER_INFO), taskAttemptName, attemptJsonMap); break; default: break; } } - scanner.close(); + source.close(); if (dagJson != null) { this.dagInfo = DagInfo.create(dagJson); setUserName(userName); @@ -202,61 +310,5 @@ private void parseContents(File historyFile, String dagId) throw new TezException( "Please provide a valid/complete history log file containing " + dagId); } - for (JSONObject jsonObject : vertexJsonMap.values()) { - VertexInfo vertexInfo = VertexInfo.create(jsonObject); - this.vertexList.add(vertexInfo); - LOG.debug("Parsed vertex {}", vertexInfo.getVertexName()); - } - for (JSONObject jsonObject : taskJsonMap.values()) { - TaskInfo taskInfo = TaskInfo.create(jsonObject); - this.taskList.add(taskInfo); - LOG.debug("Parsed task {}", taskInfo.getTaskId()); - } - for (JSONObject jsonObject : attemptJsonMap.values()) { - /** - * For converting SimpleHistoryLogging to in-memory representation - * - * We need to get "relatedEntities":[{"entity":"cn055-10.l42scl.hortonworks.com:58690", - * "entitytype":"nodeId"},{"entity":"container_1438652049951_0008_01_000152", - * "entitytype":"containerId"} and populate it in otherInfo object so that in-memory - * representation can parse it correctly - */ - JSONArray relatedEntities = jsonObject.optJSONArray(Constants.RELATED_ENTITIES); - if (relatedEntities == null) { - //This can happen when CONTAINER_EXITED abruptly. (e.g Container failed, exitCode=1) - LOG.debug("entity {} did not have related entities", - jsonObject.optJSONObject(Constants.ENTITY)); - } else { - JSONObject subJsonObject = relatedEntities.optJSONObject(0); - if (subJsonObject != null) { - String nodeId = subJsonObject.optString(Constants.ENTITY_TYPE); - if (!Strings.isNullOrEmpty(nodeId) && nodeId.equalsIgnoreCase(Constants.NODE_ID)) { - //populate it in otherInfo - JSONObject otherInfo = jsonObject.optJSONObject(Constants.OTHER_INFO); - String nodeIdVal = subJsonObject.optString(Constants.ENTITY); - if (otherInfo != null && nodeIdVal != null) { - otherInfo.put(Constants.NODE_ID, nodeIdVal); - } - } - } - - subJsonObject = relatedEntities.optJSONObject(1); - if (subJsonObject != null) { - String containerId = subJsonObject.optString(Constants.ENTITY_TYPE); - if (!Strings.isNullOrEmpty(containerId) && containerId - .equalsIgnoreCase(Constants.CONTAINER_ID)) { - //populate it in otherInfo - JSONObject otherInfo = jsonObject.optJSONObject(Constants.OTHER_INFO); - String containerIdVal = subJsonObject.optString(Constants.ENTITY); - if (otherInfo != null && containerIdVal != null) { - otherInfo.put(Constants.CONTAINER_ID, containerIdVal); - } - } - } - } - TaskAttemptInfo attemptInfo = TaskAttemptInfo.create(jsonObject); - this.attemptList.add(attemptInfo); - LOG.debug("Parsed task attempt {}", attemptInfo.getTaskAttemptId()); - } } } \ No newline at end of file diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseParser.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseParser.java index 59ec03d972..af8e292b24 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseParser.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseParser.java @@ -25,6 +25,7 @@ import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.dag.records.TezTaskID; +import java.io.File; import java.util.List; import java.util.Map; @@ -44,6 +45,26 @@ public BaseParser() { attemptList = Lists.newLinkedList(); } + + protected boolean checkFiles(List files) { + if (files.isEmpty()) { + return false; + } + for (File file : files) { + if (!file.exists()) { + return false; + } + } + return true; + } + + + protected void addRawDataToDagInfo() { + dagInfo.addMeta("vertices", vertexList); + dagInfo.addMeta("tasks", taskList); + dagInfo.addMeta("taskAttempts", attemptList); + } + /** * link the parsed contents, so that it becomes easier to iterate from DAG-->Task and Task--DAG. * e.g Link vertex to dag, task to vertex, attempt to task etc diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/DagInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/DagInfo.java index 64ddcf862e..85fcfcfb19 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/DagInfo.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/DagInfo.java @@ -44,6 +44,7 @@ import java.util.Collection; import java.util.Collections; import java.util.Comparator; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -85,6 +86,8 @@ public class DagInfo extends BaseInfo { private Multimap containerMapping; private Map config; + private Map meta = new HashMap(); + DagInfo(JSONObject jsonObject) throws JSONException { super(jsonObject); @@ -168,6 +171,10 @@ public static DagInfo create(JSONObject jsonObject) throws JSONException { return dagInfo; } + public void addMeta(String key, Object value) { + meta.put(key, value); + } + private void parseDAGPlan(JSONObject dagPlan) throws JSONException { int version = dagPlan.optInt(Constants.VERSION, 1); parseEdges(dagPlan.optJSONArray(Constants.EDGES)); @@ -320,7 +327,7 @@ void addVertexInfo(VertexInfo vertexInfo) { BasicVertexInfo basicVertexInfo = basicVertexInfoMap.get(vertexInfo.getVertexName()); Preconditions.checkArgument(basicVertexInfo != null, - "VerteName " + vertexInfo.getVertexName() + "VertexName " + vertexInfo.getVertexName() + " not present in DAG's vertices " + basicVertexInfoMap.entrySet()); //populate additional information in VertexInfo @@ -387,6 +394,19 @@ public String toString() { return sb.toString(); } + public String toExtendedString() { + StringBuilder sb = new StringBuilder(); + sb.append(toString()); + + try { + sb.append("\nmeta=").append(new JSONObject(meta).toString(3)); + } catch (JSONException e) { + throw new RuntimeException(e); + } + + return sb.toString(); + } + public Multimap getContainerMapping() { return Multimaps.unmodifiableMultimap(containerMapping); } @@ -630,5 +650,4 @@ public final String getUserName() { final void setUserName(String userName) { this.userName = userName; } - } diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java index 3f39310d1c..3ce39bdccf 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java @@ -405,7 +405,7 @@ public String toString() { sb.append("container=").append(getContainer()).append(", "); sb.append("nodeId=").append(getNodeId()).append(", "); sb.append("logURL=").append(getLogURL()).append(", "); - sb.append("status=").append(getStatus()); + sb.append("status=").append(getDetailedStatus()); sb.append("]"); return sb.toString(); } diff --git a/tez-plugins/tez-history-parser/src/test/java/org/apache/tez/history/TestHistoryParser.java b/tez-plugins/tez-history-parser/src/test/java/org/apache/tez/history/TestHistoryParser.java index 92c4ad8683..8a054655e4 100644 --- a/tez-plugins/tez-history-parser/src/test/java/org/apache/tez/history/TestHistoryParser.java +++ b/tez-plugins/tez-history-parser/src/test/java/org/apache/tez/history/TestHistoryParser.java @@ -88,6 +88,7 @@ import java.io.File; import java.io.IOException; import java.io.OutputStreamWriter; +import java.util.Arrays; import java.util.Collection; import java.util.Iterator; import java.util.List; @@ -239,7 +240,7 @@ private DagInfo getDagInfoFromSimpleHistory(String dagId) throws TezException, I File localFile = new File(DOWNLOAD_DIR, HISTORY_TXT); //Now parse via SimpleHistory - SimpleHistoryParser parser = new SimpleHistoryParser(localFile); + SimpleHistoryParser parser = new SimpleHistoryParser(Arrays.asList(localFile)); DagInfo dagInfo = parser.getDAGData(dagId); assertTrue(dagInfo.getDagId().equals(dagId)); return dagInfo; @@ -603,7 +604,7 @@ private DagInfo getDagInfo(String dagId) throws TezException { //Parse downloaded contents File downloadedFile = new File(DOWNLOAD_DIR + Path.SEPARATOR + dagId + ".zip"); - ATSFileParser parser = new ATSFileParser(downloadedFile); + ATSFileParser parser = new ATSFileParser(Arrays.asList(downloadedFile)); DagInfo dagInfo = parser.getDAGData(dagId); assertTrue(dagInfo.getDagId().equals(dagId)); return dagInfo; diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java index 44dccb6d29..09079bdf1c 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java @@ -351,11 +351,14 @@ private HistoryEventProto convertTaskAttemptFinishedEvent(TaskAttemptFinishedEve addEventData(builder, ATSConstants.CREATION_TIME, event.getCreationTime()); addEventData(builder, ATSConstants.ALLOCATION_TIME, event.getAllocationTime()); addEventData(builder, ATSConstants.START_TIME, event.getStartTime()); + if (event.getCreationCausalTA() != null) { addEventData(builder, ATSConstants.CREATION_CAUSAL_ATTEMPT, event.getCreationCausalTA().toString()); } addEventData(builder, ATSConstants.TIME_TAKEN, (event.getFinishTime() - event.getStartTime())); + addEventData(builder, ATSConstants.STATUS, event.getState().name()); + if (event.getTaskAttemptError() != null) { addEventData(builder, ATSConstants.TASK_ATTEMPT_ERROR_ENUM, event.getTaskAttemptError().name()); @@ -447,6 +450,7 @@ private HistoryEventProto convertVertexInitializedEvent(VertexInitializedEvent e null, null, null, event.getVertexID(), null, null, null); addEventData(builder, ATSConstants.VERTEX_NAME, event.getVertexName()); addEventData(builder, ATSConstants.INIT_REQUESTED_TIME, event.getInitRequestedTime()); + addEventData(builder, ATSConstants.INIT_TIME, event.getInitedTime()); addEventData(builder, ATSConstants.NUM_TASKS, event.getNumTasks()); addEventData(builder, ATSConstants.PROCESSOR_CLASS_NAME, event.getProcessorName()); if (event.getServicePluginInfo() != null) { diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoJsonConversion.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoJsonConversion.java new file mode 100644 index 0000000000..26e20abeb3 --- /dev/null +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoJsonConversion.java @@ -0,0 +1,766 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.history.logging.proto; + +import java.util.Iterator; +import java.util.Optional; + +import org.apache.tez.common.ATSConstants; +import org.apache.tez.dag.history.HistoryEventType; +import org.apache.tez.dag.history.logging.EntityTypes; +import org.apache.tez.dag.history.logging.proto.HistoryLoggerProtos.HistoryEventProto; +import org.apache.tez.dag.history.logging.proto.HistoryLoggerProtos.KVPair; +import org.codehaus.jettison.json.JSONArray; +import org.codehaus.jettison.json.JSONException; +import org.codehaus.jettison.json.JSONObject; + +/** + * Convert HistoryEventProto into JSONObject for analyzers, which can already consume the output of + * SimpleHistoryLoggingService's JSONs. This class is based on HistoryEventJsonConversion, and all + * of the specific HistoryEvent calls were transformed info HistoryEventProto calls by taking the + * corresponding HistoryEventProtoConverter methods into consideration. + */ +public final class HistoryEventProtoJsonConversion { + + private HistoryEventProtoJsonConversion() { + } + + public static JSONObject convertToJson(HistoryEventProto historyEvent) throws JSONException { + JSONObject jsonObject = null; + + switch (historyEvent.getEventType()) { + case "APP_LAUNCHED": + jsonObject = convertAppLaunchedEvent(historyEvent); + break; + case "AM_LAUNCHED": + jsonObject = convertAMLaunchedEvent(historyEvent); + break; + case "AM_STARTED": + jsonObject = convertAMStartedEvent(historyEvent); + break; + case "CONTAINER_LAUNCHED": + jsonObject = convertContainerLaunchedEvent(historyEvent); + break; + case "CONTAINER_STOPPED": + jsonObject = convertContainerStoppedEvent(historyEvent); + break; + case "DAG_SUBMITTED": + jsonObject = convertDAGSubmittedEvent(historyEvent); + break; + case "DAG_INITIALIZED": + jsonObject = convertDAGInitializedEvent(historyEvent); + break; + case "DAG_STARTED": + jsonObject = convertDAGStartedEvent(historyEvent); + break; + case "DAG_FINISHED": + jsonObject = convertDAGFinishedEvent(historyEvent); + break; + case "VERTEX_INITIALIZED": + jsonObject = convertVertexInitializedEvent(historyEvent); + break; + case "VERTEX_STARTED": + jsonObject = convertVertexStartedEvent(historyEvent); + break; + case "VERTEX_FINISHED": + jsonObject = convertVertexFinishedEvent(historyEvent); + break; + case "TASK_STARTED": + jsonObject = convertTaskStartedEvent(historyEvent); + break; + case "TASK_FINISHED": + jsonObject = convertTaskFinishedEvent(historyEvent); + break; + case "TASK_ATTEMPT_STARTED": + jsonObject = convertTaskAttemptStartedEvent(historyEvent); + break; + case "TASK_ATTEMPT_FINISHED": + jsonObject = convertTaskAttemptFinishedEvent(historyEvent); + break; + case "VERTEX_CONFIGURE_DONE": + jsonObject = convertVertexReconfigureDoneEvent(historyEvent); + break; + case "DAG_RECOVERED": + jsonObject = convertDAGRecoveredEvent(historyEvent); + break; + case "VERTEX_COMMIT_STARTED": + case "VERTEX_GROUP_COMMIT_STARTED": + case "VERTEX_GROUP_COMMIT_FINISHED": + case "DAG_COMMIT_STARTED": + throw new UnsupportedOperationException( + "Invalid Event, does not support history" + ", eventType=" + historyEvent.getEventType()); + default: + throw new UnsupportedOperationException( + "Unhandled Event" + ", eventType=" + historyEvent.getEventType()); + } + return jsonObject; + } + + private static JSONObject convertDAGRecoveredEvent(HistoryEventProto event) throws JSONException { + JSONObject jsonObject = new JSONObject(); + jsonObject.put(ATSConstants.ENTITY, event.getDagId()); + jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); + + // Related Entities not needed as should have been done in + // dag submission event + + JSONArray events = new JSONArray(); + JSONObject recoverEvent = new JSONObject(); + recoverEvent.put(ATSConstants.TIMESTAMP, event.getEventTime()); + recoverEvent.put(ATSConstants.EVENT_TYPE, HistoryEventType.DAG_RECOVERED.name()); + + JSONObject recoverEventInfo = new JSONObject(); + recoverEventInfo.put(ATSConstants.APPLICATION_ATTEMPT_ID, event.getAppAttemptId().toString()); + recoverEventInfo.put(ATSConstants.DAG_STATE, getDataValueByKey(event, ATSConstants.DAG_STATE)); + recoverEventInfo.put(ATSConstants.RECOVERY_FAILURE_REASON, + getDataValueByKey(event, ATSConstants.RECOVERY_FAILURE_REASON)); + + recoverEvent.put(ATSConstants.EVENT_INFO, recoverEventInfo); + events.put(recoverEvent); + + jsonObject.put(ATSConstants.EVENTS, events); + + return jsonObject; + } + + private static JSONObject convertAppLaunchedEvent(HistoryEventProto event) throws JSONException { + JSONObject jsonObject = new JSONObject(); + jsonObject.put(ATSConstants.ENTITY, "tez_" + event.getAppId().toString()); + jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_APPLICATION.name()); + + // Other info to tag with Tez App + JSONObject otherInfo = new JSONObject(); + otherInfo.put(ATSConstants.USER, event.getUser()); + otherInfo.put(ATSConstants.CONFIG, new JSONObject()); // TODO: config from proto? + + jsonObject.put(ATSConstants.OTHER_INFO, otherInfo); + + return jsonObject; + } + + private static JSONObject convertAMLaunchedEvent(HistoryEventProto event) throws JSONException { + JSONObject jsonObject = new JSONObject(); + jsonObject.put(ATSConstants.ENTITY, "tez_" + event.getAppAttemptId().toString()); + jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_APPLICATION_ATTEMPT.name()); + + // Related Entities + JSONArray relatedEntities = new JSONArray(); + JSONObject appEntity = new JSONObject(); + appEntity.put(ATSConstants.ENTITY, event.getAppId().toString()); + appEntity.put(ATSConstants.ENTITY_TYPE, ATSConstants.APPLICATION_ID); + JSONObject appAttemptEntity = new JSONObject(); + appAttemptEntity.put(ATSConstants.ENTITY, event.getAppAttemptId().toString()); + appAttemptEntity.put(ATSConstants.ENTITY_TYPE, ATSConstants.APPLICATION_ATTEMPT_ID); + relatedEntities.put(appEntity); + relatedEntities.put(appAttemptEntity); + jsonObject.put(ATSConstants.RELATED_ENTITIES, relatedEntities); + + // TODO decide whether this goes into different events, + // event info or other info. + JSONArray events = new JSONArray(); + JSONObject initEvent = new JSONObject(); + initEvent.put(ATSConstants.TIMESTAMP, event.getEventTime()); + initEvent.put(ATSConstants.EVENT_TYPE, HistoryEventType.AM_LAUNCHED.name()); + events.put(initEvent); + jsonObject.put(ATSConstants.EVENTS, events); + + // Other info to tag with Tez AM + JSONObject otherInfo = new JSONObject(); + otherInfo.put(ATSConstants.APP_SUBMIT_TIME, + getDataValueByKey(event, ATSConstants.APP_SUBMIT_TIME)); + jsonObject.put(ATSConstants.OTHER_INFO, otherInfo); + + return jsonObject; + } + + private static JSONObject convertAMStartedEvent(HistoryEventProto event) throws JSONException { + JSONObject jsonObject = new JSONObject(); + jsonObject.put(ATSConstants.ENTITY, "tez_" + event.getAppAttemptId().toString()); + jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_APPLICATION_ATTEMPT.name()); + + // Related Entities + JSONArray relatedEntities = new JSONArray(); + JSONObject appEntity = new JSONObject(); + appEntity.put(ATSConstants.ENTITY, event.getAppId().toString()); + appEntity.put(ATSConstants.ENTITY_TYPE, ATSConstants.APPLICATION_ID); + JSONObject appAttemptEntity = new JSONObject(); + appAttemptEntity.put(ATSConstants.ENTITY, event.getAppAttemptId().toString()); + appAttemptEntity.put(ATSConstants.ENTITY_TYPE, ATSConstants.APPLICATION_ATTEMPT_ID); + relatedEntities.put(appEntity); + relatedEntities.put(appAttemptEntity); + jsonObject.put(ATSConstants.RELATED_ENTITIES, relatedEntities); + + // TODO decide whether this goes into different events, + // event info or other info. + JSONArray events = new JSONArray(); + JSONObject startEvent = new JSONObject(); + startEvent.put(ATSConstants.TIMESTAMP, event.getEventTime()); + startEvent.put(ATSConstants.EVENT_TYPE, HistoryEventType.AM_STARTED.name()); + events.put(startEvent); + jsonObject.put(ATSConstants.EVENTS, events); + + return jsonObject; + } + + private static JSONObject convertContainerLaunchedEvent(HistoryEventProto event) + throws JSONException { + JSONObject jsonObject = new JSONObject(); + jsonObject.put(ATSConstants.ENTITY, + "tez_" + getDataValueByKey(event, ATSConstants.CONTAINER_ID)); + jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_CONTAINER_ID.name()); + + JSONArray relatedEntities = new JSONArray(); + JSONObject appAttemptEntity = new JSONObject(); + appAttemptEntity.put(ATSConstants.ENTITY, event.getAppAttemptId().toString()); + appAttemptEntity.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_APPLICATION_ATTEMPT.name()); + + JSONObject containerEntity = new JSONObject(); + containerEntity.put(ATSConstants.ENTITY, getDataValueByKey(event, ATSConstants.CONTAINER_ID)); + containerEntity.put(ATSConstants.ENTITY_TYPE, ATSConstants.CONTAINER_ID); + + relatedEntities.put(appAttemptEntity); + relatedEntities.put(containerEntity); + jsonObject.put(ATSConstants.RELATED_ENTITIES, relatedEntities); + + // TODO decide whether this goes into different events, + // event info or other info. + JSONArray events = new JSONArray(); + JSONObject launchEvent = new JSONObject(); + launchEvent.put(ATSConstants.TIMESTAMP, event.getEventTime()); + launchEvent.put(ATSConstants.EVENT_TYPE, HistoryEventType.CONTAINER_LAUNCHED.name()); + events.put(launchEvent); + jsonObject.put(ATSConstants.EVENTS, events); + + // TODO add other container info here? or assume AHS will have this? + // TODO container logs? + + return jsonObject; + } + + private static JSONObject convertContainerStoppedEvent(HistoryEventProto event) + throws JSONException { + // structure is identical to ContainerLaunchedEvent + JSONObject jsonObject = new JSONObject(); + jsonObject.put(ATSConstants.ENTITY, + "tez_" + getDataValueByKey(event, ATSConstants.CONTAINER_ID)); + jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_CONTAINER_ID.name()); + + JSONArray relatedEntities = new JSONArray(); + JSONObject appAttemptEntity = new JSONObject(); + appAttemptEntity.put(ATSConstants.ENTITY, event.getAppAttemptId().toString()); + appAttemptEntity.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_APPLICATION_ATTEMPT.name()); + + JSONObject containerEntity = new JSONObject(); + containerEntity.put(ATSConstants.ENTITY, getDataValueByKey(event, ATSConstants.CONTAINER_ID)); + containerEntity.put(ATSConstants.ENTITY_TYPE, ATSConstants.CONTAINER_ID); + + relatedEntities.put(appAttemptEntity); + relatedEntities.put(containerEntity); + jsonObject.put(ATSConstants.RELATED_ENTITIES, relatedEntities); + + // TODO decide whether this goes into different events, + // event info or other info. + JSONArray events = new JSONArray(); + JSONObject stopEvent = new JSONObject(); + stopEvent.put(ATSConstants.TIMESTAMP, event.getEventTime()); + stopEvent.put(ATSConstants.EVENT_TYPE, HistoryEventType.CONTAINER_STOPPED.name()); + events.put(stopEvent); + jsonObject.put(ATSConstants.EVENTS, events); + + // TODO add other container info here? or assume AHS will have this? + // TODO container logs? + + // Other info + JSONObject otherInfo = new JSONObject(); + otherInfo.put(ATSConstants.EXIT_STATUS, getDataValueByKey(event, ATSConstants.EXIT_STATUS)); + jsonObject.put(ATSConstants.OTHER_INFO, otherInfo); + + return jsonObject; + } + + private static JSONObject convertDAGFinishedEvent(HistoryEventProto event) throws JSONException { + JSONObject jsonObject = new JSONObject(); + jsonObject.put(ATSConstants.ENTITY, event.getDagId()); + jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); + + // Related Entities not needed as should have been done in + // dag submission event + + // TODO decide whether this goes into different events, + // event info or other info. + JSONArray events = new JSONArray(); + JSONObject finishEvent = new JSONObject(); + finishEvent.put(ATSConstants.TIMESTAMP, event.getEventTime()); + finishEvent.put(ATSConstants.EVENT_TYPE, HistoryEventType.DAG_FINISHED.name()); + events.put(finishEvent); + jsonObject.put(ATSConstants.EVENTS, events); + + JSONObject otherInfo = new JSONObject(); + + long startTime = getLongDataValueByKey(event, ATSConstants.START_TIME); + + otherInfo.put(ATSConstants.START_TIME, startTime); + otherInfo.put(ATSConstants.FINISH_TIME, event.getEventTime()); + otherInfo.put(ATSConstants.TIME_TAKEN, event.getEventTime() - startTime); + otherInfo.put(ATSConstants.STATUS, getDataValueByKey(event, ATSConstants.STATUS)); + otherInfo.put(ATSConstants.DIAGNOSTICS, getDataValueByKey(event, ATSConstants.DIAGNOSTICS)); + otherInfo.put(ATSConstants.COUNTERS, getJSONDataValueByKey(event, ATSConstants.COUNTERS)); + otherInfo.put(ATSConstants.COMPLETION_APPLICATION_ATTEMPT_ID, + event.getAppAttemptId().toString()); + + // added all info to otherInfo in order to cover + // all key/value pairs added from event.getDagTaskStats() + Iterator it = event.getEventDataList().iterator(); + while (it.hasNext()) { + KVPair pair = it.next(); + otherInfo.put(pair.getKey(), pair.getValue()); + } + + jsonObject.put(ATSConstants.OTHER_INFO, otherInfo); + + return jsonObject; + } + + private static JSONObject convertDAGInitializedEvent(HistoryEventProto event) + throws JSONException { + JSONObject jsonObject = new JSONObject(); + jsonObject.put(ATSConstants.ENTITY, event.getDagId()); + jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); + + // Related Entities not needed as should have been done in + // dag submission event + + JSONArray events = new JSONArray(); + JSONObject initEvent = new JSONObject(); + initEvent.put(ATSConstants.TIMESTAMP, event.getEventTime()); + initEvent.put(ATSConstants.EVENT_TYPE, HistoryEventType.DAG_INITIALIZED.name()); + events.put(initEvent); + jsonObject.put(ATSConstants.EVENTS, events); + + JSONObject otherInfo = new JSONObject(); + otherInfo.put(ATSConstants.VERTEX_NAME_ID_MAPPING, + getJSONDataValueByKey(event, ATSConstants.VERTEX_NAME_ID_MAPPING)); + jsonObject.put(ATSConstants.OTHER_INFO, otherInfo); + + return jsonObject; + } + + private static JSONObject convertDAGStartedEvent(HistoryEventProto event) throws JSONException { + JSONObject jsonObject = new JSONObject(); + jsonObject.put(ATSConstants.ENTITY, event.getDagId()); + jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); + + // Related Entities not needed as should have been done in + // dag submission event + + // TODO decide whether this goes into different events, + // event info or other info. + JSONArray events = new JSONArray(); + JSONObject startEvent = new JSONObject(); + startEvent.put(ATSConstants.TIMESTAMP, event.getEventTime()); + startEvent.put(ATSConstants.EVENT_TYPE, HistoryEventType.DAG_STARTED.name()); + events.put(startEvent); + jsonObject.put(ATSConstants.EVENTS, events); + + return jsonObject; + } + + private static JSONObject convertDAGSubmittedEvent(HistoryEventProto event) throws JSONException { + JSONObject jsonObject = new JSONObject(); + jsonObject.put(ATSConstants.ENTITY, event.getDagId()); + jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); + + // Related Entities + JSONArray relatedEntities = new JSONArray(); + JSONObject tezAppEntity = new JSONObject(); + tezAppEntity.put(ATSConstants.ENTITY, "tez_" + event.getAppId().toString()); + tezAppEntity.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_APPLICATION.name()); + JSONObject tezAppAttemptEntity = new JSONObject(); + tezAppAttemptEntity.put(ATSConstants.ENTITY, "tez_" + event.getAppAttemptId().toString()); + tezAppAttemptEntity.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_APPLICATION_ATTEMPT.name()); + JSONObject appEntity = new JSONObject(); + appEntity.put(ATSConstants.ENTITY, event.getAppId().toString()); + appEntity.put(ATSConstants.ENTITY_TYPE, ATSConstants.APPLICATION_ID); + JSONObject appAttemptEntity = new JSONObject(); + appAttemptEntity.put(ATSConstants.ENTITY, event.getAppAttemptId().toString()); + appAttemptEntity.put(ATSConstants.ENTITY_TYPE, ATSConstants.APPLICATION_ATTEMPT_ID); + JSONObject userEntity = new JSONObject(); + userEntity.put(ATSConstants.ENTITY, event.getUser()); + userEntity.put(ATSConstants.ENTITY_TYPE, ATSConstants.USER); + + relatedEntities.put(tezAppEntity); + relatedEntities.put(tezAppAttemptEntity); + relatedEntities.put(appEntity); + relatedEntities.put(appAttemptEntity); + relatedEntities.put(userEntity); + jsonObject.put(ATSConstants.RELATED_ENTITIES, relatedEntities); + + // filters + JSONObject primaryFilters = new JSONObject(); + primaryFilters.put(ATSConstants.DAG_NAME, getDataValueByKey(event, ATSConstants.DAG_NAME)); + primaryFilters.put(ATSConstants.CALLER_CONTEXT_ID, + getDataValueByKey(event, ATSConstants.CALLER_CONTEXT_ID)); + primaryFilters.put(ATSConstants.CALLER_CONTEXT_TYPE, + getDataValueByKey(event, ATSConstants.CALLER_CONTEXT_TYPE)); + primaryFilters.put(ATSConstants.DAG_QUEUE_NAME, + getDataValueByKey(event, ATSConstants.DAG_QUEUE_NAME)); + + jsonObject.put(ATSConstants.PRIMARY_FILTERS, primaryFilters); + + // TODO decide whether this goes into different events, + // event info or other info. + JSONArray events = new JSONArray(); + JSONObject submitEvent = new JSONObject(); + submitEvent.put(ATSConstants.TIMESTAMP, event.getEventTime()); + submitEvent.put(ATSConstants.EVENT_TYPE, HistoryEventType.DAG_SUBMITTED.name()); + events.put(submitEvent); + jsonObject.put(ATSConstants.EVENTS, events); + + // Other info such as dag plan + JSONObject otherInfo = new JSONObject(); + otherInfo.put(ATSConstants.DAG_PLAN, getJSONDataValueByKey(event, ATSConstants.DAG_PLAN)); + + otherInfo.put(ATSConstants.CALLER_CONTEXT_ID, + getDataValueByKey(event, ATSConstants.CALLER_CONTEXT_ID)); + otherInfo.put(ATSConstants.CALLER_CONTEXT_TYPE, + getDataValueByKey(event, ATSConstants.CALLER_CONTEXT_TYPE)); + otherInfo.put(ATSConstants.DAG_QUEUE_NAME, + getDataValueByKey(event, ATSConstants.DAG_QUEUE_NAME)); + + jsonObject.put(ATSConstants.OTHER_INFO, otherInfo); + + return jsonObject; + } + + private static JSONObject convertTaskAttemptFinishedEvent(HistoryEventProto event) + throws JSONException { + JSONObject jsonObject = new JSONObject(); + jsonObject.put(ATSConstants.ENTITY, event.getTaskAttemptId()); + jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_TASK_ATTEMPT_ID.name()); + + // Events + JSONArray events = new JSONArray(); + JSONObject finishEvent = new JSONObject(); + finishEvent.put(ATSConstants.TIMESTAMP, event.getEventTime()); + finishEvent.put(ATSConstants.EVENT_TYPE, HistoryEventType.TASK_ATTEMPT_FINISHED.name()); + events.put(finishEvent); + jsonObject.put(ATSConstants.EVENTS, events); + + JSONObject otherInfo = new JSONObject(); + long startTime = getLongDataValueByKey(event, ATSConstants.START_TIME); + + otherInfo.put(ATSConstants.CREATION_TIME, getDataValueByKey(event, ATSConstants.CREATION_TIME)); + otherInfo.put(ATSConstants.ALLOCATION_TIME, + getDataValueByKey(event, ATSConstants.ALLOCATION_TIME)); + otherInfo.put(ATSConstants.START_TIME, startTime); + otherInfo.put(ATSConstants.FINISH_TIME, event.getEventTime()); + otherInfo.put(ATSConstants.TIME_TAKEN, event.getEventTime() - startTime); + + otherInfo.put(ATSConstants.CREATION_CAUSAL_ATTEMPT, + getDataValueByKey(event, ATSConstants.CREATION_CAUSAL_ATTEMPT)); + otherInfo.put(ATSConstants.STATUS, getDataValueByKey(event, ATSConstants.STATUS)); + + otherInfo.put(ATSConstants.STATUS, getDataValueByKey(event, ATSConstants.STATUS)); + otherInfo.put(ATSConstants.TASK_ATTEMPT_ERROR_ENUM, + getDataValueByKey(event, ATSConstants.TASK_ATTEMPT_ERROR_ENUM)); + otherInfo.put(ATSConstants.TASK_FAILURE_TYPE, + getDataValueByKey(event, ATSConstants.TASK_FAILURE_TYPE)); + otherInfo.put(ATSConstants.DIAGNOSTICS, getDataValueByKey(event, ATSConstants.DIAGNOSTICS)); + otherInfo.put(ATSConstants.COUNTERS, getJSONDataValueByKey(event, ATSConstants.COUNTERS)); + otherInfo.put(ATSConstants.LAST_DATA_EVENTS, + getJSONDataValueByKey(event, ATSConstants.LAST_DATA_EVENTS)); + otherInfo.put(ATSConstants.NODE_ID, getDataValueByKey(event, ATSConstants.NODE_ID)); + otherInfo.put(ATSConstants.CONTAINER_ID, getDataValueByKey(event, ATSConstants.CONTAINER_ID)); + otherInfo.put(ATSConstants.IN_PROGRESS_LOGS_URL, + getDataValueByKey(event, ATSConstants.IN_PROGRESS_LOGS_URL)); + otherInfo.put(ATSConstants.COMPLETED_LOGS_URL, + getDataValueByKey(event, ATSConstants.COMPLETED_LOGS_URL)); + otherInfo.put(ATSConstants.NODE_HTTP_ADDRESS, + getDataValueByKey(event, ATSConstants.NODE_HTTP_ADDRESS)); + + jsonObject.put(ATSConstants.OTHER_INFO, otherInfo); + + return jsonObject; + } + + private static JSONObject convertTaskAttemptStartedEvent(HistoryEventProto event) + throws JSONException { + JSONObject jsonObject = new JSONObject(); + jsonObject.put(ATSConstants.ENTITY, event.getTaskAttemptId()); + jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_TASK_ATTEMPT_ID.name()); + + // Related entities + JSONArray relatedEntities = new JSONArray(); + JSONObject nodeEntity = new JSONObject(); + nodeEntity.put(ATSConstants.ENTITY, getDataValueByKey(event, ATSConstants.NODE_ID)); + nodeEntity.put(ATSConstants.ENTITY_TYPE, ATSConstants.NODE_ID); + + JSONObject containerEntity = new JSONObject(); + containerEntity.put(ATSConstants.ENTITY, getDataValueByKey(event, ATSConstants.CONTAINER_ID)); + containerEntity.put(ATSConstants.ENTITY_TYPE, ATSConstants.CONTAINER_ID); + + JSONObject taskEntity = new JSONObject(); + taskEntity.put(ATSConstants.ENTITY, event.getTaskAttemptId()); + taskEntity.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_TASK_ID.name()); + + relatedEntities.put(nodeEntity); + relatedEntities.put(containerEntity); + relatedEntities.put(taskEntity); + jsonObject.put(ATSConstants.RELATED_ENTITIES, relatedEntities); + + // Events + JSONArray events = new JSONArray(); + JSONObject startEvent = new JSONObject(); + startEvent.put(ATSConstants.TIMESTAMP, event.getEventTime()); + startEvent.put(ATSConstants.EVENT_TYPE, HistoryEventType.TASK_ATTEMPT_STARTED.name()); + events.put(startEvent); + jsonObject.put(ATSConstants.EVENTS, events); + + // Other info + JSONObject otherInfo = new JSONObject(); + otherInfo.put(ATSConstants.IN_PROGRESS_LOGS_URL, + getDataValueByKey(event, ATSConstants.IN_PROGRESS_LOGS_URL)); + otherInfo.put(ATSConstants.COMPLETED_LOGS_URL, + getDataValueByKey(event, ATSConstants.COMPLETED_LOGS_URL)); + jsonObject.put(ATSConstants.OTHER_INFO, otherInfo); + + return jsonObject; + } + + private static JSONObject convertTaskFinishedEvent(HistoryEventProto event) throws JSONException { + JSONObject jsonObject = new JSONObject(); + jsonObject.put(ATSConstants.ENTITY, event.getTaskId()); + jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_TASK_ID.name()); + + // Events + JSONArray events = new JSONArray(); + JSONObject finishEvent = new JSONObject(); + finishEvent.put(ATSConstants.TIMESTAMP, event.getEventTime()); + finishEvent.put(ATSConstants.EVENT_TYPE, HistoryEventType.TASK_FINISHED.name()); + events.put(finishEvent); + jsonObject.put(ATSConstants.EVENTS, events); + + long startTime = getLongDataValueByKey(event, ATSConstants.START_TIME); + + JSONObject otherInfo = new JSONObject(); + otherInfo.put(ATSConstants.START_TIME, startTime); + otherInfo.put(ATSConstants.FINISH_TIME, event.getEventTime()); + otherInfo.put(ATSConstants.TIME_TAKEN, event.getEventTime() - startTime); + + otherInfo.put(ATSConstants.STATUS, getDataValueByKey(event, ATSConstants.STATUS)); + otherInfo.put(ATSConstants.DIAGNOSTICS, getDataValueByKey(event, ATSConstants.DIAGNOSTICS)); + otherInfo.put(ATSConstants.COUNTERS, getJSONDataValueByKey(event, ATSConstants.COUNTERS)); + otherInfo.put(ATSConstants.SUCCESSFUL_ATTEMPT_ID, + getDataValueByKey(event, ATSConstants.SUCCESSFUL_ATTEMPT_ID)); + + jsonObject.put(ATSConstants.OTHER_INFO, otherInfo); + + return jsonObject; + } + + private static JSONObject convertTaskStartedEvent(HistoryEventProto event) throws JSONException { + JSONObject jsonObject = new JSONObject(); + jsonObject.put(ATSConstants.ENTITY, event.getTaskId()); + jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_TASK_ID.name()); + + // Related entities + JSONArray relatedEntities = new JSONArray(); + JSONObject vertexEntity = new JSONObject(); + vertexEntity.put(ATSConstants.ENTITY, event.getVertexId()); + vertexEntity.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_VERTEX_ID.name()); + relatedEntities.put(vertexEntity); + jsonObject.put(ATSConstants.RELATED_ENTITIES, relatedEntities); + + // Events + JSONArray events = new JSONArray(); + JSONObject startEvent = new JSONObject(); + startEvent.put(ATSConstants.TIMESTAMP, event.getEventTime()); + startEvent.put(ATSConstants.EVENT_TYPE, HistoryEventType.TASK_STARTED.name()); + events.put(startEvent); + jsonObject.put(ATSConstants.EVENTS, events); + + // Other info + // TODO fix schedule/launch time to be events + JSONObject otherInfo = new JSONObject(); + otherInfo.put(ATSConstants.START_TIME, event.getEventTime()); + otherInfo.put(ATSConstants.SCHEDULED_TIME, + getDataValueByKey(event, ATSConstants.SCHEDULED_TIME)); + jsonObject.put(ATSConstants.OTHER_INFO, otherInfo); + + return jsonObject; + } + + private static JSONObject convertVertexFinishedEvent(HistoryEventProto event) + throws JSONException { + JSONObject jsonObject = new JSONObject(); + jsonObject.put(ATSConstants.ENTITY, event.getVertexId()); + jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_VERTEX_ID.name()); + + // Events + JSONArray events = new JSONArray(); + JSONObject finishEvent = new JSONObject(); + finishEvent.put(ATSConstants.TIMESTAMP, event.getEventTime()); + finishEvent.put(ATSConstants.EVENT_TYPE, HistoryEventType.VERTEX_FINISHED.name()); + events.put(finishEvent); + jsonObject.put(ATSConstants.EVENTS, events); + + long startTime = getLongDataValueByKey(event, ATSConstants.START_TIME); + + JSONObject otherInfo = new JSONObject(); + otherInfo.put(ATSConstants.FINISH_TIME, event.getEventTime()); + otherInfo.put(ATSConstants.TIME_TAKEN, (event.getEventTime() - startTime)); + otherInfo.put(ATSConstants.STATUS, getDataValueByKey(event, ATSConstants.STATUS)); + otherInfo.put(ATSConstants.DIAGNOSTICS, getDataValueByKey(event, ATSConstants.DIAGNOSTICS)); + otherInfo.put(ATSConstants.COUNTERS, getJSONDataValueByKey(event, ATSConstants.COUNTERS)); + + otherInfo.put(ATSConstants.STATS, getJSONDataValueByKey(event, ATSConstants.STATS)); + + // added all info to otherInfo in order to cover + // all key/value pairs added from event.getVertexTaskStats() + Iterator it = event.getEventDataList().iterator(); + while (it.hasNext()) { + KVPair pair = it.next(); + otherInfo.put(pair.getKey(), pair.getValue()); + } + + otherInfo.put(ATSConstants.SERVICE_PLUGIN, + getJSONDataValueByKey(event, ATSConstants.SERVICE_PLUGIN)); + + jsonObject.put(ATSConstants.OTHER_INFO, otherInfo); + + return jsonObject; + } + + private static JSONObject convertVertexReconfigureDoneEvent(HistoryEventProto event) + throws JSONException { + JSONObject jsonObject = new JSONObject(); + jsonObject.put(ATSConstants.ENTITY, event.getVertexId()); + jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_VERTEX_ID.name()); + + // Events + JSONArray events = new JSONArray(); + JSONObject updateEvent = new JSONObject(); + updateEvent.put(ATSConstants.TIMESTAMP, event.getEventTime()); + updateEvent.put(ATSConstants.EVENT_TYPE, HistoryEventType.VERTEX_CONFIGURE_DONE.name()); + + JSONObject eventInfo = new JSONObject(); + eventInfo.put(ATSConstants.NUM_TASKS, getDataValueByKey(event, ATSConstants.NUM_TASKS)); + eventInfo.put(ATSConstants.UPDATED_EDGE_MANAGERS, + getJSONDataValueByKey(event, ATSConstants.UPDATED_EDGE_MANAGERS)); + updateEvent.put(ATSConstants.EVENT_INFO, eventInfo); + events.put(updateEvent); + jsonObject.put(ATSConstants.EVENTS, events); + + // Other info + JSONObject otherInfo = new JSONObject(); + jsonObject.put(ATSConstants.OTHER_INFO, otherInfo); + + // TODO add more on all other updated information + return jsonObject; + } + + private static JSONObject convertVertexInitializedEvent(HistoryEventProto event) + throws JSONException { + JSONObject jsonObject = new JSONObject(); + jsonObject.put(ATSConstants.ENTITY, event.getVertexId()); + jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_VERTEX_ID.name()); + + // Related entities + JSONArray relatedEntities = new JSONArray(); + JSONObject vertexEntity = new JSONObject(); + vertexEntity.put(ATSConstants.ENTITY, event.getDagId()); + vertexEntity.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); + relatedEntities.put(vertexEntity); + jsonObject.put(ATSConstants.RELATED_ENTITIES, relatedEntities); + + // Events + JSONArray events = new JSONArray(); + JSONObject initEvent = new JSONObject(); + initEvent.put(ATSConstants.TIMESTAMP, event.getEventTime()); + initEvent.put(ATSConstants.EVENT_TYPE, HistoryEventType.VERTEX_INITIALIZED.name()); + events.put(initEvent); + jsonObject.put(ATSConstants.EVENTS, events); + + // Other info + // TODO fix requested times to be events + JSONObject otherInfo = new JSONObject(); + otherInfo.put(ATSConstants.VERTEX_NAME, getDataValueByKey(event, ATSConstants.VERTEX_NAME)); + otherInfo.put(ATSConstants.INIT_REQUESTED_TIME, + getDataValueByKey(event, ATSConstants.INIT_REQUESTED_TIME)); + otherInfo.put(ATSConstants.INIT_TIME, getDataValueByKey(event, ATSConstants.INIT_TIME)); + otherInfo.put(ATSConstants.NUM_TASKS, getDataValueByKey(event, ATSConstants.NUM_TASKS)); + otherInfo.put(ATSConstants.PROCESSOR_CLASS_NAME, + getDataValueByKey(event, ATSConstants.PROCESSOR_CLASS_NAME)); + otherInfo.put(ATSConstants.SERVICE_PLUGIN, + getJSONDataValueByKey(event, ATSConstants.SERVICE_PLUGIN)); + jsonObject.put(ATSConstants.OTHER_INFO, otherInfo); + + return jsonObject; + } + + private static JSONObject convertVertexStartedEvent(HistoryEventProto event) + throws JSONException { + JSONObject jsonObject = new JSONObject(); + jsonObject.put(ATSConstants.ENTITY, event.getVertexId()); + jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_VERTEX_ID.name()); + + // Related entities + JSONArray relatedEntities = new JSONArray(); + JSONObject vertexEntity = new JSONObject(); + vertexEntity.put(ATSConstants.ENTITY, event.getDagId()); + vertexEntity.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); + relatedEntities.put(vertexEntity); + jsonObject.put(ATSConstants.RELATED_ENTITIES, relatedEntities); + + // Events + JSONArray events = new JSONArray(); + JSONObject startEvent = new JSONObject(); + startEvent.put(ATSConstants.TIMESTAMP, event.getEventTime()); + startEvent.put(ATSConstants.EVENT_TYPE, HistoryEventType.VERTEX_STARTED.name()); + events.put(startEvent); + jsonObject.put(ATSConstants.EVENTS, events); + + // Other info + // TODO fix requested times to be events + JSONObject otherInfo = new JSONObject(); + otherInfo.put(ATSConstants.START_REQUESTED_TIME, + getDataValueByKey(event, ATSConstants.START_REQUESTED_TIME)); + otherInfo.put(ATSConstants.START_TIME, event.getEventTime()); + jsonObject.put(ATSConstants.OTHER_INFO, otherInfo); + + return jsonObject; + } + + private static String getDataValueByKey(HistoryEventProto event, String key) { + Optional pair = + event.getEventDataList().stream().filter(p -> p.getKey().equals(key)).findAny(); + return pair.isPresent() ? pair.get().getValue() : null; + } + + private static long getLongDataValueByKey(HistoryEventProto event, String key) { + String value = getDataValueByKey(event, key); + return (value == null || value.isEmpty()) ? 0 : Long.parseLong(value); + } + + private static JSONObject getJSONDataValueByKey(HistoryEventProto event, String key) + throws JSONException { + String value = getDataValueByKey(event, key); + return (value == null || value.isEmpty()) ? null : new JSONObject(value); + } +} diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageReader.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageReader.java index d736feaa77..2cac4d8764 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageReader.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageReader.java @@ -34,7 +34,7 @@ public class ProtoMessageReader implements Closeable { private final Reader reader; private final ProtoMessageWritable writable; - ProtoMessageReader(Configuration conf, Path filePath, Parser parser) throws IOException { + public ProtoMessageReader(Configuration conf, Path filePath, Parser parser) throws IOException { this.filePath = filePath; // The writer does not flush the length during hflush. Using length options lets us read // past length in the FileStatus but it will throw EOFException during a read instead diff --git a/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestHistoryEventProtoConverter.java b/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestHistoryEventProtoConverter.java index 92d3e300d7..64f66bc12b 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestHistoryEventProtoConverter.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestHistoryEventProtoConverter.java @@ -409,7 +409,7 @@ public void testConvertTaskAttemptFinishedEvent() { "logsURL", "nodeHttpAddress"); HistoryEventProto proto = converter.convert(event); assertCommon(proto, HistoryEventType.TASK_ATTEMPT_FINISHED, finishTime, - EntityTypes.TEZ_DAG_ID, null, null, 16); + EntityTypes.TEZ_DAG_ID, null, null, 17); assertEventData(proto, ATSConstants.STATUS, state.name()); assertEventData(proto, ATSConstants.CREATION_CAUSAL_ATTEMPT, tezTaskAttemptID.toString()); @@ -499,7 +499,7 @@ public void testConvertVertexInitializedEvent() { HistoryEventProto proto = converter.convert(event); assertCommon(proto, HistoryEventType.VERTEX_INITIALIZED, initedTime, - EntityTypes.TEZ_VERTEX_ID, null, null, 5); + EntityTypes.TEZ_VERTEX_ID, null, null, 6); assertEventData(proto, ATSConstants.VERTEX_NAME, "v1"); assertEventData(proto, ATSConstants.PROCESSOR_CLASS_NAME, "proc"); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/CSVResult.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/CSVResult.java index 1da281c86e..0e167b2dcf 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/CSVResult.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/CSVResult.java @@ -32,6 +32,8 @@ import java.io.OutputStreamWriter; import java.nio.charset.Charset; import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; import java.util.Iterator; import java.util.List; @@ -65,6 +67,10 @@ public Iterator getRecordsIterator() { return Iterators.unmodifiableIterator(recordsList.iterator()); } + @SuppressWarnings({ "rawtypes", "unchecked" }) + public void sort(Comparator comparator) { + Collections.sort(recordsList, comparator); + } public void setComments(String comments) { this.comments = comments; diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/AnalyzerDriver.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/AnalyzerDriver.java index cf600c5618..cad0d98d75 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/AnalyzerDriver.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/AnalyzerDriver.java @@ -46,6 +46,8 @@ public static void main(String argv[]){ "Print spill details in a DAG"); pgd.addClass("TaskAssignmentAnalyzer", TaskAssignmentAnalyzer.class, "Print task-to-node assignment details of a DAG"); + pgd.addClass("TaskAttemptResultStatisticsAnalyzer", TaskAttemptResultStatisticsAnalyzer.class, + "Print vertex:node:status level details of task attempt results"); pgd.addClass("TaskConcurrencyAnalyzer", TaskConcurrencyAnalyzer.class, "Print the task concurrency details in a DAG"); pgd.addClass("VertexLevelCriticalPathAnalyzer", VertexLevelCriticalPathAnalyzer.class, diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskAttemptResultStatisticsAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskAttemptResultStatisticsAnalyzer.java new file mode 100644 index 0000000000..df2f95cdce --- /dev/null +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskAttemptResultStatisticsAnalyzer.java @@ -0,0 +1,125 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.analyzer.plugins; + +import java.util.Comparator; +import java.util.HashMap; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.ToolRunner; +import org.apache.tez.analyzer.Analyzer; +import org.apache.tez.analyzer.CSVResult; +import org.apache.tez.analyzer.Result; +import org.apache.tez.dag.api.TezException; +import org.apache.tez.history.parser.datamodel.DagInfo; +import org.apache.tez.history.parser.datamodel.TaskAttemptInfo; +import org.apache.tez.history.parser.datamodel.VertexInfo; + +/** + * Get simple count of task attempt states on vertex:node:status level, like below. + * + * vertex (+task stats: all/succeeded/failed/killed),node,status,numAttempts + * Map 1 (vertex_x_y_z) (216/153/0/63),node1,KILLED:INTERNAL_PREEMPTION,1185 + * Map 1 (vertex_x_y_z) (216/153/0/63),node1,KILLED:TERMINATED_AT_SHUTDOWN,22 + * Map 1 (vertex_x_y_z) (216/153/0/63),node1,KILLED:EXTERNAL_PREEMPTION,3349 + * Map 1 (vertex_x_y_z) (216/153/0/63),node1,SUCCEEDED,1 + */ +public class TaskAttemptResultStatisticsAnalyzer extends TezAnalyzerBase implements Analyzer { + private final String[] headers = + { "vertex (+task stats: all/succeeded/failed/killed)", "node", "status", "numAttempts" }; + private final Configuration config; + private final CSVResult csvResult; + + public TaskAttemptResultStatisticsAnalyzer(Configuration config) { + this.config = config; + csvResult = new CSVResult(headers); + } + + @Override + public void analyze(DagInfo dagInfo) throws TezException { + Map map = new HashMap<>(); + + for (VertexInfo vertex : dagInfo.getVertices()) { + String taskStatsInVertex = + String.format("%s/%s/%s/%s", vertex.getNumTasks(), vertex.getSucceededTasksCount(), + vertex.getFailedTasksCount(), vertex.getKilledTasksCount()); + for (TaskAttemptInfo attempt : vertex.getTaskAttempts()) { + String key = String.format("%s#%s#%s", + String.format("%s (%s) (%s)", vertex.getVertexName(), vertex.getVertexId(), + taskStatsInVertex), + attempt.getNodeId(), attempt.getDetailedStatus()); + Integer previousValue = (Integer) map.get(key); + map.put(key, previousValue == null ? 1 : previousValue + 1); + } + } + + map.forEach((key, value) -> { + addARecord(key.split("#")[0], key.split("#")[1], key.split("#")[2], value); + }); + + csvResult.sort(new Comparator() { + public int compare(String[] first, String[] second) { + int vertexOrder = first[0].compareTo(second[0]); + int nodeOrder = first[1].compareTo(second[1]); + int statusOrder = first[2].compareTo(second[2]); + + return vertexOrder == 0 ? (nodeOrder == 0 ? statusOrder : nodeOrder) : vertexOrder; + } + }); + } + + private void addARecord(String vertexData, String node, String status, + int numAttempts) { + String[] record = new String[4]; + record[0] = vertexData; + record[1] = node; + record[2] = status; + record[3] = Integer.toString(numAttempts); + csvResult.addRecord(record); + } + + @Override + public Result getResult() throws TezException { + return csvResult; + } + + @Override + public String getName() { + return "Task Attempt Result Statistics Analyzer"; + } + + @Override + public String getDescription() { + return "Get statistics about task attempts states in vertex:node:status level"; + } + + @Override + public Configuration getConfiguration() { + return config; + } + + public static void main(String[] args) throws Exception { + Configuration config = new Configuration(); + TaskAttemptResultStatisticsAnalyzer analyzer = new TaskAttemptResultStatisticsAnalyzer(config); + int res = ToolRunner.run(config, analyzer, args); + analyzer.printResults(); + System.exit(res); + } +} diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java index 59ae4a3d60..75a55a754d 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java @@ -19,7 +19,12 @@ package org.apache.tez.analyzer.plugins; import java.io.File; +import java.io.FilenameFilter; +import java.util.ArrayList; +import java.util.Arrays; import java.util.Iterator; +import java.util.List; +import java.util.stream.Collectors; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.GnuParser; @@ -27,6 +32,7 @@ import org.apache.commons.cli.OptionBuilder; import org.apache.commons.cli.Options; import org.apache.commons.cli.ParseException; +import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.fs.Path; import org.apache.hadoop.util.Tool; @@ -36,6 +42,7 @@ import org.apache.tez.dag.api.TezException; import org.apache.tez.history.ATSImportTool; import org.apache.tez.history.parser.ATSFileParser; +import org.apache.tez.history.parser.ProtoHistoryParser; import org.apache.tez.history.parser.SimpleHistoryParser; import org.apache.tez.history.parser.datamodel.DagInfo; @@ -52,6 +59,7 @@ public abstract class TezAnalyzerBase extends Configured implements Tool, Analyz private static final String SAVE_RESULTS = "saveResults"; private static final String DAG_ID = "dagId"; private static final String FROM_SIMPLE_HISTORY = "fromSimpleHistory"; + private static final String FROM_PROTO_HISTORY = "fromProtoHistory"; private static final String HELP = "help"; private static final int SEPARATOR_WIDTH = 80; @@ -81,7 +89,12 @@ private static Options buildOptions() { (FROM_SIMPLE_HISTORY) .withDescription("Event data from Simple History logging. Must also specify event file") .isRequired(false).create(); - + + Option fromProtoHistoryOption = + OptionBuilder.withArgName(FROM_PROTO_HISTORY).withLongOpt(FROM_PROTO_HISTORY) + .withDescription("Event data from Proto History logging. Must also specify event file") + .isRequired(false).create(); + Option help = OptionBuilder.withArgName(HELP).withLongOpt (HELP) .withDescription("print help") @@ -93,6 +106,7 @@ private static Options buildOptions() { opts.addOption(saveResults); opts.addOption(eventFileNameOption); opts.addOption(fromSimpleHistoryOption); + opts.addOption(fromProtoHistoryOption); opts.addOption(help); return opts; } @@ -133,21 +147,36 @@ public int run(String[] args) throws Exception { outputDir = System.getProperty("user.dir"); } - File file = null; + String dagId = cmdLine.getOptionValue(DAG_ID); + + List files = new ArrayList(); if (cmdLine.hasOption(EVENT_FILE_NAME)) { - file = new File(cmdLine.getOptionValue(EVENT_FILE_NAME)); + for (String file : cmdLine.getOptionValue(EVENT_FILE_NAME).split(",")) { + File fileOrDir = new File(file); + if (fileOrDir.exists()) { + if (fileOrDir.isFile()) { + files.add(fileOrDir); + } else { + files.addAll(collectFilesForDagId(fileOrDir, dagId)); + } + } + } } - - String dagId = cmdLine.getOptionValue(DAG_ID); - + DagInfo dagInfo = null; - if (file == null) { + if (files.isEmpty()) { if (cmdLine.hasOption(FROM_SIMPLE_HISTORY)) { System.err.println("Event file name must be specified when using simple history"); printUsage(); return -2; } + if (cmdLine.hasOption(FROM_PROTO_HISTORY)) { + System.err.println("Proto file name must be specified when using proto history"); + printUsage(); + return -2; + } + // using ATS - try to download directly String[] importArgs = { "--dagId=" + dagId, "--downloadDir=" + outputDir }; @@ -159,30 +188,60 @@ public int run(String[] args) throws Exception { //Parse ATS data and verify results //Parse downloaded contents - file = new File(outputDir - + Path.SEPARATOR + dagId + ".zip"); + files.add(new File(outputDir + + Path.SEPARATOR + dagId + ".zip")); } - Preconditions.checkState(file != null); - if (!cmdLine.hasOption(FROM_SIMPLE_HISTORY)) { - ATSFileParser parser = new ATSFileParser(file); + Preconditions.checkState(!files.isEmpty()); + if (cmdLine.hasOption(FROM_SIMPLE_HISTORY)) { + SimpleHistoryParser parser = new SimpleHistoryParser(files); + dagInfo = parser.getDAGData(dagId); + } else if (cmdLine.hasOption(FROM_PROTO_HISTORY)) { + ProtoHistoryParser parser = new ProtoHistoryParser(files); dagInfo = parser.getDAGData(dagId); } else { - SimpleHistoryParser parser = new SimpleHistoryParser(file); + ATSFileParser parser = new ATSFileParser(files); dagInfo = parser.getDAGData(dagId); } Preconditions.checkState(dagInfo.getDagId().equals(dagId)); analyze(dagInfo); Result result = getResult(); - if (saveResults && (result instanceof CSVResult)) { - String fileName = outputDir + File.separator - + this.getClass().getName() + "_" + dagInfo.getDagId() + ".csv"; - ((CSVResult) result).dumpToFile(fileName); - LOG.info("Saved results in " + fileName); + + if (saveResults) { + String dagInfoFileName = outputDir + File.separator + this.getClass().getName() + "_" + + dagInfo.getDagId() + ".dag"; + FileUtils.writeStringToFile(new File(dagInfoFileName), dagInfo.toExtendedString()); + LOG.info("Saved dag info in " + dagInfoFileName); + + if (result instanceof CSVResult) { + String fileName = outputDir + File.separator + this.getClass().getName() + "_" + + dagInfo.getDagId() + ".csv"; + ((CSVResult) result).dumpToFile(fileName); + LOG.info("Saved results in " + fileName); + } } + return 0; } + private List collectFilesForDagId(File parentDir, String dagId) { + File[] arrFiles = parentDir.listFiles(new FilenameFilter() { + @Override + public boolean accept(File dir, String name) { + return name.contains(dagId); + } + }); + if (arrFiles == null || arrFiles.length == 0) { + throw new RuntimeException( + String.format("cannot find relevant files for dag: '%s' in dir: %s", dagId, parentDir)); + } + + List files = Arrays.asList(arrFiles); + LOG.info("collected files for dag: \n" + + files.stream().map(f -> "\n" + f.getAbsolutePath()).collect(Collectors.toList())); + return files; + } + public void printResults() throws TezException { Result result = getResult(); if (result instanceof CSVResult) { diff --git a/tez-tools/analyzers/job-analyzer/src/test/java/org/apache/tez/analyzer/TestAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/test/java/org/apache/tez/analyzer/TestAnalyzer.java index d34f9c518b..606b7e3cd7 100644 --- a/tez-tools/analyzers/job-analyzer/src/test/java/org/apache/tez/analyzer/TestAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/test/java/org/apache/tez/analyzer/TestAnalyzer.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertTrue; import java.io.File; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Random; @@ -267,7 +268,7 @@ private DagInfo getDagInfo(String dagId) throws Exception { //Parse downloaded contents File downloadedFile = new File(DOWNLOAD_DIR + Path.SEPARATOR + dagId + ".zip"); - ATSFileParser parser = new ATSFileParser(downloadedFile); + ATSFileParser parser = new ATSFileParser(Arrays.asList(downloadedFile)); dagInfo = parser.getDAGData(dagId); assertTrue(dagInfo.getDagId().equals(dagId)); } else { @@ -286,7 +287,7 @@ private DagInfo getDagInfo(String dagId) throws Exception { } //Now parse via SimpleHistory File localFile = new File(DOWNLOAD_DIR, HISTORY_TXT); - SimpleHistoryParser parser = new SimpleHistoryParser(localFile); + SimpleHistoryParser parser = new SimpleHistoryParser(Arrays.asList(localFile)); dagInfo = parser.getDAGData(dagId); assertTrue(dagInfo.getDagId().equals(dagId)); } From 2c94033ea3222f78b6e2439b32d39da906127734 Mon Sep 17 00:00:00 2001 From: Attila Magyar Date: Thu, 4 Jun 2020 18:15:53 -0700 Subject: [PATCH 242/512] TEZ-4185 : Tez may skip file permission update on intermediate output (Attila Magyar via Ashutosh Chauhan) Signed-off-by: Ashutosh Chauhan --- .../common/sort/impl/PipelinedSorter.java | 14 ++----- .../common/sort/impl/TezSpillRecord.java | 9 +++-- .../common/sort/impl/dflt/DefaultSorter.java | 14 ++----- .../writers/UnorderedPartitionedKVWriter.java | 22 +++-------- .../TestUnorderedPartitionedKVWriter.java | 39 +++++++++++-------- 5 files changed, 43 insertions(+), 55 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java index 610cae9de3..bc68a5f206 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java @@ -69,7 +69,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; -import static org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord.SPILL_FILE_PERMS; +import static org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord.ensureSpillFilePermissions; @SuppressWarnings({"unchecked", "rawtypes"}) public class PipelinedSorter extends ExternalSorter { @@ -488,9 +488,7 @@ private void spillSingleRecord(final Object key, final Object value, * MAP_OUTPUT_INDEX_RECORD_LENGTH); spillFilePaths.put(numSpills, filename); FSDataOutputStream out = rfs.create(filename, true, 4096); - if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { - rfs.setPermission(filename, SPILL_FILE_PERMS); - } + ensureSpillFilePermissions(filename, rfs); try { LOG.info(outputContext.getDestinationVertexName() + ": Spilling to " + filename.toString() + @@ -576,9 +574,7 @@ public boolean spill(boolean ignoreEmptySpills) throws IOException { mapOutputFile.getSpillFileForWrite(numSpills, size); spillFilePaths.put(numSpills, filename); out = rfs.create(filename, true, 4096); - if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { - rfs.setPermission(filename, SPILL_FILE_PERMS); - } + ensureSpillFilePermissions(filename, rfs); LOG.info(outputContext.getDestinationVertexName() + ": Spilling to " + filename.toString()); for (int i = 0; i < partitions; ++i) { if (isThreadInterrupted()) { @@ -761,9 +757,7 @@ public void flush() throws IOException { } //The output stream for the final single output file FSDataOutputStream finalOut = rfs.create(finalOutputFile, true, 4096); - if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { - rfs.setPermission(finalOutputFile, SPILL_FILE_PERMS); - } + ensureSpillFilePermissions(finalOutputFile, rfs); final TezSpillRecord spillRec = new TezSpillRecord(partitions); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.java index e16b7a06b7..1c9edeead7 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.java @@ -143,10 +143,13 @@ public void writeToFile(Path loc, Configuration job, FileSystem rfs, Checksum cr } else { out.close(); } - if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(job)))) { - rfs.setPermission(loc, SPILL_FILE_PERMS); - } + ensureSpillFilePermissions(loc, rfs); } } + public static void ensureSpillFilePermissions(Path loc, FileSystem rfs) throws IOException { + if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(rfs.getConf())))) { + rfs.setPermission(loc, SPILL_FILE_PERMS); + } + } } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java index 124f0784a8..d0a18b4205 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java @@ -64,7 +64,7 @@ import org.apache.tez.common.Preconditions; -import static org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord.SPILL_FILE_PERMS; +import static org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord.ensureSpillFilePermissions; @SuppressWarnings({"unchecked", "rawtypes"}) public final class DefaultSorter extends ExternalSorter implements IndexedSortable { @@ -896,9 +896,7 @@ protected void spill(int mstart, int mend, long sameKeyCount, long totalKeysCoun mapOutputFile.getSpillFileForWrite(numSpills, size); spillFilePaths.put(numSpills, filename); out = rfs.create(filename); - if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { - rfs.setPermission(filename, SPILL_FILE_PERMS); - } + ensureSpillFilePermissions(filename, rfs); int spindex = mstart; final InMemValBytes value = createInMemValBytes(); @@ -1007,9 +1005,7 @@ private void spillSingleRecord(final Object key, final Object value, mapOutputFile.getSpillFileForWrite(numSpills, size); spillFilePaths.put(numSpills, filename); out = rfs.create(filename); - if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { - rfs.setPermission(filename, SPILL_FILE_PERMS); - } + ensureSpillFilePermissions(filename, rfs); // we don't run the combiner for a single record for (int i = 0; i < partitions; ++i) { @@ -1283,9 +1279,7 @@ private void mergeParts() throws IOException, InterruptedException { //The output stream for the final single output file FSDataOutputStream finalOut = rfs.create(finalOutputFile, true, 4096); - if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { - rfs.setPermission(finalOutputFile, SPILL_FILE_PERMS); - } + ensureSpillFilePermissions(finalOutputFile, rfs); if (numSpills == 0) { // TODO Change event generation to say there is no data rather than generating a dummy file diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java index 62170d95a3..ffce5c62ca 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java @@ -87,7 +87,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.protobuf.ByteString; -import static org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord.SPILL_FILE_PERMS; +import static org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord.ensureSpillFilePermissions; public class UnorderedPartitionedKVWriter extends BaseUnorderedPartitionedKVWriter { @@ -308,9 +308,7 @@ public UnorderedPartitionedKVWriter(OutputContext outputContext, Configuration c finalOutPath = outputFileHandler.getOutputFileForWrite(); writer = new IFile.Writer(conf, rfs, finalOutPath, keyClass, valClass, codec, outputRecordsCounter, outputRecordBytesCounter); - if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { - rfs.setPermission(finalOutPath, SPILL_FILE_PERMS); - } + ensureSpillFilePermissions(finalOutPath, rfs); } } else { skipBuffers = false; @@ -628,9 +626,7 @@ protected SpillResult callInternal() throws IOException { } LOG.info("Writing spill " + spillNumber + " to " + spillPathDetails.outputFilePath.toString()); FSDataOutputStream out = rfs.create(spillPathDetails.outputFilePath); - if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { - rfs.setPermission(spillPathDetails.outputFilePath, SPILL_FILE_PERMS); - } + ensureSpillFilePermissions(spillPathDetails.outputFilePath, rfs); TezSpillRecord spillRecord = new TezSpillRecord(numPartitions); DataInputBuffer key = new DataInputBuffer(); DataInputBuffer val = new DataInputBuffer(); @@ -728,9 +724,7 @@ private boolean canSendDataOverDME() throws IOException { if (((IFile.FileBackedInMemIFileWriter) writer).isDataFlushedToDisk()) { this.finalOutPath = ((IFile.FileBackedInMemIFileWriter) writer).getOutputPath(); - if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { - rfs.setPermission(finalOutPath, SPILL_FILE_PERMS); - } + ensureSpillFilePermissions(finalOutPath, rfs); additionalSpillBytesWritternCounter.increment(writer.getCompressedLength()); } } @@ -1080,9 +1074,7 @@ private void mergeAll() throws IOException { FSDataOutputStream out = null; try { out = rfs.create(finalOutPath); - if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { - rfs.setPermission(finalOutPath, SPILL_FILE_PERMS); - } + ensureSpillFilePermissions(finalOutPath, rfs); Writer writer = null; for (int i = 0; i < numPartitions; i++) { @@ -1171,9 +1163,7 @@ private void writeLargeRecord(final Object key, final Object value, final int pa final TezSpillRecord spillRecord = new TezSpillRecord(numPartitions); final Path outPath = spillPathDetails.outputFilePath; out = rfs.create(outPath); - if (!SPILL_FILE_PERMS.equals(SPILL_FILE_PERMS.applyUMask(FsPermission.getUMask(conf)))) { - rfs.setPermission(outPath, SPILL_FILE_PERMS); - } + ensureSpillFilePermissions(outPath, rfs); BitSet emptyPartitions = null; if (pipelinedShuffle || !isFinalMergeEnabled) { emptyPartitions = new BitSet(numPartitions); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java index 3692392518..f4e99ec0ed 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java @@ -53,6 +53,7 @@ import com.google.protobuf.ByteString; import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.fs.permission.FsAction; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.runtime.api.TaskFailureType; import org.apache.tez.runtime.api.events.VertexManagerEvent; @@ -520,10 +521,7 @@ public void textTest(int numRegularRecords, int numPartitions, long availableMem if (numRecordsWritten > 0) { assertTrue(localFs.exists(outputFilePath)); assertTrue(localFs.exists(spillFilePath)); - assertEquals("Incorrect output permissions", (short)0640, - localFs.getFileStatus(outputFilePath).getPermission().toShort()); - assertEquals("Incorrect index permissions", (short)0640, - localFs.getFileStatus(spillFilePath).getPermission().toShort()); + checkPermissions(outputFilePath, spillFilePath); } else { return; } @@ -812,16 +810,24 @@ private void baseTestWithPipelinedTransfer(int numRecords, int numPartitions, Se Path indexFile = taskOutput.getSpillIndexFileForWrite(i, 10); assertTrue(localFs.exists(outputFile)); assertTrue(localFs.exists(indexFile)); - assertEquals("Incorrect output permissions", (short)0640, - localFs.getFileStatus(outputFile).getPermission().toShort()); - assertEquals("Incorrect index permissions", (short)0640, - localFs.getFileStatus(indexFile).getPermission().toShort()); + checkPermissions(outputFile, indexFile); } } else { return; } } + private void checkPermissions(Path outputFile, Path indexFile) throws IOException { + assertEquals("Incorrect output permissions (user)", FsAction.READ_WRITE, + localFs.getFileStatus(outputFile).getPermission().getUserAction()); + assertEquals("Incorrect output permissions (group)", FsAction.READ, + localFs.getFileStatus(outputFile).getPermission().getGroupAction()); + assertEquals("Incorrect index permissions (user)", FsAction.READ_WRITE, + localFs.getFileStatus(indexFile).getPermission().getUserAction()); + assertEquals("Incorrect index permissions (group)", FsAction.READ, + localFs.getFileStatus(indexFile).getPermission().getGroupAction()); + } + private void verifyEmptyPartitions(DataMovementEventPayloadProto eventProto, int numRecordsWritten, int numPartitions, Set skippedPartitions) throws IOException { @@ -1065,10 +1071,7 @@ private void baseTestWithFinalMergeDisabled(int numRecords, int numPartitions, Path outputPath = new Path(outputContext.getWorkDirs()[0], "output/" + eventProto.getPathComponent() + "/" + Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING); Path indexPath = outputPath.suffix(Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING); - assertEquals("Incorrect output permissions", (short)0640, - localFs.getFileStatus(outputPath).getPermission().toShort()); - assertEquals("Incorrect index permissions", (short)0640, - localFs.getFileStatus(indexPath).getPermission().toShort()); + checkPermissions(outputPath, indexPath); } else { assertEquals(0, eventProto.getSpillId()); if (outputRecordsCounter.getValue() > 0) { @@ -1289,12 +1292,16 @@ private void baseTest(int numRecords, int numPartitions, Set skippedPar boolean isInMem= eventProto.getData().hasData(); assertTrue(localFs.exists(outputFilePath)); - assertEquals("Incorrect output permissions", (short) 0640, - localFs.getFileStatus(outputFilePath).getPermission().toShort()); + assertEquals("Incorrect output permissions (user)", FsAction.READ_WRITE, + localFs.getFileStatus(outputFilePath).getPermission().getUserAction()); + assertEquals("Incorrect output permissions (group)", FsAction.READ, + localFs.getFileStatus(outputFilePath).getPermission().getGroupAction()); if( !isInMem ) { assertTrue(localFs.exists(spillFilePath)); - assertEquals("Incorrect index permissions", (short) 0640, - localFs.getFileStatus(spillFilePath).getPermission().toShort()); + assertEquals("Incorrect index permissions (user)", FsAction.READ_WRITE, + localFs.getFileStatus(spillFilePath).getPermission().getUserAction()); + assertEquals("Incorrect index permissions (group)", FsAction.READ, + localFs.getFileStatus(spillFilePath).getPermission().getGroupAction()); // verify no intermediate spill files have been left around synchronized (kvWriter.spillInfoList) { From 21701261ebc02f4dea1752509b18144d70dc534f Mon Sep 17 00:00:00 2001 From: Attila Magyar Date: Thu, 2 Jul 2020 10:41:50 -0700 Subject: [PATCH 243/512] TEZ-4170 : RootInputInitializerManager could make use of ThreadPool from appContext ( Attila Magyar via Rajesh Balamohan) Signed-off-by: Ashutosh Chauhan --- .../app/dag/RootInputInitializerManager.java | 243 ++++++++---------- .../tez/dag/app/dag/impl/VertexImpl.java | 25 +- .../dag/TestRootInputInitializerManager.java | 26 +- .../tez/dag/app/dag/impl/TestDAGRecovery.java | 6 +- .../tez/dag/app/dag/impl/TestVertexImpl.java | 97 ++++++- 5 files changed, 223 insertions(+), 174 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java index 7ff9fa9c0d..5ce00504b1 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java @@ -18,35 +18,30 @@ package org.apache.tez.dag.app.dag; -import javax.annotation.Nullable; +import static org.apache.tez.dag.app.dag.VertexState.FAILED; import java.io.IOException; import java.lang.reflect.UndeclaredThrowableException; import java.security.PrivilegedExceptionAction; +import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.Objects; -import org.apache.tez.common.Preconditions; -import com.google.common.collect.LinkedListMultimap; -import com.google.common.collect.ListMultimap; -import com.google.common.collect.Lists; +import javax.annotation.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.event.EventHandler; -import org.apache.tez.common.GuavaShim; +import org.apache.tez.common.Preconditions; import org.apache.tez.common.ReflectionUtils; import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.dag.api.InputDescriptor; @@ -54,38 +49,38 @@ import org.apache.tez.dag.api.RootInputLeafOutput; import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.api.TezUncheckedException; -import org.apache.tez.dag.api.event.*; import org.apache.tez.dag.api.event.VertexState; +import org.apache.tez.dag.api.event.VertexStateUpdate; import org.apache.tez.dag.api.oldrecords.TaskState; import org.apache.tez.dag.app.AppContext; import org.apache.tez.dag.app.dag.event.VertexEventRootInputFailed; import org.apache.tez.dag.app.dag.event.VertexEventRootInputInitialized; import org.apache.tez.dag.app.dag.impl.AMUserCodeException; -import org.apache.tez.dag.app.dag.impl.TezRootInputInitializerContextImpl; import org.apache.tez.dag.app.dag.impl.AMUserCodeException.Source; +import org.apache.tez.dag.app.dag.impl.TezRootInputInitializerContextImpl; +import org.apache.tez.dag.app.dag.impl.VertexImpl; import org.apache.tez.dag.records.TezTaskID; import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.InputInitializer; import org.apache.tez.runtime.api.InputInitializerContext; +import org.apache.tez.runtime.api.events.InputInitializerEvent; +import org.apache.tez.runtime.api.impl.TezEvent; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import com.google.common.annotations.VisibleForTesting; -import com.google.common.util.concurrent.FutureCallback; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.collect.LinkedListMultimap; +import com.google.common.collect.ListMultimap; +import com.google.common.collect.Lists; import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; -import com.google.common.util.concurrent.ThreadFactoryBuilder; - -import org.apache.tez.runtime.api.events.InputInitializerEvent; -import org.apache.tez.runtime.api.impl.TezEvent; public class RootInputInitializerManager { private static final Logger LOG = LoggerFactory.getLogger(RootInputInitializerManager.class); - private final ExecutorService rawExecutor; - private final ListeningExecutorService executor; + @VisibleForTesting + protected ListeningExecutorService executor; @SuppressWarnings("rawtypes") private final EventHandler eventHandler; private volatile boolean isStopped = false; @@ -96,50 +91,106 @@ public class RootInputInitializerManager { private final AppContext appContext; @VisibleForTesting - final Map initializerMap = new HashMap(); + final Map initializerMap = new ConcurrentHashMap<>(); public RootInputInitializerManager(Vertex vertex, AppContext appContext, UserGroupInformation dagUgi, StateChangeNotifier stateTracker) { this.appContext = appContext; this.vertex = vertex; this.eventHandler = appContext.getEventHandler(); - this.rawExecutor = Executors.newCachedThreadPool(new ThreadFactoryBuilder() - .setDaemon(true).setNameFormat("InputInitializer {" + this.vertex.getName() + "} #%d").build()); - this.executor = MoreExecutors.listeningDecorator(rawExecutor); + this.executor = appContext.getExecService(); this.dagUgi = dagUgi; this.entityStateTracker = stateTracker; } - - public void runInputInitializers(List> - inputs) throws TezException { - for (RootInputLeafOutput input : inputs) { - InputInitializerContext context = - new TezRootInputInitializerContextImpl(input, vertex, appContext, this); - InputInitializer initializer; + public void runInputInitializers( + List> inputs, List pendingInitializerEvents) { + + executor.submit(() -> createAndStartInitializing(inputs, pendingInitializerEvents)); + } + + private void createAndStartInitializing(List> inputs, List pendingInitializerEvents) { + String current = null; + try { + List result = new ArrayList<>(); + for (RootInputLeafOutput each : inputs) { + current = each.getName(); + InitializerWrapper initializer = createInitializerWrapper(each); + initializerMap.put(each.getName(), initializer); + registerPendingVertex(each, initializer); + result.add(initializer); + } + handleInitializerEvents(pendingInitializerEvents); + pendingInitializerEvents.clear(); + for (InitializerWrapper inputWrapper : result) { + executor.submit(() -> runInitializerAndProcessResult(inputWrapper)); + } + } catch (Throwable t) { + VertexImpl vertexImpl = (VertexImpl) vertex; + String msg = "Fail to create InputInitializerManager, " + ExceptionUtils.getStackTrace(t); + LOG.info(msg); + vertexImpl.finished(FAILED, VertexTerminationCause.INIT_FAILURE, msg); + eventHandler.handle(new VertexEventRootInputFailed(vertex.getVertexId(), current, + new AMUserCodeException(AMUserCodeException.Source.InputInitializer, t))); + + } + } + + private void runInitializerAndProcessResult(InitializerWrapper initializer) { + try { + List result = runInitializer(initializer); + LOG.info("Succeeded InputInitializer for Input: " + initializer.getInput().getName() + + " on vertex " + initializer.getVertexLogIdentifier()); + eventHandler.handle(new VertexEventRootInputInitialized(vertex.getVertexId(), + initializer.getInput().getName(), result)); + } catch (Throwable t) { + if (t instanceof UndeclaredThrowableException) { + t = t.getCause(); + } + LOG.info("Failed InputInitializer for Input: " + initializer.getInput().getName() + + " on vertex " + initializer.getVertexLogIdentifier()); + eventHandler.handle(new VertexEventRootInputFailed(vertex.getVertexId(), initializer.getInput().getName(), + new AMUserCodeException(Source.InputInitializer,t))); + } finally { + initializer.setComplete(); + } + } + + private List runInitializer(InitializerWrapper initializer) throws IOException, InterruptedException { + return dagUgi.doAs((PrivilegedExceptionAction>) () -> { + LOG.info( + "Starting InputInitializer for Input: " + initializer.getInput().getName() + + " on vertex " + initializer.getVertexLogIdentifier()); try { - TezUtilsInternal.setHadoopCallerContext(appContext.getHadoopShim(), vertex.getVertexId()); - initializer = createInitializer(input, context); + TezUtilsInternal.setHadoopCallerContext(appContext.getHadoopShim(), + initializer.vertexId); + return initializer.getInitializer().initialize(); } finally { appContext.getHadoopShim().clearHadoopCallerContext(); } + }); + } - InitializerWrapper initializerWrapper = - new InitializerWrapper(input, initializer, context, vertex, entityStateTracker, appContext); + private InitializerWrapper createInitializerWrapper(RootInputLeafOutput input) throws TezException { + InputInitializerContext context = + new TezRootInputInitializerContextImpl(input, vertex, appContext, this); + try { + TezUtilsInternal.setHadoopCallerContext(appContext.getHadoopShim(), vertex.getVertexId()); + InputInitializer initializer = createInitializer(input, context); + return new InitializerWrapper(input, initializer, context, vertex, entityStateTracker, appContext); + } finally { + appContext.getHadoopShim().clearHadoopCallerContext(); + } + } - // Register pending vertex update registrations - List vertexUpdateRegistrations = pendingVertexRegistrations.removeAll(input.getName()); - if (vertexUpdateRegistrations != null) { - for (VertexUpdateRegistrationHolder h : vertexUpdateRegistrations) { - initializerWrapper.registerForVertexStateUpdates(h.vertexName, h.stateSet); - } + private void registerPendingVertex(RootInputLeafOutput input, InitializerWrapper initializerWrapper) { + // Register pending vertex update registrations + List vertexUpdateRegistrations = pendingVertexRegistrations.removeAll(input.getName()); + if (vertexUpdateRegistrations != null) { + for (VertexUpdateRegistrationHolder h : vertexUpdateRegistrations) { + initializerWrapper.registerForVertexStateUpdates(h.vertexName, h.stateSet); } - - initializerMap.put(input.getName(), initializerWrapper); - ListenableFuture> future = executor - .submit(new InputInitializerCallable(initializerWrapper, dagUgi, appContext)); - Futures.addCallback(future, createInputInitializerCallback(initializerWrapper), GuavaShim.directExecutor()); } } @@ -232,11 +283,6 @@ public void registerForVertexUpdates(String vertexName, String inputName, } } - @VisibleForTesting - protected InputInitializerCallback createInputInitializerCallback(InitializerWrapper initializer) { - return new InputInitializerCallback(initializer, eventHandler, vertex.getVertexId()); - } - @VisibleForTesting @InterfaceAudience.Private public InitializerWrapper getInitializerWrapper(String inputName) { @@ -244,92 +290,7 @@ public InitializerWrapper getInitializerWrapper(String inputName) { } public void shutdown() { - if (executor != null && !isStopped) { - // Don't really care about what is running if an error occurs. If no error - // occurs, all execution is complete. - executor.shutdownNow(); - isStopped = true; - } - } - - private static class InputInitializerCallable implements - Callable> { - - private final InitializerWrapper initializerWrapper; - private final UserGroupInformation ugi; - private final AppContext appContext; - - public InputInitializerCallable(InitializerWrapper initializer, UserGroupInformation ugi, - AppContext appContext) { - this.initializerWrapper = initializer; - this.ugi = ugi; - this.appContext = appContext; - } - - @Override - public List call() throws Exception { - List events = ugi.doAs(new PrivilegedExceptionAction>() { - @Override - public List run() throws Exception { - LOG.info( - "Starting InputInitializer for Input: " + initializerWrapper.getInput().getName() + - " on vertex " + initializerWrapper.getVertexLogIdentifier()); - try { - TezUtilsInternal.setHadoopCallerContext(appContext.getHadoopShim(), - initializerWrapper.vertexId); - return initializerWrapper.getInitializer().initialize(); - } finally { - appContext.getHadoopShim().clearHadoopCallerContext(); - } - } - }); - return events; - } - } - - @SuppressWarnings("rawtypes") - @VisibleForTesting - private static class InputInitializerCallback implements - FutureCallback> { - - private final InitializerWrapper initializer; - private final EventHandler eventHandler; - private final TezVertexID vertexID; - - public InputInitializerCallback(InitializerWrapper initializer, - EventHandler eventHandler, TezVertexID vertexID) { - this.initializer = initializer; - this.eventHandler = eventHandler; - this.vertexID = vertexID; - } - - @SuppressWarnings("unchecked") - @Override - public void onSuccess(List result) { - initializer.setComplete(); - LOG.info( - "Succeeded InputInitializer for Input: " + initializer.getInput().getName() + - " on vertex " + initializer.getVertexLogIdentifier()); - eventHandler.handle(new VertexEventRootInputInitialized(vertexID, - initializer.getInput().getName(), result)); - } - - @SuppressWarnings("unchecked") - @Override - public void onFailure(Throwable t) { - // catch real root cause of failure, it would throw UndeclaredThrowableException - // if using UGI.doAs - if (t instanceof UndeclaredThrowableException) { - t = t.getCause(); - } - initializer.setComplete(); - LOG.info( - "Failed InputInitializer for Input: " + initializer.getInput().getName() + - " on vertex " + initializer.getVertexLogIdentifier()); - eventHandler - .handle(new VertexEventRootInputFailed(vertexID, initializer.getInput().getName(), - new AMUserCodeException(Source.InputInitializer,t))); - } + isStopped = true; } @VisibleForTesting diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index 85ae38da47..db0cd46119 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -2416,7 +2416,7 @@ void tryEnactKill(VertexTerminationCause trigger, } } - VertexState finished(VertexState finalState, + public VertexState finished(VertexState finalState, VertexTerminationCause termCause, String diag) { if (finishTime == 0) setFinishTime(); if (termCause != null) { @@ -3073,13 +3073,7 @@ private VertexState handleInitEvent(VertexImpl vertex) { if (vertex.inputsWithInitializers != null) { if (vertex.recoveryData == null || !vertex.recoveryData.shouldSkipInit()) { LOG.info("Vertex will initialize from input initializer. " + vertex.logIdentifier); - try { - vertex.setupInputInitializerManager(); - } catch (TezException e) { - String msg = "Fail to create InputInitializerManager, " + ExceptionUtils.getStackTrace(e); - LOG.info(msg); - return vertex.finished(VertexState.FAILED, VertexTerminationCause.INIT_FAILURE, msg); - } + vertex.setupInputInitializerManager(); } return VertexState.INITIALIZING; } else { @@ -3112,13 +3106,7 @@ private VertexState handleInitEvent(VertexImpl vertex) { if (vertex.inputsWithInitializers != null && (vertex.recoveryData == null || !vertex.recoveryData.shouldSkipInit())) { LOG.info("Vertex will initialize from input initializer. " + vertex.logIdentifier); - try { - vertex.setupInputInitializerManager(); - } catch (TezException e) { - String msg = "Fail to create InputInitializerManager, " + ExceptionUtils.getStackTrace(e); - LOG.error(msg); - return vertex.finished(VertexState.FAILED, VertexTerminationCause.INIT_FAILURE, msg); - } + vertex.setupInputInitializerManager(); return VertexState.INITIALIZING; } if (!vertex.uninitializedEdges.isEmpty()) { @@ -4255,7 +4243,7 @@ public void transition(VertexImpl vertex, VertexEvent event) { } } - private void setupInputInitializerManager() throws TezException { + private void setupInputInitializerManager() { rootInputInitializerManager = createRootInputInitializerManager( getDAG().getName(), getName(), getVertexId(), eventHandler, getTotalTasks(), @@ -4270,10 +4258,7 @@ eventHandler, getTotalTasks(), LOG.info("Starting " + inputsWithInitializers.size() + " inputInitializers for vertex " + logIdentifier); initWaitsForRootInitializers = true; - rootInputInitializerManager.runInputInitializers(inputList); - // Send pending rootInputInitializerEvents - rootInputInitializerManager.handleInitializerEvents(pendingInitializerEvents); - pendingInitializerEvents.clear(); + rootInputInitializerManager.runInputInitializers(inputList, pendingInitializerEvents); } private static class VertexStateChangedCallback diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java index b79b4afe68..01cc37f18c 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java @@ -28,8 +28,14 @@ import java.io.IOException; import java.util.Collections; import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import com.google.common.collect.Lists; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import com.google.common.util.concurrent.ThreadFactoryBuilder; + import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.event.EventHandler; @@ -50,10 +56,27 @@ import org.apache.tez.runtime.api.events.InputInitializerEvent; import org.apache.tez.runtime.api.impl.EventMetaData; import org.apache.tez.runtime.api.impl.TezEvent; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import org.mockito.ArgumentCaptor; public class TestRootInputInitializerManager { + ListeningExecutorService execService; + + @Before + public void setUp() throws Exception { + ExecutorService rawExecutor = Executors.newCachedThreadPool(new ThreadFactoryBuilder() + .setDaemon(true).setNameFormat("Test App Shared Pool - " + "#%d").build()); + execService = MoreExecutors.listeningDecorator(rawExecutor); + } + + @After + public void tearDown() throws Exception { + if (execService != null) { + execService.shutdownNow(); + } + } // Simple testing. No events if task doesn't succeed. // Also exercises path where two attempts are reported as successful via the stateChangeNotifier. @@ -214,6 +237,7 @@ public void testCorrectUgiUsage() throws TezException, InterruptedException { AppContext appContext = mock(AppContext.class); doReturn(new DefaultHadoopShim()).when(appContext).getHadoopShim(); doReturn(mock(EventHandler.class)).when(appContext).getEventHandler(); + when(appContext.getExecService()).thenReturn(execService); UserGroupInformation dagUgi = UserGroupInformation.createRemoteUser("fakeuser"); StateChangeNotifier stateChangeNotifier = mock(StateChangeNotifier.class); RootInputInitializerManager rootInputInitializerManager = new RootInputInitializerManager(vertex, appContext, dagUgi, stateChangeNotifier); @@ -222,7 +246,7 @@ public void testCorrectUgiUsage() throws TezException, InterruptedException { InputInitializerDescriptor iid = InputInitializerDescriptor.create(InputInitializerForUgiTest.class.getName()); RootInputLeafOutput rootInput = new RootInputLeafOutput<>("InputName", id, iid); - rootInputInitializerManager.runInputInitializers(Collections.singletonList(rootInput)); + rootInputInitializerManager.runInputInitializers(Collections.singletonList(rootInput), Collections.emptyList()); InputInitializerForUgiTest.awaitInitialize(); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java index 95ea8a030e..9636329da3 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java @@ -330,8 +330,10 @@ public void setup() { Mockito.doAnswer(new Answer() { public ListenableFuture answer(InvocationOnMock invocation) { Object[] args = invocation.getArguments(); - CallableEvent e = (CallableEvent) args[0]; - dispatcher.getEventHandler().handle(e); + if (args[0] instanceof CallableEvent) { + CallableEvent e = (CallableEvent) args[0]; + dispatcher.getEventHandler().handle(e); + } return mockFuture; } }).when(execService).submit((Callable) any()); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java index 5722406684..5ae95560fa 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java @@ -43,11 +43,15 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; +import com.google.common.util.concurrent.MoreExecutors; +import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.protobuf.ByteString; import org.apache.commons.lang.StringUtils; @@ -2421,10 +2425,10 @@ private void setupVertices() { dagConf); } } else { - v = new VertexImpl(vertexId, vPlan, vPlan.getName(), conf, - dispatcher.getEventHandler(), taskCommunicatorManagerInterface, - clock, thh, true, appContext, locationHint, vertexGroups, taskSpecificLaunchCmdOption, - updateTracker, dagConf); + v = new VertexImplWithRunningInputInitializerWithExecutor(vertexId, vPlan, vPlan.getName(), conf, + dispatcher.getEventHandler(), taskCommunicatorManagerInterface, + clock, thh, appContext, locationHint, dispatcher, customInitializer, updateTracker, + dagConf, vertexGroups); } vertices.put(vName, v); vertexIdMap.put(vertexId, v); @@ -2528,8 +2532,10 @@ public void setupPostDagCreation() throws TezException { Mockito.doAnswer(new Answer() { public ListenableFuture answer(InvocationOnMock invocation) { Object[] args = invocation.getArguments(); - CallableEvent e = (CallableEvent) args[0]; - dispatcher.getEventHandler().handle(e); + if (args[0] instanceof CallableEvent) { + CallableEvent e = (CallableEvent) args[0]; + dispatcher.getEventHandler().handle(e); + } return mockFuture; }}) .when(execService).submit((Callable) any()); @@ -2760,12 +2766,13 @@ public void testNonExistVertexManager() throws TezException { } @Test(timeout=5000) - public void testNonExistInputInitializer() throws TezException { + public void testNonExistInputInitializer() throws Exception { setupPreDagCreation(); dagPlan = createDAGPlanWithNonExistInputInitializer(); setupPostDagCreation(); VertexImpl v1 = vertices.get("vertex1"); v1.handle(new VertexEvent(v1.getVertexId(), VertexEventType.V_INIT)); + while (v1.getTerminationCause() == null) Thread.sleep(10); Assert.assertEquals(VertexState.FAILED, v1.getState()); Assert.assertEquals(VertexTerminationCause.INIT_FAILURE, v1.getTerminationCause()); Assert.assertTrue(StringUtils.join(v1.getDiagnostics(),"") @@ -5842,6 +5849,43 @@ protected RootInputInitializerManager createRootInputInitializerManager( } } + @SuppressWarnings("rawtypes") + private static class VertexImplWithRunningInputInitializerWithExecutor extends VertexImpl { + private RootInputInitializerManagerWithExecutor rootInputInitializerManager; + + public VertexImplWithRunningInputInitializerWithExecutor(TezVertexID vertexId, + VertexPlan vertexPlan, String vertexName, + Configuration conf, + EventHandler eventHandler, + TaskCommunicatorManagerInterface taskCommunicatorManagerInterface, + Clock clock, TaskHeartbeatHandler thh, + AppContext appContext, + VertexLocationHint vertexLocationHint, + DrainDispatcher dispatcher, + InputInitializer presetInitializer, + StateChangeNotifier updateTracker, + Configuration dagConf, + Map vertexGroups) { + super(vertexId, vertexPlan, vertexName, conf, eventHandler, + taskCommunicatorManagerInterface, clock, thh, true, + appContext, vertexLocationHint, vertexGroups, taskSpecificLaunchCmdOption, + updateTracker, dagConf); + } + + @Override + protected RootInputInitializerManager createRootInputInitializerManager( + String dagName, String vertexName, TezVertexID vertexID, + EventHandler eventHandler, int numTasks, int numNodes, + Resource taskResource, Resource totalResource) { + try { + rootInputInitializerManager = new RootInputInitializerManagerWithExecutor(this, this.getAppContext(), stateChangeNotifier); + } catch (IOException e) { + throw new RuntimeException(e); + } + return rootInputInitializerManager; + } + } + @SuppressWarnings("rawtypes") private static class VertexImplWithControlledInitializerManager extends VertexImpl { @@ -5898,9 +5942,11 @@ public RootInputInitializerManagerWithRunningInitializer(Vertex vertex, AppConte IOException { super(vertex, appContext, UserGroupInformation.getCurrentUser(), tracker); this.presetInitializer = presetInitializer; + ExecutorService rawExecutor = Executors.newCachedThreadPool(new ThreadFactoryBuilder() + .setDaemon(true).setNameFormat("Test App Shared Pool - " + "#%d").build()); + this.executor = MoreExecutors.listeningDecorator(rawExecutor); } - @Override protected InputInitializer createInitializer( RootInputLeafOutput input, @@ -5910,6 +5956,31 @@ protected InputInitializer createInitializer( } return presetInitializer; } + + @Override + public void shutdown() { + super.shutdown(); + if (executor != null) { + executor.shutdown(); + } + } + } + + private static class RootInputInitializerManagerWithExecutor extends RootInputInitializerManager { + public RootInputInitializerManagerWithExecutor(Vertex vertex, AppContext appContext, StateChangeNotifier tracker) throws IOException { + super(vertex, appContext, UserGroupInformation.getCurrentUser(), tracker); + ExecutorService rawExecutor = Executors.newCachedThreadPool(new ThreadFactoryBuilder() + .setDaemon(true).setNameFormat("Test App Shared Pool - " + "#%d").build()); + this.executor = MoreExecutors.listeningDecorator(rawExecutor); + } + + @Override + public void shutdown() { + super.shutdown(); + if (executor != null) { + executor.shutdown(); + } + } } @SuppressWarnings({"rawtypes", "unchecked"}) @@ -5931,11 +6002,14 @@ public RootInputInitializerManagerControlled(Vertex vertex, AppContext appContex this.eventHandler = eventHandler; this.dispatcher = dispatcher; this.vertexID = vertex.getVertexId(); + ExecutorService rawExecutor = Executors.newCachedThreadPool(new ThreadFactoryBuilder() + .setDaemon(true).setNameFormat("Test App Shared Pool - " + "#%d").build()); + this.executor = MoreExecutors.listeningDecorator(rawExecutor); } @Override public void runInputInitializers( - List> inputs) { + List> inputs, List pendingInitializerEvents) { this.inputs = inputs; } @@ -5961,10 +6035,13 @@ public void handleInputInitializerEvent(List events) thro @Override public void shutdown() { hasShutDown = true; + if (executor != null) { + executor.shutdown(); + } } public void failInputInitialization() throws TezException { - super.runInputInitializers(inputs); + super.runInputInitializers(inputs, Collections.emptyList()); eventHandler.handle(new VertexEventRootInputFailed(vertexID, inputs .get(0).getName(), new AMUserCodeException(Source.InputInitializer, From 474bb22ba124800f4bc8a505c0895babcb65f0e6 Mon Sep 17 00:00:00 2001 From: Mustafa Iman Date: Tue, 14 Jul 2020 17:13:16 -0500 Subject: [PATCH 244/512] TEZ-4200. Precommit docker image build fails Signed-off-by: Jonathan Eagles --- build-tools/docker/Dockerfile | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/build-tools/docker/Dockerfile b/build-tools/docker/Dockerfile index 1edd8ae5ee..143861d25a 100644 --- a/build-tools/docker/Dockerfile +++ b/build-tools/docker/Dockerfile @@ -18,7 +18,7 @@ # Dockerfile for installing the necessary dependencies for building Hadoop. # See BUILDING.txt. -FROM ubuntu:xenial +FROM ubuntu:bionic WORKDIR /root @@ -55,7 +55,6 @@ RUN apt-get -q update \ git \ gnupg-agent \ libbz2-dev \ - libcurl4-openssl-dev \ libfuse-dev \ libprotobuf-dev \ libprotoc-dev \ @@ -75,7 +74,6 @@ RUN apt-get -q update \ python-wheel \ rsync \ software-properties-common \ - snappy \ sudo \ valgrind \ zlib1g-dev \ @@ -180,7 +178,6 @@ RUN apt-get -q update \ && apt-get install -y --no-install-recommends nodejs npm \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* \ - && ln -s /usr/bin/nodejs /usr/bin/node \ && npm install npm@latest -g \ && npm install -g jshint From 5944982cd12804fde987d4d559d022571d9f9d72 Mon Sep 17 00:00:00 2001 From: Mustafa Iman Date: Wed, 15 Jul 2020 23:30:22 -0500 Subject: [PATCH 245/512] TEZ-4201. findbugs-maven-plugin is not compatible with Maven 3.6.0+ Signed-off-by: Jonathan Eagles --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index fe67b50c93..b1d5f66723 100644 --- a/pom.xml +++ b/pom.xml @@ -58,7 +58,7 @@ scm:git:https://gitbox.apache.org/repos/asf/tez.git ${maven.build.timestamp} 1.4 - 3.0.1 + 3.0.5 2.10.4 2.4.3 3.0.0 From 8acfb03335e10a983e51974107520a34581f1187 Mon Sep 17 00:00:00 2001 From: Mustafa Iman Date: Fri, 17 Jul 2020 11:34:46 -0500 Subject: [PATCH 246/512] TEZ-4199. MergeManager::finalMerge should make use of compression Signed-off-by: Jonathan Eagles --- .../library/common/sort/impl/TezMerger.java | 2 +- .../orderedgrouped/DummyCompressionCodec.java | 131 ++++++++++++++++++ .../orderedgrouped/TestMergeManager.java | 89 +++++++++++- 3 files changed, 217 insertions(+), 5 deletions(-) create mode 100644 tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/DummyCompressionCodec.java diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java index 0e18eadae6..3e04e748db 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java @@ -536,7 +536,7 @@ public MergeQueue(Configuration conf, FileSystem fs, List segments, RawComparator comparator, Progressable reporter, boolean sortSegments, CompressionCodec codec, boolean considerFinalMergeForProgress) { - this(conf, fs, segments, comparator, reporter, sortSegments, null, + this(conf, fs, segments, comparator, reporter, sortSegments, codec, considerFinalMergeForProgress, true); } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/DummyCompressionCodec.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/DummyCompressionCodec.java new file mode 100644 index 0000000000..962a9e0207 --- /dev/null +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/DummyCompressionCodec.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.runtime.library.common.shuffle.orderedgrouped; + +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.io.compress.CompressionInputStream; +import org.apache.hadoop.io.compress.CompressionOutputStream; +import org.apache.hadoop.io.compress.Compressor; +import org.apache.hadoop.io.compress.Decompressor; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +import static org.mockito.Mockito.mock; + +/** + * A dummy codec. It passes everything to underlying stream + */ +public class DummyCompressionCodec implements CompressionCodec { + + @Override + public CompressionOutputStream createOutputStream(OutputStream out) throws IOException { + return new DummyCompressionOutputStream(out); + } + + @Override + public CompressionOutputStream createOutputStream(OutputStream out, Compressor compressor) throws IOException { + return new DummyCompressionOutputStream(out); + } + + @Override + public Class getCompressorType() { + return Compressor.class; + } + + @Override + public Compressor createCompressor() { + return mock(Compressor.class); + } + + @Override + public CompressionInputStream createInputStream(InputStream in) throws IOException { + return new DummyCompressionInputStream(in); + } + + @Override + public CompressionInputStream createInputStream(InputStream in, Decompressor decompressor) throws IOException { + return new DummyCompressionInputStream(in); + } + + @Override + public Class getDecompressorType() { + return Decompressor.class; + } + + @Override + public Decompressor createDecompressor() { + return mock(Decompressor.class); + } + + @Override + public String getDefaultExtension() { + return null; + } + + class DummyCompressionOutputStream extends CompressionOutputStream { + + protected DummyCompressionOutputStream(OutputStream out) { + super(out); + } + + @Override + public void write(int b) throws IOException { + out.write(b); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + out.write(b, off, len); + } + + @Override + public void finish() throws IOException { + //no-op + } + + @Override + public void resetState() throws IOException { + //no-op + } + } + + class DummyCompressionInputStream extends CompressionInputStream { + + protected DummyCompressionInputStream(InputStream in) throws IOException { + super(in); + } + + @Override + public int read() throws IOException { + return in.read(); + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + return in.read(b, off, len); + } + + @Override + public void resetState() throws IOException { + //no-op + } + } +} diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java index 92d9125dab..9cffcc7a4a 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java @@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; import static org.mockito.Mockito.atLeastOnce; -import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; @@ -38,8 +37,7 @@ import com.google.common.collect.Sets; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; +import org.apache.hadoop.io.compress.CompressionCodec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -257,6 +255,65 @@ public void testIntermediateMemoryMergeAccounting() throws Exception { assertEquals(data1.length + data2.length, mergeManager.getUsedMemory()); } + @Test + public void testDiskMergeWithCodec() throws Throwable { + Configuration conf = new TezConfiguration(defaultConf); + conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, IntWritable.class.getName()); + conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS, IntWritable.class.getName()); + conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_IO_SORT_FACTOR, 3); + + Path localDir = new Path(workDir, "local"); + localFs.mkdirs(localDir); + + conf.setStrings(TezRuntimeFrameworkConfigs.LOCAL_DIRS, localDir.toString()); + + LocalDirAllocator localDirAllocator = + new LocalDirAllocator(TezRuntimeFrameworkConfigs.LOCAL_DIRS); + InputContext inputContext = createMockInputContext(UUID.randomUUID().toString()); + + // Create a mock compressor. We will check if it is used. + CompressionCodec dummyCodec = spy(new DummyCompressionCodec()); + + MergeManager mergeManager = + new MergeManager(conf, localFs, localDirAllocator, inputContext, null, null, null, null, + mock(ExceptionReporter.class), 2000, dummyCodec, false, -1); + mergeManager.configureAndStart(); + + assertEquals(0, mergeManager.getUsedMemory()); + assertEquals(0, mergeManager.getCommitMemory()); + + InputAttemptIdentifier inputAttemptIdentifier1 = new InputAttemptIdentifier(0, 0); + InputAttemptIdentifier inputAttemptIdentifier2 = new InputAttemptIdentifier(1, 0); + InputAttemptIdentifier inputAttemptIdentifier3 = new InputAttemptIdentifier(2, 0); + InputAttemptIdentifier inputAttemptIdentifier4 = new InputAttemptIdentifier(3, 0); + byte[] data1 = generateDataBySizeAndGetBytes(conf, 500, inputAttemptIdentifier1); + byte[] data2 = generateDataBySizeAndGetBytes(conf, 500, inputAttemptIdentifier2); + byte[] data3 = generateDataBySizeAndGetBytes(conf, 500, inputAttemptIdentifier3); + byte[] data4 = generateDataBySizeAndGetBytes(conf, 500, inputAttemptIdentifier3); + + MapOutput mo1 = mergeManager.reserve(inputAttemptIdentifier1, data1.length, data1.length, 0); + MapOutput mo2 = mergeManager.reserve(inputAttemptIdentifier2, data2.length, data2.length, 0); + MapOutput mo3 = mergeManager.reserve(inputAttemptIdentifier3, data3.length, data3.length, 0); + MapOutput mo4 = mergeManager.reserve(inputAttemptIdentifier4, data4.length, data4.length, 0); + + mo1.getDisk().write(data1); + mo1.getDisk().flush(); + mo2.getDisk().write(data2); + mo2.getDisk().flush(); + mo3.getDisk().write(data3); + mo3.getDisk().flush(); + mo4.getDisk().write(data4); + mo4.getDisk().flush(); + + mo1.commit(); + mo2.commit(); + mo3.commit(); + mo4.commit(); + + mergeManager.close(true); + verify(dummyCodec, atLeastOnce()).createOutputStream(any(), any()); + } + @Test(timeout = 60000l) public void testIntermediateMemoryMerge() throws Throwable { Configuration conf = new TezConfiguration(defaultConf); @@ -592,7 +649,31 @@ private byte[] generateDataBySize(Configuration conf, int rawLen, InputAttemptId return data; } - private byte[] generateData(Configuration conf, int numEntries, InputAttemptIdentifier inputAttemptIdentifier) throws IOException { + private byte[] generateDataBySizeAndGetBytes(Configuration conf, int rawLen, + InputAttemptIdentifier inputAttemptIdentifier) throws IOException { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + FSDataOutputStream fsdos = new FSDataOutputStream(baos, null); + IFile.Writer writer = + new IFile.Writer(conf, fsdos, IntWritable.class, IntWritable.class, null, null, null); + int i = 0; + while(true) { + writer.append(new IntWritable(i), new IntWritable(i)); + i++; + if (writer.getRawLength() > rawLen) { + break; + } + } + writer.close(); + int compressedLength = (int)writer.getCompressedLength(); + int rawLength = (int)writer.getRawLength(); + byte[] data = new byte[rawLength]; + ShuffleUtils.shuffleToMemory(data, new ByteArrayInputStream(baos.toByteArray()), + rawLength, compressedLength, null, false, 0, LOG, inputAttemptIdentifier); + return baos.toByteArray(); + } + + private byte[] generateData(Configuration conf, int numEntries, + InputAttemptIdentifier inputAttemptIdentifier) throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); FSDataOutputStream fsdos = new FSDataOutputStream(baos, null); IFile.Writer writer = From 2d7c60849adf3ed62f36f00e161c5d55962206f5 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Tue, 21 Jul 2020 15:45:22 +0000 Subject: [PATCH 247/512] TEZ-4133. key class implements writableComparable and configurable use default configuration (wang qiang via jeagles) Signed-off-by: Jonathan Eagles --- .../runtime/library/common/ConfigUtils.java | 4 +- .../library/common/TestConfigUtils.java | 85 +++++++++++++++++++ 2 files changed, 87 insertions(+), 2 deletions(-) create mode 100644 tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/TestConfigUtils.java diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java index 24ad0ad1e3..76d3dff868 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java @@ -122,7 +122,7 @@ public static RawComparator getIntermediateOutputKeyComparator(Configurat if (theClass != null) return ReflectionUtils.newInstance(theClass, conf); return WritableComparator.get(getIntermediateOutputKeyClass(conf).asSubclass( - WritableComparable.class)); + WritableComparable.class), conf); } public static RawComparator getIntermediateInputKeyComparator(Configuration conf) { @@ -132,7 +132,7 @@ public static RawComparator getIntermediateInputKeyComparator(Configurati if (theClass != null) return ReflectionUtils.newInstance(theClass, conf); return WritableComparator.get(getIntermediateInputKeyClass(conf).asSubclass( - WritableComparable.class)); + WritableComparable.class), conf); } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/TestConfigUtils.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/TestConfigUtils.java new file mode 100644 index 0000000000..24f76cc832 --- /dev/null +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/TestConfigUtils.java @@ -0,0 +1,85 @@ +/** + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.runtime.library.common; + +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.io.WritableComparator; +import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; +import org.junit.Test; + +import java.io.DataInput; +import java.io.DataOutput; + +import static org.junit.Assert.assertEquals; + + +public class TestConfigUtils { + + private static class CustomKey implements WritableComparable, Configurable { + + private Configuration conf; + + @Override + public int compareTo(CustomKey o) { + return 0; + } + + @Override + public void write(DataOutput out) { + + } + + @Override + public void readFields(DataInput in) { + + } + + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } + + @Override + public Configuration getConf() { + return conf; + } + } + + @Test + public void getIntermediateOutputKeyComparator() { + Configuration conf = new Configuration(); + String testKey = "test_flag_name"; + String testValue = "tez"; + conf.set(testKey, testValue); + conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, CustomKey.class.getName()); + WritableComparator rawComparator = (WritableComparator) ConfigUtils.getIntermediateOutputKeyComparator(conf); + CustomKey customKey = (CustomKey) rawComparator.newKey(); + assertEquals(testValue, customKey.getConf().get(testKey)); + } + + @Test + public void getIntermediateInputKeyComparator() { + Configuration conf = new Configuration(); + String testKey = "test_flag_name"; + String testValue = "tez"; + conf.set(testKey, testValue); + conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, CustomKey.class.getName()); + WritableComparator rawComparator = (WritableComparator) ConfigUtils.getIntermediateInputKeyComparator(conf); + CustomKey customKey = (CustomKey) rawComparator.newKey(); + assertEquals(testValue, customKey.getConf().get(testKey)); + } +} \ No newline at end of file From 134ecc3c73098a22d34417ab69d03b69068b1354 Mon Sep 17 00:00:00 2001 From: Mustafa Iman Date: Thu, 30 Jul 2020 12:08:18 +0200 Subject: [PATCH 248/512] TEZ-4206: TestSpeculation.testBasicSpeculationPerVertexConf is flaky (Mustafa Iman via Ashutosh Chauhan) Signed-off-by: Laszlo Bodor --- .../speculation/legacy/LegacySpeculator.java | 23 +++-------- .../apache/tez/dag/app/TestSpeculation.java | 41 +++++++++++-------- 2 files changed, 31 insertions(+), 33 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java index f21b8198e2..6bf02df898 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java @@ -23,13 +23,10 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; -import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import com.google.common.annotations.VisibleForTesting; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.hadoop.service.AbstractService; @@ -101,8 +98,6 @@ public class LegacySpeculator extends AbstractService { private final Clock clock; private Thread speculationBackgroundThread = null; private volatile boolean stopped = false; - /** Allow the speculator to wait on a blockingQueue in case we use it for event notification. */ - private BlockingQueue scanControl = new LinkedBlockingQueue(); @VisibleForTesting public int getMinimumAllowedSpeculativeTasks() { return minimumAllowedSpeculativeTasks;} @@ -233,16 +228,6 @@ protected void serviceStop() throws Exception { } } - // This interface is intended to be used only for test cases. - public void scanForSpeculationsForTesting() { - if (LOG.isDebugEnabled()) { - LOG.debug("We got asked to run a debug speculation scan."); - LOG.debug("There are {} speculative events stacked already.", scanControl.size()); - } - scanControl.add(new Object()); - Thread.yield(); - } - public Runnable createThread() { return new Runnable() { @Override @@ -256,9 +241,13 @@ public void run() { long wait = Math.max(nextRecompTime, clock.getTime() - backgroundRunStartTime); if (speculations > 0) { LOG.info("We launched " + speculations - + " speculations. Waiting " + wait + " milliseconds."); + + " speculations. Waiting " + wait + " milliseconds before next evaluation."); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Waiting {} milliseconds before next evaluation.", wait); + } } - Object pollResult = scanControl.poll(wait, TimeUnit.MILLISECONDS); + Thread.sleep(wait); } catch (InterruptedException ie) { if (!stopped) { LOG.warn("Speculator thread interrupted", ie); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java index b9a7c5ae17..b6f50301c7 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java @@ -84,6 +84,16 @@ public class TestSpeculation { "Number of attempts after Speculation should be two"; private static final String UNIT_EXCEPTION_MESSAGE = "test timed out after"; + + /** + * {@link MockDAGAppMaster#launcherSleepTime} advances tasks every 1 millisecond. + * We want our test task to take at least slightly more than 1 second. This is because + * MockDAGAppMaster's mock clock advances clock 1 second at each tick. If we are unlucky + * this may cause speculator to wait 1 second between each evaluation. If we are really + * unlucky, our test tasks finish before speculator has a chance to evaluate and speculate + * them. That is why we want the tasks to take at least one second. + */ + private static final int NUM_UPDATES_FOR_TEST_TASK = 1200; private static final int ASSERT_SPECULATIONS_COUNT_RETRIES = 3; private Configuration defaultConf; private FileSystem localFs; @@ -277,7 +287,7 @@ void syncWithMockAppLauncher(boolean allowScheduling, AtomicBoolean mockAppLaunc * @throws Exception the exception */ @Retry - @Test (timeout = 10000) + @Test (timeout = 30000) public void testSingleTaskSpeculation() throws Exception { // Map Map confToExpected = new HashMap(); @@ -308,7 +318,7 @@ public void testSingleTaskSpeculation() throws Exception { TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 1); Thread.sleep(200); // cause speculation trigger - mockLauncher.setStatusUpdatesForTask(killedTaId, 100); + mockLauncher.setStatusUpdatesForTask(killedTaId, NUM_UPDATES_FOR_TEST_TASK); mockLauncher.startScheduling(true); dagClient.waitForCompletion(); @@ -350,7 +360,7 @@ public void testBasicSpeculation(boolean withProgress) throws Exception { mockLauncher.updateProgress(withProgress); // cause speculation trigger - mockLauncher.setStatusUpdatesForTask(killedTaId, 100); + mockLauncher.setStatusUpdatesForTask(killedTaId, NUM_UPDATES_FOR_TEST_TASK); mockLauncher.startScheduling(true); dagClient.waitForCompletion(); @@ -392,7 +402,7 @@ public void testBasicSpeculation(boolean withProgress) throws Exception { * @throws Exception the exception */ @Retry - @Test (timeout=10000) + @Test (timeout=30000) public void testBasicSpeculationWithProgress() throws Exception { testBasicSpeculation(true); } @@ -403,7 +413,7 @@ public void testBasicSpeculationWithProgress() throws Exception { * @throws Exception the exception */ @Retry - @Test (timeout=10000) + @Test (timeout=30000) public void testBasicSpeculationWithoutProgress() throws Exception { testBasicSpeculation(false); } @@ -414,12 +424,11 @@ public void testBasicSpeculationWithoutProgress() throws Exception { * @throws Exception the exception */ @Retry - @Test (timeout=10000) + @Test (timeout=30000) public void testBasicSpeculationPerVertexConf() throws Exception { DAG dag = DAG.create("test"); String vNameNoSpec = "A"; String vNameSpec = "B"; - String speculatorSleepTime = "50"; Vertex vA = Vertex.create(vNameNoSpec, ProcessorDescriptor.create("Proc.class"), 5); Vertex vB = Vertex.create(vNameSpec, ProcessorDescriptor.create("Proc.class"), 5); vA.setConf(TezConfiguration.TEZ_AM_SPECULATION_ENABLED, "false"); @@ -436,20 +445,20 @@ public void testBasicSpeculationPerVertexConf() throws Exception { DAGClient dagClient = tezClient.submitDAG(dag); DAGImpl dagImpl = (DAGImpl) mockApp.getContext().getCurrentDAG(); - TezVertexID vertexId = dagImpl.getVertex(vNameSpec).getVertexId(); + TezVertexID vertexIdSpec = dagImpl.getVertex(vNameSpec).getVertexId(); TezVertexID vertexIdNoSpec = dagImpl.getVertex(vNameNoSpec).getVertexId(); // original attempt is killed and speculative one is successful TezTaskAttemptID killedTaId = - TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 0); - TezTaskAttemptID noSpecTaId = TezTaskAttemptID + TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexIdSpec, 0), 0); + TezTaskAttemptID successfulTaId = TezTaskAttemptID .getInstance(TezTaskID.getInstance(vertexIdNoSpec, 0), 0); // cause speculation trigger for both - mockLauncher.setStatusUpdatesForTask(killedTaId, 100); - mockLauncher.setStatusUpdatesForTask(noSpecTaId, 100); + mockLauncher.setStatusUpdatesForTask(killedTaId, NUM_UPDATES_FOR_TEST_TASK); + mockLauncher.setStatusUpdatesForTask(successfulTaId, NUM_UPDATES_FOR_TEST_TASK); mockLauncher.startScheduling(true); - org.apache.tez.dag.app.dag.Vertex vSpec = dagImpl.getVertex(vertexId); + org.apache.tez.dag.app.dag.Vertex vSpec = dagImpl.getVertex(vertexIdSpec); org.apache.tez.dag.app.dag.Vertex vNoSpec = dagImpl.getVertex(vertexIdNoSpec); // Wait enough time to give chance for the speculator to trigger // speculation on VB. @@ -476,7 +485,7 @@ public void testBasicSpeculationPerVertexConf() throws Exception { * @throws Exception the exception */ @Retry - @Test (timeout=10000) + @Test (timeout=30000) public void testBasicSpeculationNotUseful() throws Exception { DAG dag = DAG.create("test"); Vertex vA = Vertex.create("A", ProcessorDescriptor.create("Proc.class"), 5); @@ -491,8 +500,8 @@ public void testBasicSpeculationNotUseful() throws Exception { TezTaskAttemptID successTaId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 0); TezTaskAttemptID killedTaId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 1); - mockLauncher.setStatusUpdatesForTask(successTaId, 100); - mockLauncher.setStatusUpdatesForTask(killedTaId, 100); + mockLauncher.setStatusUpdatesForTask(successTaId, NUM_UPDATES_FOR_TEST_TASK); + mockLauncher.setStatusUpdatesForTask(killedTaId, NUM_UPDATES_FOR_TEST_TASK); mockLauncher.startScheduling(true); dagClient.waitForCompletion(); From 9d2b61b576a2421ec4fb813489d896d2b89fcce9 Mon Sep 17 00:00:00 2001 From: Mustafa Iman Date: Thu, 30 Jul 2020 12:19:36 +0200 Subject: [PATCH 249/512] TEZ-4204: Data race in RootInputInitializerManager (Mustafa Iman via Ashutosh Chauhan) Signed-off-by: Laszlo Bodor --- .../app/dag/RootInputInitializerManager.java | 74 +++++++++++++------ 1 file changed, 53 insertions(+), 21 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java index 5ce00504b1..9194c1dd14 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java @@ -25,6 +25,7 @@ import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.LinkedList; @@ -33,6 +34,8 @@ import java.util.Objects; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicBoolean; import javax.annotation.Nullable; @@ -104,36 +107,65 @@ public RootInputInitializerManager(Vertex vertex, AppContext appContext, } - public void runInputInitializers( - List> inputs, List pendingInitializerEvents) { - - executor.submit(() -> createAndStartInitializing(inputs, pendingInitializerEvents)); + public void runInputInitializers(List> inputs, + List pendingInitializerEvents) { + List initWrappers = createInitializerWrappers(inputs); + if (!initWrappers.isEmpty()) { + executor.submit(() -> createAndStartInitializing(pendingInitializerEvents, initWrappers)); + } } - private void createAndStartInitializing(List> inputs, List pendingInitializerEvents) { + /** + * Create input wrappers for all inputs in parallel. + * + * @param inputs + * @return + */ + private List createInitializerWrappers( + List> inputs) { String current = null; + final List result = Collections.synchronizedList(new ArrayList<>()); try { - List result = new ArrayList<>(); + final List> fResults = new ArrayList<>(); for (RootInputLeafOutput each : inputs) { current = each.getName(); - InitializerWrapper initializer = createInitializerWrapper(each); - initializerMap.put(each.getName(), initializer); - registerPendingVertex(each, initializer); - result.add(initializer); + fResults.add(executor.submit(() -> { + InitializerWrapper initializer = createInitializerWrapper(each); + initializerMap.put(each.getName(), initializer); + registerPendingVertex(each, initializer); + result.add(initializer); + return null; + })); } - handleInitializerEvents(pendingInitializerEvents); - pendingInitializerEvents.clear(); - for (InitializerWrapper inputWrapper : result) { - executor.submit(() -> runInitializerAndProcessResult(inputWrapper)); + for(Future f : fResults) { + f.get(); } - } catch (Throwable t) { - VertexImpl vertexImpl = (VertexImpl) vertex; - String msg = "Fail to create InputInitializerManager, " + ExceptionUtils.getStackTrace(t); - LOG.info(msg); - vertexImpl.finished(FAILED, VertexTerminationCause.INIT_FAILURE, msg); - eventHandler.handle(new VertexEventRootInputFailed(vertex.getVertexId(), current, - new AMUserCodeException(AMUserCodeException.Source.InputInitializer, t))); + } catch (InterruptedException | ExecutionException t) { + failVertex(t, current); + } + return result; + } + + void failVertex(Throwable t, String inputName) { + VertexImpl vertexImpl = (VertexImpl) vertex; + String msg = "Fail to create InputInitializerManager, " + ExceptionUtils.getStackTrace(t); + LOG.info(msg); + vertexImpl.finished(FAILED, VertexTerminationCause.INIT_FAILURE, msg); + eventHandler.handle(new VertexEventRootInputFailed(vertex.getVertexId(), inputName, + new AMUserCodeException(AMUserCodeException.Source.InputInitializer, t))); + } + /** + * Start initializers in parallel. + * + * @param pendingInitializerEvents + * @param result + */ + private void createAndStartInitializing(List pendingInitializerEvents, List result) { + handleInitializerEvents(pendingInitializerEvents); + pendingInitializerEvents.clear(); + for (InitializerWrapper inputWrapper : result) { + executor.submit(() -> runInitializerAndProcessResult(inputWrapper)); } } From 69a2bdeb19cbedb9f3d868f4e7231d3c92b98c75 Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Fri, 31 Jul 2020 15:12:06 -0500 Subject: [PATCH 250/512] TEZ-4203. Findbugs: MergeThread.shuffleSchedulerThread; locked 80% of time Signed-off-by: Jonathan Eagles --- tez-runtime-library/findbugs-exclude.xml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/tez-runtime-library/findbugs-exclude.xml b/tez-runtime-library/findbugs-exclude.xml index 9e55221983..caa46c5527 100644 --- a/tez-runtime-library/findbugs-exclude.xml +++ b/tez-runtime-library/findbugs-exclude.xml @@ -213,4 +213,11 @@ + + + + + + + From 3c405c96f4da9713968ff463af051d0ca8cd25ee Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Tue, 4 Aug 2020 08:17:54 +0200 Subject: [PATCH 251/512] =?UTF-8?q?TEZ-4172:=20Let=20tasks=20be=20killed?= =?UTF-8?q?=20after=20too=20many=20overall=20attempts=20(L=C3=A1szl=C3=B3?= =?UTF-8?q?=20Bodor=20reviewed=20by=20Jonathan=20Turner=20Eagles)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../apache/tez/dag/api/TezConfiguration.java | 10 ++++ .../org/apache/tez/dag/app/dag/Vertex.java | 1 + .../apache/tez/dag/app/dag/impl/TaskImpl.java | 49 +++++++++++++----- .../tez/dag/app/dag/impl/VertexImpl.java | 8 +++ .../tez/dag/app/dag/impl/TestTaskImpl.java | 51 +++++++++++++++---- 5 files changed, 97 insertions(+), 22 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 58aecdabb1..7dc412b62b 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -708,6 +708,16 @@ public TezConfiguration(boolean loadDefaults) { TEZ_AM_PREFIX + "task.max.failed.attempts"; public static final int TEZ_AM_TASK_MAX_FAILED_ATTEMPTS_DEFAULT = 4; + /** + * Int value. The maximum number of attempts that can run for a particular task before the task is + * failed. This count every attempts, including failed, killed attempts. Task failure results in + * DAG failure. Default is 0, which disables this feature. + */ + @ConfigurationScope(Scope.VERTEX) + @ConfigurationProperty(type = "integer") + public static final String TEZ_AM_TASK_MAX_ATTEMPTS = TEZ_AM_PREFIX + "task.max.attempts"; + public static final int TEZ_AM_TASK_MAX_ATTEMPTS_DEFAULT = 0; + /** * Boolean value. Specifies whether a re-scheduled attempt of a task, caused by previous * failures gets higher priority diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java index f3ef72bbc4..ba3079d4e1 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java @@ -214,6 +214,7 @@ public TaskAttemptEventInfo getTaskAttemptTezEvents(TezTaskAttemptID attemptID, interface VertexConfig { int getMaxFailedTaskAttempts(); + int getMaxTaskAttempts(); boolean getTaskRescheduleHigherPriority(); boolean getTaskRescheduleRelaxedLocality(); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java index 34702163fe..0b4b1160f2 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java @@ -126,6 +126,7 @@ public class TaskImpl implements Task, EventHandler { private final TezTaskID taskId; private Map attempts; protected final int maxFailedAttempts; + protected final int maxAttempts; //overall max number of attempts (consider preempted task attempts) protected final Clock clock; private final Vertex vertex; private final Lock readLock; @@ -159,7 +160,7 @@ public class TaskImpl implements Task, EventHandler { private final Set nodesWithRunningAttempts = Collections .newSetFromMap(new ConcurrentHashMap()); - private static final SingleArcTransition + private static final MultipleArcTransition ATTEMPT_KILLED_TRANSITION = new AttemptKilledTransition(); private static final SingleArcTransition KILL_TRANSITION = new KillTransition(); @@ -193,7 +194,7 @@ TaskEventType.T_ATTEMPT_LAUNCHED, new LaunchTransition()) .addTransition(TaskStateInternal.SCHEDULED, TaskStateInternal.KILL_WAIT, TaskEventType.T_TERMINATE, KILL_TRANSITION) - .addTransition(TaskStateInternal.SCHEDULED, TaskStateInternal.SCHEDULED, + .addTransition(TaskStateInternal.SCHEDULED, EnumSet.of(TaskStateInternal.SCHEDULED, TaskStateInternal.FAILED), TaskEventType.T_ATTEMPT_KILLED, ATTEMPT_KILLED_TRANSITION) .addTransition(TaskStateInternal.SCHEDULED, EnumSet.of(TaskStateInternal.SCHEDULED, TaskStateInternal.FAILED), @@ -216,7 +217,7 @@ TaskEventType.T_ADD_SPEC_ATTEMPT, new RedundantScheduleTransition()) EnumSet.of(TaskStateInternal.SUCCEEDED), TaskEventType.T_ATTEMPT_SUCCEEDED, new AttemptSucceededTransition()) - .addTransition(TaskStateInternal.RUNNING, TaskStateInternal.RUNNING, + .addTransition(TaskStateInternal.RUNNING, EnumSet.of(TaskStateInternal.RUNNING, TaskStateInternal.FAILED), TaskEventType.T_ATTEMPT_KILLED, ATTEMPT_KILLED_TRANSITION) .addTransition(TaskStateInternal.RUNNING, @@ -380,6 +381,7 @@ public TaskImpl(TezVertexID vertexId, int taskIndex, writeLock = readWriteLock.writeLock(); this.attempts = Collections.emptyMap(); maxFailedAttempts = vertex.getVertexConfig().getMaxFailedTaskAttempts(); + maxAttempts = vertex.getVertexConfig().getMaxTaskAttempts(); taskId = TezTaskID.getInstance(vertexId, taskIndex); this.taskCommunicatorManagerInterface = taskCommunicatorManagerInterface; this.taskHeartbeatHandler = thh; @@ -776,7 +778,7 @@ public TaskAttempt getSuccessfulAttempt() { } // This is always called in the Write Lock - private void addAndScheduleAttempt(TezTaskAttemptID schedulingCausalTA) { + private boolean addAndScheduleAttempt(TezTaskAttemptID schedulingCausalTA) { TaskAttempt attempt = createAttempt(attempts.size(), schedulingCausalTA); if (LOG.isDebugEnabled()) { LOG.debug("Created attempt " + attempt.getID()); @@ -794,13 +796,23 @@ private void addAndScheduleAttempt(TezTaskAttemptID schedulingCausalTA) { Preconditions.checkArgument(attempts.put(attempt.getID(), attempt) == null, attempt.getID() + " already existed"); break; - default: Preconditions.checkArgument(attempts.put(attempt.getID(), attempt) == null, attempt.getID() + " already existed"); break; } + if (maxAttempts > 0 && attempts.size() == maxAttempts) { + TaskImpl task = (TaskImpl) attempt.getTask(); + LOG.error("Cannot schedule new attempt for task as max number of attempts ({}) reached: {}", + maxAttempts, task); + + task.logJobHistoryTaskFailedEvent(TaskState.FAILED); + task.eventHandler.handle(new VertexEventTaskCompleted(task.taskId, TaskState.FAILED)); + + return false; + } + // TODO: Recovery /* // Update nextATtemptNumber @@ -819,7 +831,7 @@ private void addAndScheduleAttempt(TezTaskAttemptID schedulingCausalTA) { // picture in mind eventHandler.handle(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, attempt)); - + return true; } @Override @@ -1007,7 +1019,9 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { task.locationHint = scheduleEvent.getTaskLocationHint(); task.baseTaskSpec = scheduleEvent.getBaseTaskSpec(); // For now, initial scheduling dependency is due to vertex manager scheduling - task.addAndScheduleAttempt(null); + if (!task.addAndScheduleAttempt(null)) { + return TaskStateInternal.FAILED; + } return TaskStateInternal.SCHEDULED; } } @@ -1108,7 +1122,9 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { LOG.info("Can not recover the successful task attempt, schedule new task attempt," + "taskId=" + task.getTaskId()); task.successfulAttempt = null; - task.addAndScheduleAttempt(successTaId); + if (!task.addAndScheduleAttempt(successTaId)) { + task.finished(TaskStateInternal.FAILED); + } task.eventHandler.handle(new TaskAttemptEventAttemptKilled(successTaId, errorMsg, TaskAttemptTerminationCause.TERMINATED_AT_RECOVERY, true)); return TaskStateInternal.RUNNING; @@ -1167,9 +1183,11 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { } private static class AttemptKilledTransition implements - SingleArcTransition { + MultipleArcTransition { @Override - public void transition(TaskImpl task, TaskEvent event) { + public TaskStateInternal transition(TaskImpl task, TaskEvent event) { + TaskStateInternal originalState = task.getInternalState(); + TaskEventTAUpdate castEvent = (TaskEventTAUpdate) event; task.addDiagnosticInfo("TaskAttempt " + castEvent.getTaskAttemptID().getId() + " killed"); if (task.commitAttempt !=null && @@ -1197,8 +1215,11 @@ public void transition(TaskImpl task, TaskEvent event) { task.getVertex().incrementKilledTaskAttemptCount(); } if (task.shouldScheduleNewAttempt()) { - task.addAndScheduleAttempt(castEvent.getTaskAttemptID()); + if (!task.addAndScheduleAttempt(castEvent.getTaskAttemptID())) { + return task.finished(TaskStateInternal.FAILED); + } } + return originalState; } } @@ -1258,8 +1279,10 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { if (task.shouldScheduleNewAttempt()) { LOG.info("Scheduling new attempt for task: " + task.getTaskId() + ", currentFailedAttempts: " + task.failedAttempts + ", maxFailedAttempts: " - + task.maxFailedAttempts); - task.addAndScheduleAttempt(getSchedulingCausalTA()); + + task.maxFailedAttempts + ", maxAttempts: " + task.maxAttempts); + if (!task.addAndScheduleAttempt(getSchedulingCausalTA())){ + return task.finished(TaskStateInternal.FAILED); + } } } else { if (castEvent.getTaskFailureType() == TaskFailureType.NON_FATAL) { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index db0cd46119..711d02827e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -4828,6 +4828,7 @@ void setCounters(TezCounters counters) { static class VertexConfigImpl implements VertexConfig { private final int maxFailedTaskAttempts; + private final int maxTaskAttempts; private final boolean taskRescheduleHigherPriority; private final boolean taskRescheduleRelaxedLocality; @@ -4847,6 +4848,8 @@ static class VertexConfigImpl implements VertexConfig { public VertexConfigImpl(Configuration conf) { this.maxFailedTaskAttempts = conf.getInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS, TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS_DEFAULT); + this.maxTaskAttempts = conf.getInt(TezConfiguration.TEZ_AM_TASK_MAX_ATTEMPTS, + TezConfiguration.TEZ_AM_TASK_MAX_ATTEMPTS_DEFAULT); this.taskRescheduleHigherPriority = conf.getBoolean(TezConfiguration.TEZ_AM_TASK_RESCHEDULE_HIGHER_PRIORITY, TezConfiguration.TEZ_AM_TASK_RESCHEDULE_HIGHER_PRIORITY_DEFAULT); @@ -4872,6 +4875,11 @@ public int getMaxFailedTaskAttempts() { return maxFailedTaskAttempts; } + @Override + public int getMaxTaskAttempts() { + return maxTaskAttempts; + } + @Override public boolean getTaskRescheduleHigherPriority() { return taskRescheduleHigherPriority; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java index 1af6092d1e..a28e786c6e 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java @@ -286,10 +286,15 @@ private void updateAttemptState(MockTaskAttemptImpl attempt, attempt.setState(s); } - private void killRunningTaskAttempt(TezTaskAttemptID attemptId) { + private void killRunningTaskAttempt(TezTaskAttemptID attemptId, TaskState stateToVerify) { + killRunningTaskAttempt(attemptId, stateToVerify, 1); + } + + private void killRunningTaskAttempt(TezTaskAttemptID attemptId, TaskState stateToVerify, + int killedCountToVerify) { mockTask.handle(createTaskTAKilledEvent(attemptId)); - assertTaskRunningState(); - verify(mockTask.getVertex(), times(1)).incrementKilledTaskAttemptCount(); + assertTaskState(stateToVerify); + verify(mockTask.getVertex(), times(killedCountToVerify)).incrementKilledTaskAttemptCount(); } private void failRunningTaskAttempt(TezTaskAttemptID attemptId) { @@ -310,21 +315,25 @@ private void failRunningTaskAttempt(TezTaskAttemptID attemptId, boolean verifySt * {@link TaskState#NEW} */ private void assertTaskNewState() { - assertEquals(TaskState.NEW, mockTask.getState()); + assertTaskState(TaskState.NEW); } /** * {@link TaskState#SCHEDULED} */ private void assertTaskScheduledState() { - assertEquals(TaskState.SCHEDULED, mockTask.getState()); + assertTaskState(TaskState.SCHEDULED); } /** * {@link TaskState#RUNNING} */ private void assertTaskRunningState() { - assertEquals(TaskState.RUNNING, mockTask.getState()); + assertTaskState(TaskState.RUNNING); + } + + private void assertTaskState(TaskState state) { + assertEquals(state, mockTask.getState()); } /** @@ -385,7 +394,7 @@ public void testKillRunningTask() { } @Test(timeout = 5000) - public void testTooManyFailedAtetmpts() { + public void testTooManyFailedAttempts() { LOG.info("--- START: testTooManyFailedAttempts ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId, TaskState.SCHEDULED); @@ -408,6 +417,30 @@ public void testTooManyFailedAtetmpts() { verifyOutgoingEvents(eventHandler.events, VertexEventType.V_TASK_COMPLETED); } + @Test(timeout = 5000) + public void testTooManyAttempts() { + LOG.info("--- START: testTooManyAttempts ---"); + + conf.setInt(TezConfiguration.TEZ_AM_TASK_MAX_ATTEMPTS, 3); + Vertex vertex = mock(Vertex.class); + doReturn(new VertexImpl.VertexConfigImpl(conf)).when(vertex).getVertexConfig(); + mockTask = new MockTaskImpl(vertexId, partition, + eventHandler, conf, taskCommunicatorManagerInterface, clock, + taskHeartbeatHandler, appContext, leafVertex, + taskResource, containerContext, vertex); + + TezTaskID taskId = getNewTaskID(); + scheduleTaskAttempt(taskId, TaskState.SCHEDULED); // attempt_0 + launchTaskAttempt(mockTask.getLastAttempt().getID()); + killRunningTaskAttempt(mockTask.getLastAttempt().getID(), TaskState.RUNNING, 1); // attempt_1 + + launchTaskAttempt(mockTask.getLastAttempt().getID()); + killRunningTaskAttempt(mockTask.getLastAttempt().getID(), TaskState.FAILED, 2); // attempt_2 -> reached 3 + + assertEquals(TaskStateInternal.FAILED, mockTask.getInternalState()); + verifyOutgoingEvents(eventHandler.events, VertexEventType.V_TASK_COMPLETED); + } + @Test(timeout = 5000) public void testFailedAttemptWithFatalError() { LOG.info("--- START: testFailedAttemptWithFatalError ---"); @@ -428,7 +461,7 @@ public void testKillRunningTaskPreviousKilledAttempts() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); launchTaskAttempt(mockTask.getLastAttempt().getID()); - killRunningTaskAttempt(mockTask.getLastAttempt().getID()); + killRunningTaskAttempt(mockTask.getLastAttempt().getID(), TaskState.RUNNING); assertEquals(TaskStateInternal.RUNNING, mockTask.getInternalState()); killTask(taskId); mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getID())); @@ -503,7 +536,7 @@ public void testKillRunningTaskAttempt() { scheduleTaskAttempt(taskId); TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getID(); launchTaskAttempt(mockTask.getLastAttempt().getID()); - killRunningTaskAttempt(mockTask.getLastAttempt().getID()); + killRunningTaskAttempt(mockTask.getLastAttempt().getID(), TaskState.RUNNING); // last killed attempt should be causal TA of next attempt Assert.assertEquals(lastTAId, mockTask.getLastAttempt().getSchedulingCausalTA()); } From 5f3cac509df0acb733c9f3539a6a37dd0dda1297 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Tue, 4 Aug 2020 08:58:22 -0500 Subject: [PATCH 252/512] =?UTF-8?q?TEZ-4212.=20Fix=20build=20checkstyle=20?= =?UTF-8?q?configuration=20and=20suppressions=20dtd=20URLs=20(Jonathan=20E?= =?UTF-8?q?agles=20reviewed=20by=20L=C3=A1szl=C3=B3=20Bodor)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Jonathan Eagles --- pom.xml | 4 ++-- .../src/main/resources/checkstyle/checkstyle.xml | 11 ++++++----- .../src/main/resources/checkstyle/suppressions.xml | 4 ++-- 3 files changed, 10 insertions(+), 9 deletions(-) diff --git a/pom.xml b/pom.xml index b1d5f66723..26b20d8094 100644 --- a/pom.xml +++ b/pom.xml @@ -61,8 +61,8 @@ 3.0.5 2.10.4 2.4.3 - 3.0.0 - 8.16 + 3.1.1 + 8.35 1.3.6 ${project.build.directory}/tmp diff --git a/tez-build-tools/src/main/resources/checkstyle/checkstyle.xml b/tez-build-tools/src/main/resources/checkstyle/checkstyle.xml index fd002ded72..cac5814e7f 100644 --- a/tez-build-tools/src/main/resources/checkstyle/checkstyle.xml +++ b/tez-build-tools/src/main/resources/checkstyle/checkstyle.xml @@ -1,7 +1,7 @@ + "-//Checkstyle//DTD Checkstyle Configuration 1.3//EN" + "https://checkstyle.org/dtds/configuration_1_3.dtd"> + + + + @@ -122,9 +126,6 @@ - - - diff --git a/tez-build-tools/src/main/resources/checkstyle/suppressions.xml b/tez-build-tools/src/main/resources/checkstyle/suppressions.xml index ccc89c8bf0..829751a139 100644 --- a/tez-build-tools/src/main/resources/checkstyle/suppressions.xml +++ b/tez-build-tools/src/main/resources/checkstyle/suppressions.xml @@ -13,8 +13,8 @@ limitations under the License. See accompanying LICENSE file. --> + "-//Checkstyle//DTD SuppressionFilter Configuration 1.2//EN" + "https://checkstyle.org/dtds/suppressions_1_2.dtd"> From 3ce961bc732322140db4d6120b309134958cac7b Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Wed, 5 Aug 2020 16:52:46 -0700 Subject: [PATCH 253/512] TEZ-4208 : Pipelinesorter uses single SortSpan after spill (Rajesh Balamohan via Ashutosh Chauhan) Signed-off-by: Ashutosh Chauhan --- .../common/sort/impl/PipelinedSorter.java | 11 +++++-- .../common/sort/impl/TestPipelinedSorter.java | 29 +++++++++++++++++-- 2 files changed, 35 insertions(+), 5 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java index bc68a5f206..2ace8755f0 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java @@ -116,6 +116,8 @@ public class PipelinedSorter extends ExternalSorter { //Maintain a list of ByteBuffers @VisibleForTesting final List buffers; + @VisibleForTesting + List bufferUsage; final int maxNumberOfBlocks; private int bufferIndex = -1; private final int MIN_BLOCK_SIZE; @@ -202,6 +204,7 @@ public PipelinedSorter(OutputContext outputContext, Configuration conf, int numO capacity = totalCapacityWithoutMeta; buffers = Lists.newArrayListWithCapacity(maxNumberOfBlocks); + bufferUsage = Lists.newArrayListWithCapacity(maxNumberOfBlocks); allocateSpace(); //Allocate the first block if (!lazyAllocateMem) { LOG.info("Pre allocating rest of memory buffers upfront"); @@ -257,6 +260,7 @@ ByteBuffer allocateSpace() { buffers.add(space); bufferIndex++; + bufferUsage.add(0); Preconditions.checkState(buffers.size() <= maxNumberOfBlocks, "Number of blocks " + buffers.size() @@ -337,8 +341,9 @@ public void sort() throws IOException { if (pipelinedShuffle && ret) { sendPipelinedShuffleEvents(); } - //safe to reset bufferIndex to 0; - bufferIndex = 0; + // Use the next buffer + bufferIndex = (bufferIndex + 1) % buffers.size(); + bufferUsage.set(bufferIndex, bufferUsage.get(bufferIndex) + 1); int items = 1024*1024; int perItem = 16; if(span.length() != 0) { @@ -1186,7 +1191,7 @@ public int size() { public int compareTo(SpanIterator other) { return span.compareInternal(other.getKey(), other.getPartition(), kvindex); } - + @Override public String toString() { return String.format("SpanIterator<%d:%d> (span=%s)", kvindex, maxindex, span.toString()); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java index bd7f585adb..eed423b494 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java @@ -47,7 +47,6 @@ import org.apache.tez.runtime.library.api.TezRuntimeConfiguration.ReportPartitionStats; import org.apache.tez.runtime.library.common.Constants; import org.apache.tez.runtime.library.common.combine.Combiner; -import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.runtime.library.conf.OrderedPartitionedKVOutputConfig.SorterImpl; import org.apache.tez.runtime.library.partitioner.HashPartitioner; import org.apache.tez.runtime.library.testutils.RandomTextGenerator; @@ -67,7 +66,6 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyListOf; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -499,6 +497,33 @@ public void testMultipleSpills_WithRLE() throws IOException { verifyOutputPermissions(outputContext.getUniqueIdentifier()); } + @Test + /** + * Verify whether all buffers are used evenly in sorter. + */ + public void basicTestForBufferUsage() throws IOException { + this.numOutputs = 1; + conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_PIPELINED_SORTER_LAZY_ALLOCATE_MEMORY, true); + + PipelinedSorter sorter = new PipelinedSorter(this.outputContext, conf, numOutputs, (100 << 20)); + Assert.assertTrue(sorter.maxNumberOfBlocks >= 2); + + // Start filling in with data 1MB Key, 1MB Val. + for (int i = 0; i < 200; i++) { + writeData(sorter, 1, 1024 * 1024, false); + } + + // Check if all buffers are evenly used + int avg = (int) sorter.bufferUsage.stream().mapToDouble(d -> d).average().orElse(0.0); + + for(int i = 0; i< sorter.bufferUsage.size(); i++) { + int usage = sorter.bufferUsage.get(i); + Assert.assertTrue("Buffer index " + i + " is not used correctly. " + + " usage: " + usage + ", avg: " + avg, usage >= avg); + } + conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_PIPELINED_SORTER_LAZY_ALLOCATE_MEMORY, false); + } + public void basicTest2(int partitions, int[] numkeys, int[] keysize, long initialAvailableMem, int blockSize) throws IOException { this.numOutputs = partitions; // single output From 8364f17abd9a6adc95e100bb308b22754529f8f8 Mon Sep 17 00:00:00 2001 From: Eugene Chung Date: Fri, 7 Aug 2020 16:27:22 -0500 Subject: [PATCH 254/512] TEZ-4188. Link to NodeManager Logs of Home and DAG details doesn't consider yarnProtocol Signed-off-by: Jonathan Eagles --- tez-ui/src/main/webapp/app/serializers/dag.js | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tez-ui/src/main/webapp/app/serializers/dag.js b/tez-ui/src/main/webapp/app/serializers/dag.js index de60985f88..701c1d2afd 100644 --- a/tez-ui/src/main/webapp/app/serializers/dag.js +++ b/tez-ui/src/main/webapp/app/serializers/dag.js @@ -75,8 +75,9 @@ function getContainerLogs(source) { for (var key in otherinfo) { if (key.indexOf('inProgressLogsURL_') === 0) { let logs = Ember.get(source, 'otherinfo.' + key); - if (logs.indexOf('http') !== 0) { - logs = 'http://' + logs; + if (logs.indexOf("://") === -1) { + let yarnProtocol = this.get('env.app.yarnProtocol'); + logs = yarnProtocol + '://' + logs; } let attemptID = key.substring(18); containerLogs.push({ From 6fc75ad6e9b1601b8b14dd85fa9b0aea53585fba Mon Sep 17 00:00:00 2001 From: Harish JP Date: Thu, 13 Aug 2020 08:57:35 +0530 Subject: [PATCH 255/512] TEZ-4223 - Adding new jars or resources after the first DAG runs does not work. Signed-off-by: Rajesh Balamohan --- .../apache/tez/common/ReflectionUtils.java | 43 +----------------- .../org/apache/tez/common/TezClassLoader.java | 45 +++++++------------ .../tez/common/TestReflectionUtils.java | 4 +- .../org/apache/tez/dag/app/DAGAppMaster.java | 2 + 4 files changed, 21 insertions(+), 73 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java b/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java index 9f7c5d35f0..73becdaa6a 100644 --- a/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java @@ -19,17 +19,14 @@ package org.apache.tez.common; import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.net.URL; -import java.net.URLClassLoader; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.tez.dag.api.TezReflectionException; -import org.apache.tez.dag.api.TezUncheckedException; @Private public class ReflectionUtils { @@ -109,47 +106,11 @@ public static Method getMethod(Class targetClazz, String methodName, Class } } - @Private - public static synchronized void addResourcesToClasspath(List urls) { - ClassLoader classLoader = new URLClassLoader(urls.toArray(new URL[urls.size()]), Thread - .currentThread().getContextClassLoader()); - Thread.currentThread().setContextClassLoader(classLoader); - } - - // Parameters for addResourcesToSystemClassLoader - private static final Class[] parameters = new Class[]{URL.class}; - private static Method sysClassLoaderMethod = null; - @Private public static synchronized void addResourcesToSystemClassLoader(List urls) { - ClassLoader sysLoader = getSystemClassLoader(); - if (sysClassLoaderMethod == null) { - Class sysClass = TezClassLoader.class; - Method method; - try { - method = sysClass.getDeclaredMethod("addURL", parameters); - } catch (SecurityException e) { - throw new TezUncheckedException("Failed to get handle on method addURL", e); - } catch (NoSuchMethodException e) { - throw new TezUncheckedException("Failed to get handle on method addURL", e); - } - method.setAccessible(true); - sysClassLoaderMethod = method; - } + TezClassLoader classLoader = TezClassLoader.getInstance(); for (URL url : urls) { - try { - sysClassLoaderMethod.invoke(sysLoader, new Object[] { url }); - } catch (IllegalArgumentException e) { - throw new TezUncheckedException("Failed to invoke addURL for rsrc: " + url, e); - } catch (IllegalAccessException e) { - throw new TezUncheckedException("Failed to invoke addURL for rsrc: " + url, e); - } catch (InvocationTargetException e) { - throw new TezUncheckedException("Failed to invoke addURL for rsrc: " + url, e); - } + classLoader.addURL(url); } } - - private static ClassLoader getSystemClassLoader() { - return TezClassLoader.getInstance(); - } } diff --git a/tez-api/src/main/java/org/apache/tez/common/TezClassLoader.java b/tez-api/src/main/java/org/apache/tez/common/TezClassLoader.java index 2679efa692..1842a1915b 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezClassLoader.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezClassLoader.java @@ -13,30 +13,24 @@ */ package org.apache.tez.common; -import java.net.MalformedURLException; import java.net.URL; import java.net.URLClassLoader; -import java.security.AccessController; -import java.security.PrivilegedAction; -import java.util.Arrays; +/** + * ClassLoader to allow addition of new paths to classpath in the runtime. + * + * It uses URLClassLoader with this class' classloader as parent classloader. + * And hence first delegates the resource loading to parent and then to the URLs + * added. The process must be setup to use by invoking setupTezClassLoader() which sets + * the global TezClassLoader as current thread context class loader. All threads + * created will inherit the classloader and hence will resolve the class/resource + * from TezClassLoader. + */ public class TezClassLoader extends URLClassLoader { - private static TezClassLoader INSTANCE; - - static { - INSTANCE = AccessController.doPrivileged(new PrivilegedAction() { - ClassLoader sysLoader = TezClassLoader.class.getClassLoader(); - - public TezClassLoader run() { - return new TezClassLoader( - sysLoader instanceof URLClassLoader ? ((URLClassLoader) sysLoader).getURLs() : extractClassPathEntries(), - sysLoader); - } - }); - } + private static TezClassLoader INSTANCE = new TezClassLoader(); - public TezClassLoader(URL[] urls, ClassLoader classLoader) { - super(urls, classLoader); + private TezClassLoader() { + super(new URL[] {}, TezClassLoader.class.getClassLoader()); } public void addURL(URL url) { @@ -47,16 +41,7 @@ public static TezClassLoader getInstance() { return INSTANCE; } - private static URL[] extractClassPathEntries() { - String pathSeparator = System.getProperty("path.separator"); - String[] classPathEntries = System.getProperty("java.class.path").split(pathSeparator); - URL[] cp = Arrays.asList(classPathEntries).stream().map(s -> { - try { - return new URL("file://" + s); - } catch (MalformedURLException e) { - throw new RuntimeException(e); - } - }).toArray(URL[]::new); - return cp; + public static void setupTezClassLoader() { + Thread.currentThread().setContextClassLoader(INSTANCE); } } diff --git a/tez-api/src/test/java/org/apache/tez/common/TestReflectionUtils.java b/tez-api/src/test/java/org/apache/tez/common/TestReflectionUtils.java index 2fbd35cb3d..ed3814d000 100644 --- a/tez-api/src/test/java/org/apache/tez/common/TestReflectionUtils.java +++ b/tez-api/src/test/java/org/apache/tez/common/TestReflectionUtils.java @@ -58,7 +58,7 @@ public void testConstructorWithParameters() throws TezReflectionException @Test(timeout = 5000) public void testAddResourceToClasspath() throws IOException, TezException { - + TezClassLoader.setupTezClassLoader(); String rsrcName = "dummyfile.xml"; FileSystem localFs = FileSystem.getLocal(new Configuration()); Path p = new Path(rsrcName); @@ -78,7 +78,7 @@ public void testAddResourceToClasspath() throws IOException, TezException { urlForm = urlForm.substring(0, urlForm.lastIndexOf('/') + 1); URL url = new URL(urlForm); - ReflectionUtils.addResourcesToClasspath(Collections.singletonList(url)); + ReflectionUtils.addResourcesToSystemClassLoader(Collections.singletonList(url)); loadedUrl = Thread.currentThread().getContextClassLoader().getResource(rsrcName); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index 7e5a7a9d67..fcfb883953 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -2328,6 +2328,8 @@ public boolean isSession() { public static void main(String[] args) { try { + // Install the tez class loader, which can be used add new resources + TezClassLoader.setupTezClassLoader(); Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler()); final String pid = System.getenv().get("JVM_PID"); String containerIdStr = From 59419dd3e3817d52f3a7ace34f122c92bea03d33 Mon Sep 17 00:00:00 2001 From: Harish JP Date: Thu, 13 Aug 2020 11:59:03 +0530 Subject: [PATCH 256/512] Revert "TEZ-4223 - Adding new jars or resources after the first DAG runs does not work." This reverts commit 6fc75ad6e9b1601b8b14dd85fa9b0aea53585fba. --- .../apache/tez/common/ReflectionUtils.java | 43 +++++++++++++++++- .../org/apache/tez/common/TezClassLoader.java | 45 ++++++++++++------- .../tez/common/TestReflectionUtils.java | 4 +- .../org/apache/tez/dag/app/DAGAppMaster.java | 2 - 4 files changed, 73 insertions(+), 21 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java b/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java index 73becdaa6a..9f7c5d35f0 100644 --- a/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java @@ -19,14 +19,17 @@ package org.apache.tez.common; import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.net.URL; +import java.net.URLClassLoader; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.tez.dag.api.TezReflectionException; +import org.apache.tez.dag.api.TezUncheckedException; @Private public class ReflectionUtils { @@ -106,11 +109,47 @@ public static Method getMethod(Class targetClazz, String methodName, Class } } + @Private + public static synchronized void addResourcesToClasspath(List urls) { + ClassLoader classLoader = new URLClassLoader(urls.toArray(new URL[urls.size()]), Thread + .currentThread().getContextClassLoader()); + Thread.currentThread().setContextClassLoader(classLoader); + } + + // Parameters for addResourcesToSystemClassLoader + private static final Class[] parameters = new Class[]{URL.class}; + private static Method sysClassLoaderMethod = null; + @Private public static synchronized void addResourcesToSystemClassLoader(List urls) { - TezClassLoader classLoader = TezClassLoader.getInstance(); + ClassLoader sysLoader = getSystemClassLoader(); + if (sysClassLoaderMethod == null) { + Class sysClass = TezClassLoader.class; + Method method; + try { + method = sysClass.getDeclaredMethod("addURL", parameters); + } catch (SecurityException e) { + throw new TezUncheckedException("Failed to get handle on method addURL", e); + } catch (NoSuchMethodException e) { + throw new TezUncheckedException("Failed to get handle on method addURL", e); + } + method.setAccessible(true); + sysClassLoaderMethod = method; + } for (URL url : urls) { - classLoader.addURL(url); + try { + sysClassLoaderMethod.invoke(sysLoader, new Object[] { url }); + } catch (IllegalArgumentException e) { + throw new TezUncheckedException("Failed to invoke addURL for rsrc: " + url, e); + } catch (IllegalAccessException e) { + throw new TezUncheckedException("Failed to invoke addURL for rsrc: " + url, e); + } catch (InvocationTargetException e) { + throw new TezUncheckedException("Failed to invoke addURL for rsrc: " + url, e); + } } } + + private static ClassLoader getSystemClassLoader() { + return TezClassLoader.getInstance(); + } } diff --git a/tez-api/src/main/java/org/apache/tez/common/TezClassLoader.java b/tez-api/src/main/java/org/apache/tez/common/TezClassLoader.java index 1842a1915b..2679efa692 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezClassLoader.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezClassLoader.java @@ -13,24 +13,30 @@ */ package org.apache.tez.common; +import java.net.MalformedURLException; import java.net.URL; import java.net.URLClassLoader; +import java.security.AccessController; +import java.security.PrivilegedAction; +import java.util.Arrays; -/** - * ClassLoader to allow addition of new paths to classpath in the runtime. - * - * It uses URLClassLoader with this class' classloader as parent classloader. - * And hence first delegates the resource loading to parent and then to the URLs - * added. The process must be setup to use by invoking setupTezClassLoader() which sets - * the global TezClassLoader as current thread context class loader. All threads - * created will inherit the classloader and hence will resolve the class/resource - * from TezClassLoader. - */ public class TezClassLoader extends URLClassLoader { - private static TezClassLoader INSTANCE = new TezClassLoader(); + private static TezClassLoader INSTANCE; + + static { + INSTANCE = AccessController.doPrivileged(new PrivilegedAction() { + ClassLoader sysLoader = TezClassLoader.class.getClassLoader(); + + public TezClassLoader run() { + return new TezClassLoader( + sysLoader instanceof URLClassLoader ? ((URLClassLoader) sysLoader).getURLs() : extractClassPathEntries(), + sysLoader); + } + }); + } - private TezClassLoader() { - super(new URL[] {}, TezClassLoader.class.getClassLoader()); + public TezClassLoader(URL[] urls, ClassLoader classLoader) { + super(urls, classLoader); } public void addURL(URL url) { @@ -41,7 +47,16 @@ public static TezClassLoader getInstance() { return INSTANCE; } - public static void setupTezClassLoader() { - Thread.currentThread().setContextClassLoader(INSTANCE); + private static URL[] extractClassPathEntries() { + String pathSeparator = System.getProperty("path.separator"); + String[] classPathEntries = System.getProperty("java.class.path").split(pathSeparator); + URL[] cp = Arrays.asList(classPathEntries).stream().map(s -> { + try { + return new URL("file://" + s); + } catch (MalformedURLException e) { + throw new RuntimeException(e); + } + }).toArray(URL[]::new); + return cp; } } diff --git a/tez-api/src/test/java/org/apache/tez/common/TestReflectionUtils.java b/tez-api/src/test/java/org/apache/tez/common/TestReflectionUtils.java index ed3814d000..2fbd35cb3d 100644 --- a/tez-api/src/test/java/org/apache/tez/common/TestReflectionUtils.java +++ b/tez-api/src/test/java/org/apache/tez/common/TestReflectionUtils.java @@ -58,7 +58,7 @@ public void testConstructorWithParameters() throws TezReflectionException @Test(timeout = 5000) public void testAddResourceToClasspath() throws IOException, TezException { - TezClassLoader.setupTezClassLoader(); + String rsrcName = "dummyfile.xml"; FileSystem localFs = FileSystem.getLocal(new Configuration()); Path p = new Path(rsrcName); @@ -78,7 +78,7 @@ public void testAddResourceToClasspath() throws IOException, TezException { urlForm = urlForm.substring(0, urlForm.lastIndexOf('/') + 1); URL url = new URL(urlForm); - ReflectionUtils.addResourcesToSystemClassLoader(Collections.singletonList(url)); + ReflectionUtils.addResourcesToClasspath(Collections.singletonList(url)); loadedUrl = Thread.currentThread().getContextClassLoader().getResource(rsrcName); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index fcfb883953..7e5a7a9d67 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -2328,8 +2328,6 @@ public boolean isSession() { public static void main(String[] args) { try { - // Install the tez class loader, which can be used add new resources - TezClassLoader.setupTezClassLoader(); Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler()); final String pid = System.getenv().get("JVM_PID"); String containerIdStr = From c047fde127a7ec2448c7851e89366cb3b0b03136 Mon Sep 17 00:00:00 2001 From: Harish JP Date: Thu, 13 Aug 2020 11:59:40 +0530 Subject: [PATCH 257/512] TEZ-4223 - Adding new jars or resources after the first DAG runs does not work. Signed-off-by: Rajesh Balamohan --- .../apache/tez/common/ReflectionUtils.java | 43 +----------------- .../org/apache/tez/common/TezClassLoader.java | 37 +++++++--------- .../apache/tez/client/TestTezClientUtils.java | 44 ++++++++++++++----- .../tez/common/TestReflectionUtils.java | 4 +- .../org/apache/tez/dag/app/DAGAppMaster.java | 2 + 5 files changed, 55 insertions(+), 75 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java b/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java index 9f7c5d35f0..73becdaa6a 100644 --- a/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java @@ -19,17 +19,14 @@ package org.apache.tez.common; import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.net.URL; -import java.net.URLClassLoader; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.tez.dag.api.TezReflectionException; -import org.apache.tez.dag.api.TezUncheckedException; @Private public class ReflectionUtils { @@ -109,47 +106,11 @@ public static Method getMethod(Class targetClazz, String methodName, Class } } - @Private - public static synchronized void addResourcesToClasspath(List urls) { - ClassLoader classLoader = new URLClassLoader(urls.toArray(new URL[urls.size()]), Thread - .currentThread().getContextClassLoader()); - Thread.currentThread().setContextClassLoader(classLoader); - } - - // Parameters for addResourcesToSystemClassLoader - private static final Class[] parameters = new Class[]{URL.class}; - private static Method sysClassLoaderMethod = null; - @Private public static synchronized void addResourcesToSystemClassLoader(List urls) { - ClassLoader sysLoader = getSystemClassLoader(); - if (sysClassLoaderMethod == null) { - Class sysClass = TezClassLoader.class; - Method method; - try { - method = sysClass.getDeclaredMethod("addURL", parameters); - } catch (SecurityException e) { - throw new TezUncheckedException("Failed to get handle on method addURL", e); - } catch (NoSuchMethodException e) { - throw new TezUncheckedException("Failed to get handle on method addURL", e); - } - method.setAccessible(true); - sysClassLoaderMethod = method; - } + TezClassLoader classLoader = TezClassLoader.getInstance(); for (URL url : urls) { - try { - sysClassLoaderMethod.invoke(sysLoader, new Object[] { url }); - } catch (IllegalArgumentException e) { - throw new TezUncheckedException("Failed to invoke addURL for rsrc: " + url, e); - } catch (IllegalAccessException e) { - throw new TezUncheckedException("Failed to invoke addURL for rsrc: " + url, e); - } catch (InvocationTargetException e) { - throw new TezUncheckedException("Failed to invoke addURL for rsrc: " + url, e); - } + classLoader.addURL(url); } } - - private static ClassLoader getSystemClassLoader() { - return TezClassLoader.getInstance(); - } } diff --git a/tez-api/src/main/java/org/apache/tez/common/TezClassLoader.java b/tez-api/src/main/java/org/apache/tez/common/TezClassLoader.java index 2679efa692..923d2177db 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezClassLoader.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezClassLoader.java @@ -13,30 +13,34 @@ */ package org.apache.tez.common; -import java.net.MalformedURLException; import java.net.URL; import java.net.URLClassLoader; import java.security.AccessController; import java.security.PrivilegedAction; -import java.util.Arrays; +/** + * ClassLoader to allow addition of new paths to classpath in the runtime. + * + * It uses URLClassLoader with this class' classloader as parent classloader. + * And hence first delegates the resource loading to parent and then to the URLs + * added. The process must be setup to use by invoking setupTezClassLoader() which sets + * the global TezClassLoader as current thread context class loader. All threads + * created will inherit the classloader and hence will resolve the class/resource + * from TezClassLoader. + */ public class TezClassLoader extends URLClassLoader { - private static TezClassLoader INSTANCE; + private static final TezClassLoader INSTANCE; static { INSTANCE = AccessController.doPrivileged(new PrivilegedAction() { - ClassLoader sysLoader = TezClassLoader.class.getClassLoader(); - public TezClassLoader run() { - return new TezClassLoader( - sysLoader instanceof URLClassLoader ? ((URLClassLoader) sysLoader).getURLs() : extractClassPathEntries(), - sysLoader); + return new TezClassLoader(); } }); } - public TezClassLoader(URL[] urls, ClassLoader classLoader) { - super(urls, classLoader); + private TezClassLoader() { + super(new URL[] {}, TezClassLoader.class.getClassLoader()); } public void addURL(URL url) { @@ -47,16 +51,7 @@ public static TezClassLoader getInstance() { return INSTANCE; } - private static URL[] extractClassPathEntries() { - String pathSeparator = System.getProperty("path.separator"); - String[] classPathEntries = System.getProperty("java.class.path").split(pathSeparator); - URL[] cp = Arrays.asList(classPathEntries).stream().map(s -> { - try { - return new URL("file://" + s); - } catch (MalformedURLException e) { - throw new RuntimeException(e); - } - }).toArray(URL[]::new); - return cp; + public static void setupTezClassLoader() { + Thread.currentThread().setContextClassLoader(INSTANCE); } } diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java index adcc65c3bf..29e9210538 100644 --- a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java +++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java @@ -26,6 +26,7 @@ import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; +import java.net.MalformedURLException; import java.net.URL; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -61,7 +62,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.util.Records; -import org.apache.tez.common.TezClassLoader; import org.apache.tez.common.security.JobTokenIdentifier; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.common.security.TokenCache; @@ -83,7 +83,7 @@ public class TestTezClientUtils { private static String TEST_ROOT_DIR = "target" + Path.SEPARATOR + TestTezClientUtils.class.getName() + "-tmpDir"; - private static final File STAGING_DIR = new File(System.getProperty("test.build.data"), + private static final File STAGING_DIR = new File(System.getProperty("test.build.data", "target"), TestTezClientUtils.class.getName()).getAbsoluteFile(); /** * @@ -132,12 +132,29 @@ public void validateSetTezJarLocalResourcesDefinedNonExistingDirectory() throws TezClientUtils.setupTezJarsLocalResources(conf, credentials, resources); } - /** - * - */ + private static List getDirAndFileURL() throws MalformedURLException { + String[] classpaths = System.getProperty("java.class.path") + .split(System.getProperty("path.separator")); + List urls = new ArrayList<>(2); + File lastFile = null; + // Add one file and one directory. + for (String path : classpaths) { + URL url = new URL("file://" + path); + File file = FileUtils.toFile(url); + if (lastFile == null) { + lastFile = file; + urls.add(url); + } else if (lastFile.isDirectory() != file.isDirectory()) { + urls.add(url); + break; + } + } + return urls; + } + @Test (timeout=20000) public void validateSetTezJarLocalResourcesDefinedExistingDirectory() throws Exception { - URL[] cp = TezClassLoader.getInstance().getURLs(); + List cp = getDirAndFileURL(); StringBuffer buffer = new StringBuffer(); for (URL url : cp) { buffer.append(url.toExternalForm()); @@ -151,22 +168,27 @@ public void validateSetTezJarLocalResourcesDefinedExistingDirectory() throws Exc localizedMap); Assert.assertFalse(usingArchive); Set resourceNames = localizedMap.keySet(); + boolean assertedDir = false; + boolean assertedFile = false; for (URL url : cp) { File file = FileUtils.toFile(url); - if (file.isDirectory()){ + if (file.isDirectory()) { String[] firList = file.list(); for (String fileNme : firList) { File innerFile = new File(file, fileNme); if (!innerFile.isDirectory()){ assertTrue(resourceNames.contains(innerFile.getName())); + assertedDir = true; } // not supporting deep hierarchies } - } - else { + } else { assertTrue(resourceNames.contains(file.getName())); + assertedFile = true; } } + assertTrue(assertedDir); + assertTrue(assertedFile); } /** @@ -175,7 +197,7 @@ public void validateSetTezJarLocalResourcesDefinedExistingDirectory() throws Exc */ @Test (timeout=5000) public void validateSetTezJarLocalResourcesDefinedExistingDirectoryIgnored() throws Exception { - URL[] cp = TezClassLoader.getInstance().getURLs(); + List cp = getDirAndFileURL(); StringBuffer buffer = new StringBuffer(); for (URL url : cp) { buffer.append(url.toExternalForm()); @@ -196,7 +218,7 @@ public void validateSetTezJarLocalResourcesDefinedExistingDirectoryIgnored() thr */ @Test (timeout=20000) public void validateSetTezJarLocalResourcesDefinedExistingDirectoryIgnoredSetToFalse() throws Exception { - URL[] cp = TezClassLoader.getInstance().getURLs(); + List cp = getDirAndFileURL(); StringBuffer buffer = new StringBuffer(); for (URL url : cp) { buffer.append(url.toExternalForm()); diff --git a/tez-api/src/test/java/org/apache/tez/common/TestReflectionUtils.java b/tez-api/src/test/java/org/apache/tez/common/TestReflectionUtils.java index 2fbd35cb3d..ed3814d000 100644 --- a/tez-api/src/test/java/org/apache/tez/common/TestReflectionUtils.java +++ b/tez-api/src/test/java/org/apache/tez/common/TestReflectionUtils.java @@ -58,7 +58,7 @@ public void testConstructorWithParameters() throws TezReflectionException @Test(timeout = 5000) public void testAddResourceToClasspath() throws IOException, TezException { - + TezClassLoader.setupTezClassLoader(); String rsrcName = "dummyfile.xml"; FileSystem localFs = FileSystem.getLocal(new Configuration()); Path p = new Path(rsrcName); @@ -78,7 +78,7 @@ public void testAddResourceToClasspath() throws IOException, TezException { urlForm = urlForm.substring(0, urlForm.lastIndexOf('/') + 1); URL url = new URL(urlForm); - ReflectionUtils.addResourcesToClasspath(Collections.singletonList(url)); + ReflectionUtils.addResourcesToSystemClassLoader(Collections.singletonList(url)); loadedUrl = Thread.currentThread().getContextClassLoader().getResource(rsrcName); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index 7e5a7a9d67..fcfb883953 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -2328,6 +2328,8 @@ public boolean isSession() { public static void main(String[] args) { try { + // Install the tez class loader, which can be used add new resources + TezClassLoader.setupTezClassLoader(); Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler()); final String pid = System.getenv().get("JVM_PID"); String containerIdStr = From 8f7209fcbb9f5a74dcea9e83cb53279a58e57aff Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Thu, 13 Aug 2020 14:30:29 +0530 Subject: [PATCH 258/512] TEZ-4207: Provide approximate number of input records to be processed in UnorderedKVInput (Rajesh Balamohan, reviewed by Ashutosh Chauhan) --- .../apache/tez/common/counters/TaskCounter.java | 9 ++++++++- .../impl/ShuffleInputEventHandlerImpl.java | 4 ++++ .../common/shuffle/impl/ShuffleManager.java | 15 ++++++++++++++- .../writers/UnorderedPartitionedKVWriter.java | 3 +++ .../src/main/proto/ShufflePayloads.proto | 1 + .../writers/TestUnorderedPartitionedKVWriter.java | 11 +++++++++++ 6 files changed, 41 insertions(+), 2 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java index e604f37e63..2ee82a3965 100644 --- a/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java +++ b/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java @@ -98,7 +98,14 @@ public enum TaskCounter { * Used by MROutput, OnFileSortedOutput, and OnFileUnorderedKVOutput */ OUTPUT_RECORDS, - + + /** + * Approximate number of input records that should be processed as the event keeps arriving from + * inputs. + * //TODO: As of now supporting broadcast data only. + */ + APPROXIMATE_INPUT_RECORDS, + /** * Represent the number of large records in the output - typically, records which are * spilled directly diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java index 7ad1389170..e924876628 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java @@ -190,7 +190,11 @@ private void processDataMovementEvent(DataMovementEvent dme, DataMovementEventPa numDmeEventsNoData.getAndIncrement(); shuffleManager.addCompletedInputWithNoData(srcAttemptIdentifier.expand(0)); return; + } else { + shuffleManager.updateApproximateInputRecords(shufflePayload.getNumRecord()); } + } else { + shuffleManager.updateApproximateInputRecords(shufflePayload.getNumRecord()); } CompositeInputAttemptIdentifier srcAttemptIdentifier = constructInputAttemptIdentifier(dme.getTargetIndex(), 1, dme.getVersion(), diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java index 8ae4f60667..742fc18632 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java @@ -189,6 +189,9 @@ public class ShuffleManager implements FetcherCallback { private final AtomicBoolean isShutdown = new AtomicBoolean(false); + private long inputRecordsFromEvents; + private long eventsReceived; + private final TezCounter approximateInputRecords; private final TezCounter shuffledInputsCounter; private final TezCounter failedShufflesCounter; private final TezCounter bytesShuffledCounter; @@ -222,7 +225,8 @@ public ShuffleManager(InputContext inputContext, Configuration conf, int numInpu CompressionCodec codec, FetchedInputAllocator inputAllocator) throws IOException { this.inputContext = inputContext; this.numInputs = numInputs; - + + this.approximateInputRecords = inputContext.getCounters().findCounter(TaskCounter.APPROXIMATE_INPUT_RECORDS); this.shuffledInputsCounter = inputContext.getCounters().findCounter(TaskCounter.NUM_SHUFFLED_INPUTS); this.failedShufflesCounter = inputContext.getCounters().findCounter(TaskCounter.NUM_FAILED_SHUFFLE_INPUTS); this.bytesShuffledCounter = inputContext.getCounters().findCounter(TaskCounter.SHUFFLE_BYTES); @@ -335,6 +339,15 @@ public ShuffleManager(InputContext inputContext, Configuration conf, int numInpu + httpConnectionParams.toString() + ", maxTaskOutputAtOnce=" + maxTaskOutputAtOnce); } + public void updateApproximateInputRecords(int delta) { + if (delta <= 0) { + return; + } + inputRecordsFromEvents += delta; + eventsReceived++; + approximateInputRecords.setValue((inputRecordsFromEvents / eventsReceived) * numInputs); + } + public void run() throws IOException { Preconditions.checkState(inputManager != null, "InputManager must be configured"); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java index ffce5c62ca..d9467af2bc 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java @@ -902,6 +902,9 @@ private Event generateDMEvent(boolean addSpillDetails, int spillId, outputContext.notifyProgress(); DataMovementEventPayloadProto.Builder payloadBuilder = DataMovementEventPayloadProto .newBuilder(); + if (numPartitions == 1) { + payloadBuilder.setNumRecord((int) outputRecordsCounter.getValue()); + } String host = getHost(); if (emptyPartitions.cardinality() != 0) { diff --git a/tez-runtime-library/src/main/proto/ShufflePayloads.proto b/tez-runtime-library/src/main/proto/ShufflePayloads.proto index 0d13710d73..5cbd18a9b0 100644 --- a/tez-runtime-library/src/main/proto/ShufflePayloads.proto +++ b/tez-runtime-library/src/main/proto/ShufflePayloads.proto @@ -30,6 +30,7 @@ message DataMovementEventPayloadProto { optional bool pipelined = 7; // Related to pipelined shuffle optional bool last_event = 8; // Related to pipelined shuffle optional int32 spill_id = 9; // Related to pipelined shuffle. + optional int32 num_record = 10; } message DataProto { diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java index f4e99ec0ed..c39bf3f707 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java @@ -59,6 +59,7 @@ import org.apache.tez.runtime.api.events.VertexManagerEvent; import org.apache.tez.runtime.library.common.Constants; import org.apache.tez.runtime.library.common.writers.UnorderedPartitionedKVWriter.SpillInfo; +import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads; import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.VertexManagerEventPayloadProto; import org.apache.tez.runtime.library.utils.DATA_RANGE_IN_MB; import org.roaringbitmap.RoaringBitmap; @@ -1171,6 +1172,16 @@ private void baseTest(int numRecords, int numPartitions, Set skippedPar if (numPartitions == 1) { assertEquals(true, kvWriter.skipBuffers); + + // VM & DME events + assertEquals(2, events.size()); + Event event1 = events.get(1); + assertTrue(event1 instanceof CompositeDataMovementEvent); + CompositeDataMovementEvent dme = (CompositeDataMovementEvent) event1; + ByteBuffer bb = dme.getUserPayload(); + ShuffleUserPayloads.DataMovementEventPayloadProto shufflePayload = + ShuffleUserPayloads.DataMovementEventPayloadProto.parseFrom(ByteString.copyFrom(bb)); + assertEquals(kvWriter.outputRecordsCounter.getValue(), shufflePayload.getNumRecord()); } int recordsPerBuffer = sizePerBuffer / sizePerRecordWithOverhead; From ef1d69aaf48f068aaf7a2c6b4eb72d320d5c2d1a Mon Sep 17 00:00:00 2001 From: Panagiotis Garefalakis Date: Mon, 24 Aug 2020 14:37:53 +0200 Subject: [PATCH 259/512] TEZ-4213: Bound appContext executor capacity using a configurable property (Panagiotis Garefalakis reviewed by Ashutosh Chauhan, Mustafa Iman, Attila Magyar) Signed-off-by: Laszlo Bodor --- .../apache/tez/dag/api/TezConfiguration.java | 10 ++ .../org/apache/tez/dag/app/AppContext.java | 6 +- .../org/apache/tez/dag/app/DAGAppMaster.java | 22 ++++- .../app/dag/RootInputInitializerManager.java | 4 +- .../apache/tez/dag/app/TestDAGAppMaster.java | 4 +- .../dag/TestRootInputInitializerManager.java | 94 ++++++++++++++++++- 6 files changed, 127 insertions(+), 13 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 7dc412b62b..d5e5e73d45 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -847,6 +847,16 @@ public TezConfiguration(boolean loadDefaults) { public static final int TEZ_AM_DAG_CLEANUP_THREAD_COUNT_LIMIT_DEFAULT = 10; + /** + * Int value. Upper limit on the number of threads used by app context (vertex management and input init events). + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty(type="integer") + public static final String TEZ_AM_DAG_APPCONTEXT_THREAD_COUNT_LIMIT = + TEZ_AM_PREFIX + "dag.appcontext.thread-count-limit"; + + public static final int TEZ_AM_DAG_APPCONTEXT_THREAD_COUNT_LIMIT_DEFAULT = 10; + /** Int value. The amount of memory in MB to be used by tasks. This applies to all tasks across * all vertices. Setting it to the same value for all tasks is helpful for container reuse and * thus good for performance typically. */ diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java index 4eb2ae2842..fc4ddcfcdc 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java @@ -20,6 +20,7 @@ import java.util.Map; import java.util.Set; +import java.util.concurrent.ThreadPoolExecutor; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; @@ -72,7 +73,10 @@ public interface AppContext { String getUser(); DAG getCurrentDAG(); - + + // For testing only! + ThreadPoolExecutor getThreadPool(); + ListeningExecutorService getExecService(); void setDAG(DAG dag); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index fcfb883953..5400668860 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -47,9 +47,10 @@ import java.util.Set; import java.util.Timer; import java.util.TimerTask; -import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -302,7 +303,7 @@ public class DAGAppMaster extends AbstractService { private Path tezSystemStagingDir; private FileSystem recoveryFS; - private ExecutorService rawExecutor; + private ThreadPoolExecutor rawExecutor; private ListeningExecutorService execService; // TODO May not need to be a bidi map @@ -621,8 +622,13 @@ public synchronized void serviceInit(final Configuration conf) throws Exception } } - rawExecutor = Executors.newCachedThreadPool(new ThreadFactoryBuilder().setDaemon(true) - .setNameFormat("App Shared Pool - " + "#%d").build()); + int threadCount = conf.getInt(TezConfiguration.TEZ_AM_DAG_APPCONTEXT_THREAD_COUNT_LIMIT, + TezConfiguration.TEZ_AM_DAG_APPCONTEXT_THREAD_COUNT_LIMIT_DEFAULT); + // NOTE: LinkedBlockingQueue does not have a capacity Limit and can thus + // occupy large memory chunks when numerous Runables are pending for execution + rawExecutor = new ThreadPoolExecutor(threadCount, threadCount, + 60L, TimeUnit.SECONDS, new LinkedBlockingQueue(), + new ThreadFactoryBuilder().setDaemon(true).setNameFormat("App Shared Pool - " + "#%d").build()); execService = MoreExecutors.listeningDecorator(rawExecutor); initServices(conf); @@ -1503,6 +1509,14 @@ public DAG getCurrentDAG() { return dag; } + @Override + // For Testing only! + public ThreadPoolExecutor getThreadPool() { + synchronized (DAGAppMaster.this) { + return rawExecutor; + } + } + @Override public ListeningExecutorService getExecService() { return execService; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java index 9194c1dd14..bd4bcd89a8 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java @@ -121,7 +121,7 @@ public void runInputInitializers(List createInitializerWrappers( + protected List createInitializerWrappers( List> inputs) { String current = null; final List result = Collections.synchronizedList(new ArrayList<>()); @@ -161,7 +161,7 @@ void failVertex(Throwable t, String inputName) { * @param pendingInitializerEvents * @param result */ - private void createAndStartInitializing(List pendingInitializerEvents, List result) { + protected void createAndStartInitializing(List pendingInitializerEvents, List result) { handleInitializerEvents(pendingInitializerEvents); pendingInitializerEvents.clear(); for (InitializerWrapper inputWrapper : result) { diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java index 4adf310e82..d8167dbcc4 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java @@ -627,7 +627,7 @@ public TestTokenIdentifier createIdentifier() { } } - private static class DAGAppMasterForTest extends DAGAppMaster { + public static class DAGAppMasterForTest extends DAGAppMaster { private DAGAppMasterShutdownHandler mockShutdown; private TaskSchedulerManager mockScheduler = mock(TaskSchedulerManager.class); @@ -638,7 +638,7 @@ public DAGAppMasterForTest(ApplicationAttemptId attemptId, boolean isSession) { new TezDagVersionInfo().getVersion(), createCredentials(), "jobname", null); } - private static Credentials createCredentials() { + public static Credentials createCredentials() { Credentials creds = new Credentials(); JobTokenSecretManager jtsm = new JobTokenSecretManager(); JobTokenIdentifier jtid = new JobTokenIdentifier(new Text()); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java index 01cc37f18c..000f077278 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java @@ -14,7 +14,9 @@ package org.apache.tez.dag.app.dag; +import static org.apache.tez.dag.app.TestDAGAppMaster.DAGAppMasterForTest.createCredentials; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; import static org.mockito.Mockito.RETURNS_DEEP_STUBS; import static org.mockito.Mockito.doReturn; @@ -25,26 +27,43 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import java.io.File; import java.io.IOException; import java.util.Collections; +import java.util.LinkedList; import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.ThreadPoolExecutor; import com.google.common.collect.Lists; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.util.SystemClock; +import org.apache.tez.client.TezApiVersionInfo; +import org.apache.tez.common.TezCommonUtils; import org.apache.tez.dag.api.InputDescriptor; import org.apache.tez.dag.api.InputInitializerDescriptor; import org.apache.tez.dag.api.RootInputLeafOutput; +import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.api.TezConstants; import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.api.oldrecords.TaskState; +import org.apache.tez.dag.api.records.DAGProtos; import org.apache.tez.dag.app.AppContext; +import org.apache.tez.dag.app.DAGAppMaster; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.dag.records.TezTaskID; @@ -62,20 +81,26 @@ import org.mockito.ArgumentCaptor; public class TestRootInputInitializerManager { - ListeningExecutorService execService; + + private static final File TEST_DIR = new File(System.getProperty("test.build.data"), + TestRootInputInitializerManager.class.getName()).getAbsoluteFile(); + private static ListeningExecutorService execService; @Before - public void setUp() throws Exception { + public void setUp() { ExecutorService rawExecutor = Executors.newCachedThreadPool(new ThreadFactoryBuilder() .setDaemon(true).setNameFormat("Test App Shared Pool - " + "#%d").build()); execService = MoreExecutors.listeningDecorator(rawExecutor); + FileUtil.fullyDelete(TEST_DIR); + TEST_DIR.mkdirs(); } @After - public void tearDown() throws Exception { + public void tearDown() { if (execService != null) { execService.shutdownNow(); } + FileUtil.fullyDelete(TEST_DIR); } // Simple testing. No events if task doesn't succeed. @@ -251,7 +276,68 @@ public void testCorrectUgiUsage() throws TezException, InterruptedException { InputInitializerForUgiTest.awaitInitialize(); assertEquals(dagUgi, InputInitializerForUgiTest.ctorUgi); - assertEquals(dagUgi, InputInitializerForUgiTest.initializeUgi); + assertEquals(dagUgi.getRealUser(), InputInitializerForUgiTest.initializeUgi.getRealUser()); + } + + @Test (timeout = 10000) + public synchronized void testParallelInputInitialization() throws InterruptedException, IOException { + // Create Local DAGAppMaster with default conf + Configuration conf = new Configuration(true); + conf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); + conf.set(TezConfiguration.TEZ_AM_STAGING_DIR, TEST_DIR.toString()); + + FileSystem fs = FileSystem.getLocal(conf); + FSDataOutputStream sessionJarsPBOutStream = + TezCommonUtils.createFileForAM(fs, new Path(TEST_DIR.toString(), + TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME)); + DAGProtos.PlanLocalResourcesProto.getDefaultInstance() + .writeDelimitedTo(sessionJarsPBOutStream); + sessionJarsPBOutStream.close(); + + ApplicationId appId = ApplicationId.newInstance(1, 1); + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1); + + DAGAppMaster am = new DAGAppMaster(attemptId, + ContainerId.newContainerId(attemptId, 1), + "127.0.0.1", 0, 0, new SystemClock(), 1, true, + TEST_DIR.toString(), new String[] {TEST_DIR.toString()}, + new String[] {TEST_DIR.toString()}, + new TezApiVersionInfo().getVersion(), createCredentials(), + "someuser", null); + am.init(conf); + + Vertex vertex = mock(Vertex.class); + doReturn(mock(TezVertexID.class)).when(vertex).getVertexId(); + UserGroupInformation dagUgi = UserGroupInformation.createRemoteUser("fakeuser"); + StateChangeNotifier stateChangeNotifier = mock(StateChangeNotifier.class); + RootInputInitializerManager rootInputInitializerManager = + new RootInputInitializerManager(vertex, am.getContext(), dagUgi, stateChangeNotifier); + + List> inlist = new LinkedList(); + // Make sure we dont have any OOM issue by controlling the capacity of the thread pool + // and also block producer (createInitializerWrapper when resources are saturated) + InputDescriptor id = mock(InputDescriptor.class); + InputInitializerDescriptor iid = InputInitializerDescriptor.create(InputInitializerForUgiTest.class.getName()); + for (int i=0; i < 10000; i++) { + RootInputLeafOutput rootInput = + new RootInputLeafOutput<>("InputName"+i, id, iid); + inlist.add(rootInput); + } + + List initWrappers = + rootInputInitializerManager.createInitializerWrappers(inlist); + + int max_thread_size = conf.getInt(TezConfiguration.TEZ_AM_DAG_APPCONTEXT_THREAD_COUNT_LIMIT, + TezConfiguration.TEZ_AM_DAG_APPCONTEXT_THREAD_COUNT_LIMIT_DEFAULT); + ThreadPoolExecutor amThreadPool = am.getContext().getThreadPool(); + + rootInputInitializerManager.executor.submit(() + -> rootInputInitializerManager.createAndStartInitializing(Collections.emptyList(), initWrappers)); + + while (am.getContext().getThreadPool().getQueue().size() > 0) { + assertTrue(amThreadPool.getPoolSize() <= max_thread_size); + Thread.sleep(100); + } } public static class InputInitializerForUgiTest extends InputInitializer { From 3986f5bc2c2e34ad160f77afd7478b26935fffa5 Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Mon, 24 Aug 2020 14:11:52 -0700 Subject: [PATCH 260/512] TEZ-4216 : RLE check in MergeManager::finalMerge could be disabled (Rajesh Balamohan via Ashutosh Chauhan) Signed-off-by: Ashutosh Chauhan --- .../shuffle/orderedgrouped/MergeManager.java | 8 ++++---- .../library/common/sort/impl/TezMerger.java | 15 +++++++++++++++ 2 files changed, 19 insertions(+), 4 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java index 70f9e55f70..dd23b3b617 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java @@ -1316,10 +1316,10 @@ public int compare(Segment o1, Segment o2) { LOG.info(finalMergeLog.toString()); } // This is doing nothing but creating an iterator over the segments. - return TezMerger.merge(job, fs, keyClass, valueClass, - finalSegments, finalSegments.size(), tmpDir, - comparator, progressable, spilledRecordsCounter, null, - additionalBytesRead, null); + return TezMerger.merge(job, fs, keyClass, valueClass, codec, + finalSegments, finalSegments.size(), tmpDir, + comparator, progressable, spilledRecordsCounter, null, + additionalBytesRead, null); } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java index 3e04e748db..726810bda8 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java @@ -128,6 +128,21 @@ TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, bytesReadCounter, mergePhase); } + public static TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, + Class keyClass, Class valueClass, CompressionCodec codec, + List segments, + int mergeFactor, Path tmpDir, + RawComparator comparator, Progressable reporter, + TezCounter readsCounter, + TezCounter writesCounter, + TezCounter bytesReadCounter, + Progress mergePhase) throws IOException, InterruptedException { + return new MergeQueue(conf, fs, segments, comparator, reporter, + false, codec, false, false) + .merge(keyClass, valueClass, mergeFactor, tmpDir, + readsCounter, writesCounter, bytesReadCounter, mergePhase); + } + public static TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, Class keyClass, Class valueClass, From 99895f9808170ce64fd1e7c6dfb2e932f4578489 Mon Sep 17 00:00:00 2001 From: Laszlo Bodor Date: Tue, 25 Aug 2020 13:56:57 +0200 Subject: [PATCH 261/512] TEZ-4213: Bound appContext executor capacity using a configurable property (Panagiotis Garefalakis reviewed by Ashutosh Chauhan, Mustafa Iman, Attila Magyar) - addendum checkstyle --- .../tez/dag/app/dag/TestRootInputInitializerManager.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java index 000f077278..a197e54e6e 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java @@ -327,15 +327,15 @@ public synchronized void testParallelInputInitialization() throws InterruptedExc List initWrappers = rootInputInitializerManager.createInitializerWrappers(inlist); - int max_thread_size = conf.getInt(TezConfiguration.TEZ_AM_DAG_APPCONTEXT_THREAD_COUNT_LIMIT, + int maxThreadSize = conf.getInt(TezConfiguration.TEZ_AM_DAG_APPCONTEXT_THREAD_COUNT_LIMIT, TezConfiguration.TEZ_AM_DAG_APPCONTEXT_THREAD_COUNT_LIMIT_DEFAULT); ThreadPoolExecutor amThreadPool = am.getContext().getThreadPool(); - rootInputInitializerManager.executor.submit(() - -> rootInputInitializerManager.createAndStartInitializing(Collections.emptyList(), initWrappers)); + rootInputInitializerManager.executor.submit(() -> rootInputInitializerManager + .createAndStartInitializing(Collections.emptyList(), initWrappers)); while (am.getContext().getThreadPool().getQueue().size() > 0) { - assertTrue(amThreadPool.getPoolSize() <= max_thread_size); + assertTrue(amThreadPool.getPoolSize() <= maxThreadSize); Thread.sleep(100); } } From 69e73517b4bbb6fea5d31892f020a1bb20a2adc3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Tue, 25 Aug 2020 18:56:59 +0200 Subject: [PATCH 262/512] =?UTF-8?q?TEZ-4224:=20Add=20Laszlo=20Bodor's=20pu?= =?UTF-8?q?blic=20key=20to=20KEYS=20(L=C3=A1szl=C3=B3=20Bodor=20reviewed?= =?UTF-8?q?=20by=20Jonathan=20Turner=20Eagles)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- KEYS | 59 +++++++++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 59 insertions(+) diff --git a/KEYS b/KEYS index 37818138ab..dab4415cec 100644 --- a/KEYS +++ b/KEYS @@ -592,3 +592,62 @@ aU33ZET6JYY37Zvn0KiOWLP/M08a8g/JOdTP1308wOoL4t+L9tgfy/Pwcv9EtQrj RH25lXA/2HxJ4hqptH/ClwVJJeFs65M= =p0Tr -----END PGP PUBLIC KEY BLOCK----- +pub rsa4096 2020-08-25 [SC] + D7DC 0A99 CECC 840B FC0D 2932 4ECA 5CA5 E303 605A +uid [ultimate] Laszlo Bodor (CODE SIGNING KEY) +sig 3 4ECA5CA5E303605A 2020-08-25 Laszlo Bodor (CODE SIGNING KEY) +sub rsa4096 2020-08-25 [E] +sig 4ECA5CA5E303605A 2020-08-25 Laszlo Bodor (CODE SIGNING KEY) + +-----BEGIN PGP PUBLIC KEY BLOCK----- + +mQINBF9E69kBEAC0MGD1eHddeC5etJ+SQORt7mBIo4CBStS8gvjZs13CFjKTLc+O +UfOTRIZOu1qa73yhOSWsf/iX3DR4Q/F8FL5nbvQkrcDo06PRho7T1BHo5MVipcyt +PAsdiUDBsq9EhF4p2aWOV1BkHz9Pl7AJWTh3Y/HZUhcQogQEej+tSTR3M/+I2hvN +7tTF0/1695Z1KxjTUd0qPesoYf269fD8S5WeRY1HaHWX//+j3NAJ7ss2xNM8Jg1F +NuAoSac5hIYcuXpF1TpfNg+uEAj9SIorRllT0EixrDR8Nn0HUKLgRXmSN/nA5EbX +vHYpWRu4Dm8grNxrGG6TUUJEUFGnwIEb7BS+kVIIt7TMOmELPGUmzY9BQoHyPY8h +gjcT3i7AFRVJl/eiNNQIKM9p8VUGwhH5PApDR1e/VSJcj1fNnNj/QCQ7OT95yYnB +2ZfiKHUCb5iFhL0GCzcEfaNDRuIuAET3XRsqfc9TRVAh/rb4qmvFQdChRF9aQpMm +o6mqW+NcJ9pkDrhyfeRwt+59PDHfPo2P+LQ+yxD8pHrJUlzUy1sI/ACJfsBr2v1S +ad+jQnzcUQ7S+7P8BXfnd9OgSxcapNhsHM6YWosGZAhkanf3qp+ldJxLHIQyl7dM +t/OZutoofi23UA5tbZrNbLzYGv3G+u055/jHVk/VqmsXBQn5aWtqiKLgKQARAQAB +tDtMYXN6bG8gQm9kb3IgKENPREUgU0lHTklORyBLRVkpIDxib2Rvcmxhc3psbzAy +MDJAZ21haWwuY29tPokCTgQTAQoAOBYhBNfcCpnOzIQL/A0pMk7KXKXjA2BaBQJf +ROvZAhsDBQsJCAcCBhUKCQgLAgQWAgMBAh4BAheAAAoJEE7KXKXjA2BaDREQAKb2 +5cZZ8TsFMaO0X9dZ5PHgN2uuQ4xjRnCL28kNm5WVcj5/4hIj2iIIuLqlDMUQto/a +wfzkwWvjkkIG8bKAfIotqCrdj9f5kwogF4ivmvmKL0EY5mXKF8i/IQ/fHKj8Jhdf +LpH96TOvlLW5NjNgLmK+ymhqUXKg1VswTcqd4sf5aZ35iN+JOt9lPhgu7sm0tlib +0AD4T23Vm+6fGLImhxzGrCRNZH+PhX+peRHVBQZ6RrY12WYiEaF+KUPwm2LX7Kyb +FAjncMZ9ovFQRqmwAR2Ku1yyu5t7TUZ1PqTPAopM+3uhQ1rJIEOJVOzZYFg1Tl73 +ac6U9CcqIatOgx3gsh5g2IOfPYV24hz/xcXlsaAXvoS1vWqf4+Yv5jXKYwEwnjvX +TTFyHKtxiphg6LwOratCI1qfdj6OiZ5BTJF3NYGgc5kwbAzUjk2d4IbpqyZTJ5op +UwnHMNNg72lAAXNgX0uTIWkwlQ6Q8KzV6NMmr1NQwQ0JJeOk1LiaBRgFG2+XNoXE +wC2gSJRy1FLurpb/cCO6H6BtXRh3R45JBdpsqeQkbMSWHbVDZfRUMmNqvyCZytNO +sEFPTR2T1dAi2XgXRFcNobnDdfJ+5YOimvDh24aOLVzP2jzMwnirfFeTCIxpTeVM +YvuDnC3NA4y+p0R7PDuvCVMXe/z75iz8w3VIr34AuQINBF9E69kBEACoXsSxE5wH +bWs9DzUIY9OluV9+2e9LtZkYGQkCJdHu3aXjQBZkPoWwM0ZtXqNcJ+vLMH0Qfnb1 +W+au2TBcp8k5LDfSLgv/pxOBiJVYcpkaSOABuewTvuro+qhppNzQe6fM37oYa779 +i/vBBw/at7G9I5NUcTBRYLr8zsAScvMVO9j06YANcf7lr1bbIXeIgLbUlaIC/FoZ +nOw0awr0yJvZjROJVvE3mMhXDfluLZvP7TVkbEky/6NfNUDC2nXtNEWmIys73JJi +WjD7gSdD3PiELMWHmI4VQNK2RneTYcY9n2h4MXVjkcAKTOlndbadsfqVxz5wweJV +FgYuHKr9jC3Jz5gRD9RbPAMuND3vntfavXYDoydaZfRCyDQsroBupVZWt6rkZtB2 +ElU/o1lpkdBkLVTbNw2wxcV8MYFT6XZjsW60ZkURk92vjRKAOMSDCLWmQhQkpG59 +Z50Wb+dXGbpBvdQJdh9SG7sXrl4pl7MXFafGzWZehzJOrcgaEL38lM70pOBfAbLA +mbP8YoW8rAdHnkkNDs52bu4X7qE20zUNqmFF7okw1q+shcXhIKfhMrJuNdMBITQn +UyRZxrkbiC+E2EjBUrtkdWA75T7EQPcpuF3yVdV1EW5hFLLd2GVrhDLuf2voAxe+ +f5gbCyBntMPYwxpukMoLOA6a8n+jyYxHvwARAQABiQI2BBgBCgAgFiEE19wKmc7M +hAv8DSkyTspcpeMDYFoFAl9E69kCGwwACgkQTspcpeMDYFqJsw//cuUIsEuIB0xI +jFQ3vnm857zrTNq+E2IHgZCUw9oY/l5ByJeV3TX+eFL/IzAfiUq9/fGfDl9OoruC +zMPg9bUtTM9BsgjH/J+fXlV/JEQi4/+zp085VmeSaYuQeJgO920y4CgCy3n0NUJb +gmIqMYlCN5glX4+2S/jNqR0xBKovZua8S6ZmK2yxqXo5q9yNxtHsIVuNvR7FIx/V +Epb4YMco/Mt4mu2CbPsED/OYkDtu5gp/DQuDPT3cKk8vgcci6F/92Csc6YYBTmDW +qY3otbU6JBsH4mW21OB8Dp5/K7+so5XrCa0DKC84IT4qjKV17x3k+PQqISs3WkSy +9VMBpaoJQroyTZJr6F5Tdq2wJRvXF+z06aJXoVT2/m3GKbnpyUtkkUlVK2UfD1gP +7iiqEP/f+GU1rLFzyy0SGbZmRopNmu2ytY+Hk1dFYOQJW/1BLlsGLpNfHGjiOMvd +Z0LWTBuL54lt+nAm0DqFcb5vmV/fkXdQI25V/ytQdPPnfz0k+lOyXbyxAtv4gASZ +4CLRmiNwOXSdfYmdyfD8n5jawGddtN2qXgDWCPhv8HPVlDqt0trTSCOgQQ8xwJmV +lB3/5pgPNByiaLyoB9kLomrsimlFSF5IY0R6uC7z1Aukx0NjfrJaTLYt8oUx/8bl +jAq7t18gzFWofePw7o6gv35FdrOe+PQ= +=hVV4 +-----END PGP PUBLIC KEY BLOCK----- From 0e17787c3547a0480a7ffb2069a4c62a72783cb9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Wed, 26 Aug 2020 19:58:35 +0200 Subject: [PATCH 263/512] =?UTF-8?q?TEZ-4175:=20Consider=20removing=20YarnC?= =?UTF-8?q?onfiguration=20where=20it's=20possible=20(L=C3=A1szl=C3=B3=20Bo?= =?UTF-8?q?dor=20reviewed=20by=20Rajesh=20Balamohan,=20Mustafa=20Iman,=20A?= =?UTF-8?q?shutosh=20Chauhan)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../apache/tez/client/AMConfiguration.java | 10 -------- .../apache/tez/client/FrameworkClient.java | 5 +--- .../java/org/apache/tez/client/TezClient.java | 24 +++++++------------ .../org/apache/tez/client/TezYarnClient.java | 4 ++-- .../tez/dag/api/client/DAGClientImpl.java | 6 ++--- .../org/apache/tez/client/TestTezClient.java | 3 +-- .../tez/dag/api/client/rpc/TestDAGClient.java | 18 ++++---------- .../org/apache/tez/client/LocalClient.java | 4 +--- .../org/apache/tez/dag/app/DAGAppMaster.java | 4 +--- .../tez/dag/app/rm/TestContainerReuse.java | 23 +++++++++--------- .../tez/mapreduce/client/YARNRunner.java | 2 +- .../mapreduce/hadoop/TestMRInputHelpers.java | 3 +-- .../tez/test/FaultToleranceTestRunner.java | 9 ++++--- 13 files changed, 38 insertions(+), 77 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/AMConfiguration.java b/tez-api/src/main/java/org/apache/tez/client/AMConfiguration.java index 238b155010..1b61c45d05 100644 --- a/tez-api/src/main/java/org/apache/tez/client/AMConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/client/AMConfiguration.java @@ -23,7 +23,6 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.yarn.api.records.LocalResource; -import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.tez.dag.api.TezConfiguration; import com.google.common.collect.Maps; @@ -34,7 +33,6 @@ class AMConfiguration { private Map amLocalResources = Maps.newHashMap(); private TezConfiguration tezConf; private Credentials credentials; - private YarnConfiguration yarnConfig; private LocalResource binaryConfLRsrc; AMConfiguration(TezConfiguration tezConf, Map localResources, @@ -64,10 +62,6 @@ void setCredentials(Credentials credentials) { void setTezConfiguration(TezConfiguration tezConf) { this.tezConf = tezConf; } - - void setYarnConfiguration(YarnConfiguration yarnConf) { - this.yarnConfig = yarnConf; - } String getQueueName() { return this.tezConf.get(TezConfiguration.TEZ_QUEUE_NAME); @@ -81,10 +75,6 @@ TezConfiguration getTezConfiguration() { return tezConf; } - YarnConfiguration getYarnConfiguration() { - return yarnConfig; - } - Credentials getCredentials() { return credentials; } diff --git a/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java b/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java index b3e084c27c..7c60ec100d 100644 --- a/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java @@ -26,7 +26,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.client.api.YarnClientApplication; -import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.tez.common.ReflectionUtils; import org.apache.tez.dag.api.TezConfiguration; @@ -58,10 +57,8 @@ public static FrameworkClient createFrameworkClient(TezConfiguration tezConf) { * * @param tezConf the {@link org.apache.tez.dag.api.TezConfiguration} instance being used by the * cluster - * @param yarnConf the {@link org.apache.hadoop.yarn.conf.YarnConfiguration} instance being used - * by the cluster */ - public abstract void init(TezConfiguration tezConf, YarnConfiguration yarnConf); + public abstract void init(TezConfiguration tezConf); public abstract void start(); diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java index 4fb37dce9a..fbe35094f0 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java @@ -57,7 +57,6 @@ import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.LocalResourceType; import org.apache.hadoop.yarn.api.records.YarnApplicationState; -import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.util.Time; @@ -384,7 +383,6 @@ public synchronized void setHistoryLogLevel(HistoryLogLevel historyLogLevel) { * @throws IOException */ public synchronized void start() throws TezException, IOException { - amConfig.setYarnConfiguration(new YarnConfiguration(amConfig.getTezConfiguration())); startFrameworkClient(); setupJavaOptsChecker(); @@ -434,7 +432,6 @@ public synchronized TezClient getClient(String appIdStr) throws IOException, Tez */ public synchronized TezClient getClient(ApplicationId appId) throws TezException, IOException { sessionAppId = appId; - amConfig.setYarnConfiguration(new YarnConfiguration(amConfig.getTezConfiguration())); startFrameworkClient(); setupJavaOptsChecker(); @@ -466,7 +463,7 @@ public synchronized TezClient getClient(ApplicationId appId) throws TezException private void startFrameworkClient() { frameworkClient = createFrameworkClient(); - frameworkClient.init(amConfig.getTezConfiguration(), amConfig.getYarnConfiguration()); + frameworkClient.init(amConfig.getTezConfiguration()); frameworkClient.start(); } @@ -716,7 +713,6 @@ private DAGClient submitDAGSession(DAG dag) throws TezException, IOException { + ", dagName=" + dag.getName()); return new DAGClientImpl(sessionAppId, dagId, amConfig.getTezConfiguration(), - amConfig.getYarnConfiguration(), frameworkClient, getUgi()); } @@ -1068,7 +1064,7 @@ protected FrameworkClient createFrameworkClient() { protected DAGClientAMProtocolBlockingPB getAMProxy(ApplicationId appId) throws TezException, IOException { return TezClientUtils.getAMProxy( - frameworkClient, amConfig.getYarnConfiguration(), appId, getUgi()); + frameworkClient, amConfig.getTezConfiguration(), appId, getUgi()); } private DAGClientAMProtocolBlockingPB waitForProxy() @@ -1146,8 +1142,7 @@ DAGClient submitDAGApplication(ApplicationId appId, DAG dag) } // wait for dag in non-session mode to start running, so that we can start to getDAGStatus waitNonSessionTillReady(); - return getDAGClient(appId, amConfig.getTezConfiguration(), amConfig.getYarnConfiguration(), - frameworkClient, getUgi()); + return getDAGClient(appId, amConfig.getTezConfiguration(), frameworkClient, getUgi()); } private ApplicationId createApplication() throws TezException, IOException { @@ -1170,20 +1165,17 @@ private synchronized Map getTezJarResources(Credentials c } @Private - static DAGClient getDAGClient(ApplicationId appId, TezConfiguration tezConf, YarnConfiguration - yarnConf, FrameworkClient frameworkClient, UserGroupInformation ugi) - throws IOException, TezException { - return new DAGClientImpl(appId, getDefaultTezDAGID(appId), tezConf, - yarnConf, frameworkClient, ugi); + static DAGClient getDAGClient(ApplicationId appId, TezConfiguration tezConf, + FrameworkClient frameworkClient, UserGroupInformation ugi) throws IOException, TezException { + return new DAGClientImpl(appId, getDefaultTezDAGID(appId), tezConf, frameworkClient, ugi); } @Private // Used only for MapReduce compatibility code static DAGClient getDAGClient(ApplicationId appId, TezConfiguration tezConf, - FrameworkClient frameworkClient) - throws IOException, TezException { + FrameworkClient frameworkClient) throws IOException, TezException { UserGroupInformation ugi = UserGroupInformation.createRemoteUser(UserGroupInformation.getCurrentUser().getUserName()); - return getDAGClient(appId, tezConf, new YarnConfiguration(tezConf), frameworkClient, ugi); + return getDAGClient(appId, tezConf, frameworkClient, ugi); } // DO NOT CHANGE THIS. This code is replicated from TezDAGID.java diff --git a/tez-api/src/main/java/org/apache/tez/client/TezYarnClient.java b/tez-api/src/main/java/org/apache/tez/client/TezYarnClient.java index 2a0c79ab65..f89664d863 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezYarnClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezYarnClient.java @@ -45,8 +45,8 @@ protected TezYarnClient(YarnClient yarnClient) { } @Override - public void init(TezConfiguration tezConf, YarnConfiguration yarnConf) { - yarnClient.init(yarnConf); + public void init(TezConfiguration tezConf) { + yarnClient.init(new YarnConfiguration(tezConf)); } @Override diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java index 0b899fd27d..01a10b269c 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java @@ -40,7 +40,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.api.records.YarnApplicationState; -import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.tez.client.FrameworkClient; import org.apache.tez.common.counters.TezCounters; @@ -80,8 +79,7 @@ public class DAGClientImpl extends DAGClient { private boolean cleanupFrameworkClient; public DAGClientImpl(ApplicationId appId, String dagId, TezConfiguration conf, - YarnConfiguration yarnConf, @Nullable FrameworkClient frameworkClient, - UserGroupInformation ugi) { + @Nullable FrameworkClient frameworkClient, UserGroupInformation ugi) { this.appId = appId; this.dagId = dagId; this.conf = conf; @@ -89,7 +87,7 @@ public DAGClientImpl(ApplicationId appId, String dagId, TezConfiguration conf, this.frameworkClient = frameworkClient; } else { this.frameworkClient = FrameworkClient.createFrameworkClient(conf); - this.frameworkClient.init(conf, yarnConf); + this.frameworkClient.init(conf); this.frameworkClient.start(); cleanupFrameworkClient = true; } diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java index 91dce5ed5f..7316452df3 100644 --- a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java +++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java @@ -67,7 +67,6 @@ import org.apache.hadoop.yarn.api.records.URL; import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.client.api.YarnClient; -import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.tez.common.counters.LimitExceededException; @@ -889,7 +888,7 @@ public void testMissingYarnAppStatus() throws Exception { when(yarnClient.createApplication().getNewApplicationResponse().getApplicationId()).thenReturn(appId1); when(yarnClient.getApplicationReport(appId1)).thenReturn(mockReport); TezYarnClient tezClient = new TezYarnClient(yarnClient); - tezClient.init(new TezConfiguration(false), new YarnConfiguration()); + tezClient.init(new TezConfiguration(false)); try { tezClient.getApplicationReport(appId1); fail("getApplicationReport should have thrown"); diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java index 57087cb7b4..211baf257e 100644 --- a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java +++ b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java @@ -35,7 +35,6 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; -import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.tez.client.FrameworkClient; import org.apache.tez.dag.api.TezConfiguration; @@ -204,8 +203,7 @@ public void setUp() throws YarnException, IOException, TezException, ServiceExce .thenReturn(GetVertexStatusResponseProto.newBuilder().setVertexStatus(vertexStatusProtoWithCounters).build()); TezConfiguration tezConf = new TezConfiguration(); - YarnConfiguration yarnConf = new YarnConfiguration(tezConf); - dagClient = new DAGClientImpl(mockAppId, dagIdStr, tezConf, yarnConf, null, + dagClient = new DAGClientImpl(mockAppId, dagIdStr, tezConf, null, UserGroupInformation.getCurrentUser()); DAGClientRPCImpl realClient = (DAGClientRPCImpl)((DAGClientImpl)dagClient).getRealClient(); realClient.appReport = mockAppReport; @@ -342,17 +340,14 @@ public void testGetDagStatusWithTimeout() throws Exception { TezConfiguration tezConf = new TezConfiguration(); tezConf.setLong(TezConfiguration.TEZ_DAG_STATUS_POLLINTERVAL_MS, 800l); - YarnConfiguration yarnConf = new YarnConfiguration(tezConf); - DAGClientImplForTest dagClient = new DAGClientImplForTest(mockAppId, dagIdStr, tezConf, - yarnConf,null); + DAGClientImplForTest dagClient = new DAGClientImplForTest(mockAppId, dagIdStr, tezConf, null); DAGClientRPCImplForTest dagClientRpc = new DAGClientRPCImplForTest(mockAppId, dagIdStr, tezConf, null); dagClient.setRealClient(dagClientRpc); DAGStatus dagStatus; - // Fetch from RM. AM not up yet. dagClientRpc.setAMProxy(null); DAGStatus rmDagStatus = @@ -426,14 +421,12 @@ private static void testAtsEnabled(ApplicationId appId, String dagIdStr, boolean String loggingClass, boolean amHistoryLoggingEnabled, boolean dagHistoryLoggingEnabled) throws IOException { TezConfiguration tezConf = new TezConfiguration(); - YarnConfiguration yarnConf = new YarnConfiguration(tezConf); tezConf.set(TezConfiguration.TEZ_HISTORY_LOGGING_SERVICE_CLASS, loggingClass); tezConf.setBoolean(TezConfiguration.TEZ_AM_HISTORY_LOGGING_ENABLED, amHistoryLoggingEnabled); tezConf.setBoolean(TezConfiguration.TEZ_DAG_HISTORY_LOGGING_ENABLED, dagHistoryLoggingEnabled); - DAGClientImplForTest dagClient = new DAGClientImplForTest(appId, dagIdStr, tezConf, - yarnConf,null); + DAGClientImplForTest dagClient = new DAGClientImplForTest(appId, dagIdStr, tezConf, null); assertEquals(expected, dagClient.getIsATSEnabled()); } @@ -478,9 +471,8 @@ private static class DAGClientImplForTest extends DAGClientImpl { int numGetStatusViaRmInvocations = 0; public DAGClientImplForTest(ApplicationId appId, String dagId, TezConfiguration conf, - YarnConfiguration yarnConf, @Nullable FrameworkClient frameworkClient) throws IOException { - super(appId, dagId, conf, yarnConf, frameworkClient, UserGroupInformation.getCurrentUser()); + super(appId, dagId, conf, frameworkClient, UserGroupInformation.getCurrentUser()); } private void setRealClient(DAGClientRPCImplForTest dagClientRpcImplForTest) { @@ -518,7 +510,7 @@ private DAGClientAMProtocolBlockingPB createMockProxy(final DAGStatusStateProto ServiceException { DAGClientAMProtocolBlockingPB mock = mock(DAGClientAMProtocolBlockingPB.class); - doAnswer(new Answer() { + doAnswer(new Answer() { @Override public Object answer(InvocationOnMock invocation) throws Throwable { GetDAGStatusRequestProto request = (GetDAGStatusRequestProto) invocation.getArguments()[1]; diff --git a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java index 9006971a55..5a6bb9a22c 100644 --- a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java +++ b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java @@ -18,14 +18,12 @@ package org.apache.tez.client; -import java.io.File; import java.io.IOException; import java.net.InetAddress; import java.nio.ByteBuffer; import java.util.List; -import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; @@ -81,7 +79,7 @@ public LocalClient() { } @Override - public void init(TezConfiguration tezConf, YarnConfiguration yarnConf) { + public void init(TezConfiguration tezConf) { this.conf = tezConf; // Tez libs already in the client's classpath this.conf.setBoolean(TezConfiguration.TEZ_IGNORE_LIB_URIS, true); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index 5400668860..f4a8923d4a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -105,7 +105,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.LocalResource; -import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.event.Event; import org.apache.hadoop.yarn.event.EventHandler; @@ -2390,8 +2389,7 @@ public static void main(String[] args) { + ", localDirs=" + System.getenv(Environment.LOCAL_DIRS.name()) + ", logDirs=" + System.getenv(Environment.LOG_DIRS.name())); - // TODO Does this really need to be a YarnConfiguration ? - Configuration conf = new Configuration(new YarnConfiguration()); + Configuration conf = new Configuration(); ConfigurationProto confProto = TezUtilsInternal.readUserSpecifiedTezConfiguration(System.getenv(Environment.PWD.name())); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java index 4e29dd5917..92ea8a87f8 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java @@ -55,7 +55,6 @@ import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.client.api.AMRMClient; -import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.tez.common.MockDNSToSwitchMapping; import org.apache.tez.dag.api.InputDescriptor; import org.apache.tez.dag.api.OutputDescriptor; @@ -114,7 +113,7 @@ public static void setup() { public void testDelayedReuseContainerBecomesAvailable() throws IOException, InterruptedException, ExecutionException { LOG.info("Test testDelayedReuseContainerBecomesAvailable"); - Configuration conf = new Configuration(new YarnConfiguration()); + Configuration conf = new Configuration(); conf.setBoolean( TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, true); conf.setBoolean( @@ -249,7 +248,7 @@ public void testDelayedReuseContainerBecomesAvailable() public void testDelayedReuseContainerNotAvailable() throws IOException, InterruptedException, ExecutionException { LOG.info("Test testDelayedReuseContainerNotAvailable"); - Configuration conf = new Configuration(new YarnConfiguration()); + Configuration conf = new Configuration(); conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, true); conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, false); conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED, false); @@ -351,7 +350,7 @@ public void testDelayedReuseContainerNotAvailable() @Test(timeout = 10000l) public void testSimpleReuse() throws IOException, InterruptedException, ExecutionException { LOG.info("Test testSimpleReuse"); - Configuration tezConf = new Configuration(new YarnConfiguration()); + Configuration tezConf = new Configuration(); tezConf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, true); tezConf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, true); tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 0); @@ -492,7 +491,7 @@ public void testSimpleReuse() throws IOException, InterruptedException, Executio @Test(timeout = 10000l) public void testReuseWithTaskSpecificLaunchCmdOption() throws IOException, InterruptedException, ExecutionException { LOG.info("Test testReuseWithTaskSpecificLaunchCmdOption"); - Configuration tezConf = new Configuration(new YarnConfiguration()); + Configuration tezConf = new Configuration(); tezConf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, true); tezConf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, true); tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 0); @@ -682,7 +681,7 @@ public void testReuseWithTaskSpecificLaunchCmdOption() throws IOException, Inter public void testReuseNonLocalRequest() throws IOException, InterruptedException, ExecutionException { LOG.info("Test testReuseNonLocalRequest"); - Configuration tezConf = new Configuration(new YarnConfiguration()); + Configuration tezConf = new Configuration(); tezConf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, true); tezConf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, true); tezConf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED, true); @@ -803,7 +802,7 @@ public void testReuseNonLocalRequest() public void testReuseAcrossVertices() throws IOException, InterruptedException, ExecutionException { LOG.info("Test testReuseAcrossVertices"); - Configuration tezConf = new Configuration(new YarnConfiguration()); + Configuration tezConf = new Configuration(); tezConf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, true); tezConf.setLong( TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 1l); @@ -923,7 +922,7 @@ public void testReuseAcrossVertices() @Test(timeout = 30000l) public void testReuseLocalResourcesChanged() throws IOException, InterruptedException, ExecutionException { LOG.info("Test testReuseLocalResourcesChanged"); - Configuration tezConf = new Configuration(new YarnConfiguration()); + Configuration tezConf = new Configuration(); tezConf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, true); tezConf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, true); tezConf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED, true); @@ -1080,7 +1079,7 @@ public void testReuseLocalResourcesChanged() throws IOException, InterruptedExce @Test(timeout = 30000l) public void testReuseConflictLocalResources() throws IOException, InterruptedException, ExecutionException { LOG.info("Test testReuseLocalResourcesChanged"); - Configuration tezConf = new Configuration(new YarnConfiguration()); + Configuration tezConf = new Configuration(); tezConf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, true); tezConf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, true); tezConf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED, true); @@ -1317,7 +1316,7 @@ public void testReuseConflictLocalResources() throws IOException, InterruptedExc public void testAssignmentOnShutdown() throws IOException, InterruptedException, ExecutionException { LOG.info("Test testAssignmentOnShutdown"); - Configuration tezConf = new Configuration(new YarnConfiguration()); + Configuration tezConf = new Configuration(); tezConf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, false); tezConf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, true); tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 0); @@ -1384,7 +1383,7 @@ public void testAssignmentOnShutdown() @Test(timeout=5000) public void testDifferentResourceContainerReuse() throws Exception { - Configuration tezConf = new Configuration(new YarnConfiguration()); + Configuration tezConf = new Configuration(); tezConf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, true); tezConf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, true); tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 0); @@ -1520,7 +1519,7 @@ eventHandler, rmClient, new AlwaysMatchesContainerMatcher(), @Test(timeout=5000) public void testEnvironmentVarsContainerReuse() throws Exception { - Configuration tezConf = new Configuration(new YarnConfiguration()); + Configuration tezConf = new Configuration(); tezConf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, true); tezConf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, true); tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 0); diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java index 00a68cd998..5a7d754ba5 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java @@ -148,7 +148,7 @@ public class YARNRunner implements ClientProtocol { * @param conf the configuration object for the client */ public YARNRunner(Configuration conf) { - this(conf, new ResourceMgrDelegate(new YarnConfiguration(conf))); + this(conf, new ResourceMgrDelegate(new YarnConfiguration(conf))); } /** diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestMRInputHelpers.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestMRInputHelpers.java index 88cc4a51f1..11b1271a8a 100644 --- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestMRInputHelpers.java +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestMRInputHelpers.java @@ -40,7 +40,6 @@ import org.apache.hadoop.mapreduce.split.JobSplit; import org.apache.hadoop.mapreduce.split.SplitMetaInfoReader; import org.apache.hadoop.yarn.api.records.LocalResource; -import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.tez.dag.api.DataSourceDescriptor; import org.apache.tez.dag.api.TaskLocationHint; import org.junit.Assert; @@ -71,7 +70,7 @@ public static void setup() throws IOException { throw new RuntimeException("problem starting mini dfs cluster", io); } - Configuration testConf = new YarnConfiguration( + Configuration testConf = new Configuration( dfsCluster.getFileSystem().getConf()); diff --git a/tez-tests/src/test/java/org/apache/tez/test/FaultToleranceTestRunner.java b/tez-tests/src/test/java/org/apache/tez/test/FaultToleranceTestRunner.java index 94242a7fd2..9305711744 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/FaultToleranceTestRunner.java +++ b/tez-tests/src/test/java/org/apache/tez/test/FaultToleranceTestRunner.java @@ -25,7 +25,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.tez.client.TezClientUtils; import org.apache.tez.client.TezClient; import org.apache.tez.dag.api.DAG; @@ -52,10 +51,10 @@ public class FaultToleranceTestRunner { void setup() throws Exception { TezConfiguration tezConf = null; - if (conf == null ) { - tezConf = new TezConfiguration(new YarnConfiguration()); - }else { - tezConf = new TezConfiguration(new YarnConfiguration(this.conf)); + if (conf == null) { + tezConf = new TezConfiguration(); + } else { + tezConf = new TezConfiguration(this.conf); } FileSystem defaultFs = FileSystem.get(tezConf); From d90b1042e97993eb103f161b40e523638e317f9a Mon Sep 17 00:00:00 2001 From: Jonathan Turner Eagles Date: Thu, 27 Aug 2020 14:57:59 +0200 Subject: [PATCH 264/512] TEZ-3645: Reuse SerializationFactory while sorting, merging, and writing IFiles (Jonathan Turner Eagles reviewed by Rajesh Balamohan, Laszlo Bodor) Signed-off-by: Laszlo Bodor --- .../serializer/SerializationContext.java | 87 +++++++++++++++++++ .../shuffle/orderedgrouped/MergeManager.java | 72 ++++++++------- .../common/sort/impl/ExternalSorter.java | 19 ++-- .../library/common/sort/impl/IFile.java | 43 +++++---- .../common/sort/impl/PipelinedSorter.java | 28 +++--- .../library/common/sort/impl/TezMerger.java | 55 ++++++------ .../common/sort/impl/dflt/DefaultSorter.java | 33 ++++--- .../BaseUnorderedPartitionedKVWriter.java | 9 +- .../writers/UnorderedPartitionedKVWriter.java | 15 ++-- .../library/common/TestValuesIterator.java | 57 ++++++------ .../common/readers/TestUnorderedKVReader.java | 5 +- .../orderedgrouped/TestMergeManager.java | 16 ++-- .../library/common/sort/impl/TestIFile.java | 47 +++++----- .../common/sort/impl/TestTezMerger.java | 38 ++++---- 14 files changed, 318 insertions(+), 206 deletions(-) create mode 100644 tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/serializer/SerializationContext.java diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/serializer/SerializationContext.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/serializer/SerializationContext.java new file mode 100644 index 0000000000..2398b8f930 --- /dev/null +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/serializer/SerializationContext.java @@ -0,0 +1,87 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.runtime.library.common.serializer; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.serializer.Serialization; +import org.apache.hadoop.io.serializer.SerializationFactory; +import org.apache.hadoop.io.serializer.Serializer; +import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; +import org.apache.tez.runtime.library.common.ConfigUtils; + +/** + * SerializationContext is a wrapper class for serialization related fields. + */ +public class SerializationContext { + + private Class keyClass; + private Class valueClass; + private Serialization keySerialization; + private Serialization valSerialization; + + public SerializationContext(Configuration conf) { + this.keyClass = ConfigUtils.getIntermediateInputKeyClass(conf); + this.valueClass = ConfigUtils.getIntermediateInputValueClass(conf); + SerializationFactory serializationFactory = new SerializationFactory(conf); + if (keyClass != null) { + this.keySerialization = serializationFactory.getSerialization(keyClass); + } + if (valueClass != null) { + this.valSerialization = serializationFactory.getSerialization(valueClass); + } + } + + public SerializationContext(Class keyClass, Class valueClass, + Serialization keySerialization, Serialization valSerialization) { + this.keyClass = keyClass; + this.valueClass = valueClass; + this.keySerialization = keySerialization; + this.valSerialization = valSerialization; + } + + public Class getKeyClass() { + return keyClass; + } + + public Class getValueClass() { + return valueClass; + } + + public Serialization getKeySerialization() { + return keySerialization; + } + + public Serialization getValSerialization() { + return valSerialization; + } + + @SuppressWarnings({ "rawtypes", "unchecked" }) + public Serializer getKeySerializer() { + return keySerialization.getSerializer((Class) keyClass); + } + + @SuppressWarnings({ "rawtypes", "unchecked" }) + public Serializer getValueSerializer() { + return valSerialization.getSerializer((Class) valueClass); + } + + public void applyToConf(Configuration conf) { + conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, keyClass.getName()); + conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS, valueClass.getName()); + } +} diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java index dd23b3b617..8565e7111c 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java @@ -30,7 +30,6 @@ import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.BoundedByteArrayOutputStream; import org.apache.hadoop.io.DataInputBuffer; import org.apache.hadoop.io.FileChunk; import org.apache.hadoop.io.RawComparator; @@ -47,6 +46,7 @@ import org.apache.tez.runtime.library.common.Constants; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; import org.apache.tez.runtime.library.common.combine.Combiner; +import org.apache.tez.runtime.library.common.serializer.SerializationContext; import org.apache.tez.runtime.library.common.sort.impl.IFile; import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer; import org.apache.tez.runtime.library.common.sort.impl.TezMerger; @@ -158,6 +158,8 @@ public void progress() { private final boolean cleanup; + private SerializationContext serializationContext; + /** * Construct the MergeManager. Must call start before it becomes usable. */ @@ -297,21 +299,21 @@ public MergeManager(Configuration conf, + ", mergeThreshold: " + this.mergeThreshold); } - boolean allowMemToMemMerge = - conf.getBoolean( - TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_ENABLE_MEMTOMEM, + boolean allowMemToMemMerge = + conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_ENABLE_MEMTOMEM, TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_ENABLE_MEMTOMEM_DEFAULT); - if (allowMemToMemMerge) { - this.memToMemMerger = - new IntermediateMemoryToMemoryMerger(this, - memToMemMergeOutputsThreshold); - } else { - this.memToMemMerger = null; - } - - this.inMemoryMerger = new InMemoryMerger(this); - - this.onDiskMerger = new OnDiskMerger(this); + if (allowMemToMemMerge) { + this.memToMemMerger = + new IntermediateMemoryToMemoryMerger(this, memToMemMergeOutputsThreshold); + } else { + this.memToMemMerger = null; + } + + this.inMemoryMerger = new InMemoryMerger(this); + + this.onDiskMerger = new OnDiskMerger(this); + + this.serializationContext = new SerializationContext(conf); } void setupParentThread(Thread shuffleSchedulerThread) { @@ -802,8 +804,7 @@ public void merge(List inputs) throws IOException, InterruptedExcepti // TODO Is this doing any combination ? TezRawKeyValueIterator rIter = TezMerger.merge(conf, rfs, - ConfigUtils.getIntermediateInputKeyClass(conf), - ConfigUtils.getIntermediateInputValueClass(conf), + serializationContext, inMemorySegments, inMemorySegments.size(), new Path(inputContext.getUniqueIdentifier()), (RawComparator)ConfigUtils.getIntermediateInputKeyComparator(conf), @@ -885,10 +886,10 @@ public void merge(List inputs) throws IOException, InterruptedExcepti long outFileLen = 0; try { writer = - new Writer(conf, rfs, outputPath, - (Class)ConfigUtils.getIntermediateInputKeyClass(conf), - (Class)ConfigUtils.getIntermediateInputValueClass(conf), - codec, null, null); + new Writer(serializationContext.getKeySerialization(), + serializationContext.getValSerialization(), rfs, outputPath, + serializationContext.getKeyClass(), serializationContext.getValueClass(), codec, + null, null); TezRawKeyValueIterator rIter = null; LOG.info("Initiating in-memory merge with " + noInMemorySegments + @@ -897,8 +898,7 @@ public void merge(List inputs) throws IOException, InterruptedExcepti tmpDir = new Path(inputContext.getUniqueIdentifier()); // Nothing actually materialized to disk - controlled by setting sort-factor to #segments. rIter = TezMerger.merge(conf, rfs, - (Class)ConfigUtils.getIntermediateInputKeyClass(conf), - (Class)ConfigUtils.getIntermediateInputValueClass(conf), + serializationContext, inMemorySegments, inMemorySegments.size(), tmpDir, (RawComparator)ConfigUtils.getIntermediateInputKeyComparator(conf), progressable, spilledRecordsCounter, null, additionalBytesRead, null); @@ -1027,16 +1027,14 @@ public void merge(List inputs) throws IOException, InterruptedExcepti outputPath = localDirAllocator.getLocalPathForWrite(namePart, approxOutputSize, conf); outputPath = outputPath.suffix(Constants.MERGED_OUTPUT_PREFIX + mergeFileSequenceId.getAndIncrement()); - Writer writer = - new Writer(conf, rfs, outputPath, - (Class)ConfigUtils.getIntermediateInputKeyClass(conf), - (Class)ConfigUtils.getIntermediateInputValueClass(conf), - codec, null, null); + Writer writer = new Writer(serializationContext.getKeySerialization(), + serializationContext.getValSerialization(), rfs, outputPath, + serializationContext.getKeyClass(), serializationContext.getValueClass(), codec, null, + null); tmpDir = new Path(inputContext.getUniqueIdentifier()); try { TezRawKeyValueIterator iter = TezMerger.merge(conf, rfs, - (Class)ConfigUtils.getIntermediateInputKeyClass(conf), - (Class)ConfigUtils.getIntermediateInputValueClass(conf), + serializationContext, inputSegments, ioSortFactor, tmpDir, (RawComparator)ConfigUtils.getIntermediateInputKeyComparator(conf), @@ -1165,8 +1163,7 @@ private TezRawKeyValueIterator finalMerge(Configuration job, FileSystem fs, inputContext.notifyProgress(); // merge config params - Class keyClass = (Class)ConfigUtils.getIntermediateInputKeyClass(job); - Class valueClass = (Class)ConfigUtils.getIntermediateInputValueClass(job); + SerializationContext serContext = new SerializationContext(job); final Path tmpDir = new Path(inputContext.getUniqueIdentifier()); final RawComparator comparator = (RawComparator)ConfigUtils.getIntermediateInputKeyComparator(job); @@ -1198,11 +1195,12 @@ private TezRawKeyValueIterator finalMerge(Configuration job, FileSystem fs, final Path outputPath = mapOutputFile.getInputFileForWrite(srcTaskId, Integer.MAX_VALUE, inMemToDiskBytes).suffix(Constants.MERGED_OUTPUT_PREFIX); - final TezRawKeyValueIterator rIter = TezMerger.merge(job, fs, keyClass, valueClass, + final TezRawKeyValueIterator rIter = TezMerger.merge(job, fs, serContext, memDiskSegments, numMemDiskSegments, tmpDir, comparator, progressable, spilledRecordsCounter, null, additionalBytesRead, null); - final Writer writer = new Writer(job, fs, outputPath, - keyClass, valueClass, codec, null, null); + final Writer writer = new Writer(serContext.getKeySerialization(), + serContext.getValSerialization(), fs, outputPath, serContext.getKeyClass(), + serContext.getValueClass(), codec, null, null); try { TezMerger.writeFile(rIter, writer, progressable, TezRuntimeConfiguration.TEZ_RUNTIME_RECORDS_BEFORE_PROGRESS_DEFAULT); } catch (IOException e) { @@ -1302,7 +1300,7 @@ public int compare(Segment o1, Segment o2) { diskSegments.addAll(0, memDiskSegments); memDiskSegments.clear(); TezRawKeyValueIterator diskMerge = TezMerger.merge( - job, fs, keyClass, valueClass, codec, diskSegments, + job, fs, serContext, codec, diskSegments, ioSortFactor, numInMemSegments, tmpDir, comparator, progressable, false, spilledRecordsCounter, null, additionalBytesRead, null); diskSegments.clear(); @@ -1316,7 +1314,7 @@ public int compare(Segment o1, Segment o2) { LOG.info(finalMergeLog.toString()); } // This is doing nothing but creating an iterator over the segments. - return TezMerger.merge(job, fs, keyClass, valueClass, codec, + return TezMerger.merge(job, fs, serContext, codec, finalSegments, finalSegments.size(), tmpDir, comparator, progressable, spilledRecordsCounter, null, additionalBytesRead, null); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java index 16d58492fb..194e899fea 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java @@ -44,7 +44,6 @@ import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.Compressor; import org.apache.hadoop.io.compress.DefaultCodec; -import org.apache.hadoop.io.serializer.SerializationFactory; import org.apache.hadoop.io.serializer.Serializer; import org.apache.hadoop.util.IndexedSorter; import org.apache.hadoop.util.Progressable; @@ -60,13 +59,14 @@ import org.apache.tez.runtime.library.common.ConfigUtils; import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.apache.tez.runtime.library.common.combine.Combiner; +import org.apache.tez.runtime.library.common.serializer.SerializationContext; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.ShuffleHeader; import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer; import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput; import org.apache.tez.common.Preconditions; -@SuppressWarnings({"unchecked", "rawtypes"}) +@SuppressWarnings({"rawtypes"}) public abstract class ExternalSorter { private static final Logger LOG = LoggerFactory.getLogger(ExternalSorter.class); @@ -106,10 +106,9 @@ public void progress() { protected final FileSystem rfs; protected final TezTaskOutput mapOutputFile; protected final int partitions; - protected final Class keyClass; - protected final Class valClass; protected final RawComparator comparator; - protected final SerializationFactory serializationFactory; + + protected final SerializationContext serializationContext; protected final Serializer keySerializer; protected final Serializer valSerializer; @@ -201,14 +200,12 @@ public ExternalSorter(OutputContext outputContext, Configuration conf, int numOu comparator = ConfigUtils.getIntermediateOutputKeyComparator(this.conf); // k/v serialization - keyClass = ConfigUtils.getIntermediateOutputKeyClass(this.conf); - valClass = ConfigUtils.getIntermediateOutputValueClass(this.conf); - serializationFactory = new SerializationFactory(this.conf); - keySerializer = serializationFactory.getSerializer(keyClass); - valSerializer = serializationFactory.getSerializer(valClass); + this.serializationContext = new SerializationContext(this.conf); + keySerializer = serializationContext.getKeySerializer(); + valSerializer = serializationContext.getValueSerializer(); LOG.info(outputContext.getDestinationVertexName() + " using: " + "memoryMb=" + assignedMb - + ", keySerializerClass=" + keyClass + + ", keySerializerClass=" + serializationContext.getKeyClass() + ", valueSerializerClass=" + valSerializer + ", comparator=" + (RawComparator) ConfigUtils.getIntermediateOutputKeyComparator(conf) + ", partitioner=" + conf.get(TezRuntimeConfiguration.TEZ_RUNTIME_PARTITIONER_CLASS) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java index 5d8e944424..6aa44e206a 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java @@ -51,7 +51,7 @@ import org.apache.hadoop.io.compress.CompressionOutputStream; import org.apache.hadoop.io.compress.Compressor; import org.apache.hadoop.io.compress.Decompressor; -import org.apache.hadoop.io.serializer.SerializationFactory; +import org.apache.hadoop.io.serializer.Serialization; import org.apache.hadoop.io.serializer.Serializer; import org.apache.tez.common.counters.TezCounter; @@ -110,7 +110,8 @@ public static class FileBackedInMemIFileWriter extends Writer { * Note that we do not allow compression in in-mem stream. * When spilled over to file, compression gets enabled. * - * @param conf + * @param keySerialization + * @param valSerialization * @param fs * @param taskOutput * @param keyClass @@ -121,13 +122,11 @@ public static class FileBackedInMemIFileWriter extends Writer { * @param cacheSize * @throws IOException */ - public FileBackedInMemIFileWriter(Configuration conf, FileSystem fs, - TezTaskOutput taskOutput, Class keyClass, Class valueClass, - CompressionCodec codec, - TezCounter writesCounter, - TezCounter serializedBytesCounter, - int cacheSize) throws IOException { - super(conf, new FSDataOutputStream(createBoundedBuffer(cacheSize), null), + public FileBackedInMemIFileWriter(Serialization keySerialization, + Serialization valSerialization, FileSystem fs, TezTaskOutput taskOutput, + Class keyClass, Class valueClass, CompressionCodec codec, TezCounter writesCounter, + TezCounter serializedBytesCounter, int cacheSize) throws IOException { + super(keySerialization, valSerialization, new FSDataOutputStream(createBoundedBuffer(cacheSize), null), keyClass, valueClass, null, writesCounter, serializedBytesCounter); this.fs = fs; this.cacheStream = (BoundedByteArrayOutputStream) this.rawOut.getWrappedStream(); @@ -315,12 +314,12 @@ public static class Writer { protected final boolean rle; - public Writer(Configuration conf, FileSystem fs, Path file, + public Writer(Serialization keySerialization, Serialization valSerialization, FileSystem fs, Path file, Class keyClass, Class valueClass, CompressionCodec codec, TezCounter writesCounter, TezCounter serializedBytesCounter) throws IOException { - this(conf, fs.create(file), keyClass, valueClass, codec, + this(keySerialization, valSerialization, fs.create(file), keyClass, valueClass, codec, writesCounter, serializedBytesCounter); ownOutputStream = true; } @@ -331,17 +330,17 @@ protected Writer(TezCounter writesCounter, TezCounter serializedBytesCounter, bo this.rle = rle; } - public Writer(Configuration conf, FSDataOutputStream outputStream, + public Writer(Serialization keySerialization, Serialization valSerialization, FSDataOutputStream outputStream, Class keyClass, Class valueClass, CompressionCodec codec, TezCounter writesCounter, TezCounter serializedBytesCounter) throws IOException { - this(conf, outputStream, keyClass, valueClass, codec, writesCounter, + this(keySerialization, valSerialization, outputStream, keyClass, valueClass, codec, writesCounter, serializedBytesCounter, false); } - public Writer(Configuration conf, FSDataOutputStream outputStream, - Class keyClass, Class valueClass, - CompressionCodec codec, TezCounter writesCounter, TezCounter serializedBytesCounter, - boolean rle) throws IOException { + public Writer(Serialization keySerialization, Serialization valSerialization, FSDataOutputStream outputStream, + Class keyClass, Class valueClass, + CompressionCodec codec, TezCounter writesCounter, TezCounter serializedBytesCounter, + boolean rle) throws IOException { this.rawOut = outputStream; this.writtenRecordsCounter = writesCounter; this.serializedUncompressedBytes = serializedBytesCounter; @@ -354,11 +353,9 @@ public Writer(Configuration conf, FSDataOutputStream outputStream, if (keyClass != null) { this.closeSerializers = true; - SerializationFactory serializationFactory = - new SerializationFactory(conf); - this.keySerializer = serializationFactory.getSerializer(keyClass); + this.keySerializer = keySerialization.getSerializer(keyClass); this.keySerializer.open(buffer); - this.valueSerializer = serializationFactory.getSerializer(valueClass); + this.valueSerializer = valSerialization.getSerializer(valueClass); this.valueSerializer.open(buffer); } else { this.closeSerializers = false; @@ -383,8 +380,8 @@ void setupOutputStream(CompressionCodec codec) throws IOException { } } - public Writer(Configuration conf, FileSystem fs, Path file) throws IOException { - this(conf, fs, file, null, null, null, null, null); + public Writer(Serialization keySerialization, Serialization valSerialization, FileSystem fs, Path file) throws IOException { + this(keySerialization, valSerialization, fs, file, null, null, null, null, null); } protected void writeHeader(OutputStream outputStream) throws IOException { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java index 2ace8755f0..b70d6c4360 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java @@ -398,14 +398,14 @@ public void write(Object key, Object value) */ synchronized void collect(Object key, Object value, final int partition ) throws IOException { - if (key.getClass() != keyClass) { + if (key.getClass() != serializationContext.getKeyClass()) { throw new IOException("Type mismatch in key from map: expected " - + keyClass.getName() + ", received " + + serializationContext.getKeyClass().getName() + ", received " + key.getClass().getName()); } - if (value.getClass() != valClass) { + if (value.getClass() != serializationContext.getValueClass()) { throw new IOException("Type mismatch in value from map: expected " - + valClass.getName() + ", received " + + serializationContext.getValueClass().getName() + ", received " + value.getClass().getName()); } if (partition < 0 || partition >= partitions) { @@ -506,8 +506,9 @@ private void spillSingleRecord(final Object key, final Object value, try { long segmentStart = out.getPos(); if (!sendEmptyPartitionDetails || (i == partition)) { - writer = new Writer(conf, out, keyClass, valClass, codec, - spilledRecordsCounter, null, false); + writer = new Writer(serializationContext.getKeySerialization(), + serializationContext.getValSerialization(), out, serializationContext.getKeyClass(), + serializationContext.getValueClass(), codec, spilledRecordsCounter, null, false); } // we need not check for combiner since its a single record if (i == partition) { @@ -592,8 +593,10 @@ public boolean spill(boolean ignoreEmptySpills) throws IOException { Writer writer = null; boolean hasNext = kvIter.hasNext(); if (hasNext || !sendEmptyPartitionDetails) { - writer = new Writer(conf, out, keyClass, valClass, codec, - spilledRecordsCounter, null, merger.needsRLE()); + writer = new Writer(serializationContext.getKeySerialization(), + serializationContext.getValSerialization(), out, serializationContext.getKeyClass(), + serializationContext.getValueClass(), codec, spilledRecordsCounter, null, + merger.needsRLE()); } if (combiner == null) { while (kvIter.next()) { @@ -791,7 +794,7 @@ public void flush() throws IOException { boolean sortSegments = segmentList.size() > mergeFactor; //merge TezRawKeyValueIterator kvIter = TezMerger.merge(conf, rfs, - keyClass, valClass, codec, + serializationContext, codec, segmentList, mergeFactor, new Path(uniqueIdentifier), (RawComparator) ConfigUtils.getIntermediateOutputKeyComparator(conf), @@ -803,9 +806,10 @@ public void flush() throws IOException { long rawLength = 0; long partLength = 0; if (shouldWrite) { - Writer writer = - new Writer(conf, finalOut, keyClass, valClass, codec, - spilledRecordsCounter, null, merger.needsRLE()); + Writer writer = new Writer(serializationContext.getKeySerialization(), + serializationContext.getValSerialization(), finalOut, + serializationContext.getKeyClass(), serializationContext.getValueClass(), codec, + spilledRecordsCounter, null, merger.needsRLE()); if (combiner == null || numSpills < minSpillsForCombine) { TezMerger.writeFile(kvIter, writer, progressable, TezRuntimeConfiguration.TEZ_RUNTIME_RECORDS_BEFORE_PROGRESS_DEFAULT); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java index 726810bda8..e83b34e114 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java @@ -47,6 +47,7 @@ import org.apache.tez.common.counters.TezCounter; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; import org.apache.tez.runtime.library.common.Constants; +import org.apache.tez.runtime.library.common.serializer.SerializationContext; import org.apache.tez.runtime.library.common.sort.impl.IFile.Reader; import org.apache.tez.runtime.library.common.sort.impl.IFile.Reader.KeyState; import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer; @@ -70,7 +71,7 @@ public class TezMerger { public static TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, - Class keyClass, Class valueClass, + SerializationContext serializationContext, CompressionCodec codec, boolean ifileReadAhead, int ifileReadAheadLength, int ifileBufferSize, Path[] inputs, boolean deleteInputs, @@ -84,7 +85,7 @@ TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, return new MergeQueue(conf, fs, inputs, deleteInputs, codec, ifileReadAhead, ifileReadAheadLength, ifileBufferSize, false, comparator, - reporter, null).merge(keyClass, valueClass, + reporter, null).merge(serializationContext, mergeFactor, tmpDir, readsCounter, writesCounter, bytesReadCounter, @@ -94,7 +95,7 @@ TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, // Used by the in-memory merger. public static TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, - Class keyClass, Class valueClass, + SerializationContext serializationContext, List segments, int mergeFactor, Path tmpDir, RawComparator comparator, Progressable reporter, @@ -104,14 +105,14 @@ TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, Progress mergePhase) throws IOException, InterruptedException { // Get rid of this ? - return merge(conf, fs, keyClass, valueClass, segments, mergeFactor, tmpDir, + return merge(conf, fs, serializationContext, segments, mergeFactor, tmpDir, comparator, reporter, false, readsCounter, writesCounter, bytesReadCounter, mergePhase); } public static TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, - Class keyClass, Class valueClass, + SerializationContext serializationContext, List segments, int mergeFactor, Path tmpDir, RawComparator comparator, Progressable reporter, @@ -122,14 +123,14 @@ TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, Progress mergePhase) throws IOException, InterruptedException { return new MergeQueue(conf, fs, segments, comparator, reporter, - sortSegments, false).merge(keyClass, valueClass, - mergeFactor, tmpDir, + sortSegments, false).merge(serializationContext, mergeFactor, tmpDir, readsCounter, writesCounter, bytesReadCounter, mergePhase); } public static TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, - Class keyClass, Class valueClass, CompressionCodec codec, + SerializationContext serializationContext, + CompressionCodec codec, List segments, int mergeFactor, Path tmpDir, RawComparator comparator, Progressable reporter, @@ -139,13 +140,13 @@ public static TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, Progress mergePhase) throws IOException, InterruptedException { return new MergeQueue(conf, fs, segments, comparator, reporter, false, codec, false, false) - .merge(keyClass, valueClass, mergeFactor, tmpDir, + .merge(serializationContext, mergeFactor, tmpDir, readsCounter, writesCounter, bytesReadCounter, mergePhase); } public static TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, - Class keyClass, Class valueClass, + SerializationContext serializationContext, CompressionCodec codec, List segments, int mergeFactor, Path tmpDir, @@ -159,7 +160,7 @@ TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, throws IOException, InterruptedException { return new MergeQueue(conf, fs, segments, comparator, reporter, sortSegments, codec, considerFinalMergeForProgress, checkForSameKeys). - merge(keyClass, valueClass, + merge(serializationContext, mergeFactor, tmpDir, readsCounter, writesCounter, bytesReadCounter, @@ -168,7 +169,7 @@ TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, public static TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, - Class keyClass, Class valueClass, + SerializationContext serializationContext, CompressionCodec codec, List segments, int mergeFactor, Path tmpDir, @@ -182,8 +183,7 @@ TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, throws IOException, InterruptedException { return new MergeQueue(conf, fs, segments, comparator, reporter, sortSegments, codec, considerFinalMergeForProgress). - merge(keyClass, valueClass, - mergeFactor, tmpDir, + merge(serializationContext, mergeFactor, tmpDir, readsCounter, writesCounter, bytesReadCounter, mergePhase); @@ -191,7 +191,7 @@ TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, public static TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, - Class keyClass, Class valueClass, + SerializationContext serializationContext, CompressionCodec codec, List segments, int mergeFactor, int inMemSegments, Path tmpDir, @@ -203,7 +203,7 @@ TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, Progress mergePhase) throws IOException, InterruptedException { return new MergeQueue(conf, fs, segments, comparator, reporter, - sortSegments, codec, false).merge(keyClass, valueClass, + sortSegments, codec, false).merge(serializationContext, mergeFactor, inMemSegments, tmpDir, readsCounter, writesCounter, @@ -704,18 +704,18 @@ protected boolean lessThan(Object a, Object b) { return comparator.compare(key1.getData(), s1, l1, key2.getData(), s2, l2) < 0; } - public TezRawKeyValueIterator merge(Class keyClass, Class valueClass, - int factor, Path tmpDir, - TezCounter readsCounter, - TezCounter writesCounter, - TezCounter bytesReadCounter, - Progress mergePhase) + public TezRawKeyValueIterator merge(SerializationContext serializationContext, + int factor, Path tmpDir, + TezCounter readsCounter, + TezCounter writesCounter, + TezCounter bytesReadCounter, + Progress mergePhase) throws IOException, InterruptedException { - return merge(keyClass, valueClass, factor, 0, tmpDir, + return merge(serializationContext, factor, 0, tmpDir, readsCounter, writesCounter, bytesReadCounter, mergePhase); } - TezRawKeyValueIterator merge(Class keyClass, Class valueClass, + TezRawKeyValueIterator merge(SerializationContext serializationContext, int factor, int inMem, Path tmpDir, TezCounter readsCounter, TezCounter writesCounter, @@ -866,9 +866,10 @@ TezRawKeyValueIterator merge(Class keyClass, Class valueClass, // TODO Would it ever make sense to make this an in-memory writer ? // Merging because of too many disk segments - might fit in memory. - Writer writer = - new Writer(conf, fs, outputFile, keyClass, valueClass, codec, - writesCounter, null); + Writer writer = new Writer(serializationContext.getKeySerialization(), + serializationContext.getValSerialization(), fs, outputFile, + serializationContext.getKeyClass(), serializationContext.getValueClass(), codec, + writesCounter, null); writeFile(this, writer, reporter, recordsBeforeProgress); writer.close(); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java index d0a18b4205..dd6c083109 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java @@ -32,7 +32,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import org.apache.commons.lang.exception.ExceptionUtils; -import org.apache.hadoop.fs.permission.FsPermission; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.runtime.library.api.IOInterruptedException; import org.slf4j.Logger; @@ -250,14 +249,14 @@ public void write(Object key, Object value) synchronized void collect(Object key, Object value, final int partition ) throws IOException { - if (key.getClass() != keyClass) { + if (key.getClass() != serializationContext.getKeyClass()) { throw new IOException("Type mismatch in key from map: expected " - + keyClass.getName() + ", received " + + serializationContext.getKeyClass().getName() + ", received " + key.getClass().getName()); } - if (value.getClass() != valClass) { + if (value.getClass() != serializationContext.getValueClass()) { throw new IOException("Type mismatch in value from map: expected " - + valClass.getName() + ", received " + + serializationContext.getValueClass().getName() + ", received " + value.getClass().getName()); } if (partition < 0 || partition >= partitions) { @@ -907,8 +906,9 @@ protected void spill(int mstart, int mend, long sameKeyCount, long totalKeysCoun long segmentStart = out.getPos(); if (spindex < mend && kvmeta.get(offsetFor(spindex) + PARTITION) == i || !sendEmptyPartitionDetails) { - writer = new Writer(conf, out, keyClass, valClass, codec, - spilledRecordsCounter, null, rle); + writer = new Writer(serializationContext.getKeySerialization(), + serializationContext.getValSerialization(), out, serializationContext.getKeyClass(), + serializationContext.getValueClass(), codec, spilledRecordsCounter, null, rle); } if (combiner == null) { // spill directly @@ -1014,8 +1014,9 @@ private void spillSingleRecord(final Object key, final Object value, long segmentStart = out.getPos(); // Create a new codec, don't care! if (!sendEmptyPartitionDetails || (i == partition)) { - writer = new Writer(conf, out, keyClass, valClass, codec, - spilledRecordsCounter, null, false); + writer = new Writer(serializationContext.getKeySerialization(), + serializationContext.getValSerialization(), out, serializationContext.getKeyClass(), + serializationContext.getValueClass(), codec, spilledRecordsCounter, null, false); } if (i == partition) { final long recordStart = out.getPos(); @@ -1292,7 +1293,10 @@ private void mergeParts() throws IOException, InterruptedException { long segmentStart = finalOut.getPos(); if (!sendEmptyPartitionDetails) { Writer writer = - new Writer(conf, finalOut, keyClass, valClass, codec, null, null); + new Writer(serializationContext.getKeySerialization(), + serializationContext.getValSerialization(), finalOut, + serializationContext.getKeyClass(), serializationContext.getValueClass(), codec, + null, null); writer.close(); rawLength = writer.getRawLength(); partLength = writer.getCompressedLength(); @@ -1350,7 +1354,7 @@ private void mergeParts() throws IOException, InterruptedException { boolean sortSegments = segmentList.size() > mergeFactor; //merge TezRawKeyValueIterator kvIter = TezMerger.merge(conf, rfs, - keyClass, valClass, codec, + serializationContext, codec, segmentList, mergeFactor, new Path(taskIdentifier), (RawComparator)ConfigUtils.getIntermediateOutputKeyComparator(conf), @@ -1363,9 +1367,10 @@ private void mergeParts() throws IOException, InterruptedException { long rawLength = 0; long partLength = 0; if (shouldWrite) { - Writer writer = - new Writer(conf, finalOut, keyClass, valClass, codec, - spilledRecordsCounter, null); + Writer writer = new Writer(serializationContext.getKeySerialization(), + serializationContext.getValSerialization(), finalOut, + serializationContext.getKeyClass(), serializationContext.getValueClass(), codec, + spilledRecordsCounter, null); if (combiner == null || numSpills < minSpillsForCombine) { TezMerger.writeFile(kvIter, writer, progressable, TezRuntimeConfiguration.TEZ_RUNTIME_RECORDS_BEFORE_PROGRESS_DEFAULT); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/BaseUnorderedPartitionedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/BaseUnorderedPartitionedKVWriter.java index 26da98f378..ecc9e03a5d 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/BaseUnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/BaseUnorderedPartitionedKVWriter.java @@ -22,6 +22,7 @@ import java.util.Iterator; import java.util.List; +import org.apache.hadoop.io.serializer.Serialization; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -57,6 +58,8 @@ public abstract class BaseUnorderedPartitionedKVWriter extends KeyValuesWriter { protected final Serializer keySerializer; protected final Serializer valSerializer; protected final SerializationFactory serializationFactory; + protected final Serialization keySerialization; + protected final Serialization valSerialization; protected final int numPartitions; protected final CompressionCodec codec; protected final TezTaskOutput outputFileHandler; @@ -124,8 +127,10 @@ public BaseUnorderedPartitionedKVWriter(OutputContext outputContext, Configurati keyClass = ConfigUtils.getIntermediateOutputKeyClass(this.conf); valClass = ConfigUtils.getIntermediateOutputValueClass(this.conf); serializationFactory = new SerializationFactory(this.conf); - keySerializer = serializationFactory.getSerializer(keyClass); - valSerializer = serializationFactory.getSerializer(valClass); + keySerialization = serializationFactory.getSerialization(keyClass); + valSerialization = serializationFactory.getSerialization(valClass); + keySerializer = keySerialization.getSerializer(keyClass); + valSerializer = valSerialization.getSerializer(valClass); outputRecordBytesCounter = outputContext.getCounters().findCounter(TaskCounter.OUTPUT_BYTES); outputRecordsCounter = outputContext.getCounters().findCounter(TaskCounter.OUTPUT_RECORDS); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java index d9467af2bc..5ff2944766 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java @@ -47,7 +47,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.io.DataInputBuffer; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.compress.CompressionCodec; @@ -301,12 +300,12 @@ public UnorderedPartitionedKVWriter(OutputContext outputContext, Configuration c //special case, where in only one partition is available. skipBuffers = true; if (this.useCachedStream) { - writer = new IFile.FileBackedInMemIFileWriter(conf, rfs, outputFileHandler, keyClass, - valClass, codec, outputRecordsCounter, outputRecordBytesCounter, - dataViaEventsMaxSize); + writer = new IFile.FileBackedInMemIFileWriter(keySerialization, valSerialization, rfs, + outputFileHandler, keyClass, valClass, codec, outputRecordsCounter, + outputRecordBytesCounter, dataViaEventsMaxSize); } else { finalOutPath = outputFileHandler.getOutputFileForWrite(); - writer = new IFile.Writer(conf, rfs, finalOutPath, keyClass, valClass, + writer = new IFile.Writer(keySerialization, valSerialization, rfs, finalOutPath, keyClass, valClass, codec, outputRecordsCounter, outputRecordBytesCounter); ensureSpillFilePermissions(finalOutPath, rfs); } @@ -643,7 +642,7 @@ protected SpillResult callInternal() throws IOException { continue; } if (writer == null) { - writer = new Writer(conf, out, keyClass, valClass, codec, null, null); + writer = new Writer(keySerialization, valSerialization, out, keyClass, valClass, codec, null, null); } numRecords += writePartition(buffer.partitionPositions[i], buffer, writer, key, val); } @@ -1086,7 +1085,7 @@ private void mergeAll() throws IOException { LOG.info(destNameTrimmed + ": " + "Skipping partition: " + i + " in final merge since it has no records"); continue; } - writer = new Writer(conf, out, keyClass, valClass, codec, null, null); + writer = new Writer(keySerialization, valSerialization, out, keyClass, valClass, codec, null, null); try { if (currentBuffer.nextPosition != 0 && currentBuffer.partitionPositions[i] != WrappedBuffer.PARTITION_ABSENT_POSITION) { @@ -1177,7 +1176,7 @@ private void writeLargeRecord(final Object key, final Object value, final int pa spilledRecordsCounter.increment(1); Writer writer = null; try { - writer = new IFile.Writer(conf, out, keyClass, valClass, codec, null, null); + writer = new IFile.Writer(keySerialization, valSerialization, out, keyClass, valClass, codec, null, null); writer.append(key, value); outputLargeRecordsCounter.increment(1); numRecordsPerPartition[i]++; diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/TestValuesIterator.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/TestValuesIterator.java index 642f02b569..eaded181b9 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/TestValuesIterator.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/TestValuesIterator.java @@ -6,6 +6,7 @@ import com.google.common.collect.Lists; import java.nio.ByteBuffer; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -36,6 +37,7 @@ import org.apache.tez.runtime.api.InputContext; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; import org.apache.tez.runtime.library.common.comparator.TezBytesComparator; +import org.apache.tez.runtime.library.common.serializer.SerializationContext; import org.apache.tez.runtime.library.common.serializer.TezBytesWritableSerialization; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.InMemoryReader; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.InMemoryWriter; @@ -102,8 +104,7 @@ enum TestWithComparator { FileSystem fs; static final Random rnd = new Random(); - final Class keyClass; - final Class valClass; + private SerializationContext serializationContext; final RawComparator comparator; final RawComparator correctComparator; final boolean expectedTestResult; @@ -129,20 +130,18 @@ enum TestWithComparator { * @param testResult expected result * @throws IOException */ - public TestValuesIterator(String serializationClassName, Class key, Class val, + public TestValuesIterator(String serializationClassName, Class key, Class val, TestWithComparator comparator, TestWithComparator correctComparator, boolean testResult) throws IOException { - this.keyClass = key; - this.valClass = val; this.comparator = getComparator(comparator); this.correctComparator = (correctComparator == null) ? this.comparator : getComparator(correctComparator); this.expectedTestResult = testResult; originalData = LinkedListMultimap.create(); - setupConf(serializationClassName); + setupConf(key, val, serializationClassName); } - private void setupConf(String serializationClassName) throws IOException { + private void setupConf(Class key, Class val, String serializationClassName) throws IOException { mergeFactor = 2; conf = new Configuration(); conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_IO_SORT_FACTOR, mergeFactor); @@ -154,6 +153,11 @@ private void setupConf(String serializationClassName) throws IOException { String localDirs = baseDir.toString(); conf.setStrings(TezRuntimeFrameworkConfigs.LOCAL_DIRS, localDirs); fs = FileSystem.getLocal(conf); + + SerializationFactory serializationFactory = new SerializationFactory(conf); + serializationContext = new SerializationContext(key, val, + serializationFactory.getSerialization(key), serializationFactory.getSerialization(val)); + serializationContext.applyToConf(conf); } @Before @@ -231,20 +235,21 @@ private ValuesIterator createEmptyIterator(boolean inMemory) streamPaths = new Path[0]; //This will return EmptyIterator rawKeyValueIterator = - TezMerger.merge(conf, fs, keyClass, valClass, null, + TezMerger.merge(conf, fs, serializationContext, null, false, -1, 1024, streamPaths, false, mergeFactor, tmpDir, comparator, new ProgressReporter(), null, null, null, null); } else { List segments = Lists.newLinkedList(); //This will return EmptyIterator rawKeyValueIterator = - TezMerger.merge(conf, fs, keyClass, valClass, segments, mergeFactor, tmpDir, + TezMerger.merge(conf, fs, serializationContext, segments, mergeFactor, tmpDir, comparator, new ProgressReporter(), new GenericCounter("readsCounter", "y"), new GenericCounter("writesCounter", "y1"), new GenericCounter("bytesReadCounter", "y2"), new Progress()); } return new ValuesIterator(rawKeyValueIterator, comparator, - keyClass, valClass, conf, (TezCounter) new GenericCounter("inputKeyCounter", "y3"), + serializationContext.getKeyClass(), serializationContext.getValueClass(), conf, + (TezCounter) new GenericCounter("inputKeyCounter", "y3"), (TezCounter) new GenericCounter("inputValueCounter", "y4")); } @@ -332,19 +337,20 @@ private ValuesIterator createIterator(boolean inMemory) throws IOException, Inte streamPaths = createFiles(); //Merge all files to get KeyValueIterator rawKeyValueIterator = - TezMerger.merge(conf, fs, keyClass, valClass, null, + TezMerger.merge(conf, fs, serializationContext, null, false, -1, 1024, streamPaths, false, mergeFactor, tmpDir, comparator, new ProgressReporter(), null, null, null, null); } else { List segments = createInMemStreams(); rawKeyValueIterator = - TezMerger.merge(conf, fs, keyClass, valClass, segments, mergeFactor, tmpDir, + TezMerger.merge(conf, fs, serializationContext, segments, mergeFactor, tmpDir, comparator, new ProgressReporter(), new GenericCounter("readsCounter", "y"), new GenericCounter("writesCounter", "y1"), new GenericCounter("bytesReadCounter", "y2"), new Progress()); } - return new ValuesIterator(rawKeyValueIterator, comparator, - keyClass, valClass, conf, (TezCounter) new GenericCounter("inputKeyCounter", "y3"), + return new ValuesIterator(rawKeyValueIterator, comparator, serializationContext.getKeyClass(), + serializationContext.getValueClass(), conf, + (TezCounter) new GenericCounter("inputKeyCounter", "y3"), (TezCounter) new GenericCounter("inputValueCounter", "y4")); } @@ -364,19 +370,19 @@ private ValuesIterator createCountedIterator(boolean inMemory, TezCounter keyCou streamPaths = createFiles(); //Merge all files to get KeyValueIterator rawKeyValueIterator = - TezMerger.merge(conf, fs, keyClass, valClass, null, + TezMerger.merge(conf, fs, serializationContext, null, false, -1, 1024, streamPaths, false, mergeFactor, tmpDir, comparator, new ProgressReporter(), null, null, null, null); } else { List segments = createInMemStreams(); rawKeyValueIterator = - TezMerger.merge(conf, fs, keyClass, valClass, segments, mergeFactor, tmpDir, + TezMerger.merge(conf, fs, serializationContext, segments, mergeFactor, tmpDir, comparator, new ProgressReporter(), new GenericCounter("readsCounter", "y"), new GenericCounter("writesCounter", "y1"), new GenericCounter("bytesReadCounter", "y2"), new Progress()); } - return new ValuesIterator(rawKeyValueIterator, comparator, - keyClass, valClass, conf, keyCounter, tupleCounter); + return new ValuesIterator(rawKeyValueIterator, comparator, serializationContext.getKeyClass(), + serializationContext.getValueClass(), conf, keyCounter, tupleCounter); } @Parameterized.Parameters(name = "test[{0}, {1}, {2}, {3} {4} {5} {6}]") @@ -454,7 +460,9 @@ private Path[] createFiles() throws IOException { paths[i] = new Path(baseDir, "ifile_" + i + ".out"); FSDataOutputStream out = fs.create(paths[i]); //write data with RLE - IFile.Writer writer = new IFile.Writer(conf, out, keyClass, valClass, null, null, null, true); + IFile.Writer writer = new IFile.Writer(serializationContext.getKeySerialization(), + serializationContext.getValSerialization(), out, serializationContext.getKeyClass(), + serializationContext.getValueClass(), null, null, null, true); Map data = createData(); for (Map.Entry entry : data.entrySet()) { @@ -487,9 +495,8 @@ public List createInMemStreams() throws IOException { int numberOfStreams = Math.max(2, rnd.nextInt(10)); LOG.info("No of streams : " + numberOfStreams); - SerializationFactory serializationFactory = new SerializationFactory(conf); - Serializer keySerializer = serializationFactory.getSerializer(keyClass); - Serializer valueSerializer = serializationFactory.getSerializer(valClass); + Serializer keySerializer = serializationContext.getKeySerializer(); + Serializer valueSerializer = serializationContext.getValueSerializer(); LocalDirAllocator localDirAllocator = new LocalDirAllocator(TezRuntimeFrameworkConfigs.LOCAL_DIRS); @@ -549,8 +556,8 @@ private InputContext createTezInputContext() { private Map createData() { Map map = new TreeMap(comparator); for (int j = 0; j < Math.max(10, rnd.nextInt(50)); j++) { - Writable key = createData(keyClass); - Writable value = createData(valClass); + Writable key = createData(serializationContext.getKeyClass()); + Writable value = createData(serializationContext.getValueClass()); map.put(key, value); //sortedDataMap.put(key, value); } @@ -558,7 +565,7 @@ private Map createData() { } - private Writable createData(Class c) { + private Writable createData(Class c) { if (c.getName().equalsIgnoreCase(BytesWritable.class.getName())) { return new BytesWritable(new BigInteger(256, rnd).toString().getBytes()); } else if (c.getName().equalsIgnoreCase(IntWritable.class.getName())) { diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/readers/TestUnorderedKVReader.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/readers/TestUnorderedKVReader.java index 6fef9440dd..42231e988e 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/readers/TestUnorderedKVReader.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/readers/TestUnorderedKVReader.java @@ -23,6 +23,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.serializer.WritableSerialization; import org.apache.tez.common.counters.TaskCounter; import org.apache.tez.common.counters.TezCounter; import org.apache.tez.common.counters.TezCounters; @@ -131,8 +132,8 @@ public void freeResources(FetchedInput fetchedInput) { private void createIFile(Path path, int recordCount) throws IOException { FSDataOutputStream out = localFs.create(path); - IFile.Writer writer = - new IFile.Writer(defaultConf, out, Text.class, Text.class, null, null, null, true); + IFile.Writer writer = new IFile.Writer(new WritableSerialization(), new WritableSerialization(), + out, Text.class, Text.class, null, null, null, true); for (int i = 0; i < recordCount; i++) { writer.append(new Text("Key_" + i), new Text("Value_" + i)); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java index 9cffcc7a4a..13f090cffb 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java @@ -38,6 +38,7 @@ import com.google.common.collect.Sets; import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.io.serializer.WritableSerialization; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -630,8 +631,8 @@ public void testIntermediateMemoryMerge() throws Throwable { private byte[] generateDataBySize(Configuration conf, int rawLen, InputAttemptIdentifier inputAttemptIdentifier) throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); FSDataOutputStream fsdos = new FSDataOutputStream(baos, null); - IFile.Writer writer = - new IFile.Writer(conf, fsdos, IntWritable.class, IntWritable.class, null, null, null); + IFile.Writer writer = new IFile.Writer(new WritableSerialization(), new WritableSerialization(), + fsdos, IntWritable.class, IntWritable.class, null, null, null); int i = 0; while(true) { writer.append(new IntWritable(i), new IntWritable(i)); @@ -653,8 +654,8 @@ private byte[] generateDataBySizeAndGetBytes(Configuration conf, int rawLen, InputAttemptIdentifier inputAttemptIdentifier) throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); FSDataOutputStream fsdos = new FSDataOutputStream(baos, null); - IFile.Writer writer = - new IFile.Writer(conf, fsdos, IntWritable.class, IntWritable.class, null, null, null); + IFile.Writer writer = new IFile.Writer(new WritableSerialization(), new WritableSerialization(), + fsdos, IntWritable.class, IntWritable.class, null, null, null); int i = 0; while(true) { writer.append(new IntWritable(i), new IntWritable(i)); @@ -676,8 +677,8 @@ private byte[] generateData(Configuration conf, int numEntries, InputAttemptIdentifier inputAttemptIdentifier) throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); FSDataOutputStream fsdos = new FSDataOutputStream(baos, null); - IFile.Writer writer = - new IFile.Writer(conf, fsdos, IntWritable.class, IntWritable.class, null, null, null); + IFile.Writer writer = new IFile.Writer(new WritableSerialization(), new WritableSerialization(), + fsdos, IntWritable.class, IntWritable.class, null, null, null); for (int i = 0; i < numEntries; ++i) { writer.append(new IntWritable(i), new IntWritable(i)); } @@ -1015,7 +1016,8 @@ private SrcFileInfo createFile(Configuration conf, FileSystem fs, Path path, int for (int i = 0; i < numPartitions; i++) { long pos = outStream.getPos(); IFile.Writer writer = - new IFile.Writer(conf, outStream, IntWritable.class, IntWritable.class, null, null, null); + new IFile.Writer(new WritableSerialization(), new WritableSerialization(), outStream, + IntWritable.class, IntWritable.class, null, null, null); for (int j = 0; j < numKeysPerPartition; j++) { writer.append(new IntWritable(currentKey), new IntWritable(currentKey)); currentKey++; diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java index b9c556df7f..c74496e46b 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java @@ -52,6 +52,7 @@ import org.apache.hadoop.io.compress.CompressionCodecFactory; import org.apache.hadoop.io.serializer.Deserializer; import org.apache.hadoop.io.serializer.SerializationFactory; +import org.apache.hadoop.io.serializer.WritableSerialization; import org.apache.tez.common.TezRuntimeFrameworkConfigs; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; import org.apache.tez.runtime.library.common.TezRuntimeUtils; @@ -144,7 +145,7 @@ public void testCompressedFlag() throws IOException { public void testWritingEmptyKeyValues() throws IOException { DataInputBuffer key = new DataInputBuffer(); DataInputBuffer value = new DataInputBuffer(); - IFile.Writer writer = new IFile.Writer(defaultConf, localFs, outputPath, null, null, null, + IFile.Writer writer = new IFile.Writer(null, null, localFs, outputPath, null, null, null, null, null); writer.append(key, value); writer.append(key, value); @@ -203,7 +204,7 @@ public void testExceedMaxSize() throws IOException { // Check Key length exceeding MAX_BUFFER_SIZE out = localFs.create(outputPath); - writer = new IFile.Writer(defaultConf, out, + writer = new IFile.Writer(new WritableSerialization(), new WritableSerialization(), out, Text.class, Text.class, null, null, null, false); writer.append(longString, shortString); writer.close(); @@ -226,7 +227,7 @@ public void testExceedMaxSize() throws IOException { // Check Value length exceeding MAX_BUFFER_SIZE out = localFs.create(outputPath); - writer = new IFile.Writer(defaultConf, out, + writer = new IFile.Writer(new WritableSerialization(), new WritableSerialization(), out, Text.class, Text.class, null, null, null, false); writer.append(shortString, longString); writer.close(); @@ -250,7 +251,7 @@ public void testExceedMaxSize() throws IOException { // Check Key length not getting doubled out = localFs.create(outputPath); - writer = new IFile.Writer(defaultConf, out, + writer = new IFile.Writer(new WritableSerialization(), new WritableSerialization(), out, Text.class, Text.class, null, null, null, false); writer.append(longString, shortString); writer.close(); @@ -269,7 +270,7 @@ public void testExceedMaxSize() throws IOException { // Check Value length not getting doubled out = localFs.create(outputPath); - writer = new IFile.Writer(defaultConf, out, + writer = new IFile.Writer(new WritableSerialization(), new WritableSerialization(), out, Text.class, Text.class, null, null, null, false); writer.append(shortString, longString); writer.close(); @@ -296,7 +297,7 @@ public void testExceedMaxSize() throws IOException { public void testWithRLEMarker() throws IOException { //Test with append(Object, Object) FSDataOutputStream out = localFs.create(outputPath); - IFile.Writer writer = new IFile.Writer(defaultConf, out, + IFile.Writer writer = new IFile.Writer(new WritableSerialization(), new WritableSerialization(), out, Text.class, IntWritable.class, codec, null, null, true); Text key = new Text("key0"); @@ -322,7 +323,7 @@ public void testWithRLEMarker() throws IOException { int valueLength = 6; int pos = 0; out = localFs.create(outputPath); - writer = new IFile.Writer(defaultConf, out, + writer = new IFile.Writer(new WritableSerialization(), new WritableSerialization(), out, Text.class, IntWritable.class, codec, null, null, true); BoundedByteArrayOutputStream boundedOut = new BoundedByteArrayOutputStream(1024*1024); @@ -465,8 +466,8 @@ public void testInMemoryWriter() throws IOException { //Test appendValue feature public void testAppendValue() throws IOException { List data = KVDataGen.generateTestData(false, rnd.nextInt(100)); - IFile.Writer writer = new IFile.Writer(defaultConf, localFs, outputPath, - Text.class, IntWritable.class, codec, null, null); + IFile.Writer writer = new IFile.Writer(new WritableSerialization(), new WritableSerialization(), + localFs, outputPath, Text.class, IntWritable.class, codec, null, null); Text previousKey = null; for (KVPair kvp : data) { @@ -496,8 +497,8 @@ public void testAppendValues() throws IOException { values.add(val); } - IFile.Writer writer = new IFile.Writer(defaultConf, localFs, outputPath, - Text.class, IntWritable.class, codec, null, null); + IFile.Writer writer = new IFile.Writer(new WritableSerialization(), new WritableSerialization(), + localFs, outputPath, Text.class, IntWritable.class, codec, null, null); writer.append(data.get(0).getKey(), data.get(0).getvalue()); //write first KV pair writer.appendValues(values.subList(1, values.size()).iterator()); //add the rest here @@ -524,7 +525,8 @@ public void testFileBackedInMemIFileWriter() throws IOException { } TezTaskOutputFiles tezTaskOutput = new TezTaskOutputFiles(defaultConf, "uniqueId", 1); - IFile.FileBackedInMemIFileWriter writer = new IFile.FileBackedInMemIFileWriter(defaultConf, localFs, tezTaskOutput, + IFile.FileBackedInMemIFileWriter writer = new IFile.FileBackedInMemIFileWriter( + new WritableSerialization(), new WritableSerialization(), localFs, tezTaskOutput, Text.class, IntWritable.class, codec, null, null, 200); @@ -547,7 +549,8 @@ public void testFileBackedInMemIFileWriter() throws IOException { public void testFileBackedInMemIFileWriterWithSmallBuffer() throws IOException { List data = new ArrayList<>(); TezTaskOutputFiles tezTaskOutput = new TezTaskOutputFiles(defaultConf, "uniqueId", 1); - IFile.FileBackedInMemIFileWriter writer = new IFile.FileBackedInMemIFileWriter(defaultConf, localFs, tezTaskOutput, + IFile.FileBackedInMemIFileWriter writer = new IFile.FileBackedInMemIFileWriter( + new WritableSerialization(), new WritableSerialization(), localFs, tezTaskOutput, Text.class, IntWritable.class, codec, null, null, 2); @@ -581,7 +584,8 @@ public void testFileBackedInMemIFileWriter_withSpill() throws IOException { // Setting cache limit to 20. Actual data would be around 43 bytes, so it would spill over. TezTaskOutputFiles tezTaskOutput = new TezTaskOutputFiles(defaultConf, "uniqueId", 1); - IFile.FileBackedInMemIFileWriter writer = new IFile.FileBackedInMemIFileWriter(defaultConf, localFs, tezTaskOutput, + IFile.FileBackedInMemIFileWriter writer = new IFile.FileBackedInMemIFileWriter( + new WritableSerialization(), new WritableSerialization(), localFs, tezTaskOutput, Text.class, IntWritable.class, codec, null, null, 20); writer.setOutputPath(outputPath); @@ -614,7 +618,8 @@ public void testEmptyFileBackedInMemIFileWriter() throws IOException { TezTaskOutputFiles tezTaskOutput = new TezTaskOutputFiles(defaultConf, "uniqueId", 1); - IFile.FileBackedInMemIFileWriter writer = new IFile.FileBackedInMemIFileWriter(defaultConf, localFs, tezTaskOutput, + IFile.FileBackedInMemIFileWriter writer = new IFile.FileBackedInMemIFileWriter( + new WritableSerialization(), new WritableSerialization(), localFs, tezTaskOutput, Text.class, IntWritable.class, codec, null, null, 100); @@ -644,8 +649,8 @@ public void testAppendKeyValues() throws IOException { values.add(val); } - IFile.Writer writer = new IFile.Writer(defaultConf, localFs, outputPath, - Text.class, IntWritable.class, codec, null, null); + IFile.Writer writer = new IFile.Writer(new WritableSerialization(), new WritableSerialization(), + localFs, outputPath, Text.class, IntWritable.class, codec, null, null); writer.appendKeyValues(data.get(0).getKey(), values.iterator()); Text lastKey = new Text("key3"); @@ -662,8 +667,8 @@ public void testAppendKeyValues() throws IOException { //Test appendValue with DataInputBuffer public void testAppendValueWithDataInputBuffer() throws IOException { List data = KVDataGen.generateTestData(false, rnd.nextInt(100)); - IFile.Writer writer = new IFile.Writer(defaultConf, localFs, outputPath, - Text.class, IntWritable.class, codec, null, null); + IFile.Writer writer = new IFile.Writer(new WritableSerialization(), new WritableSerialization(), + localFs, outputPath, Text.class, IntWritable.class, codec, null, null); final DataInputBuffer previousKey = new DataInputBuffer(); DataInputBuffer key = new DataInputBuffer(); @@ -914,7 +919,7 @@ private void verifyData(Reader reader, List data) private Writer writeTestFile(boolean rle, boolean repeatKeys, List data, CompressionCodec codec) throws IOException { FSDataOutputStream out = localFs.create(outputPath); - IFile.Writer writer = new IFile.Writer(defaultConf, out, + IFile.Writer writer = new IFile.Writer(new WritableSerialization(), new WritableSerialization(), out, Text.class, IntWritable.class, codec, null, null, rle); writeTestFile(writer, repeatKeys, data); out.close(); @@ -947,7 +952,7 @@ private Writer writeTestFile(IFile.Writer writer, boolean repeatKeys, private Writer writeTestFileUsingDataBuffer(boolean rle, boolean repeatKeys, List data, CompressionCodec codec) throws IOException { FSDataOutputStream out = localFs.create(outputPath); - IFile.Writer writer = new IFile.Writer(defaultConf, out, + IFile.Writer writer = new IFile.Writer(new WritableSerialization(), new WritableSerialization(), out, Text.class, IntWritable.class, codec, null, null, rle); writeTestFileUsingDataBuffer(writer, repeatKeys, data); out.close(); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestTezMerger.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestTezMerger.java index af10700d89..c6574d74f4 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestTezMerger.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestTezMerger.java @@ -36,11 +36,13 @@ import org.apache.hadoop.io.RawComparator; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.WritableComparator; +import org.apache.hadoop.io.serializer.WritableSerialization; import org.apache.hadoop.util.Progress; import org.apache.hadoop.util.Progressable; import org.apache.tez.common.TezRuntimeFrameworkConfigs; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; import org.apache.tez.runtime.library.common.ConfigUtils; +import org.apache.tez.runtime.library.common.serializer.SerializationContext; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.InMemoryReader; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.InMemoryWriter; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.MergeManager; @@ -136,7 +138,7 @@ public void testMerge() throws Exception { private Path createIFileWithTextData(List data) throws IOException { Path path = new Path(workDir + "/src", "data_" + System.nanoTime() + ".out"); FSDataOutputStream out = localFs.create(path); - IFile.Writer writer = new IFile.Writer(defaultConf, out, Text.class, + IFile.Writer writer = new IFile.Writer(new WritableSerialization(), new WritableSerialization(), out, Text.class, Text.class, null, null, null, true); for (String key : data) { writer.append(new Text(key), new Text(key + "_" + System.nanoTime())); @@ -568,11 +570,12 @@ public void testWithCustomComparator_allEmptyFiles() throws Exception { private TezRawKeyValueIterator merge(List pathList, RawComparator rc) throws IOException, InterruptedException { TezMerger merger = new TezMerger(); - TezRawKeyValueIterator records = merger.merge(defaultConf, localFs, IntWritable.class, - LongWritable.class, null, false, 0, 1024, pathList.toArray(new Path[pathList.size()]), - true, 4, new Path(workDir, "tmp_" + System.nanoTime()), ((rc == null) ? comparator : rc), - new Reporter(), null, null, - null, new Progress()); + TezRawKeyValueIterator records = merger.merge(defaultConf, localFs, + new SerializationContext(IntWritable.class, LongWritable.class, new WritableSerialization(), + new WritableSerialization()), + null, false, 0, 1024, pathList.toArray(new Path[pathList.size()]), true, 4, + new Path(workDir, "tmp_" + System.nanoTime()), ((rc == null) ? comparator : rc), + new Reporter(), null, null, null, new Progress()); return records; } @@ -602,12 +605,12 @@ private void merge(int fileCount, int keysPerFile, int mergeFactor) throws Excep private void merge(List pathList, int mergeFactor, RawComparator rc) throws Exception { //Merge datasets TezMerger merger = new TezMerger(); - TezRawKeyValueIterator records = merger.merge(defaultConf, localFs, IntWritable.class, - LongWritable.class, null, false, 0, 1024, pathList.toArray(new Path[pathList.size()]), - true, mergeFactor, new Path(workDir, "tmp_" + System.nanoTime()), - ((rc == null) ? comparator : rc), new Reporter(), null, null, - null, - new Progress()); + TezRawKeyValueIterator records = merger.merge(defaultConf, localFs, + new SerializationContext(IntWritable.class, LongWritable.class, new WritableSerialization(), + new WritableSerialization()), + null, false, 0, 1024, pathList.toArray(new Path[pathList.size()]), true, mergeFactor, + new Path(workDir, "tmp_" + System.nanoTime()), ((rc == null) ? comparator : rc), + new Reporter(), null, null, null, new Progress()); verifyData(records); verificationDataSet.clear(); @@ -701,9 +704,10 @@ private void mergeSegments(List segmentList, int mergeFactor, TezMerger.MergeQueue mergeQueue = new TezMerger.MergeQueue(defaultConf, localFs, segmentList, comparator, new Reporter(), false, false); - TezRawKeyValueIterator records = mergeQueue.merge(IntWritable.class, LongWritable.class, - mergeFactor, new Path(workDir, "tmp_" - + System.nanoTime()), null, null, null, null); + TezRawKeyValueIterator records = mergeQueue.merge( + new SerializationContext(IntWritable.class, LongWritable.class, new WritableSerialization(), + new WritableSerialization()), + mergeFactor, new Path(workDir, "tmp_" + System.nanoTime()), null, null, null, null); //Verify the merged data is correct verifyData(records); @@ -770,8 +774,8 @@ static Path writeIFile(int keysPerFile, int repeatCount) throws Path path = new Path(workDir + "/src", "data_" + System.nanoTime() + ".out"); FSDataOutputStream out = localFs.create(path); //create IFile with RLE - IFile.Writer writer = new IFile.Writer(defaultConf, out, IntWritable.class - , LongWritable.class, null, null, null, true); + IFile.Writer writer = new IFile.Writer(new WritableSerialization(), new WritableSerialization(), + out, IntWritable.class, LongWritable.class, null, null, null, true); for (Integer key : dataSet.keySet()) { for (Long value : dataSet.get(key)) { From 79048f9e44cc41154a3d741090053a0b0e6f21b0 Mon Sep 17 00:00:00 2001 From: Laszlo Bodor Date: Tue, 1 Sep 2020 12:42:10 +0200 Subject: [PATCH 265/512] TEZ-4228: TezClassLoader should be used in TezChild and for Configuration objects --- .../org/apache/tez/common/TezClassLoader.java | 18 ++++++++++++++++++ .../java/org/apache/tez/common/TezUtils.java | 3 +++ .../org/apache/tez/runtime/task/TezChild.java | 4 +++- 3 files changed, 24 insertions(+), 1 deletion(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/TezClassLoader.java b/tez-api/src/main/java/org/apache/tez/common/TezClassLoader.java index 923d2177db..53c9e51fc6 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezClassLoader.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezClassLoader.java @@ -18,6 +18,10 @@ import java.security.AccessController; import java.security.PrivilegedAction; +import org.apache.hadoop.conf.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * ClassLoader to allow addition of new paths to classpath in the runtime. * @@ -30,6 +34,7 @@ */ public class TezClassLoader extends URLClassLoader { private static final TezClassLoader INSTANCE; + private static final Logger LOG = LoggerFactory.getLogger(TezClassLoader.class); static { INSTANCE = AccessController.doPrivileged(new PrivilegedAction() { @@ -41,6 +46,11 @@ public TezClassLoader run() { private TezClassLoader() { super(new URL[] {}, TezClassLoader.class.getClassLoader()); + + LOG.info( + "Created TezClassLoader with parent classloader: {}, thread: {}, system classloader: {}", + TezClassLoader.class.getClassLoader(), Thread.currentThread().getId(), + ClassLoader.getSystemClassLoader()); } public void addURL(URL url) { @@ -52,6 +62,14 @@ public static TezClassLoader getInstance() { } public static void setupTezClassLoader() { + LOG.debug( + "Setting up TezClassLoader: thread: {}, current thread classloader: {} system classloader: {}", + Thread.currentThread().getId(), Thread.currentThread().getContextClassLoader(), + ClassLoader.getSystemClassLoader()); Thread.currentThread().setContextClassLoader(INSTANCE); } + + public static void setupForConfiguration(Configuration configuration) { + configuration.setClassLoader(INSTANCE); + } } diff --git a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java index 23811aa7f1..3f0241834a 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java @@ -117,6 +117,7 @@ public static Configuration createConfFromByteString(ByteString byteString) thro DAGProtos.ConfigurationProto confProto = DAGProtos.ConfigurationProto.parseFrom(in); Configuration conf = new Configuration(false); readConfFromPB(confProto, conf); + TezClassLoader.setupForConfiguration(conf); return conf; } } @@ -130,6 +131,7 @@ public static Configuration createConfFromBaseConfAndPayload(TaskContext context try(SnappyInputStream uncompressIs = new SnappyInputStream(byteString.newInput())) { DAGProtos.ConfigurationProto confProto = DAGProtos.ConfigurationProto.parseFrom(uncompressIs); readConfFromPB(confProto, configuration); + TezClassLoader.setupForConfiguration(configuration); return configuration; } } @@ -139,6 +141,7 @@ public static void addToConfFromByteString(Configuration configuration, ByteStri try(SnappyInputStream uncompressIs = new SnappyInputStream(byteString.newInput())) { DAGProtos.ConfigurationProto confProto = DAGProtos.ConfigurationProto.parseFrom(uncompressIs); readConfFromPB(confProto, configuration); + TezClassLoader.setupForConfiguration(configuration); } } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java index 6dd146a8fb..86ceb12d7d 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java @@ -77,6 +77,8 @@ import com.google.common.base.Function; import org.apache.tez.common.Preconditions; +import org.apache.tez.common.TezClassLoader; + import com.google.common.collect.HashMultimap; import com.google.common.collect.Maps; import com.google.common.collect.Multimap; @@ -480,7 +482,7 @@ public static TezChild newTezChild(Configuration conf, String host, int port, St } public static void main(String[] args) throws IOException, InterruptedException, TezException { - + TezClassLoader.setupTezClassLoader(); final Configuration defaultConf = new Configuration(); Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler()); From 1fba8f71a0f496e1fcb67a223fb32b389bdfb2e0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Tue, 8 Sep 2020 18:43:23 +0200 Subject: [PATCH 266/512] =?UTF-8?q?TEZ-4230:=20LocalContainerLauncher=20ca?= =?UTF-8?q?n=20kill=20task=20future=20too=20early,=20causing=20app=20hang?= =?UTF-8?q?=20(L=C3=A1szl=C3=B3=20Bodor=20reviewed=20by=20Jonathan=20Turne?= =?UTF-8?q?r=20Eagles)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../tez/dag/app/launcher/LocalContainerLauncher.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java index a63ce3fdfe..ae308098a7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java @@ -299,10 +299,11 @@ private void stop(ContainerStopRequest event) { if (future == null) { LOG.info("Ignoring stop request for containerId: " + event.getContainerId()); } else { - LOG.info( - "Stopping containerId: {}", - event.getContainerId()); - future.cancel(true); + LOG.info("Stopping containerId: {}, isDone: {}", event.getContainerId(), + future.isDone()); + future.cancel(false); + LOG.debug("Stopped containerId: {}, isCancelled: {}", event.getContainerId(), + future.isCancelled()); } // Send this event to maintain regular control flow. This isn't of much use though. getContext().containerStopRequested(event.getContainerId()); From 629714afd692770d64aed73f6e48b7b46838a592 Mon Sep 17 00:00:00 2001 From: Laszlo Bodor Date: Mon, 28 Sep 2020 08:28:53 +0200 Subject: [PATCH 267/512] TEZ-4233: Map task should be blamed earlier for local fetch failures (Laszlo Bdoor reviewed by Rajesh Balamohan) --- .../api/events/InputReadErrorEvent.java | 40 +++- .../shuffle/api/ShuffleHandlerError.java | 27 +++ .../common/shuffle/api/package-info.java | 22 +++ tez-api/src/main/proto/Events.proto | 2 + .../tez/dag/app/dag/impl/TaskAttemptImpl.java | 9 +- .../tez/dag/app/dag/impl/TestTaskAttempt.java | 49 ++++- .../tez/auxservices/ShuffleHandler.java | 52 ++++-- .../tez/auxservices/TestShuffleHandler.java | 102 +++++++++- .../apache/tez/runtime/api/impl/TezEvent.java | 9 +- .../library/common/shuffle/Fetcher.java | 63 ++++--- .../common/shuffle/FetcherCallback.java | 3 +- .../shuffle/InputAttemptFetchFailure.java | 115 ++++++++++++ .../common/shuffle/impl/ShuffleManager.java | 22 ++- .../orderedgrouped/FetcherOrderedGrouped.java | 74 +++++--- .../shuffle/orderedgrouped/ShuffleHeader.java | 7 + .../orderedgrouped/ShuffleScheduler.java | 45 ++--- .../library/common/shuffle/TestFetcher.java | 54 +++++- .../shuffle/impl/TestShuffleManager.java | 15 +- .../shuffle/orderedgrouped/TestFetcher.java | 66 +++++-- .../orderedgrouped/TestShuffleScheduler.java | 174 +++++++++--------- .../library/testutils/RuntimeTestUtils.java | 44 +++++ 21 files changed, 766 insertions(+), 228 deletions(-) create mode 100644 tez-api/src/main/java/org/apache/tez/runtime/library/common/shuffle/api/ShuffleHandlerError.java create mode 100644 tez-api/src/main/java/org/apache/tez/runtime/library/common/shuffle/api/package-info.java create mode 100644 tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/InputAttemptFetchFailure.java create mode 100644 tez-runtime-library/src/test/java/org/apache/tez/runtime/library/testutils/RuntimeTestUtils.java diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java index cabc39fc8e..8ef50ebac4 100644 --- a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java +++ b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java @@ -51,26 +51,44 @@ public final class InputReadErrorEvent extends Event { */ private final int numFailures; - private InputReadErrorEvent(final String diagnostics, final int index, - final int version, final int numFailures) { + /** + * Whether this input read error is caused while fetching local file. + */ + private final boolean isLocalFetch; + + /** + * Whether this input read error is caused because the fetcher detected a fatal, unrecoverable, + * local file read issue from the shuffle handler. + */ + private final boolean isDiskErrorAtSource; + + private InputReadErrorEvent(final String diagnostics, final int index, final int version, + final int numFailures, boolean isLocalFetch, boolean isDiskErrorAtSource) { super(); this.diagnostics = diagnostics; this.index = index; this.version = version; this.numFailures = numFailures; + this.isLocalFetch = isLocalFetch; + this.isDiskErrorAtSource = isDiskErrorAtSource; } - public static InputReadErrorEvent create(String diagnostics, int index, - int version) { - return create(diagnostics, index, version, 1); + public static InputReadErrorEvent create(String diagnostics, int index, int version, + boolean isLocalFetch, boolean isDiskErrorAtSource) { + return create(diagnostics, index, version, 1, isLocalFetch, isDiskErrorAtSource); + } + + public static InputReadErrorEvent create(String diagnostics, int index, int version) { + return create(diagnostics, index, version, 1, false, false); } /** * Create an InputReadErrorEvent. */ public static InputReadErrorEvent create(final String diagnostics, final int index, - final int version, final int numFailures) { - return new InputReadErrorEvent(diagnostics, index, version, numFailures); + final int version, final int numFailures, boolean isLocalFetch, boolean isDiskErrorAtSource) { + return new InputReadErrorEvent(diagnostics, index, version, numFailures, isLocalFetch, + isDiskErrorAtSource); } public String getDiagnostics() { @@ -92,6 +110,14 @@ public int getNumFailures() { return numFailures; } + public boolean isLocalFetch() { + return isLocalFetch; + } + + public boolean isDiskErrorAtSource() { + return isDiskErrorAtSource; + } + @Override public int hashCode() { return Objects.hash(index, version); diff --git a/tez-api/src/main/java/org/apache/tez/runtime/library/common/shuffle/api/ShuffleHandlerError.java b/tez-api/src/main/java/org/apache/tez/runtime/library/common/shuffle/api/ShuffleHandlerError.java new file mode 100644 index 0000000000..09137de673 --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/runtime/library/common/shuffle/api/ShuffleHandlerError.java @@ -0,0 +1,27 @@ +/** +* Licensed to the Apache Software Foundation (ASF) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. The ASF licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ +package org.apache.tez.runtime.library.common.shuffle.api; + +/** + * ShuffleHandlerError enum encapsulates possible error messages that can be propagated from + * ShuffleHandler to fetchers. Depending on the message, fetchers can make better decisions, or give + * AM a hint in order to let it make better decisions in case of shuffle issues. + */ +public enum ShuffleHandlerError { + DISK_ERROR_EXCEPTION +} diff --git a/tez-api/src/main/java/org/apache/tez/runtime/library/common/shuffle/api/package-info.java b/tez-api/src/main/java/org/apache/tez/runtime/library/common/shuffle/api/package-info.java new file mode 100644 index 0000000000..9ad8e61d50 --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/runtime/library/common/shuffle/api/package-info.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +@Private +package org.apache.tez.runtime.library.common.shuffle.api; + +import org.apache.hadoop.classification.InterfaceAudience.Private; \ No newline at end of file diff --git a/tez-api/src/main/proto/Events.proto b/tez-api/src/main/proto/Events.proto index 71235004da..e041c33f60 100644 --- a/tez-api/src/main/proto/Events.proto +++ b/tez-api/src/main/proto/Events.proto @@ -39,6 +39,8 @@ message InputReadErrorEventProto { optional int32 index = 1; optional string diagnostics = 2; optional int32 version = 3; + optional bool is_local_fetch = 4; + optional bool is_disk_error_at_source = 5; } message InputFailedEventProto { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java index 37e818e271..9a5e73de5a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java @@ -1825,6 +1825,7 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl attempt, + " at inputIndex " + failedInputIndexOnDestTa); long time = attempt.clock.getTime(); Long firstErrReportTime = attempt.uniquefailedOutputReports.get(failedDestTaId); + if (firstErrReportTime == null) { attempt.uniquefailedOutputReports.put(failedDestTaId, time); firstErrReportTime = time; @@ -1851,7 +1852,8 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl attempt, // If needed we can launch a background task without failing this task // to generate a copy of the output just in case. // If needed we can consider only running consumer tasks - if (!crossTimeDeadline && withinFailureFractionLimits && withinOutputFailureLimits) { + if (!crossTimeDeadline && withinFailureFractionLimits && withinOutputFailureLimits + && !(readErrorEvent.isLocalFetch() || readErrorEvent.isDiskErrorAtSource())) { return attempt.getInternalState(); } String message = attempt.getID() + " being failed for too many output errors. " @@ -1862,7 +1864,10 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl attempt, + ", MAX_ALLOWED_OUTPUT_FAILURES=" + maxAllowedOutputFailures + ", MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC=" + maxAllowedTimeForTaskReadErrorSec - + ", readErrorTimespan=" + readErrorTimespanSec; + + ", readErrorTimespan=" + readErrorTimespanSec + + ", isLocalFetch=" + readErrorEvent.isLocalFetch() + + ", isDiskErrorAtSource=" + readErrorEvent.isDiskErrorAtSource(); + LOG.info(message); attempt.addDiagnosticInfo(message); // send input failed event diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java index 41cce3b60e..6862bec2ee 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java @@ -19,7 +19,6 @@ package org.apache.tez.dag.app.dag.impl; import org.apache.tez.dag.app.MockClock; -import org.apache.tez.dag.app.rm.AMSchedulerEventTAStateUpdated; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.fail; @@ -83,6 +82,7 @@ import org.apache.tez.dag.app.TaskCommunicatorWrapper; import org.apache.tez.dag.app.TaskHeartbeatHandler; import org.apache.tez.dag.app.dag.TaskAttemptStateInternal; +import org.apache.tez.dag.app.dag.DAG; import org.apache.tez.dag.app.dag.Task; import org.apache.tez.dag.app.dag.Vertex; import org.apache.tez.dag.app.dag.event.DAGEvent; @@ -127,6 +127,7 @@ import org.junit.BeforeClass; import org.junit.Test; import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -2154,6 +2155,52 @@ taListener, taskConf, new SystemClock(), Assert.assertEquals(1, taImpl.taskAttemptFinishedEventLogged); } + @Test + public void testMapTaskIsBlamedImmediatelyOnLocalFetchFailure() throws ServicePluginException { + // local fetch failure or disk read error at source -> turn source attempt to FAIL_IN_PROGRESS + testMapTaskFailingForFetchFailureType(true, true, TaskAttemptStateInternal.FAIL_IN_PROGRESS); + testMapTaskFailingForFetchFailureType(true, false, TaskAttemptStateInternal.FAIL_IN_PROGRESS); + testMapTaskFailingForFetchFailureType(false, true, TaskAttemptStateInternal.FAIL_IN_PROGRESS); + + // remote fetch failure -> won't change current state + testMapTaskFailingForFetchFailureType(false, false, TaskAttemptStateInternal.NEW); + } + + private void testMapTaskFailingForFetchFailureType(boolean isLocalFetch, + boolean isDiskErrorAtSource, TaskAttemptStateInternal expectedState) { + EventHandler eventHandler = mock(EventHandler.class); + TezTaskID taskID = + TezTaskID.getInstance(TezVertexID.getInstance(TezDAGID.getInstance("1", 1, 1), 1), 1); + TaskAttemptImpl sourceAttempt = new MockTaskAttemptImpl(taskID, 1, eventHandler, null, + new Configuration(), SystemClock.getInstance(), mock(TaskHeartbeatHandler.class), appCtx, + false, null, null, false); + + // the original read error event, sent by reducer task + InputReadErrorEvent inputReadErrorEvent = + InputReadErrorEvent.create("", 0, 1, 1, isLocalFetch, isDiskErrorAtSource); + TezTaskAttemptID destTaskAttemptId = mock(TezTaskAttemptID.class); + when(destTaskAttemptId.getTaskID()).thenReturn(mock(TezTaskID.class)); + when(destTaskAttemptId.getTaskID().getVertexID()).thenReturn(mock(TezVertexID.class)); + when(appCtx.getCurrentDAG()).thenReturn(mock(DAG.class)); + when(appCtx.getCurrentDAG().getVertex(Mockito.any(TezVertexID.class))) + .thenReturn(mock(Vertex.class)); + when(appCtx.getCurrentDAG().getVertex(Mockito.any(TezVertexID.class)).getRunningTasks()) + .thenReturn(100); + + EventMetaData mockMeta = mock(EventMetaData.class); + when(mockMeta.getTaskAttemptID()).thenReturn(destTaskAttemptId); + TezEvent tezEvent = new TezEvent(inputReadErrorEvent, mockMeta); + + // the event is propagated to map task's event handler + TaskAttemptEventOutputFailed outputFailedEvent = + new TaskAttemptEventOutputFailed(sourceAttempt.getID(), tezEvent, 11); + + Assert.assertEquals(TaskAttemptStateInternal.NEW, sourceAttempt.getInternalState()); + TaskAttemptStateInternal resultState = new TaskAttemptImpl.OutputReportedFailedTransition() + .transition(sourceAttempt, outputFailedEvent); + Assert.assertEquals(expectedState, resultState); + } + private Event verifyEventType(List events, Class eventClass, int expectedOccurences) { int count = 0; diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java index f294edced9..b67883dfcf 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java @@ -71,6 +71,7 @@ import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.runtime.library.common.Constants; import org.apache.tez.runtime.library.common.security.SecureShuffleUtils; +import org.apache.tez.runtime.library.common.shuffle.api.ShuffleHandlerError; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.ShuffleHeader; import org.apache.hadoop.metrics2.MetricsSystem; import org.apache.hadoop.metrics2.annotation.Metric; @@ -84,6 +85,7 @@ import org.apache.hadoop.security.token.Token; import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord; import org.apache.hadoop.util.Shell; +import org.apache.hadoop.util.DiskChecker.DiskErrorException; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.VersionProto; @@ -101,6 +103,7 @@ import org.iq80.leveldb.Logger; import org.iq80.leveldb.Options; import org.jboss.netty.bootstrap.ServerBootstrap; +import org.jboss.netty.buffer.ChannelBuffer; import org.jboss.netty.buffer.ChannelBuffers; import org.jboss.netty.channel.Channel; import org.jboss.netty.channel.ChannelFactory; @@ -608,6 +611,10 @@ protected Shuffle getShuffle(Configuration conf) { return new Shuffle(conf); } + protected JobTokenSecretManager getSecretManager() { + return secretManager; + } + private void recoverState(Configuration conf) throws IOException { Path recoveryRoot = getRecoveryPath(); if (recoveryRoot != null) { @@ -727,7 +734,7 @@ private void checkVersion() throws IOException { private void addJobToken(JobID jobId, String user, Token jobToken) { userRsrc.put(jobId.toString(), user); - secretManager.addTokenForJob(jobId.toString(), jobToken); + getSecretManager().addTokenForJob(jobId.toString(), jobToken); LOG.info("Added token for " + jobId.toString()); } @@ -772,7 +779,7 @@ private void recordJobShuffleInfo(JobID jobId, String user, private void removeJobShuffleInfo(JobID jobId) throws IOException { String jobIdStr = jobId.toString(); - secretManager.removeTokenForJob(jobIdStr); + getSecretManager().removeTokenForJob(jobIdStr); userRsrc.remove(jobIdStr); if (stateDb != null) { try { @@ -1080,11 +1087,19 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) try { populateHeaders(mapIds, jobId, dagId, user, reduceRange, response, keepAliveParam, mapOutputInfoMap); - } catch(IOException e) { + } catch (DiskErrorException e) { // fatal error: fetcher should be aware of that + LOG.error("Shuffle error in populating headers (fatal: DiskErrorException):", e); + String errorMessage = getErrorMessage(e); + // custom message, might be noticed by fetchers + // it should reuse the current response object, as headers have been already set for it + sendFakeShuffleHeaderWithError(ctx, + ShuffleHandlerError.DISK_ERROR_EXCEPTION + ": " + errorMessage, response); + return; + } catch (IOException e) { ch.write(response); LOG.error("Shuffle error in populating headers :", e); String errorMessage = getErrorMessage(e); - sendError(ctx,errorMessage , INTERNAL_SERVER_ERROR); + sendError(ctx, errorMessage, INTERNAL_SERVER_ERROR); return; } ch.write(response); @@ -1337,7 +1352,7 @@ public void finish() { protected void verifyRequest(String appid, ChannelHandlerContext ctx, HttpRequest request, HttpResponse response, URL requestUri) throws IOException { - SecretKey tokenSecret = secretManager.retrieveTokenSecret(appid); + SecretKey tokenSecret = getSecretManager().retrieveTokenSecret(appid); if (null == tokenSecret) { LOG.info("Request for unknown token " + appid); throw new IOException("could not find jobid"); @@ -1444,22 +1459,37 @@ public void operationComplete(ChannelFuture future) { return writeFuture; } - protected void sendError(ChannelHandlerContext ctx, - HttpResponseStatus status) { + protected void sendError(ChannelHandlerContext ctx, HttpResponseStatus status) { sendError(ctx, "", status); } - protected void sendError(ChannelHandlerContext ctx, String message, - HttpResponseStatus status) { + protected void sendError(ChannelHandlerContext ctx, String message, HttpResponseStatus status) { HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status); + sendError(ctx, message, response); + } + + protected void sendError(ChannelHandlerContext ctx, String message, HttpResponse response) { + sendError(ctx, ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8), response); + } + + private void sendFakeShuffleHeaderWithError(ChannelHandlerContext ctx, String message, + HttpResponse response) throws IOException { + ShuffleHeader header = new ShuffleHeader(message, -1, -1, -1); + DataOutputBuffer out = new DataOutputBuffer(); + header.write(out); + + sendError(ctx, wrappedBuffer(out.getData(), 0, out.getLength()), response); + } + + protected void sendError(ChannelHandlerContext ctx, ChannelBuffer content, + HttpResponse response) { response.headers().set(CONTENT_TYPE, "text/plain; charset=UTF-8"); // Put shuffle version into http header response.headers().set(ShuffleHeader.HTTP_HEADER_NAME, ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); response.headers().set(ShuffleHeader.HTTP_HEADER_VERSION, ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); - response.setContent( - ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8)); + response.setContent(content); // Close the connection as soon as the error message is sent. ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE); diff --git a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java index a7f444680b..5ca4ed8e26 100644 --- a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java @@ -20,6 +20,7 @@ //import static org.apache.hadoop.test.MetricsAsserts.assertCounter; //import static org.apache.hadoop.test.MetricsAsserts.assertGauge; //import static org.apache.hadoop.test.MetricsAsserts.getMetrics; +import org.apache.hadoop.util.DiskChecker.DiskErrorException; import static org.junit.Assert.assertTrue; import static org.jboss.netty.buffer.ChannelBuffers.wrappedBuffer; import static org.jboss.netty.handler.codec.http.HttpResponseStatus.OK; @@ -41,15 +42,12 @@ import java.net.URL; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Map; -import java.util.zip.CheckedOutputStream; import java.util.zip.Checksum; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; -import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; @@ -58,11 +56,14 @@ import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.io.nativeio.NativeIO; import org.apache.hadoop.mapred.JobID; -import org.apache.hadoop.mapred.MapTask; import org.apache.hadoop.mapreduce.TypeConverter; import org.apache.tez.runtime.library.common.security.SecureShuffleUtils; import org.apache.tez.common.security.JobTokenIdentifier; import org.apache.tez.common.security.JobTokenSecretManager; +import org.apache.tez.http.BaseHttpConnection; +import org.apache.tez.http.HttpConnectionParams; +import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; +import org.apache.tez.runtime.library.common.shuffle.api.ShuffleHandlerError; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.ShuffleHeader; import org.apache.hadoop.metrics2.MetricsSystem; import org.apache.hadoop.metrics2.impl.MetricsSystemImpl; @@ -172,6 +173,52 @@ protected boolean isSocketKeepAlive() { } } + class MockShuffleHandlerWithFatalDiskError extends org.apache.tez.auxservices.ShuffleHandler { + public static final String MESSAGE = + "Could not find application_1234/240/output/attempt_1234_0/file.out.index"; + + private JobTokenSecretManager secretManager = + new JobTokenSecretManager(JobTokenSecretManager.createSecretKey(getSecret().getBytes())); + + protected JobTokenSecretManager getSecretManager(){ + return secretManager; + } + + @Override + protected Shuffle getShuffle(final Configuration conf) { + return new Shuffle(conf) { + @Override + protected void verifyRequest(String appid, ChannelHandlerContext ctx, HttpRequest request, + HttpResponse response, URL requestUri) throws IOException { + super.verifyRequest(appid, ctx, request, response, requestUri); + } + + @Override + protected MapOutputInfo getMapOutputInfo(String dagId, String mapId, Range reduceRange, + String jobId, String user) { + return null; + } + + @Override + protected void populateHeaders(List mapIds, String jobId, String dagId, String user, + Range reduceRange, HttpResponse response, boolean keepAliveParam, + Map infoMap) throws IOException { + throw new DiskErrorException(MESSAGE); + } + + @Override + protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch, String user, + String mapId, Range reduceRange, MapOutputInfo info) throws IOException { + return null; + } + }; + } + + public String getSecret() { + return "secret"; + } + } + /** * Test the validation of ShuffleHandler's meta-data's serialization and * de-serialization. @@ -1322,6 +1369,53 @@ public void testSendMapCount() throws Exception { sh.close(); } + @Test + public void testShuffleHandlerSendsDiskError() throws Exception { + Configuration conf = new Configuration(); + conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); + + DataInputStream input = null; + MockShuffleHandlerWithFatalDiskError shuffleHandler = + new MockShuffleHandlerWithFatalDiskError(); + try { + shuffleHandler.init(conf); + shuffleHandler.start(); + + String shuffleBaseURL = "http://127.0.0.1:" + + shuffleHandler.getConfig().get(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY); + URL url = new URL( + shuffleBaseURL + "/mapOutput?job=job_12345_1&dag=1&reduce=1&map=attempt_12345_1_m_1_0"); + shuffleHandler.secretManager.addTokenForJob("job_12345_1", + new Token<>("id".getBytes(), shuffleHandler.getSecret().getBytes(), null, null)); + + HttpConnectionParams httpConnectionParams = ShuffleUtils.getHttpConnectionParams(conf); + BaseHttpConnection httpConnection = ShuffleUtils.getHttpConnection(true, url, + httpConnectionParams, "testFetcher", shuffleHandler.secretManager); + + boolean connectSucceeded = httpConnection.connect(); + Assert.assertTrue(connectSucceeded); + + input = httpConnection.getInputStream(); + httpConnection.validate(); + + ShuffleHeader header = new ShuffleHeader(); + header.readFields(input); + + // message is encoded in the shuffle header, and can be checked by fetchers + Assert.assertEquals( + ShuffleHandlerError.DISK_ERROR_EXCEPTION + ": " + MockShuffleHandlerWithFatalDiskError.MESSAGE, + header.getMapId()); + Assert.assertEquals(-1, header.getCompressedLength()); + Assert.assertEquals(-1, header.getUncompressedLength()); + Assert.assertEquals(-1, header.getPartition()); + } finally { + if (input != null) { + input.close(); + } + shuffleHandler.close(); + } + } + public ChannelFuture createMockChannelFuture(Channel mockCh, final List listenerList) { final ChannelFuture mockFuture = mock(ChannelFuture.class); diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java index e7af4a1ebe..ebea9a4f3f 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java @@ -191,6 +191,8 @@ private void serializeEvent(DataOutput out) throws IOException { .setIndex(ideEvt.getIndex()) .setDiagnostics(ideEvt.getDiagnostics()) .setVersion(ideEvt.getVersion()) + .setIsLocalFetch(ideEvt.isLocalFetch()) + .setIsDiskErrorAtSource(ideEvt.isDiskErrorAtSource()) .build(); break; case TASK_ATTEMPT_FAILED_EVENT: @@ -294,10 +296,9 @@ private void deserializeEvent(DataInput in) throws IOException { event = ProtoConverters.convertVertexManagerEventFromProto(vmProto); break; case INPUT_READ_ERROR_EVENT: - InputReadErrorEventProto ideProto = - InputReadErrorEventProto.parseFrom(input); - event = InputReadErrorEvent.create(ideProto.getDiagnostics(), - ideProto.getIndex(), ideProto.getVersion()); + InputReadErrorEventProto ideProto = InputReadErrorEventProto.parseFrom(input); + event = InputReadErrorEvent.create(ideProto.getDiagnostics(), ideProto.getIndex(), + ideProto.getVersion(), ideProto.getIsLocalFetch(), ideProto.getIsDiskErrorAtSource()); break; case TASK_ATTEMPT_FAILED_EVENT: TaskAttemptFailedEventProto tfProto = diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java index 18f66cc8ef..f295c06b8e 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java @@ -66,7 +66,7 @@ import org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord; import org.apache.tez.runtime.library.exceptions.FetcherReadTimeoutException; import org.apache.tez.runtime.library.common.shuffle.FetchedInput.Type; - +import org.apache.tez.runtime.library.common.shuffle.api.ShuffleHandlerError; import org.apache.tez.common.Preconditions; /** @@ -277,7 +277,8 @@ public FetchResult callInternal() throws Exception { HostFetchResult hostFetchResult; - if (localDiskFetchEnabled && host.equals(localHostname) && port == shufflePort) { + boolean isLocalFetch = localDiskFetchEnabled && host.equals(localHostname) && port == shufflePort; + if (isLocalFetch) { hostFetchResult = setupLocalDiskFetch(); } else if (multiplex) { hostFetchResult = doSharedFetch(); @@ -288,7 +289,7 @@ public FetchResult callInternal() throws Exception { if (hostFetchResult.failedInputs != null && hostFetchResult.failedInputs.length > 0) { if (!isShutDown.get()) { LOG.warn("copyInputs failed for tasks " + Arrays.toString(hostFetchResult.failedInputs)); - for (InputAttemptIdentifier left : hostFetchResult.failedInputs) { + for (InputAttemptFetchFailure left : hostFetchResult.failedInputs) { fetcherCallback.fetchFailed(host, left, hostFetchResult.connectFailed); } } else { @@ -504,7 +505,7 @@ private HostFetchResult setupConnection(Collection attem // ioErrs.increment(1); // If connect did not succeed, just mark all the maps as failed, // indirectly penalizing the host - InputAttemptIdentifier[] failedFetches = null; + InputAttemptFetchFailure[] failedFetches = null; if (isShutDown.get()) { if (isDebugEnabled) { LOG.debug( @@ -512,8 +513,7 @@ private HostFetchResult setupConnection(Collection attem e.getClass().getName() + ", Message: " + e.getMessage()); } } else { - failedFetches = srcAttemptsRemaining.values(). - toArray(new InputAttemptIdentifier[srcAttemptsRemaining.values().size()]); + failedFetches = InputAttemptFetchFailure.fromAttempts(srcAttemptsRemaining.values()); } return new HostFetchResult(new FetchResult(host, port, partition, partitionCount, srcAttemptsRemaining.values()), failedFetches, true); } @@ -547,7 +547,7 @@ private HostFetchResult setupConnection(Collection attem "Fetch Failure while connecting from %s to: %s:%d, attempt: %s Informing ShuffleManager: ", localHostname, host, port, firstAttempt), e); return new HostFetchResult(new FetchResult(host, port, partition, partitionCount, srcAttemptsRemaining.values()), - new InputAttemptIdentifier[] { firstAttempt }, true); + new InputAttemptFetchFailure[] { new InputAttemptFetchFailure(firstAttempt) }, true); } } catch (InterruptedException e) { Thread.currentThread().interrupt(); //reset status @@ -584,7 +584,7 @@ protected HostFetchResult doHttpFetch(CachingCallBack callback) { // On any error, faildTasks is not null and we exit // after putting back the remaining maps to the // yet_to_be_fetched list and marking the failed tasks. - InputAttemptIdentifier[] failedInputs = null; + InputAttemptFetchFailure[] failedInputs = null; while (!srcAttemptsRemaining.isEmpty() && failedInputs == null) { InputAttemptIdentifier inputAttemptIdentifier = srcAttemptsRemaining.entrySet().iterator().next().getValue(); @@ -711,7 +711,7 @@ public void freeResources(FetchedInput fetchedInput) { } } - InputAttemptIdentifier[] failedFetches = null; + InputAttemptFetchFailure[] failedFetches = null; if (failMissing && srcAttemptsRemaining.size() > 0) { if (isShutDown.get()) { if (isDebugEnabled) { @@ -720,8 +720,8 @@ public void freeResources(FetchedInput fetchedInput) { " remaining inputs"); } } else { - failedFetches = srcAttemptsRemaining.values(). - toArray(new InputAttemptIdentifier[srcAttemptsRemaining.values().size()]); + failedFetches = + InputAttemptFetchFailure.fromAttemptsLocalFetchFailure(srcAttemptsRemaining.values()); } } else { // nothing needs to be done to requeue remaining entries @@ -770,10 +770,10 @@ public Map getPathToAttemptMap() { static class HostFetchResult { private final FetchResult fetchResult; - private final InputAttemptIdentifier[] failedInputs; + private final InputAttemptFetchFailure[] failedInputs; private final boolean connectFailed; - public HostFetchResult(FetchResult fetchResult, InputAttemptIdentifier[] failedInputs, + public HostFetchResult(FetchResult fetchResult, InputAttemptFetchFailure[] failedInputs, boolean connectFailed) { this.fetchResult = fetchResult; this.failedInputs = failedInputs; @@ -831,8 +831,11 @@ public String toString() { return "id: " + srcAttemptId + ", decompressed length: " + decompressedLength + ", compressed length: " + compressedLength + ", reduce: " + forReduce; } } - private InputAttemptIdentifier[] fetchInputs(DataInputStream input, - CachingCallBack callback, InputAttemptIdentifier inputAttemptIdentifier) throws FetcherReadTimeoutException { + + @VisibleForTesting + InputAttemptFetchFailure[] fetchInputs(DataInputStream input, CachingCallBack callback, + InputAttemptIdentifier inputAttemptIdentifier) + throws FetcherReadTimeoutException { FetchedInput fetchedInput = null; InputAttemptIdentifier srcAttemptId = null; long decompressedLength = 0; @@ -856,9 +859,19 @@ private InputAttemptIdentifier[] fetchInputs(DataInputStream input, header.readFields(input); pathComponent = header.getMapId(); if (!pathComponent.startsWith(InputAttemptIdentifier.PATH_PREFIX)) { - throw new IllegalArgumentException("Invalid map id: " + header.getMapId() + ", expected to start with " + - InputAttemptIdentifier.PATH_PREFIX + ", partition: " + header.getPartition() - + " while fetching " + inputAttemptIdentifier); + if (pathComponent.startsWith(ShuffleHandlerError.DISK_ERROR_EXCEPTION.toString())) { + LOG.warn("Invalid map id: " + header.getMapId() + ", expected to start with " + + InputAttemptIdentifier.PATH_PREFIX + ", partition: " + header.getPartition() + + " while fetching " + inputAttemptIdentifier); + // this should be treated as local fetch failure while reporting later + return new InputAttemptFetchFailure[] { + InputAttemptFetchFailure.fromDiskErrorAtSource(inputAttemptIdentifier) }; + } else { + throw new IllegalArgumentException( + "Invalid map id: " + header.getMapId() + ", expected to start with " + + InputAttemptIdentifier.PATH_PREFIX + ", partition: " + header.getPartition() + + " while fetching " + inputAttemptIdentifier); + } } srcAttemptId = pathToAttemptMap.get(new PathPartition(pathComponent, header.getPartition())); @@ -883,7 +896,7 @@ private InputAttemptIdentifier[] fetchInputs(DataInputStream input, if (!isShutDown.get()) { LOG.warn("Invalid src id ", e); // Don't know which one was bad, so consider all of them as bad - return srcAttemptsRemaining.values().toArray(new InputAttemptIdentifier[srcAttemptsRemaining.size()]); + return InputAttemptFetchFailure.fromAttempts(srcAttemptsRemaining.values()); } else { if (isDebugEnabled) { LOG.debug("Already shutdown. Ignoring badId error with message: " + e.getMessage()); @@ -902,7 +915,8 @@ private InputAttemptIdentifier[] fetchInputs(DataInputStream input, srcAttemptId = getNextRemainingAttempt(); } assert (srcAttemptId != null); - return new InputAttemptIdentifier[]{srcAttemptId}; + return new InputAttemptFetchFailure[] { + InputAttemptFetchFailure.fromAttempt(srcAttemptId) }; } else { if (isDebugEnabled) { LOG.debug("Already shutdown. Ignoring verification failure."); @@ -1004,10 +1018,10 @@ private InputAttemptIdentifier[] fetchInputs(DataInputStream input, // Cleanup the fetchedInput before returning. cleanupFetchedInput(fetchedInput); if (srcAttemptId == null) { - return srcAttemptsRemaining.values() - .toArray(new InputAttemptIdentifier[srcAttemptsRemaining.size()]); + return InputAttemptFetchFailure.fromAttempts(srcAttemptsRemaining.values()); } else { - return new InputAttemptIdentifier[] { srcAttemptId }; + return new InputAttemptFetchFailure[] { + new InputAttemptFetchFailure(srcAttemptId) }; } } LOG.warn("Failed to shuffle output of " + srcAttemptId + " from " + host + " (to " @@ -1016,7 +1030,8 @@ private InputAttemptIdentifier[] fetchInputs(DataInputStream input, // Cleanup the fetchedInput cleanupFetchedInput(fetchedInput); // metrics.failedFetch(); - return new InputAttemptIdentifier[] { srcAttemptId }; + return new InputAttemptFetchFailure[] { + new InputAttemptFetchFailure(srcAttemptId) }; } return null; } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherCallback.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherCallback.java index 34bd272909..b751fb9ce0 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherCallback.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherCallback.java @@ -28,6 +28,7 @@ public void fetchSucceeded(String host, InputAttemptIdentifier srcAttemptIdentif FetchedInput fetchedInput, long fetchedBytes, long decompressedLength, long copyDuration) throws IOException; - public void fetchFailed(String host, InputAttemptIdentifier srcAttemptIdentifier, boolean connectFailed); + public void fetchFailed(String host, InputAttemptFetchFailure srcAttemptFetchFailure, + boolean connectFailed); } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/InputAttemptFetchFailure.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/InputAttemptFetchFailure.java new file mode 100644 index 0000000000..d94db35c2f --- /dev/null +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/InputAttemptFetchFailure.java @@ -0,0 +1,115 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.runtime.library.common.shuffle; + +import java.util.Arrays; +import java.util.Collection; + +import org.apache.tez.runtime.library.common.CompositeInputAttemptIdentifier; +import org.apache.tez.runtime.library.common.InputAttemptIdentifier; + +/** + * InputAttemptFetchFailure is supposed to wrap an InputAttemptIdentifier with any kind of failure + * information during fetch. It can be useful for propagating as a single object instead of multiple + * parameters (local fetch error, remote fetch error, connect failed, read failed, etc.). + */ +public class InputAttemptFetchFailure { + + private final InputAttemptIdentifier inputAttemptIdentifier; + private final boolean isLocalFetch; + private final boolean isDiskErrorAtSource; + + public InputAttemptFetchFailure(InputAttemptIdentifier inputAttemptIdentifier) { + this(inputAttemptIdentifier, false, false); + } + + public InputAttemptFetchFailure(InputAttemptIdentifier inputAttemptIdentifier, + boolean isLocalFetch, boolean isDiskErrorAtSource) { + this.inputAttemptIdentifier = inputAttemptIdentifier; + this.isLocalFetch = isLocalFetch; + this.isDiskErrorAtSource = isDiskErrorAtSource; + } + + public InputAttemptIdentifier getInputAttemptIdentifier() { + return inputAttemptIdentifier; + } + + public boolean isLocalFetch() { + return isLocalFetch; + } + + public boolean isDiskErrorAtSource() { + return isDiskErrorAtSource; + } + + public static InputAttemptFetchFailure fromAttempt(InputAttemptIdentifier attempt) { + return new InputAttemptFetchFailure(attempt, false, false); + } + + public static InputAttemptFetchFailure fromLocalFetchFailure(InputAttemptIdentifier attempt) { + return new InputAttemptFetchFailure(attempt, true, false); + } + + public static InputAttemptFetchFailure fromDiskErrorAtSource(InputAttemptIdentifier attempt) { + return new InputAttemptFetchFailure(attempt, false, true); + } + + public static InputAttemptFetchFailure[] fromAttempts(Collection values) { + return values.stream().map(identifier -> new InputAttemptFetchFailure(identifier, false, false)) + .toArray(InputAttemptFetchFailure[]::new); + } + + public static InputAttemptFetchFailure[] fromAttempts(InputAttemptIdentifier[] values) { + return Arrays.asList(values).stream() + .map(identifier -> new InputAttemptFetchFailure(identifier, false, false)) + .toArray(InputAttemptFetchFailure[]::new); + } + + public static InputAttemptFetchFailure[] fromAttemptsLocalFetchFailure( + Collection values) { + return values.stream().map(identifier -> new InputAttemptFetchFailure(identifier, true, false)) + .toArray(InputAttemptFetchFailure[]::new); + } + + public static InputAttemptFetchFailure fromCompositeAttemptLocalFetchFailure( + CompositeInputAttemptIdentifier compositeInputAttemptIdentifier) { + return new InputAttemptFetchFailure(compositeInputAttemptIdentifier, true, false); + } + + @Override + public boolean equals(Object obj) { + if (obj == null || (obj.getClass() != this.getClass())) { + return false; + } + return inputAttemptIdentifier.equals(((InputAttemptFetchFailure) obj).inputAttemptIdentifier) + && isLocalFetch == ((InputAttemptFetchFailure) obj).isLocalFetch + && isDiskErrorAtSource == ((InputAttemptFetchFailure) obj).isDiskErrorAtSource; + } + + @Override + public int hashCode() { + return 31 * inputAttemptIdentifier.hashCode() + 31 * (isLocalFetch ? 0 : 1) + + 31 * (isDiskErrorAtSource ? 0 : 1); + } + + @Override + public String toString() { + return String.format("%s, isLocalFetch: %s, isDiskErrorAtSource: %s", + inputAttemptIdentifier.toString(), isLocalFetch, isDiskErrorAtSource); + } +} diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java index 742fc18632..2b83ad8920 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java @@ -84,6 +84,7 @@ import org.apache.tez.runtime.library.common.shuffle.Fetcher.FetcherBuilder; import org.apache.tez.runtime.library.common.shuffle.FetcherCallback; import org.apache.tez.runtime.library.common.shuffle.HostPort; +import org.apache.tez.runtime.library.common.shuffle.InputAttemptFetchFailure; import org.apache.tez.runtime.library.common.shuffle.InputHost; import org.apache.tez.runtime.library.common.shuffle.InputHost.PartitionToInputs; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; @@ -391,9 +392,9 @@ protected Void callInternal() throws Exception { List failedEventsToSend = Lists.newArrayListWithCapacity( failedEvents.size()); for (InputReadErrorEvent key : failedEvents.keySet()) { - failedEventsToSend.add(InputReadErrorEvent - .create(key.getDiagnostics(), key.getIndex(), - key.getVersion(), failedEvents.get(key))); + failedEventsToSend.add(InputReadErrorEvent.create(key.getDiagnostics(), + key.getIndex(), key.getVersion(), failedEvents.get(key), key.isLocalFetch(), + key.isDiskErrorAtSource())); } inputContext.sendEvents(failedEventsToSend); failedEvents.clear(); @@ -939,12 +940,15 @@ private void reportFatalError(Throwable exception, String message) { @Override public void fetchFailed(String host, - InputAttemptIdentifier srcAttemptIdentifier, boolean connectFailed) { + InputAttemptFetchFailure inputAttemptFetchFailure, boolean connectFailed) { // TODO NEWTEZ. Implement logic to report fetch failures after a threshold. // For now, reporting immediately. - LOG.info(srcNameTrimmed + ": " + "Fetch failed for src: " + srcAttemptIdentifier - + "InputIdentifier: " + srcAttemptIdentifier + ", connectFailed: " - + connectFailed); + InputAttemptIdentifier srcAttemptIdentifier = inputAttemptFetchFailure.getInputAttemptIdentifier(); + LOG.info( + "{}: Fetch failed for src: {} InputIdentifier: {}, connectFailed: {}, " + + "local fetch: {}, remote fetch failure reported as local failure: {})", + srcNameTrimmed, srcAttemptIdentifier, srcAttemptIdentifier, connectFailed, + inputAttemptFetchFailure.isLocalFetch(), inputAttemptFetchFailure.isDiskErrorAtSource()); failedShufflesCounter.increment(1); inputContext.notifyProgress(); if (srcAttemptIdentifier == null) { @@ -957,7 +961,9 @@ public void fetchFailed(String host, srcAttemptIdentifier.getInputIdentifier(), srcAttemptIdentifier.getAttemptNumber()), srcAttemptIdentifier.getInputIdentifier(), - srcAttemptIdentifier.getAttemptNumber()); + srcAttemptIdentifier.getAttemptNumber(), + inputAttemptFetchFailure.isLocalFetch(), + inputAttemptFetchFailure.isDiskErrorAtSource()); if (maxTimeToWaitForReportMillis > 0) { try { reportLock.lock(); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java index e732ab0063..327232710b 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java @@ -50,7 +50,9 @@ import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord; import org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord; import org.apache.tez.runtime.library.exceptions.FetcherReadTimeoutException; +import org.apache.tez.runtime.library.common.shuffle.InputAttemptFetchFailure; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; +import org.apache.tez.runtime.library.common.shuffle.api.ShuffleHandlerError; import com.google.common.annotations.VisibleForTesting; @@ -272,7 +274,8 @@ protected void copyFromHost(MapHost host) throws IOException { // On any error, faildTasks is not null and we exit // after putting back the remaining maps to the // yet_to_be_fetched list and marking the failed tasks. - InputAttemptIdentifier[] failedTasks = null; + InputAttemptFetchFailure[] failedTasks = null; + while (!remaining.isEmpty() && failedTasks == null) { InputAttemptIdentifier inputAttemptIdentifier = remaining.entrySet().iterator().next().getValue(); @@ -300,25 +303,14 @@ protected void copyFromHost(MapHost host) throws IOException { } return; } - failedTasks = new InputAttemptIdentifier[] {getNextRemainingAttempt()}; + failedTasks = new InputAttemptFetchFailure[] { + new InputAttemptFetchFailure(getNextRemainingAttempt()) }; break; } } } - if (failedTasks != null && failedTasks.length > 0) { - if (stopped) { - if (LOG.isDebugEnabled()) { - LOG.debug("Ignoring copyMapOutput failures for tasks: " + Arrays.toString(failedTasks) + - " since Fetcher has been stopped"); - } - } else { - LOG.warn("copyMapOutput failed for tasks " + Arrays.toString(failedTasks)); - for (InputAttemptIdentifier left : failedTasks) { - scheduler.copyFailed(left, host, true, false, false); - } - } - } + invokeCopyFailedForFailedTasks(host, failedTasks); cleanupCurrentConnection(false); @@ -332,6 +324,23 @@ protected void copyFromHost(MapHost host) throws IOException { } } + private void invokeCopyFailedForFailedTasks(MapHost host, + InputAttemptFetchFailure[] failedTasks) { + if (failedTasks != null && failedTasks.length > 0) { + if (stopped) { + if (LOG.isDebugEnabled()) { + LOG.debug("Ignoring copyMapOutput failures for tasks: " + Arrays.toString(failedTasks) + + " since Fetcher has been stopped"); + } + } else { + LOG.warn("copyMapOutput failed for tasks " + Arrays.toString(failedTasks)); + for (InputAttemptFetchFailure left : failedTasks) { + scheduler.copyFailed(left, host, true, false); + } + } + } + } + @VisibleForTesting boolean setupConnection(MapHost host, Collection attempts) throws IOException { @@ -380,7 +389,8 @@ boolean setupConnection(MapHost host, Collection attempt for (InputAttemptIdentifier left : remaining.values()) { // Need to be handling temporary glitches .. // Report read error to the AM to trigger source failure heuristics - scheduler.copyFailed(left, host, connectSucceeded, !connectSucceeded, false); + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(left), host, connectSucceeded, + !connectSucceeded); } return false; } @@ -404,7 +414,8 @@ protected void putBackRemainingMapOutputs(MapHost host) { } } - private static InputAttemptIdentifier[] EMPTY_ATTEMPT_ID_ARRAY = new InputAttemptIdentifier[0]; + private static final InputAttemptFetchFailure[] EMPTY_ATTEMPT_ID_ARRAY = + new InputAttemptFetchFailure[0]; private static class MapOutputStat { final InputAttemptIdentifier srcAttemptId; @@ -425,8 +436,8 @@ public String toString() { } } - protected InputAttemptIdentifier[] copyMapOutput(MapHost host, - DataInputStream input, InputAttemptIdentifier inputAttemptIdentifier) throws FetcherReadTimeoutException { + protected InputAttemptFetchFailure[] copyMapOutput(MapHost host, DataInputStream input, + InputAttemptIdentifier inputAttemptIdentifier) throws FetcherReadTimeoutException { MapOutput mapOutput = null; InputAttemptIdentifier srcAttemptId = null; long decompressedLength = 0; @@ -452,7 +463,13 @@ protected InputAttemptIdentifier[] copyMapOutput(MapHost host, badIdErrs.increment(1); LOG.warn("Invalid map id: " + header.mapId + ", expected to start with " + InputAttemptIdentifier.PATH_PREFIX + ", partition: " + header.forReduce); - return new InputAttemptIdentifier[]{getNextRemainingAttempt()}; + if (header.mapId.startsWith(ShuffleHandlerError.DISK_ERROR_EXCEPTION.toString())) { + //this should be treated as local fetch failure while reporting later + return new InputAttemptFetchFailure[] { + InputAttemptFetchFailure.fromDiskErrorAtSource(getNextRemainingAttempt()) }; + } + return new InputAttemptFetchFailure[] { + InputAttemptFetchFailure.fromAttempt(getNextRemainingAttempt()) }; } else { if (LOG.isDebugEnabled()) { LOG.debug("Already shutdown. Ignoring invalid map id error"); @@ -477,7 +494,8 @@ protected InputAttemptIdentifier[] copyMapOutput(MapHost host, LOG.warn("Invalid map id ", e); // Don't know which one was bad, so consider this one bad and dont read // the remaining because we dont know where to start reading from. YARN-1773 - return new InputAttemptIdentifier[]{getNextRemainingAttempt()}; + return new InputAttemptFetchFailure[] { + new InputAttemptFetchFailure(getNextRemainingAttempt()) }; } else { if (LOG.isDebugEnabled()) { LOG.debug("Already shutdown. Ignoring invalid map id error. Exception: " + @@ -497,7 +515,8 @@ protected InputAttemptIdentifier[] copyMapOutput(MapHost host, LOG.warn("Was expecting " + srcAttemptId + " but got null"); } assert (srcAttemptId != null); - return new InputAttemptIdentifier[]{srcAttemptId}; + return new InputAttemptFetchFailure[] { + new InputAttemptFetchFailure(getNextRemainingAttempt()) }; } else { if (LOG.isDebugEnabled()) { LOG.debug("Already stopped. Ignoring verification failure."); @@ -595,9 +614,10 @@ protected InputAttemptIdentifier[] copyMapOutput(MapHost host, srcAttemptId + " decomp: " + decompressedLength + ", " + compressedLength, ioe); if (srcAttemptId == null) { - return remaining.values().toArray(new InputAttemptIdentifier[remaining.values().size()]); + return InputAttemptFetchFailure.fromAttempts(remaining.values()); } else { - return new InputAttemptIdentifier[]{srcAttemptId}; + return new InputAttemptFetchFailure[] { + new InputAttemptFetchFailure(srcAttemptId) }; } } LOG.warn("Failed to shuffle output of " + srcAttemptId + @@ -605,7 +625,8 @@ protected InputAttemptIdentifier[] copyMapOutput(MapHost host, // Inform the shuffle-scheduler mapOutput.abort(); - return new InputAttemptIdentifier[] {srcAttemptId}; + return new InputAttemptFetchFailure[] { + new InputAttemptFetchFailure(srcAttemptId) }; } return null; } @@ -734,7 +755,8 @@ protected void setupLocalDiskFetch(MapHost host) throws InterruptedException { if (!stopped) { hasFailures = true; ioErrs.increment(1); - scheduler.copyFailed(srcAttemptId, host, true, false, true); + scheduler.copyFailed(InputAttemptFetchFailure.fromLocalFetchFailure(srcAttemptId), + host, true, false); LOG.warn("Failed to read local disk output of " + srcAttemptId + " from " + host.getHostIdentifier(), e); } else { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleHeader.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleHeader.java index 9f883dbfdc..f074e897e8 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleHeader.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleHeader.java @@ -103,4 +103,11 @@ public void write(DataOutput out) throws IOException { WritableUtils.writeVLong(out, uncompressedLength); WritableUtils.writeVInt(out, forReduce); } + + @Override + public String toString() { + return String.format( + "ShuffleHeader [mapId=%s, uncompressedLength=%d, compressedLength=%d, forReduce=%d]", mapId, + uncompressedLength, compressedLength, forReduce); + } } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java index 0954a76ec9..e7f63ab7ad 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java @@ -80,6 +80,7 @@ import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils.FetchStatsLogger; import org.apache.tez.runtime.library.common.shuffle.HostPort; +import org.apache.tez.runtime.library.common.shuffle.InputAttemptFetchFailure; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.MapHost.HostPortPartition; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.MapOutput.Type; @@ -755,16 +756,13 @@ private void logProgress() { } } - public synchronized void copyFailed(InputAttemptIdentifier srcAttempt, - MapHost host, - boolean readError, - boolean connectError, - boolean isLocalFetch) { + public synchronized void copyFailed(InputAttemptFetchFailure fetchFailure, MapHost host, + boolean readError, boolean connectError) { failedShuffleCounter.increment(1); inputContext.notifyProgress(); - int failures = incrementAndGetFailureAttempt(srcAttempt); + int failures = incrementAndGetFailureAttempt(fetchFailure.getInputAttemptIdentifier()); - if (!isLocalFetch) { + if (!fetchFailure.isLocalFetch()) { /** * Track the number of failures that has happened since last completion. * This gets reset on a successful copy. @@ -789,11 +787,11 @@ public synchronized void copyFailed(InputAttemptIdentifier srcAttempt, if (shouldInformAM) { //Inform AM. In case producer needs to be restarted, it is handled at AM. - informAM(srcAttempt); + informAM(fetchFailure); } //Restart consumer in case shuffle is not healthy - if (!isShuffleHealthy(srcAttempt)) { + if (!isShuffleHealthy(fetchFailure.getInputAttemptIdentifier())) { return; } @@ -868,21 +866,24 @@ public void reportLocalError(IOException ioe) { } // Notify AM - private void informAM(InputAttemptIdentifier srcAttempt) { + private void informAM(InputAttemptFetchFailure fetchFailure) { + InputAttemptIdentifier srcAttempt = fetchFailure.getInputAttemptIdentifier(); LOG.info( - srcNameTrimmed + ": " + "Reporting fetch failure for InputIdentifier: " - + srcAttempt + " taskAttemptIdentifier: " + TezRuntimeUtils - .getTaskAttemptIdentifier(inputContext.getSourceVertexName(), - srcAttempt.getInputIdentifier(), - srcAttempt.getAttemptNumber()) + " to AM."); + "{}: Reporting fetch failure for InputIdentifier: {} taskAttemptIdentifier: {}, " + + "local fetch: {}, remote fetch failure reported as local failure: {}) to AM.", + srcNameTrimmed, srcAttempt, + TezRuntimeUtils.getTaskAttemptIdentifier(inputContext.getSourceVertexName(), + srcAttempt.getInputIdentifier(), srcAttempt.getAttemptNumber()), + fetchFailure.isLocalFetch(), fetchFailure.isDiskErrorAtSource()); List failedEvents = Lists.newArrayListWithCapacity(1); - failedEvents.add(InputReadErrorEvent.create( - "Fetch failure for " + TezRuntimeUtils - .getTaskAttemptIdentifier(inputContext.getSourceVertexName(), - srcAttempt.getInputIdentifier(), - srcAttempt.getAttemptNumber()) + " to jobtracker.", - srcAttempt.getInputIdentifier(), - srcAttempt.getAttemptNumber())); + failedEvents.add( + InputReadErrorEvent.create( + "Fetch failure for " + + TezRuntimeUtils.getTaskAttemptIdentifier(inputContext.getSourceVertexName(), + srcAttempt.getInputIdentifier(), srcAttempt.getAttemptNumber()) + + " to jobtracker.", + srcAttempt.getInputIdentifier(), srcAttempt.getAttemptNumber(), + fetchFailure.isLocalFetch(), fetchFailure.isDiskErrorAtSource())); inputContext.sendEvents(failedEvents); } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java index db9c7afad0..05d4eb4145 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java @@ -32,6 +32,7 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; +import java.io.DataInputStream; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -46,10 +47,16 @@ import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.runtime.api.InputContext; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; +import org.apache.tez.runtime.library.common.shuffle.api.ShuffleHandlerError; +import org.apache.tez.runtime.library.common.shuffle.impl.ShuffleManager; +import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.ShuffleHeader; import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord; +import org.apache.tez.runtime.library.testutils.RuntimeTestUtils; import org.junit.Assert; import org.junit.Test; import org.mockito.ArgumentCaptor; @@ -79,7 +86,8 @@ public void testLocalFetchModeSetting() throws Exception { Fetcher fetcher = spy(builder.build()); FetchResult fr = new FetchResult(HOST, PORT, 0, 1, Arrays.asList(srcAttempts)); - Fetcher.HostFetchResult hfr = new Fetcher.HostFetchResult(fr, srcAttempts, false); + Fetcher.HostFetchResult hfr = + new Fetcher.HostFetchResult(fr, InputAttemptFetchFailure.fromAttempts(srcAttempts), false); doReturn(hfr).when(fetcher).setupLocalDiskFetch(); doReturn(null).when(fetcher).doHttpFetch(); doNothing().when(fetcher).shutdown(); @@ -151,7 +159,7 @@ public void testSetupLocalDiskFetch() throws Exception { }; final int FIRST_FAILED_ATTEMPT_IDX = 2; final int SECOND_FAILED_ATTEMPT_IDX = 4; - final int[] sucessfulAttempts = {0, 1, 3}; + final int[] successfulAttempts = {0, 1, 3}; TezConfiguration conf = new TezConfiguration(); conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, "true"); @@ -206,18 +214,24 @@ public TezIndexRecord answer(InvocationOnMock invocation) throws Throwable { doNothing().when(fetcher).shutdown(); doNothing().when(callback).fetchSucceeded(anyString(), any(InputAttemptIdentifier.class), any(FetchedInput.class), anyLong(), anyLong(), anyLong()); - doNothing().when(callback).fetchFailed(anyString(), any(InputAttemptIdentifier.class), eq(false)); + doNothing().when(callback).fetchFailed(anyString(), any(InputAttemptFetchFailure.class), eq(false)); FetchResult fetchResult = fetcher.call(); verify(fetcher).setupLocalDiskFetch(); - // expect 3 sucesses and 2 failures - for (int i : sucessfulAttempts) { + // expect 3 successes and 2 failures + for (int i : successfulAttempts) { verifyFetchSucceeded(callback, srcAttempts[i], conf); } - verify(callback).fetchFailed(eq(HOST), eq(srcAttempts[FIRST_FAILED_ATTEMPT_IDX]), eq(false)); - verify(callback).fetchFailed(eq(HOST), eq(srcAttempts[SECOND_FAILED_ATTEMPT_IDX]), eq(false)); + verify(callback).fetchFailed(eq(HOST), + eq(InputAttemptFetchFailure + .fromCompositeAttemptLocalFetchFailure(srcAttempts[FIRST_FAILED_ATTEMPT_IDX])), + eq(false)); + verify(callback).fetchFailed(eq(HOST), + eq(InputAttemptFetchFailure + .fromCompositeAttemptLocalFetchFailure(srcAttempts[SECOND_FAILED_ATTEMPT_IDX])), + eq(false)); Assert.assertEquals("fetchResult host", fetchResult.getHost(), HOST); Assert.assertEquals("fetchResult partition", fetchResult.getPartition(), partition); @@ -304,4 +318,30 @@ public void testInputAttemptIdentifierMap() { Assert.assertTrue(expectedSrcAttempts[count++].toString().compareTo(key) == 0); } } + + @Test + public void testShuffleHandlerDiskErrorUnordered() + throws Exception { + Configuration conf = new Configuration(); + + InputContext inputContext = mock(InputContext.class); + doReturn(new TezCounters()).when(inputContext).getCounters(); + doReturn("vertex").when(inputContext).getSourceVertexName(); + + Fetcher.FetcherBuilder builder = new Fetcher.FetcherBuilder(mock(ShuffleManager.class), null, + null, ApplicationId.newInstance(0, 1), 1, null, "fetcherTest", conf, true, HOST, PORT, + false, true, false); + builder.assignWork(HOST, PORT, 0, 1, Arrays.asList(new InputAttemptIdentifier(0, 0))); + + Fetcher fetcher = builder.build(); + ShuffleHeader header = + new ShuffleHeader(ShuffleHandlerError.DISK_ERROR_EXCEPTION.toString(), -1, -1, -1); + DataInputStream input = RuntimeTestUtils.shuffleHeaderToDataInput(header); + + InputAttemptFetchFailure[] failures = + fetcher.fetchInputs(input, null, new InputAttemptIdentifier(0, 0)); + Assert.assertEquals(1, failures.length); + Assert.assertTrue(failures[0].isDiskErrorAtSource()); + Assert.assertFalse(failures[0].isLocalFetch()); + } } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java index 94f7f5a487..041fd03854 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java @@ -22,7 +22,6 @@ import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyList; import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doAnswer; @@ -37,7 +36,6 @@ import java.io.InputStream; import java.io.OutputStream; import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.LinkedList; import java.util.List; import java.util.concurrent.ExecutorService; @@ -55,7 +53,6 @@ import org.apache.tez.common.security.JobTokenIdentifier; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.dag.api.TezConfiguration; -import org.apache.tez.dag.api.TezConstants; import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.ExecutionContext; import org.apache.tez.runtime.api.InputContext; @@ -66,9 +63,9 @@ import org.apache.tez.runtime.library.common.shuffle.FetchedInput; import org.apache.tez.runtime.library.common.shuffle.FetchedInputAllocator; import org.apache.tez.runtime.library.common.shuffle.Fetcher; +import org.apache.tez.runtime.library.common.shuffle.InputAttemptFetchFailure; import org.apache.tez.runtime.library.common.shuffle.FetchResult; import org.apache.tez.runtime.library.common.shuffle.InputHost; -import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto; import org.junit.After; import org.junit.Assert; @@ -234,12 +231,12 @@ public void run() { } } }); - InputAttemptIdentifier inputAttemptIdentifier - = new InputAttemptIdentifier(1, 1); + InputAttemptFetchFailure inputAttemptFetchFailure = + new InputAttemptFetchFailure(new InputAttemptIdentifier(1, 1)); schedulerGetHostThread.start(); Thread.sleep(1000); - shuffleManager.fetchFailed("host1", inputAttemptIdentifier, false); + shuffleManager.fetchFailed("host1", inputAttemptFetchFailure, false); Thread.sleep(1000); ArgumentCaptor captor = ArgumentCaptor.forClass(List.class); @@ -254,8 +251,8 @@ public void run() { Assert.assertEquals("Number of failures was: " + inputEvent.getNumFailures(), inputEvent.getNumFailures(), 1); - shuffleManager.fetchFailed("host1", inputAttemptIdentifier, false); - shuffleManager.fetchFailed("host1", inputAttemptIdentifier, false); + shuffleManager.fetchFailed("host1", inputAttemptFetchFailure, false); + shuffleManager.fetchFailed("host1", inputAttemptFetchFailure, false); Thread.sleep(1000); verify(inputContext, times(1)).sendEvents(any()); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java index 5f7fe4ba7e..028fbce96a 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java @@ -42,6 +42,7 @@ import java.net.URL; import java.util.Arrays; import java.util.Collection; +import java.util.HashMap; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.LinkedList; @@ -72,7 +73,10 @@ import org.apache.tez.runtime.library.common.security.SecureShuffleUtils; import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord; import org.apache.tez.runtime.library.exceptions.FetcherReadTimeoutException; +import org.apache.tez.runtime.library.testutils.RuntimeTestUtils; +import org.apache.tez.runtime.library.common.shuffle.InputAttemptFetchFailure; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; +import org.apache.tez.runtime.library.common.shuffle.api.ShuffleHandlerError; import org.junit.Assert; import org.junit.Test; import org.mockito.ArgumentCaptor; @@ -241,7 +245,7 @@ public void testSetupLocalDiskFetch() throws Exception { ); final int FIRST_FAILED_ATTEMPT_IDX = 2; final int SECOND_FAILED_ATTEMPT_IDX = 4; - final int[] sucessfulAttemptsIndexes = { 0, 1, 3 }; + final int[] successfulAttemptsIndexes = { 0, 1, 3 }; doReturn(srcAttempts).when(scheduler).getMapsForHost(host); @@ -311,13 +315,17 @@ public TezIndexRecord answer(InvocationOnMock invocation) throws Throwable { spyFetcher.setupLocalDiskFetch(host); // should have exactly 3 success and 1 failure. - for (int i : sucessfulAttemptsIndexes) { + for (int i : successfulAttemptsIndexes) { for (int j = 0; j < host.getPartitionCount(); j++) { verifyCopySucceeded(scheduler, host, srcAttempts, i, j); } } - verify(scheduler).copyFailed(srcAttempts.get(FIRST_FAILED_ATTEMPT_IDX).expand(0), host, true, false, true); - verify(scheduler).copyFailed(srcAttempts.get(SECOND_FAILED_ATTEMPT_IDX).expand(0), host, true, false, true); + verify(scheduler).copyFailed( + eq(InputAttemptFetchFailure.fromLocalFetchFailure(srcAttempts.get(FIRST_FAILED_ATTEMPT_IDX).expand(0))), + eq(host), eq(true), eq(false)); + verify(scheduler).copyFailed( + eq(InputAttemptFetchFailure.fromLocalFetchFailure(srcAttempts.get(SECOND_FAILED_ATTEMPT_IDX).expand(0))), + eq(host), eq(true), eq(false)); verify(spyFetcher).putBackRemainingMapOutputs(host); verify(scheduler).putBackKnownMapOutput(host, srcAttempts.get(FIRST_FAILED_ATTEMPT_IDX)); @@ -426,7 +434,7 @@ public void testSetupLocalDiskFetchAutoReduce() throws Exception { ); final int FIRST_FAILED_ATTEMPT_IDX = 2; final int SECOND_FAILED_ATTEMPT_IDX = 4; - final int[] sucessfulAttemptsIndexes = { 0, 1, 3 }; + final int[] successfulAttemptsIndexes = { 0, 1, 3 }; doReturn(srcAttempts).when(scheduler).getMapsForHost(host); final ConcurrentMap pathToIdentifierMap @@ -503,15 +511,23 @@ public TezIndexRecord answer(InvocationOnMock invocation) throws Throwable { spyFetcher.setupLocalDiskFetch(host); // should have exactly 3 success and 1 failure. - for (int i : sucessfulAttemptsIndexes) { + for (int i : successfulAttemptsIndexes) { for (int j = 0; j < host.getPartitionCount(); j++) { verifyCopySucceeded(scheduler, host, srcAttempts, i, j); } } - verify(scheduler).copyFailed(srcAttempts.get(FIRST_FAILED_ATTEMPT_IDX).expand(0), host, true, false, true); - verify(scheduler).copyFailed(srcAttempts.get(FIRST_FAILED_ATTEMPT_IDX).expand(1), host, true, false, true); - verify(scheduler).copyFailed(srcAttempts.get(SECOND_FAILED_ATTEMPT_IDX).expand(0), host, true, false, true); - verify(scheduler).copyFailed(srcAttempts.get(SECOND_FAILED_ATTEMPT_IDX).expand(1), host, true, false, true); + verify(scheduler).copyFailed( + eq(InputAttemptFetchFailure.fromLocalFetchFailure(srcAttempts.get(FIRST_FAILED_ATTEMPT_IDX).expand(0))), + eq(host), eq(true), eq(false)); + verify(scheduler).copyFailed( + eq(InputAttemptFetchFailure.fromLocalFetchFailure(srcAttempts.get(FIRST_FAILED_ATTEMPT_IDX).expand(1))), + eq(host), eq(true), eq(false)); + verify(scheduler).copyFailed(eq( + InputAttemptFetchFailure.fromLocalFetchFailure(srcAttempts.get(SECOND_FAILED_ATTEMPT_IDX).expand(0))), + eq(host), eq(true), eq(false)); + verify(scheduler).copyFailed(eq( + InputAttemptFetchFailure.fromLocalFetchFailure(srcAttempts.get(SECOND_FAILED_ATTEMPT_IDX).expand(1))), + eq(host), eq(true), eq(false)); verify(spyFetcher).putBackRemainingMapOutputs(host); verify(scheduler).putBackKnownMapOutput(host, srcAttempts.get(FIRST_FAILED_ATTEMPT_IDX)); @@ -630,8 +646,8 @@ public MapOutput answer(InvocationOnMock invocation) throws Throwable { //setup connection should be called twice (1 for connect and another for retry) verify(fetcher, times(2)).setupConnection(any(MapHost.class), any(Collection.class)); //since copyMapOutput consistently fails, it should call copyFailed once - verify(scheduler, times(1)).copyFailed(any(InputAttemptIdentifier.class), any(MapHost.class), - anyBoolean(), anyBoolean(), anyBoolean()); + verify(scheduler, times(1)).copyFailed(any(InputAttemptFetchFailure.class), any(MapHost.class), + anyBoolean(), anyBoolean()); verify(fetcher, times(1)).putBackRemainingMapOutputs(any(MapHost.class)); verify(scheduler, times(3)).putBackKnownMapOutput(any(MapHost.class), @@ -750,6 +766,32 @@ public void testInputAttemptIdentifierMap() { } } + @Test + public void testShuffleHandlerDiskErrorOrdered() + throws Exception { + MapHost mapHost = new MapHost(HOST, PORT, 0, 1); + InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(0, 0, "attempt"); + + FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, null, null, null, null, false, + 0, null, new TezConfiguration(), null, false, HOST, PORT, "src vertex", mapHost, + ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, + connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); + fetcher.remaining = new HashMap(); + + ShuffleHeader header = + new ShuffleHeader(ShuffleHandlerError.DISK_ERROR_EXCEPTION.toString(), -1, -1, -1); + DataInputStream input = RuntimeTestUtils.shuffleHeaderToDataInput(header); + + // copyMapOutput is used for remote fetch, this time it returns a fetch failure, which is fatal + // and should be treated as a local fetch failure + InputAttemptFetchFailure[] failures = + fetcher.copyMapOutput(mapHost, input, inputAttemptIdentifier); + + Assert.assertEquals(1, failures.length); + Assert.assertTrue(failures[0].isDiskErrorAtSource()); + Assert.assertFalse(failures[0].isLocalFetch()); + } + private RawLocalFileSystem getRawFs(Configuration conf) { try { return (RawLocalFileSystem) FileSystem.getLocal(conf).getRaw(); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java index fabfa270d7..b89ffb0ce9 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java @@ -55,6 +55,7 @@ import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; import org.apache.tez.runtime.library.common.CompositeInputAttemptIdentifier; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; +import org.apache.tez.runtime.library.common.shuffle.InputAttemptFetchFailure; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -248,8 +249,8 @@ public void _testReducerHealth_1(Configuration conf) throws IOException { for (int i = 100; i < 199; i++) { InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(i, 0, "attempt_"); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier), + new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), false, true); } @@ -257,9 +258,8 @@ public void _testReducerHealth_1(Configuration conf) throws IOException { new InputAttemptIdentifier(200, 0, "attempt_"); //Should fail here and report exception as reducer is not healthy - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (200 % - totalProducerNodes), - 10000, 200, 1), false, true, false); + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier), + new MapHost("host" + (200 % totalProducerNodes), 10000, 200, 1), false, true); int minFailurePerHost = conf.getInt( TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_MIN_FAILURES_PER_HOST, @@ -330,8 +330,8 @@ public void testReducerHealth_2() throws IOException, InterruptedException { for (int i = 190; i < 200; i++) { InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(i, 0, "attempt_"); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier), + new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), false, true); } //Shuffle has not stalled. so no issues. @@ -342,9 +342,8 @@ public void testReducerHealth_2() throws IOException, InterruptedException { InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(190, 0, "attempt_"); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + - (190 % totalProducerNodes), - 10000, 190, 1), false, true, false); + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier), + new MapHost("host" + (190 % totalProducerNodes), 10000, 190, 1), false, true); //Even when it is stalled, need (320 - 300 = 20) * 3 = 60 failures verify(scheduler.reporter, times(0)).reportException(any(Throwable.class)); @@ -355,16 +354,17 @@ public void testReducerHealth_2() throws IOException, InterruptedException { for (int i = 190; i < 200; i++) { inputAttemptIdentifier = new InputAttemptIdentifier(i, 0, "attempt_"); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); + InputAttemptFetchFailure failure = InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), + 10000, i, 1), false, true); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), + 10000, i, 1), false, true); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), + 10000, i, 1), false, true); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), + 10000, i, 1), false, true); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), + 10000, i, 1), false, true); } assertEquals(61, scheduler.failedShufflesSinceLastCompletion); @@ -376,12 +376,14 @@ public void testReducerHealth_2() throws IOException, InterruptedException { for (int i = 110; i < 120; i++) { inputAttemptIdentifier = new InputAttemptIdentifier(i, 0, "attempt_"); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); + InputAttemptFetchFailure failure = + InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), + false, true); } // Should fail now due to fetcherHealthy. (stall has already happened and @@ -432,8 +434,8 @@ public void testReducerHealth_3() throws IOException { //1 fails (last fetch) InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(319, 0, "attempt_"); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (319 % totalProducerNodes), - 10000, 319, 1), false, true, false); + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier), + new MapHost("host" + (319 % totalProducerNodes), 10000, 319, 1), false, true); //stall the shuffle scheduler.lastProgressTime = System.currentTimeMillis() - 1000000; @@ -441,15 +443,13 @@ public void testReducerHealth_3() throws IOException { assertEquals(scheduler.remainingMaps.get(), 1); //Retry for 3 more times - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (319 % - totalProducerNodes), - 10000, 319, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (319 % - totalProducerNodes), - 10000, 310, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (319 % - totalProducerNodes), - 10000, 310, 1), false, true, false); + InputAttemptFetchFailure failure = InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier); + scheduler.copyFailed(failure, new MapHost("host" + (319 % totalProducerNodes), 10000, 319, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (319 % totalProducerNodes), 10000, 310, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (319 % totalProducerNodes), 10000, 310, 1), + false, true); // failedShufflesSinceLastCompletion has crossed the limits. Throw error verify(shuffle, times(0)).reportException(any(Throwable.class)); @@ -487,15 +487,15 @@ public void testReducerHealth_4() throws IOException { for (int i = 0; i < 64; i++) { InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(i, 0, "attempt_"); + InputAttemptFetchFailure failure = + InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % - totalProducerNodes), 10000, i, 1), false, true, false); - - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % - totalProducerNodes), 10000, i, 1), false, true, false); - - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % - totalProducerNodes), 10000, i, 1), false, true, false); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), + false, true); MapOutput mapOutput = MapOutput .createMemoryMapOutput(inputAttemptIdentifier, mock(FetchedInputAllocatorOrderedGrouped.class), @@ -518,8 +518,8 @@ public void testReducerHealth_4() throws IOException { //1 fails (last fetch) InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(319, 0, "attempt_"); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (319 % totalProducerNodes), - 10000, 319, 1), false, true, false); + scheduler.copyFailed(new InputAttemptFetchFailure(inputAttemptIdentifier), + new MapHost("host" + (319 % totalProducerNodes), 10000, 319, 1), false, true); //stall the shuffle (but within limits) scheduler.lastProgressTime = System.currentTimeMillis() - 100000; @@ -527,15 +527,13 @@ public void testReducerHealth_4() throws IOException { assertEquals(scheduler.remainingMaps.get(), 1); //Retry for 3 more times - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (319 % - totalProducerNodes), - 10000, 319, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (319 % - totalProducerNodes), - 10000, 319, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (319 % - totalProducerNodes), - 10000, 319, 1), false, true, false); + InputAttemptFetchFailure failure = InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier); + scheduler.copyFailed(failure, new MapHost("host" + (319 % totalProducerNodes), 10000, 319, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (319 % totalProducerNodes), 10000, 319, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (319 % totalProducerNodes), 10000, 319, 1), + false, true); // failedShufflesSinceLastCompletion has crossed the limits. 20% of other nodes had failures as // well. However, it has failed only in one host. So this should proceed @@ -544,9 +542,8 @@ public void testReducerHealth_4() throws IOException { //stall the shuffle (but within limits) scheduler.lastProgressTime = System.currentTimeMillis() - 300000; - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (319 % - totalProducerNodes), - 10000, 319, 1), false, true, false); + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier), + new MapHost("host" + (319 % totalProducerNodes), 10000, 319, 1), false, true); verify(shuffle, times(1)).reportException(any(Throwable.class)); } @@ -592,8 +589,9 @@ public void testReducerHealth_5() throws IOException { //1 fails (last fetch) InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(318, 0, "attempt_"); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (318 % totalProducerNodes), - 10000, 318, 1), false, true, false); + InputAttemptFetchFailure failure = new InputAttemptFetchFailure(inputAttemptIdentifier); + scheduler.copyFailed(failure, new MapHost("host" + (318 % totalProducerNodes), + 10000, 318, 1), false, true); //stall the shuffle scheduler.lastProgressTime = System.currentTimeMillis() - 1000000; @@ -601,15 +599,12 @@ public void testReducerHealth_5() throws IOException { assertEquals(scheduler.remainingMaps.get(), 1); //Retry for 3 more times - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (318 % - totalProducerNodes), - 10000, 318, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (318 % - totalProducerNodes), - 10000, 318, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (318 % - totalProducerNodes), - 10000, 318, 1), false, true, false); + scheduler.copyFailed(failure, new MapHost("host" + (318 % totalProducerNodes), 10000, 318, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (318 % totalProducerNodes), 10000, 318, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (318 % totalProducerNodes), 10000, 318, 1), + false, true); //Shuffle has not received the events completely. So do not bail out yet. verify(shuffle, times(0)).reportException(any(Throwable.class)); @@ -672,8 +667,8 @@ public void _testReducerHealth_6(Configuration conf) throws IOException { for (int i = 10; i < 15; i++) { InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(i, 0, "attempt_"); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier), + new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), false, true); } assertTrue(scheduler.failureCounts.size() >= 5); @@ -686,10 +681,10 @@ public void _testReducerHealth_6(Configuration conf) throws IOException { for (int i = 10; i < 15; i++) { InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(i, 0, "attempt_"); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier), + new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), false, true); + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier), + new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), false, true); } boolean checkFailedFetchSinceLastCompletion = conf.getBoolean @@ -749,18 +744,15 @@ public void testReducerHealth_7() throws IOException { for (int i = 100; i < 199; i++) { InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(i, 0, "attempt_"); - scheduler.copyFailed(inputAttemptIdentifier, - new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), - false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, - new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), - false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, - new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), - false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, - new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), - false, true, false); + InputAttemptFetchFailure failure = InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), + false, true); } verify(shuffle, atLeast(1)).reportException(any(Throwable.class)); @@ -799,7 +791,8 @@ public void testPenalty() throws IOException, InterruptedException { MapHost mapHost = scheduler.pendingHosts.iterator().next(); //Fails to pull from host0. host0 should be added to penalties - scheduler.copyFailed(inputAttemptIdentifier, mapHost, false, true, false); + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier), mapHost, + false, true); //Should not get host, as it is added to penalty loop MapHost host = scheduler.getHost(); @@ -993,7 +986,8 @@ public Void call() throws Exception { } for (int i = 0; i < 10; i++) { - scheduler.copyFailed(identifiers[0], mapHosts[0], false, false, false); + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(identifiers[0]), mapHosts[0], false, + false); } ShuffleScheduler.Penalty[] penaltyArray = new ShuffleScheduler.Penalty[scheduler.getPenalties().size()]; scheduler.getPenalties().toArray(penaltyArray); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/testutils/RuntimeTestUtils.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/testutils/RuntimeTestUtils.java new file mode 100644 index 0000000000..0885178ee5 --- /dev/null +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/testutils/RuntimeTestUtils.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.runtime.library.testutils; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; + +import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.ShuffleHeader; + +public final class RuntimeTestUtils { + + private RuntimeTestUtils() { + } + + public static DataInputStream shuffleHeaderToDataInput(ShuffleHeader header) throws IOException { + ByteArrayOutputStream byteOutput = new ByteArrayOutputStream(1000); + DataOutputStream output = new DataOutputStream(byteOutput); + header.write(output); + + InputStream inputStream = new ByteArrayInputStream(byteOutput.toByteArray()); + DataInputStream input = new DataInputStream(inputStream); + + return input; + } +} From d0f498781e9675e88138db49fd8e9e140cc07f0c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Tue, 6 Oct 2020 08:56:57 +0200 Subject: [PATCH 268/512] =?UTF-8?q?TEZ-4234:=20Compressor=20can=20cause=20?= =?UTF-8?q?IllegalArgumentException=20in=20Buffer.limit=20where=20limit=20?= =?UTF-8?q?exceeds=20capacity=20(L=C3=A1szl=C3=B3=20Bodor=20reviewed=20by?= =?UTF-8?q?=20Rajesh=20Balamohan,=20Jonathan=20Turner=20Eagles)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../hadoop/TestConfigTranslationMRToTez.java | 1 - .../runtime/library/common/ConfigUtils.java | 23 ---- .../library/common/TezRuntimeUtils.java | 18 ++- .../shuffle/orderedgrouped/Shuffle.java | 15 +-- .../common/sort/impl/ExternalSorter.java | 29 +--- .../library/common/sort/impl/IFile.java | 25 +--- .../BaseUnorderedPartitionedKVWriter.java | 17 +-- .../library/input/UnorderedKVInput.java | 14 +- .../tez/runtime/library/utils/CodecUtils.java | 127 ++++++++++++++++++ .../common/shuffle/TestShuffleUtils.java | 8 +- .../library/common/sort/impl/TestIFile.java | 59 +++++++- 11 files changed, 217 insertions(+), 119 deletions(-) create mode 100644 tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/CodecUtils.java diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestConfigTranslationMRToTez.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestConfigTranslationMRToTez.java index deab64feef..df68c8dff8 100644 --- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestConfigTranslationMRToTez.java +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestConfigTranslationMRToTez.java @@ -70,6 +70,5 @@ public void testMRToTezKeyTranslation() { assertEquals(LongWritable.class.getName(), ConfigUtils .getIntermediateInputValueClass(confVertex1).getName()); assertTrue(ConfigUtils.shouldCompressIntermediateOutput(confVertex1)); - assertTrue(ConfigUtils.isIntermediateInputCompressed(confVertex1)); } } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java index 76d3dff868..f83fdc99d7 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java @@ -56,24 +56,6 @@ public static Class getIntermediateOutputCompressorC } return codecClass; } - - public static Class getIntermediateInputCompressorClass( - Configuration conf, Class defaultValue) { - Class codecClass = defaultValue; - String name = conf - .get(TezRuntimeConfiguration.TEZ_RUNTIME_COMPRESS_CODEC); - if (name != null) { - try { - codecClass = conf.getClassByName(name).asSubclass( - CompressionCodec.class); - } catch (ClassNotFoundException e) { - throw new IllegalArgumentException("Compression codec " + name - + " was not found.", e); - } - } - return codecClass; - } - // TODO Move defaults over to a constants file. @@ -82,11 +64,6 @@ public static boolean shouldCompressIntermediateOutput(Configuration conf) { TezRuntimeConfiguration.TEZ_RUNTIME_COMPRESS, false); } - public static boolean isIntermediateInputCompressed(Configuration conf) { - return conf.getBoolean( - TezRuntimeConfiguration.TEZ_RUNTIME_COMPRESS, false); - } - public static Class getIntermediateOutputValueClass(Configuration conf) { Class retv = (Class) conf.getClass( TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS, null, diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java index 8be8fa20d1..daeafbc6fe 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java @@ -265,16 +265,22 @@ public static int deserializeShuffleProviderMetaData(ByteBuffer meta) } public static String getBufferSizeProperty(CompressionCodec codec) { - switch (codec.getClass().getSimpleName().toString()) { - case "DefaultCodec": + return getBufferSizeProperty(codec.getClass().getName()); + } + + public static String getBufferSizeProperty(String className) { + switch (className) { + case "org.apache.hadoop.io.compress.DefaultCodec": return "io.file.buffer.size"; - case "SnappyCodec": + case "org.apache.hadoop.io.compress.SnappyCodec": return CommonConfigurationKeys.IO_COMPRESSION_CODEC_SNAPPY_BUFFERSIZE_KEY; - case "ZStandardCodec": + case "org.apache.hadoop.io.compress.ZStandardCodec": return CommonConfigurationKeys.IO_COMPRESSION_CODEC_ZSTD_BUFFER_SIZE_KEY; - case "LzoCodec": + case "org.apache.hadoop.io.compress.LzoCodec": + return CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY; + case "com.hadoop.compression.lzo.LzoCodec": return CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY; - case "Lz4Codec": + case "org.apache.hadoop.io.compress.Lz4Codec": return CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZ4_BUFFERSIZE_KEY; default: return null; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java index 38f079a20b..db5ef734de 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java @@ -39,8 +39,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.io.compress.CompressionCodec; -import org.apache.hadoop.io.compress.DefaultCodec; -import org.apache.hadoop.util.ReflectionUtils; import org.apache.tez.common.CallableWithNdc; import org.apache.tez.common.GuavaShim; import org.apache.tez.common.TezRuntimeFrameworkConfigs; @@ -51,12 +49,11 @@ import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.InputContext; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; -import org.apache.tez.runtime.library.common.ConfigUtils; import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.apache.tez.runtime.library.common.combine.Combiner; import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator; import org.apache.tez.runtime.library.exceptions.InputAlreadyClosedException; - +import org.apache.tez.runtime.library.utils.CodecUtils; import org.apache.tez.common.Preconditions; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; @@ -114,16 +111,8 @@ public Shuffle(InputContext inputContext, Configuration conf, int numInputs, this.srcNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName()); + this.codec = CodecUtils.getCodec(conf); - if (ConfigUtils.isIntermediateInputCompressed(conf)) { - Class codecClass = - ConfigUtils.getIntermediateInputCompressorClass(conf, DefaultCodec.class); - codec = ReflectionUtils.newInstance(codecClass, conf); - // Work around needed for HADOOP-12191. Avoids the native initialization synchronization race - codec.getDecompressorType(); - } else { - codec = null; - } this.ifileReadAhead = conf.getBoolean( TezRuntimeConfiguration.TEZ_RUNTIME_IFILE_READAHEAD, TezRuntimeConfiguration.TEZ_RUNTIME_IFILE_READAHEAD_DEFAULT); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java index 194e899fea..3ff74f72bb 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java @@ -42,8 +42,6 @@ import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.hadoop.io.RawComparator; import org.apache.hadoop.io.compress.CompressionCodec; -import org.apache.hadoop.io.compress.Compressor; -import org.apache.hadoop.io.compress.DefaultCodec; import org.apache.hadoop.io.serializer.Serializer; import org.apache.hadoop.util.IndexedSorter; import org.apache.hadoop.util.Progressable; @@ -63,7 +61,7 @@ import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.ShuffleHeader; import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer; import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput; - +import org.apache.tez.runtime.library.utils.CodecUtils; import org.apache.tez.common.Preconditions; @SuppressWarnings({"rawtypes"}) @@ -224,30 +222,7 @@ public ExternalSorter(OutputContext outputContext, Configuration conf, int numOu numShuffleChunks = outputContext.getCounters().findCounter(TaskCounter.SHUFFLE_CHUNK_COUNT); // compression - if (ConfigUtils.shouldCompressIntermediateOutput(this.conf)) { - Class codecClass = - ConfigUtils.getIntermediateOutputCompressorClass(this.conf, DefaultCodec.class); - codec = ReflectionUtils.newInstance(codecClass, this.conf); - - if (codec != null) { - Class compressorType = null; - Throwable cause = null; - try { - compressorType = codec.getCompressorType(); - } catch (RuntimeException e) { - cause = e; - } - if (compressorType == null) { - String errMsg = - String.format("Unable to get CompressorType for codec (%s). This is most" + - " likely due to missing native libraries for the codec.", - conf.get(TezRuntimeConfiguration.TEZ_RUNTIME_COMPRESS_CODEC)); - throw new IOException(errMsg, cause); - } - } - } else { - codec = null; - } + this.codec = CodecUtils.getCodec(conf); this.ifileReadAhead = this.conf.getBoolean( TezRuntimeConfiguration.TEZ_RUNTIME_IFILE_READAHEAD, diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java index 6aa44e206a..1b2aefff41 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java @@ -30,20 +30,18 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.io.BoundedByteArrayOutputStream; -import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.conf.Configurable; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.DataInputBuffer; import org.apache.hadoop.io.DataOutputBuffer; import org.apache.tez.runtime.library.utils.BufferUtils; +import org.apache.tez.runtime.library.utils.CodecUtils; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.io.compress.CodecPool; @@ -823,7 +821,8 @@ public static void readToMemory(byte[] buffer, InputStream in, int compressedLen decompressor = CodecPool.getDecompressor(codec); if (decompressor != null) { decompressor.reset(); - in = getDecompressedInputStreamWithBufferSize(codec, checksumIn, decompressor, compressedLength); + in = CodecUtils.getDecompressedInputStreamWithBufferSize(codec, checksumIn, decompressor, + compressedLength); } else { LOG.warn("Could not obtain decompressor from CodecPool"); in = checksumIn; @@ -859,24 +858,6 @@ public static void readToMemory(byte[] buffer, InputStream in, int compressedLen } } - private static InputStream getDecompressedInputStreamWithBufferSize(CompressionCodec codec, - IFileInputStream checksumIn, Decompressor decompressor, int compressedLength) - throws IOException { - String bufferSizeProp = TezRuntimeUtils.getBufferSizeProperty(codec); - - if (bufferSizeProp != null) { - Configurable configurableCodec = (Configurable) codec; - Configuration conf = configurableCodec.getConf(); - - int bufSize = Math.min(compressedLength, DEFAULT_BUFFER_SIZE); - LOG.trace("buffer size was set according to min(compressedLength, {}): {}={}", - DEFAULT_BUFFER_SIZE, bufferSizeProp, bufSize); - conf.setInt(bufferSizeProp, bufSize); - } - - return codec.createInputStream(checksumIn, decompressor); - } - /** * Read entire IFile content to disk. * diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/BaseUnorderedPartitionedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/BaseUnorderedPartitionedKVWriter.java index ecc9e03a5d..adea49fe80 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/BaseUnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/BaseUnorderedPartitionedKVWriter.java @@ -29,10 +29,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.hadoop.io.compress.CompressionCodec; -import org.apache.hadoop.io.compress.DefaultCodec; import org.apache.hadoop.io.serializer.SerializationFactory; import org.apache.hadoop.io.serializer.Serializer; -import org.apache.hadoop.util.ReflectionUtils; import org.apache.tez.common.counters.TaskCounter; import org.apache.tez.common.counters.TezCounter; import org.apache.tez.runtime.api.Event; @@ -43,6 +41,7 @@ import org.apache.tez.runtime.library.common.ConfigUtils; import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput; +import org.apache.tez.runtime.library.utils.CodecUtils; @SuppressWarnings("rawtypes") public abstract class BaseUnorderedPartitionedKVWriter extends KeyValuesWriter { @@ -141,16 +140,14 @@ public BaseUnorderedPartitionedKVWriter(OutputContext outputContext, Configurati additionalSpillBytesReadCounter = outputContext.getCounters().findCounter(TaskCounter.ADDITIONAL_SPILLS_BYTES_READ); numAdditionalSpillsCounter = outputContext.getCounters().findCounter(TaskCounter.ADDITIONAL_SPILL_COUNT); dataViaEventSize = outputContext.getCounters().findCounter(TaskCounter.DATA_BYTES_VIA_EVENT); - + // compression - if (ConfigUtils.shouldCompressIntermediateOutput(this.conf)) { - Class codecClass = - ConfigUtils.getIntermediateOutputCompressorClass(this.conf, DefaultCodec.class); - codec = ReflectionUtils.newInstance(codecClass, this.conf); - } else { - codec = null; + try { + this.codec = CodecUtils.getCodec(conf); + } catch (IOException e) { + throw new RuntimeException(e); } - + this.ifileReadAhead = this.conf.getBoolean( TezRuntimeConfiguration.TEZ_RUNTIME_IFILE_READAHEAD, TezRuntimeConfiguration.TEZ_RUNTIME_IFILE_READAHEAD_DEFAULT); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java index 1db786995a..c67c405b43 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java @@ -35,8 +35,6 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.compress.CompressionCodec; -import org.apache.hadoop.io.compress.DefaultCodec; -import org.apache.hadoop.util.ReflectionUtils; import org.apache.tez.common.TezRuntimeFrameworkConfigs; import org.apache.tez.common.counters.TaskCounter; import org.apache.tez.common.counters.TezCounter; @@ -46,14 +44,13 @@ import org.apache.tez.runtime.api.InputContext; import org.apache.tez.runtime.library.api.KeyValueReader; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; -import org.apache.tez.runtime.library.common.ConfigUtils; import org.apache.tez.runtime.library.common.MemoryUpdateCallbackHandler; import org.apache.tez.runtime.library.common.readers.UnorderedKVReader; import org.apache.tez.runtime.library.common.shuffle.ShuffleEventHandler; import org.apache.tez.runtime.library.common.shuffle.impl.ShuffleInputEventHandlerImpl; import org.apache.tez.runtime.library.common.shuffle.impl.ShuffleManager; import org.apache.tez.runtime.library.common.shuffle.impl.SimpleFetchedInputAllocator; - +import org.apache.tez.runtime.library.utils.CodecUtils; import org.apache.tez.common.Preconditions; /** @@ -114,14 +111,7 @@ public synchronized void start() throws IOException { if (!isStarted.get()) { ////// Initial configuration memoryUpdateCallbackHandler.validateUpdateReceived(); - CompressionCodec codec; - if (ConfigUtils.isIntermediateInputCompressed(conf)) { - Class codecClass = ConfigUtils - .getIntermediateInputCompressorClass(conf, DefaultCodec.class); - codec = ReflectionUtils.newInstance(codecClass, conf); - } else { - codec = null; - } + CompressionCodec codec = CodecUtils.getCodec(conf); boolean compositeFetch = ShuffleUtils.isTezShuffleHandler(conf); boolean ifileReadAhead = conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_IFILE_READAHEAD, diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/CodecUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/CodecUtils.java new file mode 100644 index 0000000000..99d22c5dcc --- /dev/null +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/CodecUtils.java @@ -0,0 +1,127 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.runtime.library.utils; + +import java.io.IOException; +import java.io.InputStream; + +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.io.compress.CompressionInputStream; +import org.apache.hadoop.io.compress.Compressor; +import org.apache.hadoop.io.compress.Decompressor; +import org.apache.hadoop.io.compress.DefaultCodec; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; +import org.apache.tez.runtime.library.common.ConfigUtils; +import org.apache.tez.runtime.library.common.TezRuntimeUtils; +import org.apache.tez.runtime.library.common.sort.impl.IFile; +import org.apache.tez.runtime.library.common.sort.impl.IFileInputStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public final class CodecUtils { + + private static final Logger LOG = LoggerFactory.getLogger(IFile.class); + private static final int DEFAULT_BUFFER_SIZE = 128 * 1024; + + private CodecUtils() { + } + + public static CompressionCodec getCodec(Configuration conf) throws IOException { + if (ConfigUtils.shouldCompressIntermediateOutput(conf)) { + Class codecClass = + ConfigUtils.getIntermediateOutputCompressorClass(conf, DefaultCodec.class); + CompressionCodec codec = ReflectionUtils.newInstance(codecClass, conf); + + if (codec != null) { + Class compressorType = null; + Throwable cause = null; + try { + compressorType = codec.getCompressorType(); + } catch (RuntimeException e) { + cause = e; + } + if (compressorType == null) { + String errMsg = String.format( + "Unable to get CompressorType for codec (%s). This is most" + + " likely due to missing native libraries for the codec.", + conf.get(TezRuntimeConfiguration.TEZ_RUNTIME_COMPRESS_CODEC)); + throw new IOException(errMsg, cause); + } + } + return codec; + } else { + return null; + } + } + + public static InputStream getDecompressedInputStreamWithBufferSize(CompressionCodec codec, + IFileInputStream checksumIn, Decompressor decompressor, int compressedLength) + throws IOException { + String bufferSizeProp = TezRuntimeUtils.getBufferSizeProperty(codec); + Configurable configurableCodec = (Configurable) codec; + int originalSize = configurableCodec.getConf().getInt(bufferSizeProp, DEFAULT_BUFFER_SIZE); + + CompressionInputStream in = null; + + if (bufferSizeProp != null) { + Configuration conf = configurableCodec.getConf(); + int newBufSize = Math.min(compressedLength, DEFAULT_BUFFER_SIZE); + LOG.trace("buffer size was set according to min(compressedLength, {}): {}={}", + DEFAULT_BUFFER_SIZE, bufferSizeProp, newBufSize); + + synchronized (codec) { + conf.setInt(bufferSizeProp, newBufSize); + + in = codec.createInputStream(checksumIn, decompressor); + /* + * We would better reset the original buffer size into the codec. Basically the buffer size + * is used at 2 places. + * + * 1. It can tell the inputstream/outputstream buffersize (which is created by + * codec.createInputStream/codec.createOutputStream). This is something which might and + * should be optimized in config, as inputstreams instantiate and use their own buffer and + * won't reuse buffers from previous streams (TEZ-4135). + * + * 2. The same buffersize is used when a codec creates a new Compressor/Decompressor. The + * fundamental difference is that Compressor/Decompressor instances are expensive and reused + * by hadoop's CodecPool. Here is a hidden mismatch, which can happen when a codec is + * created with a small buffersize config. Once it creates a Compressor/Decompressor + * instance from its config field, the reused Compressor/Decompressor instance will be + * reused later, even when application handles large amount of data. This way we can end up + * in large stream buffers + small compressor/decompressor buffers, which can be suboptimal, + * moreover, it can lead to strange errors, when a compressed output exceeds the size of the + * buffer (TEZ-4234). + * + * An interesting outcome is that - as the codec buffersize config affects both + * compressor(output) and decompressor(input) paths - an altered codec config can cause the + * issues above for Compressor instances as well, even when we tried to leverage from + * smaller buffer size only on decompression paths. + */ + configurableCodec.getConf().setInt(bufferSizeProp, originalSize); + } + } else { + in = codec.createInputStream(checksumIn, decompressor); + } + + return in; + } +} \ No newline at end of file diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java index 520dec7e70..446801ac2f 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java @@ -290,6 +290,7 @@ public void testInternalErrorTranslation() throws Exception { .thenThrow(new InternalError(codecErrorMsg)); Decompressor mockDecoder = mock(Decompressor.class); CompressionCodec mockCodec = mock(ConfigurableCodecForTest.class); + when(((ConfigurableCodecForTest) mockCodec).getConf()).thenReturn(mock(Configuration.class)); when(mockCodec.createDecompressor()).thenReturn(mockDecoder); when(mockCodec.createInputStream(any(InputStream.class), any(Decompressor.class))) .thenReturn(mockCodecStream); @@ -312,6 +313,7 @@ public void testExceptionTranslation() throws Exception { .thenThrow(new IllegalArgumentException(codecErrorMsg)); Decompressor mockDecoder = mock(Decompressor.class); CompressionCodec mockCodec = mock(ConfigurableCodecForTest.class); + when(((ConfigurableCodecForTest) mockCodec).getConf()).thenReturn(mock(Configuration.class)); when(mockCodec.createDecompressor()).thenReturn(mockDecoder); when(mockCodec.createInputStream(any(InputStream.class), any(Decompressor.class))) .thenReturn(mockCodecStream); @@ -327,7 +329,8 @@ public void testExceptionTranslation() throws Exception { CompressionInputStream mockCodecStream1 = mock(CompressionInputStream.class); when(mockCodecStream1.read(any(byte[].class), anyInt(), anyInt())) .thenThrow(new SocketTimeoutException(codecErrorMsg)); - CompressionCodec mockCodec1 = mock(CompressionCodec.class); + CompressionCodec mockCodec1 = mock(ConfigurableCodecForTest.class); + when(((ConfigurableCodecForTest) mockCodec1).getConf()).thenReturn(mock(Configuration.class)); when(mockCodec1.createDecompressor()).thenReturn(mockDecoder); when(mockCodec1.createInputStream(any(InputStream.class), any(Decompressor.class))) .thenReturn(mockCodecStream1); @@ -342,7 +345,8 @@ public void testExceptionTranslation() throws Exception { CompressionInputStream mockCodecStream2 = mock(CompressionInputStream.class); when(mockCodecStream2.read(any(byte[].class), anyInt(), anyInt())) .thenThrow(new InternalError(codecErrorMsg)); - CompressionCodec mockCodec2 = mock(CompressionCodec.class); + CompressionCodec mockCodec2 = mock(ConfigurableCodecForTest.class); + when(((ConfigurableCodecForTest) mockCodec2).getConf()).thenReturn(mock(Configuration.class)); when(mockCodec2.createDecompressor()).thenReturn(mockDecoder); when(mockCodec2.createInputStream(any(InputStream.class), any(Decompressor.class))) .thenReturn(mockCodecStream2); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java index c74496e46b..bf35955625 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java @@ -50,9 +50,11 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.CompressionCodecFactory; +import org.apache.hadoop.io.compress.lz4.Lz4Compressor; import org.apache.hadoop.io.serializer.Deserializer; import org.apache.hadoop.io.serializer.SerializationFactory; import org.apache.hadoop.io.serializer.WritableSerialization; +import org.apache.hadoop.util.NativeCodeLoader; import org.apache.tez.common.TezRuntimeFrameworkConfigs; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; import org.apache.tez.runtime.library.common.TezRuntimeUtils; @@ -66,6 +68,7 @@ import org.apache.tez.runtime.library.utils.BufferUtils; import org.junit.After; import org.junit.Assert; +import org.junit.Assume; import org.junit.Before; import org.junit.Test; import org.slf4j.Logger; @@ -729,13 +732,16 @@ public void testReadToDisk() throws IOException { @Test public void testInMemoryBufferSize() throws IOException { + Configurable configurableCodec = (Configurable) codec; + int originalCodecBufferSize = + configurableCodec.getConf().getInt(TezRuntimeUtils.getBufferSizeProperty(codec), -1); + // for smaller amount of data, codec buffer should be sized according to compressed data length List data = KVDataGen.generateTestData(false, rnd.nextInt(100)); Writer writer = writeTestFile(false, false, data, codec); readAndVerifyData(writer.getRawLength(), writer.getCompressedLength(), data, codec); - Configurable configurableCodec = (Configurable) codec; - Assert.assertEquals(writer.getCompressedLength(), + Assert.assertEquals(originalCodecBufferSize, // original size is repaired configurableCodec.getConf().getInt(TezRuntimeUtils.getBufferSizeProperty(codec), 0)); // buffer size cannot grow infinitely with compressed data size @@ -743,10 +749,57 @@ public void testInMemoryBufferSize() throws IOException { writer = writeTestFile(false, false, data, codec); readAndVerifyData(writer.getRawLength(), writer.getCompressedLength(), data, codec); - Assert.assertEquals(128*1024, + Assert.assertEquals(originalCodecBufferSize, // original size is repaired configurableCodec.getConf().getInt(TezRuntimeUtils.getBufferSizeProperty(codec), 0)); } + @Test(expected = IllegalArgumentException.class) + public void testSmallDataCompression() throws IOException { + Assume.assumeTrue(NativeCodeLoader.isNativeCodeLoaded()); + + tryWriteFileWithBufferSize(17, "org.apache.hadoop.io.compress.Lz4Codec"); + tryWriteFileWithBufferSize(32, "org.apache.hadoop.io.compress.Lz4Codec"); + } + + private void tryWriteFileWithBufferSize(int bufferSize, String codecClassName) + throws IOException { + Configuration conf = new Configuration(); + + System.out.println("trying with buffer size: " + bufferSize); + conf.set(TezRuntimeUtils.getBufferSizeProperty(codecClassName), Integer.toString(bufferSize)); + CompressionCodecFactory codecFactory = new CompressionCodecFactory(conf); + CompressionCodec codecToTest = + codecFactory.getCodecByClassName(codecClassName); + List data = KVDataGen.generateTestDataOfKeySize(false, 1, 0); + writeTestFile(false, false, data, codecToTest); + } + + @Test(expected = IllegalArgumentException.class) + public void testLz4CompressedDataIsLargerThanOriginal() throws IOException { + Assume.assumeTrue(NativeCodeLoader.isNativeCodeLoaded()); + + // this one succeeds + byte[] buf = new byte[32]; + initBufWithNumbers(buf, 24, 45, 55, 49, 54, 55, 55, 54, 49, 48, 50, 55, 49, 56, 54, 48, 57, 48); + Lz4Compressor comp = new Lz4Compressor(32, false); + comp.setInput(buf, 0, 32); + comp.compress(buf, 0, 32); + + // adding 1 more element makes that fail + buf = new byte[32]; + initBufWithNumbers(buf, 24, 45, 55, 49, 54, 55, 55, 54, 49, 48, 50, 55, 49, 56, 54, 48, 57, 48, + 50); + comp = new Lz4Compressor(32, false); + comp.setInput(buf, 0, 32); + comp.compress(buf, 0, 32); + } + + private void initBufWithNumbers(byte[] buf, int... args) { + for (int i = 0; i < args.length; i++) { + buf[i] = (byte) args[i]; + } + } + /** * Test different options (RLE, repeat keys, compression) on reader/writer * From 6e9c1b2d605f3a4197a5185cab81cc66d13a1437 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Tue, 6 Oct 2020 12:00:17 +0200 Subject: [PATCH 269/512] =?UTF-8?q?TEZ-4238:=20Check=20null=20mrReader=20i?= =?UTF-8?q?n=20MRInput.close=20(L=C3=A1szl=C3=B3=20Bodor=20reviewed=20by?= =?UTF-8?q?=20Hadoop=20QA,=20Jonathan=20Turner=20Eagles,=20Hadoop=20QA)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../apache/tez/mapreduce/input/MRInput.java | 5 ++++- .../tez/mapreduce/input/TestMRInput.java | 22 +++++++++++++++++++ 2 files changed, 26 insertions(+), 1 deletion(-) diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java index 5c8ad4ea68..891249b63e 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java @@ -592,7 +592,10 @@ public void handleEvents(List inputEvents) throws Exception { @Override public List close() throws IOException { - mrReader.close(); + if (mrReader != null) { + mrReader.close(); + mrReader = null; + } long inputRecords = getContext().getCounters() .findCounter(TaskCounter.INPUT_RECORDS_PROCESSED).getValue(); getContext().getStatisticsReporter().reportItemsProcessed(inputRecords); diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/TestMRInput.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/TestMRInput.java index 5ca5c26619..844ea51cf1 100644 --- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/TestMRInput.java +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/TestMRInput.java @@ -47,6 +47,7 @@ import org.apache.tez.mapreduce.protos.MRRuntimeProtos; import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.InputContext; +import org.apache.tez.runtime.api.InputStatisticsReporter; import org.apache.tez.runtime.api.events.InputDataInformationEvent; import org.junit.Test; @@ -186,6 +187,16 @@ public void testConfigMerge() throws Exception { assertEquals("payload-value", mergedConfig.get("payload-key")); } + @Test + public void testMRInputCloseWithUnintializedReader() throws IOException { + InputContext inputContext = mock(InputContext.class); + doReturn(new TezCounters()).when(inputContext).getCounters(); + doReturn(new InputStatisticsReporterImplForTest()).when(inputContext).getStatisticsReporter(); + + MRInput mrInput = new MRInput(inputContext, 0); + mrInput.close(); // shouldn't throw NPE + } + /** * Test class to verify */ @@ -276,4 +287,15 @@ public void readFields(DataInput in) throws IOException { } } + + public static class InputStatisticsReporterImplForTest implements InputStatisticsReporter { + + @Override + public synchronized void reportDataSize(long size) { + } + + @Override + public void reportItemsProcessed(long items) { + } + } } From 16a0050909ac5928a131ede8ffdb9a4d12c82cf7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Tue, 13 Oct 2020 07:48:36 +0200 Subject: [PATCH 270/512] =?UTF-8?q?TEZ-4229:=20Improve=20TezLocalCacheMana?= =?UTF-8?q?ger=20to=20use=20configured=20root=20directory=20(L=C3=A1szl?= =?UTF-8?q?=C3=B3=20Bodor=20reviewed=20by=20Panagiotis=20Garefalakis,=20As?= =?UTF-8?q?hutosh=20Chauhan)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../apache/tez/dag/api/TezConfiguration.java | 9 ++++ .../app/launcher/TezLocalCacheManager.java | 10 ++++- .../launcher/TestTezLocalCacheManager.java | 41 +++++++++++++++++++ 3 files changed, 58 insertions(+), 2 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index d5e5e73d45..2af08a9044 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -1748,6 +1748,15 @@ public TezConfiguration(boolean loadDefaults) { public static final boolean TEZ_LOCAL_MODE_DEFAULT = false; + /** + * String value. TezLocalCacheManager uses this folder as a root for temp and localized files. + */ + @ConfigurationScope(Scope.VERTEX) + @ConfigurationProperty + public static final String TEZ_LOCAL_CACHE_ROOT_FOLDER = TEZ_PREFIX + "local.cache.root.folder"; + + public static final String TEZ_LOCAL_CACHE_ROOT_FOLDER_DEFAULT = "."; + /** * Tez AM Inline Mode flag. Not valid till Tez-684 get checked-in */ diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java index 9bcbb1530a..f4892ab81e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java @@ -26,6 +26,7 @@ import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.LocalResourceType; import org.apache.hadoop.yarn.util.FSDownload; +import org.apache.tez.dag.api.TezConfiguration; import java.io.IOException; import java.nio.file.Files; @@ -63,7 +64,7 @@ public TezLocalCacheManager(Map resources, Configuration this.fileContext = FileContext.getLocalFSFileContext(); this.resources = resources; this.conf = conf; - this.tempDir = Files.createTempDirectory(Paths.get("."), "tez-local-cache"); + this.tempDir = Files.createTempDirectory(getLocalCacheRoot(), "tez-local-cache"); } /** @@ -72,7 +73,7 @@ public TezLocalCacheManager(Map resources, Configuration * @throws IOException when an error occurs in download or link */ public void localize() throws IOException { - String absPath = Paths.get(".").toAbsolutePath().normalize().toString(); + String absPath = getLocalCacheRoot().toAbsolutePath().normalize().toString(); Path cwd = fileContext.makeQualified(new Path(absPath)); ExecutorService threadPool = null; @@ -181,6 +182,11 @@ private boolean createSymlink(Path target, Path link) throws IOException { } } + private java.nio.file.Path getLocalCacheRoot() { + return Paths.get(conf.get(TezConfiguration.TEZ_LOCAL_CACHE_ROOT_FOLDER, + TezConfiguration.TEZ_LOCAL_CACHE_ROOT_FOLDER_DEFAULT)); + } + /** * Wrapper to keep track of download path and link path. */ diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestTezLocalCacheManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestTezLocalCacheManager.java index beca047349..5596dc8feb 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestTezLocalCacheManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestTezLocalCacheManager.java @@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.util.ConverterUtils; +import org.apache.tez.dag.api.TezConfiguration; import org.junit.Assert; import org.junit.Test; @@ -110,4 +111,44 @@ private static LocalResource createFile(String content) throws IOException { ret.setTimestamp(fs.getFileStatus(p).getModificationTime()); return ret; } + + @Test + public void testLocalizeRootDirectory() throws URISyntaxException, IOException { + // default directory + Map resources = new HashMap<>(); + + LocalResource resourceOne = createFile("content-one"); + resources.put("file-one", resourceOne); + + TezLocalCacheManager manager = new TezLocalCacheManager(resources, new Configuration()); + + try { + Assert.assertFalse(Files.exists(Paths.get("./file-one"))); + manager.localize(); + Assert.assertTrue(Files.exists(Paths.get("./file-one"))); + + } finally { + manager.cleanup(); + Assert.assertFalse(Files.exists(Paths.get("./file-one"))); + } + + // configured directory + Configuration conf = new Configuration(); + conf.set(TezConfiguration.TEZ_LOCAL_CACHE_ROOT_FOLDER, "target"); + manager = new TezLocalCacheManager(resources, conf); + + try { + // files don't exist at all + Assert.assertFalse(Files.exists(Paths.get("./file-one"))); + Assert.assertFalse(Files.exists(Paths.get("./target/file-one"))); + manager.localize(); + // file appears only at configured location + Assert.assertFalse(Files.exists(Paths.get("./file-one"))); + Assert.assertTrue(Files.exists(Paths.get("./target/file-one"))); + + } finally { + manager.cleanup(); + Assert.assertFalse(Files.exists(Paths.get("./target/file-one"))); + } + } } From 970d46b5f8be72cc9abf1769f8c7c80307737fc5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Wed, 14 Oct 2020 17:24:57 +0200 Subject: [PATCH 271/512] =?UTF-8?q?TEZ-4070:=20SSLFactory=20not=20closed?= =?UTF-8?q?=20in=20DAGClientTimelineImpl=20caused=20native=20memory=20issu?= =?UTF-8?q?es=20(L=C3=A1szl=C3=B3=20Bodor=20reviewed=20by=20Jonathan=20Tur?= =?UTF-8?q?ner=20Eagles)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- tez-api/pom.xml | 5 ++ .../dag/api/client/DAGClientTimelineImpl.java | 3 ++ .../dag/api/client/TimelineReaderFactory.java | 36 ++++++++++--- .../tez/dag/api/client/rpc/TestDAGClient.java | 52 +++++++++++++++++++ 4 files changed, 88 insertions(+), 8 deletions(-) diff --git a/tez-api/pom.xml b/tez-api/pom.xml index 6659f4f9e4..95a7610ee4 100644 --- a/tez-api/pom.xml +++ b/tez-api/pom.xml @@ -110,6 +110,11 @@ mockito-all test + + org.bouncycastle + bcprov-jdk15on + test + diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientTimelineImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientTimelineImpl.java index d34dbf008d..17d2386860 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientTimelineImpl.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientTimelineImpl.java @@ -213,6 +213,9 @@ public void close() throws IOException { httpClient.destroy(); httpClient = null; } + if (timelineReaderStrategy != null) { + timelineReaderStrategy.close(); + } } private DAGStatusProto.Builder parseDagStatus(JSONObject jsonRoot, Set statusOptions) diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/TimelineReaderFactory.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/TimelineReaderFactory.java index c0569dda98..40340cc44e 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/TimelineReaderFactory.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/TimelineReaderFactory.java @@ -133,6 +133,7 @@ public static boolean isTimelineClientSupported() { public interface TimelineReaderStrategy { Client getHttpClient() throws IOException; + void close(); } /* @@ -142,6 +143,8 @@ private static class TimelineReaderTokenAuthenticatedStrategy implements Timelin private final Configuration conf; private final boolean useHttps; private final int connTimeout; + private ConnectionConfigurator connectionConfigurator; + private SSLFactory sslFactory; public TimelineReaderTokenAuthenticatedStrategy(final Configuration conf, final boolean useHttps, @@ -150,6 +153,7 @@ public TimelineReaderTokenAuthenticatedStrategy(final Configuration conf, this.conf = conf; this.useHttps = useHttps; this.connTimeout = connTimeout; + this.sslFactory = useHttps ? new SSLFactory(CLIENT, conf) : null; } @Override @@ -160,8 +164,8 @@ public Client getHttpClient() throws IOException { UserGroupInformation authUgi; String doAsUser; ClientConfig clientConfig = new DefaultClientConfig(JSONRootElementProvider.App.class); - ConnectionConfigurator connectionConfigurator = getNewConnectionConf(conf, useHttps, - connTimeout); + connectionConfigurator = getNewConnectionConf(conf, useHttps, + connTimeout, sslFactory); try { authenticator = getTokenAuthenticator(); @@ -238,6 +242,13 @@ public HttpURLConnection getHttpURLConnection(URL url) throws IOException { } } } + + @Override + public void close() { + if (sslFactory != null) { + sslFactory.destroy(); + } + } } /* @@ -247,11 +258,13 @@ public HttpURLConnection getHttpURLConnection(URL url) throws IOException { protected static class TimelineReaderPseudoAuthenticatedStrategy implements TimelineReaderStrategy { private final ConnectionConfigurator connectionConf; + private final SSLFactory sslFactory; public TimelineReaderPseudoAuthenticatedStrategy(final Configuration conf, final boolean useHttps, final int connTimeout) { - connectionConf = getNewConnectionConf(conf, useHttps, connTimeout); + sslFactory = useHttps ? new SSLFactory(CLIENT, conf) : null; + connectionConf = getNewConnectionConf(conf, useHttps, connTimeout, sslFactory); } @Override @@ -282,15 +295,23 @@ public HttpURLConnection getHttpURLConnection(URL url) throws IOException { return httpURLConnection; } } + + @Override + public void close() { + if (sslFactory != null) { + sslFactory.destroy(); + } + } } private static ConnectionConfigurator getNewConnectionConf(final Configuration conf, final boolean useHttps, - final int connTimeout) { + final int connTimeout, + final SSLFactory sslFactory) { ConnectionConfigurator connectionConf = null; if (useHttps) { try { - connectionConf = getNewSSLConnectionConf(conf, connTimeout); + connectionConf = getNewSSLConnectionConf(conf, connTimeout, sslFactory); } catch (IOException e) { if (LOG.isDebugEnabled()) { LOG.debug("Cannot load customized ssl related configuration." @@ -313,13 +334,12 @@ public HttpURLConnection configure(HttpURLConnection httpURLConnection) throws I } private static ConnectionConfigurator getNewSSLConnectionConf(final Configuration conf, - final int connTimeout) + final int connTimeout, + final SSLFactory sslFactory) throws IOException { - final SSLFactory sslFactory; final SSLSocketFactory sslSocketFactory; final HostnameVerifier hostnameVerifier; - sslFactory = new SSLFactory(CLIENT, conf); try { sslFactory.init(); sslSocketFactory = sslFactory.createSSLSocketFactory(); diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java index 211baf257e..6a5e817d44 100644 --- a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java +++ b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java @@ -28,11 +28,15 @@ import static org.mockito.Mockito.when; import javax.annotation.Nullable; + +import java.io.File; import java.io.IOException; +import java.lang.reflect.Field; import java.util.EnumSet; import java.util.Set; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.ssl.KeyStoreTestUtil; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.exceptions.YarnException; @@ -46,6 +50,7 @@ import org.apache.tez.dag.api.client.DagStatusSource; import org.apache.tez.dag.api.client.StatusGetOpts; import org.apache.tez.dag.api.client.VertexStatus; +import org.apache.tez.dag.api.client.TimelineReaderFactory.TimelineReaderStrategy; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetDAGStatusRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetDAGStatusResponseProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetVertexStatusRequestProto; @@ -62,6 +67,7 @@ import org.apache.tez.dag.api.records.DAGProtos.TezCountersProto; import org.apache.tez.dag.api.records.DAGProtos.VertexStatusProto; import org.apache.tez.dag.api.records.DAGProtos.VertexStatusStateProto; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.mockito.ArgumentCaptor; @@ -525,4 +531,50 @@ public Object answer(InvocationOnMock invocation) throws Throwable { }).when(mock).getDAGStatus(isNull(RpcController.class), any(GetDAGStatusRequestProto.class)); return mock; } + + @Test + /* testing idea is borrowed from YARN-5309 */ + public void testTimelineClientCleanup() throws Exception { + TezConfiguration tezConf = new TezConfiguration(); + tezConf.set("yarn.http.policy", "HTTPS_ONLY"); + + File testDir = new File(System.getProperty("java.io.tmpdir")); + String sslConfDir = KeyStoreTestUtil.getClasspathDir(TestDAGClient.class); + KeyStoreTestUtil.setupSSLConfig(testDir.getAbsolutePath(), sslConfDir, tezConf, false); + + DAGClientTimelineImpl dagClient = + new DAGClientTimelineImpl(mockAppId, dagIdStr, tezConf, mock(FrameworkClient.class), 10000); + Field field = DAGClientTimelineImpl.class.getDeclaredField("timelineReaderStrategy"); + field.setAccessible(true); + TimelineReaderStrategy strategy = (TimelineReaderStrategy) field.get(dagClient); + strategy.getHttpClient(); // calls SSLFactory.init + + ThreadGroup threadGroup = Thread.currentThread().getThreadGroup(); + + while (threadGroup.getParent() != null) { + threadGroup = threadGroup.getParent(); + } + + Thread[] threads = new Thread[threadGroup.activeCount()]; + + threadGroup.enumerate(threads); + Thread reloaderThread = null; + for (Thread thread : threads) { + if ((thread.getName() != null) && (thread.getName().contains("Truststore reloader thread"))) { + reloaderThread = thread; + } + } + Assert.assertTrue("Reloader is not alive", reloaderThread.isAlive()); + + dagClient.close(); + boolean reloaderStillAlive = true; + for (int i = 0; i < 10; i++) { + reloaderStillAlive = reloaderThread.isAlive(); + if (!reloaderStillAlive) { + break; + } + Thread.sleep(1000); + } + Assert.assertFalse("Reloader is still alive", reloaderStillAlive); + } } From 6c53307adb0ec1e31d3c20ab599f38e6df449300 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Thu, 22 Oct 2020 09:13:46 +0200 Subject: [PATCH 272/512] =?UTF-8?q?TEZ-4243:=20Changes=20for=200.10.0=20re?= =?UTF-8?q?lease=20(L=C3=A1szl=C3=B3=20Bodor=20reviewed=20by=20Jonathan=20?= =?UTF-8?q?Turner=20Eagles)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- Tez_DOAP.rdf | 7 +++++ .../markdown/releases/apache-tez-0-10-0.md | 30 +++++++++++++++++++ docs/src/site/markdown/releases/index.md | 1 + docs/src/site/site.xml | 1 + 4 files changed, 39 insertions(+) create mode 100644 docs/src/site/markdown/releases/apache-tez-0-10-0.md diff --git a/Tez_DOAP.rdf b/Tez_DOAP.rdf index 68142ec096..055a45cb5f 100644 --- a/Tez_DOAP.rdf +++ b/Tez_DOAP.rdf @@ -34,6 +34,13 @@ Java + + + Version 0.10.0 + 2020-10-15 + 0.10.0 + + Version 0.9.2 diff --git a/docs/src/site/markdown/releases/apache-tez-0-10-0.md b/docs/src/site/markdown/releases/apache-tez-0-10-0.md new file mode 100644 index 0000000000..0ba6f7e243 --- /dev/null +++ b/docs/src/site/markdown/releases/apache-tez-0-10-0.md @@ -0,0 +1,30 @@ + + +Apache TEZ® 0.10.0 + +Apache TEZ® 0.10.0 +---------------------- + +- [Download Release Artifacts](http://www.apache.org/dyn/closer.lua/tez/0.10.0/) +- [Release Notes](0.10.0/release-notes.txt) +- Documentation + - [API Javadocs](0.10.0/tez-api-javadocs/index.html) : Documentation for the Tez APIs + - [Runtime Library Javadocs](0.10.0/tez-runtime-library-javadocs/index.html) : Documentation for built-in implementations of useful Inputs, Outputs, Processors etc. written based on the Tez APIs + - [Tez Mapreduce Javadocs](0.10.0/tez-mapreduce-javadocs/index.html) : Documentation for built-in implementations of Mapreduce compatible Inputs, Outputs, Processors etc. written based on the Tez APIs + - [Tez Configuration](0.10.0/tez-api-javadocs/configs/TezConfiguration.html) : Documentation for configurations of Tez. These configurations are typically specified in tez-site.xml. + - [Tez Runtime Configuration](0.10.0/tez-runtime-library-javadocs/configs/TezRuntimeConfiguration.html) : Documentation for runtime configurations of Tez. These configurations are typically specified by job submitters. diff --git a/docs/src/site/markdown/releases/index.md b/docs/src/site/markdown/releases/index.md index 23ddd43f67..26c18b0017 100644 --- a/docs/src/site/markdown/releases/index.md +++ b/docs/src/site/markdown/releases/index.md @@ -19,6 +19,7 @@ Releases ------------ +- [Apache TEZ® 0.10.0](./apache-tez-0-10-0.html) (Oct 15, 2020) - [Apache TEZ® 0.9.2](./apache-tez-0-9-2.html) (Mar 29, 2019) - [Apache TEZ® 0.9.1](./apache-tez-0-9-1.html) (Jan 04, 2018) - [Apache TEZ® 0.9.0](./apache-tez-0-9-0.html) (Jul 27, 2017) diff --git a/docs/src/site/site.xml b/docs/src/site/site.xml index b4fbfbf878..04187dd3f7 100644 --- a/docs/src/site/site.xml +++ b/docs/src/site/site.xml @@ -129,6 +129,7 @@ + From 4c339011383b0501e8080729c015ef642b7a2bbc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Wed, 11 Nov 2020 14:58:24 +0100 Subject: [PATCH 273/512] =?UTF-8?q?TEZ-4237:=20Upgrade=20async-http-client?= =?UTF-8?q?-1.9.40=20due=20to=20CVE-2017-14063=20(L=C3=A1szl=C3=B3=20Bodor?= =?UTF-8?q?=20reviewed=20by=20Ashutosh=20Chauhan)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- pom.xml | 6 +- tez-plugins/tez-aux-services/pom.xml | 12 +++ tez-runtime-library/pom.xml | 2 +- .../java/org/apache/tez/http/SSLFactory.java | 24 ++++-- .../http/async/netty/AsyncHttpConnection.java | 27 +++---- .../netty/TezBodyDeferringAsyncHandler.java | 30 ++++---- .../common/shuffle/impl/ShuffleManager.java | 3 +- .../orderedgrouped/ShuffleScheduler.java | 1 + .../apache/tez/http/TestHttpConnection.java | 24 ++---- .../apache/tez/test/TestSecureShuffle.java | 75 +++++++++++++++++-- 10 files changed, 140 insertions(+), 64 deletions(-) diff --git a/pom.xml b/pom.xml index 26b20d8094..04088bd9f8 100644 --- a/pom.xml +++ b/pom.xml @@ -233,9 +233,9 @@ ${slf4j.version} - com.ning - async-http-client - 1.9.40 + org.asynchttpclient + async-http-client + 2.12.1 org.slf4j diff --git a/tez-plugins/tez-aux-services/pom.xml b/tez-plugins/tez-aux-services/pom.xml index 0a80788d2a..97096c8332 100644 --- a/tez-plugins/tez-aux-services/pom.xml +++ b/tez-plugins/tez-aux-services/pom.xml @@ -55,6 +55,12 @@ org.apache.hadoop hadoop-hdfs provided + + + io.netty + * + + org.apache.hadoop @@ -126,6 +132,12 @@ hadoop-hdfs test test-jar + + + io.netty + * + + org.apache.tez diff --git a/tez-runtime-library/pom.xml b/tez-runtime-library/pom.xml index b59d2fb774..ecabf8976a 100644 --- a/tez-runtime-library/pom.xml +++ b/tez-runtime-library/pom.xml @@ -30,7 +30,7 @@ RoaringBitmap - com.ning + org.asynchttpclient async-http-client diff --git a/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java b/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java index 203eb40dcd..4147be8980 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java @@ -18,7 +18,12 @@ package org.apache.tez.http; -import com.ning.http.client.AsyncHttpClientConfig; +import org.asynchttpclient.DefaultAsyncHttpClientConfig; + +import io.netty.handler.ssl.ClientAuth; +import io.netty.handler.ssl.JdkSslContext; +import io.netty.handler.ssl.SupportedCipherSuiteFilter; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; @@ -223,17 +228,20 @@ public HttpURLConnection configure(HttpURLConnection conn) throws IOException { } /** - * Set ssl context for {@link com.ning.http.client.AsyncHttpClientConfig.Builder} + * Set ssl context for {@link org.asynchttpclient.DefaultAsyncHttpClientConfig.Builder} * - * @param asyncNingBuilder {@link com.ning.http.client.AsyncHttpClientConfig.Builder} instance to + * @param builder {@link org.asynchttpclient.DefaultAsyncHttpClientConfig.Builder} instance to * configure. * @throws IOException if an IO error occurred. */ - public void configure(AsyncHttpClientConfig.Builder asyncNingBuilder) throws IOException { - if (asyncNingBuilder != null) { - asyncNingBuilder.setSSLContext(context); - asyncNingBuilder.setHostnameVerifier(getHostnameVerifier()); + public void configure(DefaultAsyncHttpClientConfig.Builder builder) throws IOException { + if (builder != null) { + JdkSslContext jdkSslContext = + new JdkSslContext(context, mode.equals(Mode.CLIENT), /* ciphers */null, + SupportedCipherSuiteFilter.INSTANCE, /* ApplicationProtocolConfig */ null, + requireClientCert ? ClientAuth.REQUIRE : ClientAuth.OPTIONAL, enabledProtocols, + /* startTls */ true); + builder.setSslContext(jdkSslContext); } } - } \ No newline at end of file diff --git a/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java b/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java index ac0a49c401..43f64b82b3 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java @@ -20,12 +20,13 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.tez.common.Preconditions; -import com.ning.http.client.AsyncHttpClient; -import com.ning.http.client.AsyncHttpClientConfig; -import com.ning.http.client.ListenableFuture; -import com.ning.http.client.Request; -import com.ning.http.client.RequestBuilder; -import com.ning.http.client.Response; +import org.asynchttpclient.AsyncHttpClient; +import org.asynchttpclient.DefaultAsyncHttpClient; +import org.asynchttpclient.DefaultAsyncHttpClientConfig; +import org.asynchttpclient.ListenableFuture; +import org.asynchttpclient.Request; +import org.asynchttpclient.RequestBuilder; +import org.asynchttpclient.Response; import org.apache.commons.io.IOUtils; import org.apache.tez.http.BaseHttpConnection; import org.apache.tez.http.HttpConnectionParams; @@ -76,7 +77,7 @@ private void initClient(HttpConnectionParams httpConnParams) throws IOException synchronized (AsyncHttpConnection.class) { if (httpAsyncClient == null) { LOG.info("Initializing AsyncClient (TezBodyDeferringAsyncHandler)"); - AsyncHttpClientConfig.Builder builder = new AsyncHttpClientConfig.Builder(); + DefaultAsyncHttpClientConfig.Builder builder = new DefaultAsyncHttpClientConfig.Builder(); if (httpConnParams.isSslShuffle()) { //Configure SSL SSLFactory sslFactory = httpConnParams.getSslFactory(); @@ -91,16 +92,16 @@ private void initClient(HttpConnectionParams httpConnParams) throws IOException * setMaxConnections & addRequestFilter. */ builder - .setAllowPoolingConnections(httpConnParams.isKeepAlive()) - .setAllowPoolingSslConnections(httpConnParams.isKeepAlive()) + .setKeepAlive(httpConnParams.isKeepAlive()) .setCompressionEnforced(false) //.setExecutorService(applicationThreadPool) - //.addRequestFilter(new ThrottleRequestFilter()) + //.addRequestFilter(new ThrottleRequestFilter(1)) .setMaxConnectionsPerHost(1) .setConnectTimeout(httpConnParams.getConnectionTimeout()) - .setDisableUrlEncodingForBoundedRequests(true) + .setDisableUrlEncodingForBoundRequests(true) .build(); - httpAsyncClient = new AsyncHttpClient(builder.build()); + DefaultAsyncHttpClientConfig config = builder.build(); + httpAsyncClient = new DefaultAsyncHttpClient(config); } } } @@ -208,7 +209,7 @@ public DataInputStream getInputStream() throws IOException, InterruptedException } @VisibleForTesting - public void close() { + public void close() throws IOException { httpAsyncClient.close(); httpAsyncClient = null; } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/TezBodyDeferringAsyncHandler.java b/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/TezBodyDeferringAsyncHandler.java index 8e83eaca64..acbdab7df9 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/TezBodyDeferringAsyncHandler.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/TezBodyDeferringAsyncHandler.java @@ -17,15 +17,16 @@ */ package org.apache.tez.http.async.netty; -import com.ning.http.client.AsyncHandler; -import com.ning.http.client.HttpResponseBodyPart; -import com.ning.http.client.HttpResponseHeaders; -import com.ning.http.client.HttpResponseStatus; -import com.ning.http.client.Response; +import org.asynchttpclient.AsyncHandler; +import org.asynchttpclient.HttpResponseBodyPart; +import org.asynchttpclient.HttpResponseStatus; +import org.asynchttpclient.Response; import org.apache.hadoop.classification.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import io.netty.handler.codec.http.HttpHeaders; + import java.io.FilterInputStream; import java.io.IOException; import java.io.InputStream; @@ -38,8 +39,8 @@ import java.util.concurrent.TimeUnit; /** - * Same as {@link com.ning.http.client.BodyDeferringAsyncHandler} with additional checks handle - * errors in getResponse(). Based on testing, at very high load {@link com.ning.http.client + * Same as {@link org.asynchttpclient.BodyDeferringAsyncHandler} with additional checks handle + * errors in getResponse(). Based on testing, at very high load {@link org.asynchttpclient * .BodyDeferringAsyncHandler} gets to hung state in getResponse() as it tries to wait * indefinitely for headers to arrive. This class tries to fix the problem by waiting only for * the connection timeout. @@ -92,27 +93,28 @@ public void onThrowable(Throwable t) { } } - public AsyncHandler.STATE onStatusReceived(HttpResponseStatus responseStatus) throws Exception { + public AsyncHandler.State onStatusReceived(HttpResponseStatus responseStatus) throws Exception { responseBuilder.reset(); responseBuilder.accumulate(responseStatus); statusReceived = true; - return AsyncHandler.STATE.CONTINUE; + return AsyncHandler.State.CONTINUE; } - public AsyncHandler.STATE onHeadersReceived(HttpResponseHeaders headers) throws Exception { + @Override + public AsyncHandler.State onHeadersReceived(HttpHeaders headers) throws Exception { responseBuilder.accumulate(headers); - return AsyncHandler.STATE.CONTINUE; + return AsyncHandler.State.CONTINUE; } - public AsyncHandler.STATE onBodyPartReceived(HttpResponseBodyPart bodyPart) throws Exception { + public AsyncHandler.State onBodyPartReceived(HttpResponseBodyPart bodyPart) throws Exception { // body arrived, flush headers if (!responseSet) { response = responseBuilder.build(); responseSet = true; headersArrived.countDown(); } - bodyPart.writeTo(output); - return AsyncHandler.STATE.CONTINUE; + output.write(bodyPart.getBodyPartBytes()); + return AsyncHandler.State.CONTINUE; } protected void closeOut() throws IOException { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java index 2b83ad8920..901ee08a7a 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java @@ -337,7 +337,8 @@ public ShuffleManager(InputContext inputContext, Configuration conf, int numInpu + ifileReadAhead + ", ifileReadAheadLength=" + ifileReadAheadLength +", " + "localDiskFetchEnabled=" + localDiskFetchEnabled + ", " + "sharedFetchEnabled=" + sharedFetchEnabled + ", " - + httpConnectionParams.toString() + ", maxTaskOutputAtOnce=" + maxTaskOutputAtOnce); + + httpConnectionParams.toString() + ", maxTaskOutputAtOnce=" + maxTaskOutputAtOnce + + ", asyncHttp=" + asyncHttp); } public void updateApproximateInputRecords(int delta) { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java index e7f63ab7ad..416041e005 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java @@ -441,6 +441,7 @@ public ShuffleScheduler(InputContext inputContext, + ", maxStallTimeFraction=" + maxStallTimeFraction + ", minReqProgressFraction=" + minReqProgressFraction + ", checkFailedFetchSinceLastCompletion=" + checkFailedFetchSinceLastCompletion + + ", asyncHttp=" + asyncHttp ); } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/http/TestHttpConnection.java b/tez-runtime-library/src/test/java/org/apache/tez/http/TestHttpConnection.java index ed4ed54c2e..afeb6e561f 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/http/TestHttpConnection.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/http/TestHttpConnection.java @@ -24,8 +24,6 @@ import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; import java.io.IOException; import java.net.ConnectException; @@ -42,7 +40,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -78,6 +76,7 @@ public Thread newThread(Runnable r) { }); url = new URL(NOT_HOSTED_URL); tokenSecretManager = mock(JobTokenSecretManager.class); + when(tokenSecretManager.computeHash(any())).thenReturn("1234".getBytes()); } @AfterClass @@ -89,7 +88,7 @@ public void baseTest(Callable worker, CountDownLatch latch, String message InterruptedException { long startTime = System.currentTimeMillis(); try { - Future future = executorService.submit(worker); + Future future = executorService.submit(worker); future.get(); } catch (ExecutionException e) { assertTrue(e.getCause().getCause() instanceof IOException); @@ -117,14 +116,13 @@ public void testConnectionTimeout() throws IOException, InterruptedException { } @Test(timeout = 20000) - @SuppressWarnings("unchecked") //Should be interruptible public void testAsyncHttpConnectionInterrupt() throws IOException, InterruptedException, ExecutionException { CountDownLatch latch = new CountDownLatch(1); HttpConnectionParams params = getConnectionParams(); AsyncHttpConnection asyncHttpConn = getAsyncHttpConnection(params); - Future future = executorService.submit(new Worker(latch, asyncHttpConn, true)); + Future future = executorService.submit(new Worker(latch, asyncHttpConn, true)); while(currentThread == null) { synchronized (this) { @@ -153,24 +151,14 @@ HttpConnectionParams getConnectionParams() { HttpConnection getHttpConnection(HttpConnectionParams params) throws IOException { HttpConnection realConn = new HttpConnection(url, params, "log", tokenSecretManager); HttpConnection connection = spy(realConn); - - doAnswer(new Answer() { - public Void answer(InvocationOnMock invocation) { - return null; - } - }).when(connection).computeEncHash(); + realConn.computeEncHash(); return connection; } AsyncHttpConnection getAsyncHttpConnection(HttpConnectionParams params) throws IOException { AsyncHttpConnection realConn = new AsyncHttpConnection(url, params, "log", tokenSecretManager); AsyncHttpConnection connection = spy(realConn); - - doAnswer(new Answer() { - public Void answer(InvocationOnMock invocation) { - return null; - } - }).when(connection).computeEncHash(); + realConn.computeEncHash(); return connection; } diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java b/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java index 6d34464c04..aed240997e 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java @@ -25,8 +25,13 @@ import java.io.File; import java.io.IOException; import java.io.OutputStreamWriter; +import java.net.InetAddress; +import java.security.KeyPair; +import java.security.cert.X509Certificate; import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; +import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; @@ -37,7 +42,7 @@ import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream; import org.apache.hadoop.mapreduce.MRConfig; import org.apache.hadoop.security.ssl.KeyStoreTestUtil; -import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.security.ssl.SSLFactory; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.mapreduce.examples.TestOrderedWordCount; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; @@ -133,7 +138,7 @@ public void setupTezCluster() throws Exception { conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES_LIMIT, 2); conf.setLong(TezConfiguration.TEZ_AM_SLEEP_TIME_BEFORE_EXIT_MILLIS, 500); - + conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_USE_ASYNC_HTTP, asyncHttp); String sslConf = conf.get(SSL_CLIENT_CONF_KEY, "ssl-client.xml"); conf.addResource(sslConf); @@ -205,10 +210,68 @@ private static void createSampleFile(Path inputLoc) throws IOException { */ private static void setupKeyStores() throws Exception { keysStoresDir.mkdirs(); - String sslConfsDir = - KeyStoreTestUtil.getClasspathDir(TestSecureShuffle.class); + String sslConfsDir = KeyStoreTestUtil.getClasspathDir(TestSecureShuffle.class); + + setupSSLConfig(keysStoresDir.getAbsolutePath(), sslConfsDir, conf, true, true, ""); + } + + /** + * This is a copied version of hadoop's KeyStoreTestUtil.setupSSLConfig which was needed to create + * server certs with actual hostname in CN instead of "localhost". While upgrading async http + * client in TEZ-4237, it turned out that netty doesn't support custom hostname verifiers anymore + * (as discussed in https://github.com/AsyncHttpClient/async-http-client/issues/928), that's why + * it cannot be set for an async http connection. So instead of hacking an ALLOW_ALL verifier + * somehow (which cannot be propagated to netty), a valid certificate with the actual hostname + * should be generated in setupSSLConfig, so the only change is the usage of + * "InetAddress.getLocalHost().getHostName()". + */ + public static void setupSSLConfig(String keystoresDir, String sslConfDir, Configuration config, + boolean useClientCert, boolean trustStore, String excludeCiphers) throws Exception { + String clientKS = keystoresDir + "/clientKS.jks"; + String clientPassword = "clientP"; + String serverKS = keystoresDir + "/serverKS.jks"; + String serverPassword = "serverP"; + String trustKS = null; + String trustPassword = "trustP"; + + File sslClientConfFile = new File(sslConfDir, KeyStoreTestUtil.getClientSSLConfigFileName()); + File sslServerConfFile = new File(sslConfDir, KeyStoreTestUtil.getServerSSLConfigFileName()); + + Map certs = new HashMap(); + + if (useClientCert) { + KeyPair cKP = KeyStoreTestUtil.generateKeyPair("RSA"); + X509Certificate cCert = + KeyStoreTestUtil.generateCertificate("CN=localhost, O=client", cKP, 30, "SHA1withRSA"); + KeyStoreTestUtil.createKeyStore(clientKS, clientPassword, "client", cKP.getPrivate(), cCert); + certs.put("client", cCert); + } + + String localhostName = InetAddress.getLocalHost().getHostName(); + KeyPair sKP = KeyStoreTestUtil.generateKeyPair("RSA"); + X509Certificate sCert = + KeyStoreTestUtil.generateCertificate("CN="+localhostName+", O=server", sKP, 30, "SHA1withRSA"); + KeyStoreTestUtil.createKeyStore(serverKS, serverPassword, "server", sKP.getPrivate(), sCert); + certs.put("server", sCert); + + if (trustStore) { + trustKS = keystoresDir + "/trustKS.jks"; + KeyStoreTestUtil.createTrustStore(trustKS, trustPassword, certs); + } + + Configuration clientSSLConf = KeyStoreTestUtil.createClientSSLConfig(clientKS, clientPassword, + clientPassword, trustKS, excludeCiphers); + Configuration serverSSLConf = KeyStoreTestUtil.createServerSSLConfig(serverKS, serverPassword, + serverPassword, trustKS, excludeCiphers); + + KeyStoreTestUtil.saveConfig(sslClientConfFile, clientSSLConf); + KeyStoreTestUtil.saveConfig(sslServerConfFile, serverSSLConf); + + // this will be ignored for AsyncHttpConnection, see method comments above + config.set(SSLFactory.SSL_HOSTNAME_VERIFIER_KEY, "ALLOW_ALL"); - KeyStoreTestUtil.setupSSLConfig(keysStoresDir.getAbsolutePath(), - sslConfsDir, conf, true); + config.set(SSLFactory.SSL_CLIENT_CONF_KEY, sslClientConfFile.getName()); + config.set(SSLFactory.SSL_SERVER_CONF_KEY, sslServerConfFile.getName()); + config.setBoolean(SSLFactory.SSL_REQUIRE_CLIENT_CERT_KEY, useClientCert); } } From 989d286d09cac7c4e4c5a0e06dd75ea5a6f15478 Mon Sep 17 00:00:00 2001 From: Marton Bod Date: Sat, 21 Nov 2020 07:45:38 +0100 Subject: [PATCH 274/512] =?UTF-8?q?TEZ-4248:=20MRReaderMapred=20should=20p?= =?UTF-8?q?ropagate=20properties=20based=20on=20config=20(Marton=20Bod=20v?= =?UTF-8?q?ia=20L=C3=A1szl=C3=B3=20Bodor)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../apache/tez/dag/api/TezConfiguration.java | 7 +++ .../tez/mapreduce/lib/MRReaderMapred.java | 25 +++++++++-- .../mapreduce/lib/TestKVReadersWithMR.java | 45 +++++++++++++++++++ 3 files changed, 73 insertions(+), 4 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 2af08a9044..85f85181ca 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -2114,4 +2114,11 @@ static Set getPropertySet() { @ConfigurationProperty public static final String TEZ_JOB_FS_SERVERS_TOKEN_RENEWAL_EXCLUDE = "tez.job.fs-servers.token-renewal.exclude"; + /** + * Comma-separated list of properties that MRReaderMapred should return (if present) when calling for config updates. + */ + @ConfigurationScope(Scope.VERTEX) + @ConfigurationProperty + public static final String TEZ_MRREADER_CONFIG_UPDATE_PROPERTIES = "tez.mrreader.config.update.properties"; + } diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapred.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapred.java index 21a4f96df4..e04ae7f046 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapred.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapred.java @@ -30,6 +30,7 @@ import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.JobContext; import org.apache.hadoop.mapred.RecordReader; +import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.common.counters.TezCounter; import org.apache.tez.common.counters.TezCounters; import org.apache.tez.mapreduce.hadoop.mapred.MRReporter; @@ -145,6 +146,13 @@ public Object getCurrentValue() throws IOException { * @return the additional fields set by {@link MRInput} */ public Configuration getConfigUpdates() { + String propertyList = jobConf.get(TezConfiguration.TEZ_MRREADER_CONFIG_UPDATE_PROPERTIES); + if (propertyList != null) { + String[] properties = propertyList.split(","); + for (String prop : properties) { + addToIncrementalConfFromJobConf(prop); + } + } if (incrementalConf != null) { return new Configuration(incrementalConf); } @@ -161,15 +169,24 @@ private void setupOldRecordReader() throws IOException { setupComplete = true; } - private void setIncrementalConfigParams(InputSplit inputSplit) { - if (inputSplit instanceof FileSplit) { - FileSplit fileSplit = (FileSplit) inputSplit; + private void setIncrementalConfigParams(InputSplit split) { + if (split instanceof FileSplit) { + FileSplit fileSplit = (FileSplit) split; this.incrementalConf = new Configuration(false); this.incrementalConf.set(JobContext.MAP_INPUT_FILE, fileSplit.getPath().toString()); this.incrementalConf.setLong(JobContext.MAP_INPUT_START, fileSplit.getStart()); this.incrementalConf.setLong(JobContext.MAP_INPUT_PATH, fileSplit.getLength()); } - LOG.info("Processing split: " + inputSplit); + LOG.info("Processing split: " + split); + } + + private void addToIncrementalConfFromJobConf(String property) { + if (jobConf.get(property) != null) { + if (incrementalConf == null) { + incrementalConf = new Configuration(false); + } + incrementalConf.set(property, jobConf.get(property)); + } } } diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/lib/TestKVReadersWithMR.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/lib/TestKVReadersWithMR.java index dad18deeca..21a92469e2 100644 --- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/lib/TestKVReadersWithMR.java +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/lib/TestKVReadersWithMR.java @@ -18,6 +18,7 @@ package org.apache.tez.mapreduce.lib; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapreduce.InputSplit; @@ -25,12 +26,15 @@ import org.apache.tez.common.counters.TaskCounter; import org.apache.tez.common.counters.TezCounter; import org.apache.tez.common.counters.TezCounters; +import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.runtime.api.InputContext; import org.junit.Before; import org.junit.Test; import java.io.IOException; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Mockito.mock; @@ -107,6 +111,47 @@ public void testWithSpecificNumberOfKV_MapReduce(int kvPairs) throws IOException } } + @Test + public void testIncrementalConfigWithMultipleProperties() throws IOException { + InputContext mockContext = mock(InputContext.class); + MRReaderMapred reader = new MRReaderMapred(conf, counters, inputRecordCounter, mockContext); + conf.set(TezConfiguration.TEZ_MRREADER_CONFIG_UPDATE_PROPERTIES, "column.names,does_not_exist,column.ids"); + conf.set("column.names", "first_name,last_name,id"); + conf.set("column.ids", "1,2,3"); + conf.set("random", "value"); + + Configuration incrementalConf = reader.getConfigUpdates(); + + assertEquals(2, incrementalConf.size()); + assertEquals("first_name,last_name,id", incrementalConf.get("column.names")); + assertEquals("1,2,3", incrementalConf.get("column.ids")); + } + + @Test + public void testIncrementalConfigWithSingleProperty() throws IOException { + InputContext mockContext = mock(InputContext.class); + MRReaderMapred reader = new MRReaderMapred(conf, counters, inputRecordCounter, mockContext); + conf.set(TezConfiguration.TEZ_MRREADER_CONFIG_UPDATE_PROPERTIES, "column.names"); + conf.set("column.names", "first_name,last_name,id"); + conf.set("random", "value"); + + Configuration incrementalConf = reader.getConfigUpdates(); + + assertEquals(1, incrementalConf.size()); + assertEquals("first_name,last_name,id", incrementalConf.get("column.names")); + } + + @Test + public void testIncrementalConfigWithZeroProperty() throws IOException { + InputContext mockContext = mock(InputContext.class); + MRReaderMapred reader = new MRReaderMapred(conf, counters, inputRecordCounter, mockContext); + conf.set("random", "value"); + + Configuration incrementalConf = reader.getConfigUpdates(); + + assertNull(incrementalConf); + } + static class DummyRecordReader implements RecordReader { int records; From 938d6a105c398531774a8c341dc851b7ceee4eeb Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Tue, 24 Nov 2020 13:10:15 +0530 Subject: [PATCH 275/512] TEZ-4251: Acquiring locks for getInputVertices and getOutputVertices is not consistent (Krisztian Kasa reviewed by Rajesh Balamohan) --- .../java/org/apache/tez/dag/app/dag/impl/VertexImpl.java | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index 711d02827e..f86dc987b3 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -4460,12 +4460,7 @@ public int hashCode() { @Override public Map getInputVertices() { - readLock.lock(); - try { - return Collections.unmodifiableMap(this.sourceVertices); - } finally { - readLock.unlock(); - } + return Collections.unmodifiableMap(this.sourceVertices); } @Override From dadc09f5a44c1cb61af00efecb3d27b92c92aa8f Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Wed, 2 Dec 2020 03:58:42 +0530 Subject: [PATCH 276/512] =?UTF-8?q?TEZ-4244:=20Consider=20using=20RawLocal?= =?UTF-8?q?FileSystem=20in=20LocalDiskFetchedInput=20(Rajesh=20Balamohan?= =?UTF-8?q?=20reviewed=20by=20L=C3=A1szl=C3=B3=20Bodor)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../runtime/library/common/shuffle/LocalDiskFetchedInput.java | 2 +- .../apache/tez/runtime/library/common/shuffle/TestFetcher.java | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/LocalDiskFetchedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/LocalDiskFetchedInput.java index cb6ea0599f..d7ec265048 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/LocalDiskFetchedInput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/LocalDiskFetchedInput.java @@ -49,7 +49,7 @@ public LocalDiskFetchedInput(long startOffset, long compressedSize, this.size = compressedSize; this.startOffset = startOffset; this.inputFile = inputFile; - localFS = FileSystem.getLocal(conf); + localFS = FileSystem.getLocal(conf).getRaw(); } @Override diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java index 05d4eb4145..cea3272818 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java @@ -18,6 +18,7 @@ package org.apache.tez.runtime.library.common.shuffle; +import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.tez.runtime.library.common.CompositeInputAttemptIdentifier; import static org.mockito.Matchers.anyInt; import static org.mockito.Matchers.anyLong; @@ -258,7 +259,7 @@ protected void verifyFetchSucceeded(FetcherCallback callback, CompositeInputAtte LocalDiskFetchedInput f = capturedFetchedInput.getValue(); Assert.assertEquals("success callback filename", f.getInputFile().toString(), SHUFFLE_INPUT_FILE_PREFIX + pathComponent); - Assert.assertTrue("success callback fs", f.getLocalFS() instanceof LocalFileSystem); + Assert.assertTrue("success callback fs", f.getLocalFS() instanceof RawLocalFileSystem); Assert.assertEquals("success callback filesystem", f.getStartOffset(), p * 10); Assert.assertEquals("success callback compressed size", f.getSize(), p * 100); Assert.assertEquals("success callback input id", f.getInputAttemptIdentifier(), srcAttempId.expand(0)); From 51953a67641392eb5cf4b2c4fb035baab0badf27 Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Mon, 7 Dec 2020 12:35:26 -0800 Subject: [PATCH 277/512] TEZ-4256 : Reduce key comparisons in reducer side (Rajesh Balamohan via Gopal V) Signed-off-by: Ashutosh Chauhan --- .../library/common/shuffle/orderedgrouped/MergeManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java index 8565e7111c..59ff577fed 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java @@ -1128,7 +1128,7 @@ public KeyState readRawKey(DataInputBuffer key) throws IOException { final int klen = kb.getLength() - kp; key.reset(kb.getData(), kp, klen); bytesRead += klen; - return KeyState.NEW_KEY; + return kvIter.isSameKey() ? KeyState.SAME_KEY : KeyState.NEW_KEY; } return KeyState.NO_KEY; } From b2a05115dcce28a2b188435b445b6d7e43103ab1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Mustafa=20=C4=B0man?= Date: Mon, 7 Dec 2020 22:40:07 +0100 Subject: [PATCH 278/512] =?UTF-8?q?TEZ-4253:=20Revert=20TEZ-4170=20(Mustaf?= =?UTF-8?q?a=20=C4=B0man=20via=20Attila=20Magyar,=20Ashutosh=20Chauhan)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../app/dag/RootInputInitializerManager.java | 274 +++++++++--------- .../tez/dag/app/dag/impl/VertexImpl.java | 25 +- .../dag/TestRootInputInitializerManager.java | 116 +------- .../tez/dag/app/dag/impl/TestDAGRecovery.java | 6 +- .../dag/impl/TestRootInputVertexManager.java | 5 +- .../tez/dag/app/dag/impl/TestVertexImpl.java | 99 +------ 6 files changed, 180 insertions(+), 345 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java index bd4bcd89a8..3c4a05eb84 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java @@ -18,33 +18,35 @@ package org.apache.tez.dag.app.dag; -import static org.apache.tez.dag.app.dag.VertexState.FAILED; +import javax.annotation.Nullable; import java.io.IOException; import java.lang.reflect.UndeclaredThrowableException; import java.security.PrivilegedExceptionAction; -import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.Objects; -import javax.annotation.Nullable; +import org.apache.tez.common.Preconditions; +import com.google.common.collect.LinkedListMultimap; +import com.google.common.collect.ListMultimap; +import com.google.common.collect.Lists; -import org.apache.commons.lang.exception.ExceptionUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.event.EventHandler; -import org.apache.tez.common.Preconditions; +import org.apache.tez.common.GuavaShim; import org.apache.tez.common.ReflectionUtils; import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.dag.api.InputDescriptor; @@ -52,38 +54,38 @@ import org.apache.tez.dag.api.RootInputLeafOutput; import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.api.TezUncheckedException; +import org.apache.tez.dag.api.event.*; import org.apache.tez.dag.api.event.VertexState; -import org.apache.tez.dag.api.event.VertexStateUpdate; import org.apache.tez.dag.api.oldrecords.TaskState; import org.apache.tez.dag.app.AppContext; import org.apache.tez.dag.app.dag.event.VertexEventRootInputFailed; import org.apache.tez.dag.app.dag.event.VertexEventRootInputInitialized; import org.apache.tez.dag.app.dag.impl.AMUserCodeException; -import org.apache.tez.dag.app.dag.impl.AMUserCodeException.Source; import org.apache.tez.dag.app.dag.impl.TezRootInputInitializerContextImpl; -import org.apache.tez.dag.app.dag.impl.VertexImpl; +import org.apache.tez.dag.app.dag.impl.AMUserCodeException.Source; import org.apache.tez.dag.records.TezTaskID; import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.InputInitializer; import org.apache.tez.runtime.api.InputInitializerContext; -import org.apache.tez.runtime.api.events.InputInitializerEvent; -import org.apache.tez.runtime.api.impl.TezEvent; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.LinkedListMultimap; -import com.google.common.collect.ListMultimap; -import com.google.common.collect.Lists; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import com.google.common.util.concurrent.ThreadFactoryBuilder; + +import org.apache.tez.runtime.api.events.InputInitializerEvent; +import org.apache.tez.runtime.api.impl.TezEvent; public class RootInputInitializerManager { private static final Logger LOG = LoggerFactory.getLogger(RootInputInitializerManager.class); - @VisibleForTesting - protected ListeningExecutorService executor; + private final ExecutorService rawExecutor; + private final ListeningExecutorService executor; @SuppressWarnings("rawtypes") private final EventHandler eventHandler; private volatile boolean isStopped = false; @@ -94,135 +96,51 @@ public class RootInputInitializerManager { private final AppContext appContext; @VisibleForTesting - final Map initializerMap = new ConcurrentHashMap<>(); + final Map initializerMap = new HashMap(); public RootInputInitializerManager(Vertex vertex, AppContext appContext, UserGroupInformation dagUgi, StateChangeNotifier stateTracker) { this.appContext = appContext; this.vertex = vertex; this.eventHandler = appContext.getEventHandler(); - this.executor = appContext.getExecService(); + this.rawExecutor = Executors.newCachedThreadPool(new ThreadFactoryBuilder() + .setDaemon(true).setNameFormat("InputInitializer {" + this.vertex.getName() + "} #%d").build()); + this.executor = MoreExecutors.listeningDecorator(rawExecutor); this.dagUgi = dagUgi; this.entityStateTracker = stateTracker; } + public void runInputInitializers(List> + inputs) throws TezException { + for (RootInputLeafOutput input : inputs) { - public void runInputInitializers(List> inputs, - List pendingInitializerEvents) { - List initWrappers = createInitializerWrappers(inputs); - if (!initWrappers.isEmpty()) { - executor.submit(() -> createAndStartInitializing(pendingInitializerEvents, initWrappers)); - } - } - - /** - * Create input wrappers for all inputs in parallel. - * - * @param inputs - * @return - */ - protected List createInitializerWrappers( - List> inputs) { - String current = null; - final List result = Collections.synchronizedList(new ArrayList<>()); - try { - final List> fResults = new ArrayList<>(); - for (RootInputLeafOutput each : inputs) { - current = each.getName(); - fResults.add(executor.submit(() -> { - InitializerWrapper initializer = createInitializerWrapper(each); - initializerMap.put(each.getName(), initializer); - registerPendingVertex(each, initializer); - result.add(initializer); - return null; - })); - } - for(Future f : fResults) { - f.get(); - } - } catch (InterruptedException | ExecutionException t) { - failVertex(t, current); - } - return result; - } - - void failVertex(Throwable t, String inputName) { - VertexImpl vertexImpl = (VertexImpl) vertex; - String msg = "Fail to create InputInitializerManager, " + ExceptionUtils.getStackTrace(t); - LOG.info(msg); - vertexImpl.finished(FAILED, VertexTerminationCause.INIT_FAILURE, msg); - eventHandler.handle(new VertexEventRootInputFailed(vertex.getVertexId(), inputName, - new AMUserCodeException(AMUserCodeException.Source.InputInitializer, t))); - } + InputInitializerContext context = + new TezRootInputInitializerContextImpl(input, vertex, appContext, this); - /** - * Start initializers in parallel. - * - * @param pendingInitializerEvents - * @param result - */ - protected void createAndStartInitializing(List pendingInitializerEvents, List result) { - handleInitializerEvents(pendingInitializerEvents); - pendingInitializerEvents.clear(); - for (InitializerWrapper inputWrapper : result) { - executor.submit(() -> runInitializerAndProcessResult(inputWrapper)); - } - } - - private void runInitializerAndProcessResult(InitializerWrapper initializer) { - try { - List result = runInitializer(initializer); - LOG.info("Succeeded InputInitializer for Input: " + initializer.getInput().getName() + - " on vertex " + initializer.getVertexLogIdentifier()); - eventHandler.handle(new VertexEventRootInputInitialized(vertex.getVertexId(), - initializer.getInput().getName(), result)); - } catch (Throwable t) { - if (t instanceof UndeclaredThrowableException) { - t = t.getCause(); - } - LOG.info("Failed InputInitializer for Input: " + initializer.getInput().getName() + - " on vertex " + initializer.getVertexLogIdentifier()); - eventHandler.handle(new VertexEventRootInputFailed(vertex.getVertexId(), initializer.getInput().getName(), - new AMUserCodeException(Source.InputInitializer,t))); - } finally { - initializer.setComplete(); - } - } - - private List runInitializer(InitializerWrapper initializer) throws IOException, InterruptedException { - return dagUgi.doAs((PrivilegedExceptionAction>) () -> { - LOG.info( - "Starting InputInitializer for Input: " + initializer.getInput().getName() + - " on vertex " + initializer.getVertexLogIdentifier()); + InputInitializer initializer; try { - TezUtilsInternal.setHadoopCallerContext(appContext.getHadoopShim(), - initializer.vertexId); - return initializer.getInitializer().initialize(); + TezUtilsInternal.setHadoopCallerContext(appContext.getHadoopShim(), vertex.getVertexId()); + initializer = createInitializer(input, context); } finally { appContext.getHadoopShim().clearHadoopCallerContext(); } - }); - } - private InitializerWrapper createInitializerWrapper(RootInputLeafOutput input) throws TezException { - InputInitializerContext context = - new TezRootInputInitializerContextImpl(input, vertex, appContext, this); - try { - TezUtilsInternal.setHadoopCallerContext(appContext.getHadoopShim(), vertex.getVertexId()); - InputInitializer initializer = createInitializer(input, context); - return new InitializerWrapper(input, initializer, context, vertex, entityStateTracker, appContext); - } finally { - appContext.getHadoopShim().clearHadoopCallerContext(); - } - } + InitializerWrapper initializerWrapper = + new InitializerWrapper(input, initializer, context, vertex, entityStateTracker, appContext); - private void registerPendingVertex(RootInputLeafOutput input, InitializerWrapper initializerWrapper) { - // Register pending vertex update registrations - List vertexUpdateRegistrations = pendingVertexRegistrations.removeAll(input.getName()); - if (vertexUpdateRegistrations != null) { - for (VertexUpdateRegistrationHolder h : vertexUpdateRegistrations) { - initializerWrapper.registerForVertexStateUpdates(h.vertexName, h.stateSet); + // Register pending vertex update registrations + List vertexUpdateRegistrations = + pendingVertexRegistrations.removeAll(input.getName()); + if (vertexUpdateRegistrations != null) { + for (VertexUpdateRegistrationHolder h : vertexUpdateRegistrations) { + initializerWrapper.registerForVertexStateUpdates(h.vertexName, h.stateSet); + } } + + initializerMap.put(input.getName(), initializerWrapper); + ListenableFuture> future = executor + .submit(new InputInitializerCallable(initializerWrapper, dagUgi, appContext)); + Futures.addCallback(future, createInputInitializerCallback(initializerWrapper), GuavaShim.directExecutor()); } } @@ -315,6 +233,11 @@ public void registerForVertexUpdates(String vertexName, String inputName, } } + @VisibleForTesting + protected InputInitializerCallback createInputInitializerCallback(InitializerWrapper initializer) { + return new InputInitializerCallback(initializer, eventHandler, vertex.getVertexId()); + } + @VisibleForTesting @InterfaceAudience.Private public InitializerWrapper getInitializerWrapper(String inputName) { @@ -322,7 +245,92 @@ public InitializerWrapper getInitializerWrapper(String inputName) { } public void shutdown() { - isStopped = true; + if (executor != null && !isStopped) { + // Don't really care about what is running if an error occurs. If no error + // occurs, all execution is complete. + executor.shutdownNow(); + isStopped = true; + } + } + + private static class InputInitializerCallable implements + Callable> { + + private final InitializerWrapper initializerWrapper; + private final UserGroupInformation ugi; + private final AppContext appContext; + + InputInitializerCallable(InitializerWrapper initializer, UserGroupInformation ugi, + AppContext appContext) { + this.initializerWrapper = initializer; + this.ugi = ugi; + this.appContext = appContext; + } + + @Override + public List call() throws Exception { + List events = ugi.doAs(new PrivilegedExceptionAction>() { + @Override + public List run() throws Exception { + LOG.info( + "Starting InputInitializer for Input: " + initializerWrapper.getInput().getName() + + " on vertex " + initializerWrapper.getVertexLogIdentifier()); + try { + TezUtilsInternal.setHadoopCallerContext(appContext.getHadoopShim(), + initializerWrapper.vertexId); + return initializerWrapper.getInitializer().initialize(); + } finally { + appContext.getHadoopShim().clearHadoopCallerContext(); + } + } + }); + return events; + } + } + + @SuppressWarnings("rawtypes") + @VisibleForTesting + private static class InputInitializerCallback implements + FutureCallback> { + + private final InitializerWrapper initializer; + private final EventHandler eventHandler; + private final TezVertexID vertexID; + + InputInitializerCallback(InitializerWrapper initializer, + EventHandler eventHandler, TezVertexID vertexID) { + this.initializer = initializer; + this.eventHandler = eventHandler; + this.vertexID = vertexID; + } + + @SuppressWarnings("unchecked") + @Override + public void onSuccess(List result) { + initializer.setComplete(); + LOG.info( + "Succeeded InputInitializer for Input: " + initializer.getInput().getName() + + " on vertex " + initializer.getVertexLogIdentifier()); + eventHandler.handle(new VertexEventRootInputInitialized(vertexID, + initializer.getInput().getName(), result)); + } + + @SuppressWarnings("unchecked") + @Override + public void onFailure(Throwable t) { + // catch real root cause of failure, it would throw UndeclaredThrowableException + // if using UGI.doAs + if (t instanceof UndeclaredThrowableException) { + t = t.getCause(); + } + initializer.setComplete(); + LOG.info( + "Failed InputInitializer for Input: " + initializer.getInput().getName() + + " on vertex " + initializer.getVertexLogIdentifier()); + eventHandler + .handle(new VertexEventRootInputFailed(vertexID, initializer.getInput().getName(), + new AMUserCodeException(Source.InputInitializer, t))); + } } @VisibleForTesting diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index f86dc987b3..6ae3ba55a8 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -2416,7 +2416,7 @@ void tryEnactKill(VertexTerminationCause trigger, } } - public VertexState finished(VertexState finalState, + VertexState finished(VertexState finalState, VertexTerminationCause termCause, String diag) { if (finishTime == 0) setFinishTime(); if (termCause != null) { @@ -3073,7 +3073,13 @@ private VertexState handleInitEvent(VertexImpl vertex) { if (vertex.inputsWithInitializers != null) { if (vertex.recoveryData == null || !vertex.recoveryData.shouldSkipInit()) { LOG.info("Vertex will initialize from input initializer. " + vertex.logIdentifier); - vertex.setupInputInitializerManager(); + try { + vertex.setupInputInitializerManager(); + } catch (TezException e) { + String msg = "Fail to create InputInitializerManager, " + ExceptionUtils.getStackTrace(e); + LOG.info(msg); + return vertex.finished(VertexState.FAILED, VertexTerminationCause.INIT_FAILURE, msg); + } } return VertexState.INITIALIZING; } else { @@ -3106,7 +3112,13 @@ private VertexState handleInitEvent(VertexImpl vertex) { if (vertex.inputsWithInitializers != null && (vertex.recoveryData == null || !vertex.recoveryData.shouldSkipInit())) { LOG.info("Vertex will initialize from input initializer. " + vertex.logIdentifier); - vertex.setupInputInitializerManager(); + try { + vertex.setupInputInitializerManager(); + } catch (TezException e) { + String msg = "Fail to create InputInitializerManager, " + ExceptionUtils.getStackTrace(e); + LOG.error(msg); + return vertex.finished(VertexState.FAILED, VertexTerminationCause.INIT_FAILURE, msg); + } return VertexState.INITIALIZING; } if (!vertex.uninitializedEdges.isEmpty()) { @@ -4243,7 +4255,7 @@ public void transition(VertexImpl vertex, VertexEvent event) { } } - private void setupInputInitializerManager() { + private void setupInputInitializerManager() throws TezException { rootInputInitializerManager = createRootInputInitializerManager( getDAG().getName(), getName(), getVertexId(), eventHandler, getTotalTasks(), @@ -4258,7 +4270,10 @@ eventHandler, getTotalTasks(), LOG.info("Starting " + inputsWithInitializers.size() + " inputInitializers for vertex " + logIdentifier); initWaitsForRootInitializers = true; - rootInputInitializerManager.runInputInitializers(inputList, pendingInitializerEvents); + rootInputInitializerManager.runInputInitializers(inputList); + // Send pending rootInputInitializerEvents + rootInputInitializerManager.handleInitializerEvents(pendingInitializerEvents); + pendingInitializerEvents.clear(); } private static class VertexStateChangedCallback diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java index a197e54e6e..ffa4309602 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java @@ -14,9 +14,7 @@ package org.apache.tez.dag.app.dag; -import static org.apache.tez.dag.app.TestDAGAppMaster.DAGAppMasterForTest.createCredentials; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; import static org.mockito.Mockito.RETURNS_DEEP_STUBS; import static org.mockito.Mockito.doReturn; @@ -27,43 +25,20 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import java.io.File; import java.io.IOException; import java.util.Collections; -import java.util.LinkedList; import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.ThreadPoolExecutor; import com.google.common.collect.Lists; -import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; -import com.google.common.util.concurrent.ThreadFactoryBuilder; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.event.EventHandler; -import org.apache.hadoop.yarn.util.SystemClock; -import org.apache.tez.client.TezApiVersionInfo; -import org.apache.tez.common.TezCommonUtils; import org.apache.tez.dag.api.InputDescriptor; import org.apache.tez.dag.api.InputInitializerDescriptor; import org.apache.tez.dag.api.RootInputLeafOutput; -import org.apache.tez.dag.api.TezConfiguration; -import org.apache.tez.dag.api.TezConstants; import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.api.oldrecords.TaskState; -import org.apache.tez.dag.api.records.DAGProtos; import org.apache.tez.dag.app.AppContext; -import org.apache.tez.dag.app.DAGAppMaster; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.dag.records.TezTaskID; @@ -75,34 +50,11 @@ import org.apache.tez.runtime.api.events.InputInitializerEvent; import org.apache.tez.runtime.api.impl.EventMetaData; import org.apache.tez.runtime.api.impl.TezEvent; -import org.junit.After; -import org.junit.Before; import org.junit.Test; import org.mockito.ArgumentCaptor; public class TestRootInputInitializerManager { - private static final File TEST_DIR = new File(System.getProperty("test.build.data"), - TestRootInputInitializerManager.class.getName()).getAbsoluteFile(); - private static ListeningExecutorService execService; - - @Before - public void setUp() { - ExecutorService rawExecutor = Executors.newCachedThreadPool(new ThreadFactoryBuilder() - .setDaemon(true).setNameFormat("Test App Shared Pool - " + "#%d").build()); - execService = MoreExecutors.listeningDecorator(rawExecutor); - FileUtil.fullyDelete(TEST_DIR); - TEST_DIR.mkdirs(); - } - - @After - public void tearDown() { - if (execService != null) { - execService.shutdownNow(); - } - FileUtil.fullyDelete(TEST_DIR); - } - // Simple testing. No events if task doesn't succeed. // Also exercises path where two attempts are reported as successful via the stateChangeNotifier. // Primarily a failure scenario, when a Task moves back to running from success @@ -262,7 +214,6 @@ public void testCorrectUgiUsage() throws TezException, InterruptedException { AppContext appContext = mock(AppContext.class); doReturn(new DefaultHadoopShim()).when(appContext).getHadoopShim(); doReturn(mock(EventHandler.class)).when(appContext).getEventHandler(); - when(appContext.getExecService()).thenReturn(execService); UserGroupInformation dagUgi = UserGroupInformation.createRemoteUser("fakeuser"); StateChangeNotifier stateChangeNotifier = mock(StateChangeNotifier.class); RootInputInitializerManager rootInputInitializerManager = new RootInputInitializerManager(vertex, appContext, dagUgi, stateChangeNotifier); @@ -271,73 +222,12 @@ public void testCorrectUgiUsage() throws TezException, InterruptedException { InputInitializerDescriptor iid = InputInitializerDescriptor.create(InputInitializerForUgiTest.class.getName()); RootInputLeafOutput rootInput = new RootInputLeafOutput<>("InputName", id, iid); - rootInputInitializerManager.runInputInitializers(Collections.singletonList(rootInput), Collections.emptyList()); + rootInputInitializerManager.runInputInitializers(Collections.singletonList(rootInput)); InputInitializerForUgiTest.awaitInitialize(); assertEquals(dagUgi, InputInitializerForUgiTest.ctorUgi); - assertEquals(dagUgi.getRealUser(), InputInitializerForUgiTest.initializeUgi.getRealUser()); - } - - @Test (timeout = 10000) - public synchronized void testParallelInputInitialization() throws InterruptedException, IOException { - // Create Local DAGAppMaster with default conf - Configuration conf = new Configuration(true); - conf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); - conf.set(TezConfiguration.TEZ_AM_STAGING_DIR, TEST_DIR.toString()); - - FileSystem fs = FileSystem.getLocal(conf); - FSDataOutputStream sessionJarsPBOutStream = - TezCommonUtils.createFileForAM(fs, new Path(TEST_DIR.toString(), - TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME)); - DAGProtos.PlanLocalResourcesProto.getDefaultInstance() - .writeDelimitedTo(sessionJarsPBOutStream); - sessionJarsPBOutStream.close(); - - ApplicationId appId = ApplicationId.newInstance(1, 1); - ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1); - - DAGAppMaster am = new DAGAppMaster(attemptId, - ContainerId.newContainerId(attemptId, 1), - "127.0.0.1", 0, 0, new SystemClock(), 1, true, - TEST_DIR.toString(), new String[] {TEST_DIR.toString()}, - new String[] {TEST_DIR.toString()}, - new TezApiVersionInfo().getVersion(), createCredentials(), - "someuser", null); - am.init(conf); - - Vertex vertex = mock(Vertex.class); - doReturn(mock(TezVertexID.class)).when(vertex).getVertexId(); - UserGroupInformation dagUgi = UserGroupInformation.createRemoteUser("fakeuser"); - StateChangeNotifier stateChangeNotifier = mock(StateChangeNotifier.class); - RootInputInitializerManager rootInputInitializerManager = - new RootInputInitializerManager(vertex, am.getContext(), dagUgi, stateChangeNotifier); - - List> inlist = new LinkedList(); - // Make sure we dont have any OOM issue by controlling the capacity of the thread pool - // and also block producer (createInitializerWrapper when resources are saturated) - InputDescriptor id = mock(InputDescriptor.class); - InputInitializerDescriptor iid = InputInitializerDescriptor.create(InputInitializerForUgiTest.class.getName()); - for (int i=0; i < 10000; i++) { - RootInputLeafOutput rootInput = - new RootInputLeafOutput<>("InputName"+i, id, iid); - inlist.add(rootInput); - } - - List initWrappers = - rootInputInitializerManager.createInitializerWrappers(inlist); - - int maxThreadSize = conf.getInt(TezConfiguration.TEZ_AM_DAG_APPCONTEXT_THREAD_COUNT_LIMIT, - TezConfiguration.TEZ_AM_DAG_APPCONTEXT_THREAD_COUNT_LIMIT_DEFAULT); - ThreadPoolExecutor amThreadPool = am.getContext().getThreadPool(); - - rootInputInitializerManager.executor.submit(() -> rootInputInitializerManager - .createAndStartInitializing(Collections.emptyList(), initWrappers)); - - while (am.getContext().getThreadPool().getQueue().size() > 0) { - assertTrue(amThreadPool.getPoolSize() <= maxThreadSize); - Thread.sleep(100); - } + assertEquals(dagUgi, InputInitializerForUgiTest.initializeUgi); } public static class InputInitializerForUgiTest extends InputInitializer { @@ -379,4 +269,4 @@ static void awaitInitialize() throws InterruptedException { } } } -} +} \ No newline at end of file diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java index 9636329da3..95ea8a030e 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java @@ -330,10 +330,8 @@ public void setup() { Mockito.doAnswer(new Answer() { public ListenableFuture answer(InvocationOnMock invocation) { Object[] args = invocation.getArguments(); - if (args[0] instanceof CallableEvent) { - CallableEvent e = (CallableEvent) args[0]; - dispatcher.getEventHandler().handle(e); - } + CallableEvent e = (CallableEvent) args[0]; + dispatcher.getEventHandler().handle(e); return mockFuture; } }).when(execService).submit((Callable) any()); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestRootInputVertexManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestRootInputVertexManager.java index 16a97d4374..39a291ebbd 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestRootInputVertexManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestRootInputVertexManager.java @@ -24,7 +24,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyList; +import static org.mockito.Matchers.anyListOf; import static org.mockito.Matchers.eq; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; @@ -176,7 +176,7 @@ public void testRootInputVertexManagerSlowStart() { final List scheduledTasks = Lists.newLinkedList(); doAnswer(new ScheduledTasksAnswer(scheduledTasks)).when( - mockContext).scheduleTasks(anyList()); + mockContext).scheduleTasks(anyListOf(VertexManagerPluginContext.ScheduleTaskRequest.class)); // source vertices have 0 tasks. immediate start of all managed tasks when(mockContext.getVertexNumTasks(mockSrcVertexId1)).thenReturn(0); @@ -568,6 +568,7 @@ public ScheduledTasksAnswer(List scheduledTasks) { public Object answer(InvocationOnMock invocation) throws IOException { Object[] args = invocation.getArguments(); scheduledTasks.clear(); + @SuppressWarnings("unchecked") List tasks = (List)args[0]; for (VertexManagerPluginContext.ScheduleTaskRequest task : tasks) { diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java index 5ae95560fa..fb7872ff9a 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java @@ -43,15 +43,11 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.Callable; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; -import com.google.common.util.concurrent.MoreExecutors; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.protobuf.ByteString; import org.apache.commons.lang.StringUtils; @@ -2425,10 +2421,10 @@ private void setupVertices() { dagConf); } } else { - v = new VertexImplWithRunningInputInitializerWithExecutor(vertexId, vPlan, vPlan.getName(), conf, - dispatcher.getEventHandler(), taskCommunicatorManagerInterface, - clock, thh, appContext, locationHint, dispatcher, customInitializer, updateTracker, - dagConf, vertexGroups); + v = new VertexImpl(vertexId, vPlan, vPlan.getName(), conf, + dispatcher.getEventHandler(), taskCommunicatorManagerInterface, + clock, thh, true, appContext, locationHint, vertexGroups, taskSpecificLaunchCmdOption, + updateTracker, dagConf); } vertices.put(vName, v); vertexIdMap.put(vertexId, v); @@ -2532,10 +2528,8 @@ public void setupPostDagCreation() throws TezException { Mockito.doAnswer(new Answer() { public ListenableFuture answer(InvocationOnMock invocation) { Object[] args = invocation.getArguments(); - if (args[0] instanceof CallableEvent) { - CallableEvent e = (CallableEvent) args[0]; - dispatcher.getEventHandler().handle(e); - } + CallableEvent e = (CallableEvent) args[0]; + dispatcher.getEventHandler().handle(e); return mockFuture; }}) .when(execService).submit((Callable) any()); @@ -2766,13 +2760,12 @@ public void testNonExistVertexManager() throws TezException { } @Test(timeout=5000) - public void testNonExistInputInitializer() throws Exception { + public void testNonExistInputInitializer() throws TezException { setupPreDagCreation(); dagPlan = createDAGPlanWithNonExistInputInitializer(); setupPostDagCreation(); VertexImpl v1 = vertices.get("vertex1"); v1.handle(new VertexEvent(v1.getVertexId(), VertexEventType.V_INIT)); - while (v1.getTerminationCause() == null) Thread.sleep(10); Assert.assertEquals(VertexState.FAILED, v1.getState()); Assert.assertEquals(VertexTerminationCause.INIT_FAILURE, v1.getTerminationCause()); Assert.assertTrue(StringUtils.join(v1.getDiagnostics(),"") @@ -5849,43 +5842,6 @@ protected RootInputInitializerManager createRootInputInitializerManager( } } - @SuppressWarnings("rawtypes") - private static class VertexImplWithRunningInputInitializerWithExecutor extends VertexImpl { - private RootInputInitializerManagerWithExecutor rootInputInitializerManager; - - public VertexImplWithRunningInputInitializerWithExecutor(TezVertexID vertexId, - VertexPlan vertexPlan, String vertexName, - Configuration conf, - EventHandler eventHandler, - TaskCommunicatorManagerInterface taskCommunicatorManagerInterface, - Clock clock, TaskHeartbeatHandler thh, - AppContext appContext, - VertexLocationHint vertexLocationHint, - DrainDispatcher dispatcher, - InputInitializer presetInitializer, - StateChangeNotifier updateTracker, - Configuration dagConf, - Map vertexGroups) { - super(vertexId, vertexPlan, vertexName, conf, eventHandler, - taskCommunicatorManagerInterface, clock, thh, true, - appContext, vertexLocationHint, vertexGroups, taskSpecificLaunchCmdOption, - updateTracker, dagConf); - } - - @Override - protected RootInputInitializerManager createRootInputInitializerManager( - String dagName, String vertexName, TezVertexID vertexID, - EventHandler eventHandler, int numTasks, int numNodes, - Resource taskResource, Resource totalResource) { - try { - rootInputInitializerManager = new RootInputInitializerManagerWithExecutor(this, this.getAppContext(), stateChangeNotifier); - } catch (IOException e) { - throw new RuntimeException(e); - } - return rootInputInitializerManager; - } - } - @SuppressWarnings("rawtypes") private static class VertexImplWithControlledInitializerManager extends VertexImpl { @@ -5942,11 +5898,9 @@ public RootInputInitializerManagerWithRunningInitializer(Vertex vertex, AppConte IOException { super(vertex, appContext, UserGroupInformation.getCurrentUser(), tracker); this.presetInitializer = presetInitializer; - ExecutorService rawExecutor = Executors.newCachedThreadPool(new ThreadFactoryBuilder() - .setDaemon(true).setNameFormat("Test App Shared Pool - " + "#%d").build()); - this.executor = MoreExecutors.listeningDecorator(rawExecutor); } + @Override protected InputInitializer createInitializer( RootInputLeafOutput input, @@ -5956,31 +5910,6 @@ protected InputInitializer createInitializer( } return presetInitializer; } - - @Override - public void shutdown() { - super.shutdown(); - if (executor != null) { - executor.shutdown(); - } - } - } - - private static class RootInputInitializerManagerWithExecutor extends RootInputInitializerManager { - public RootInputInitializerManagerWithExecutor(Vertex vertex, AppContext appContext, StateChangeNotifier tracker) throws IOException { - super(vertex, appContext, UserGroupInformation.getCurrentUser(), tracker); - ExecutorService rawExecutor = Executors.newCachedThreadPool(new ThreadFactoryBuilder() - .setDaemon(true).setNameFormat("Test App Shared Pool - " + "#%d").build()); - this.executor = MoreExecutors.listeningDecorator(rawExecutor); - } - - @Override - public void shutdown() { - super.shutdown(); - if (executor != null) { - executor.shutdown(); - } - } } @SuppressWarnings({"rawtypes", "unchecked"}) @@ -6002,15 +5931,12 @@ public RootInputInitializerManagerControlled(Vertex vertex, AppContext appContex this.eventHandler = eventHandler; this.dispatcher = dispatcher; this.vertexID = vertex.getVertexId(); - ExecutorService rawExecutor = Executors.newCachedThreadPool(new ThreadFactoryBuilder() - .setDaemon(true).setNameFormat("Test App Shared Pool - " + "#%d").build()); - this.executor = MoreExecutors.listeningDecorator(rawExecutor); } @Override public void runInputInitializers( - List> inputs, List pendingInitializerEvents) { - this.inputs = inputs; + List> inputList) { + this.inputs = inputList; } @Override @@ -6035,13 +5961,10 @@ public void handleInputInitializerEvent(List events) thro @Override public void shutdown() { hasShutDown = true; - if (executor != null) { - executor.shutdown(); - } } public void failInputInitialization() throws TezException { - super.runInputInitializers(inputs, Collections.emptyList()); + super.runInputInitializers(inputs); eventHandler.handle(new VertexEventRootInputFailed(vertexID, inputs .get(0).getName(), new AMUserCodeException(Source.InputInitializer, From 1f1a177ef70b6e4fba514a02e027fe3b071117e4 Mon Sep 17 00:00:00 2001 From: Lewis John McGibbney Date: Thu, 10 Dec 2020 23:13:01 -0600 Subject: [PATCH 279/512] TEZ-4258. Fix travisci build Signed-off-by: Jonathan Eagles --- .travis.yml | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/.travis.yml b/.travis.yml index bfc6b34bbf..0437531837 100644 --- a/.travis.yml +++ b/.travis.yml @@ -20,12 +20,10 @@ sudo: required env: MAVEN_OPTS="-Xmx2G -XX:MaxPermSize=512M" -jdk: - - oraclejdk8 - before_install: - ./build-tools/install-protobuf.sh script: + - jdk_switcher use oraclejdk8 - mvn -B clean install package -DskipTests=true -Dmaven.javadoc.skip=true From 849e1d7694cdfd2432d631830940bc95c6f26ead Mon Sep 17 00:00:00 2001 From: Lewis John McGibbney Date: Fri, 11 Dec 2020 09:42:25 -0600 Subject: [PATCH 280/512] TEZ-4258. Fix travisci build (addendum 1) Signed-off-by: Jonathan Eagles --- .travis.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.travis.yml b/.travis.yml index 0437531837..3637a0defe 100644 --- a/.travis.yml +++ b/.travis.yml @@ -14,6 +14,8 @@ # See the License for the specific language governing permissions and # limitations under the License. +dist: trusty + language: java sudo: required From 979bf5d7d7a1d3d66839e330388d3c9ec975b1c3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Fri, 15 Jan 2021 10:08:50 -0600 Subject: [PATCH 281/512] TEZ-3706. add option to skip Tez UI build Signed-off-by: Jonathan Eagles --- BUILDING.txt | 10 ++ pom.xml | 4 +- tez-ui/pom.xml | 277 +++++++++++++++++++++++++++++++++++++++++-------- 3 files changed, 245 insertions(+), 46 deletions(-) diff --git a/BUILDING.txt b/BUILDING.txt index f2231ea088..875bf3e097 100644 --- a/BUILDING.txt +++ b/BUILDING.txt @@ -104,6 +104,16 @@ Issue with PhantomJS on building in PowerPC. please try installing PhantomJS manually and rerun. Refer https://github.com/ibmsoe/phantomjs-1/blob/v2.1.1-ppc64/README.md and install it globally for the build to work. +---------------------------------------------------------------------------------- +Skip UI build: + +In case you want to completely skip UI build, you can use 'noui' profile. +For instance, a full build without tests and tez-ui looks like: + + $ mvn clean install -DskipTests -Pnoui + +It's important to note that maven will still include tez-ui project, but all of the maven plugins are skipped. + ---------------------------------------------------------------------------------- Protocol Buffer compiler: diff --git a/pom.xml b/pom.xml index 04088bd9f8..59ec193dfb 100644 --- a/pom.xml +++ b/pom.xml @@ -825,7 +825,7 @@ org.apache.maven.plugins maven-war-plugin - 2.5 + 3.2.3 org.apache.maven.plugins @@ -921,7 +921,7 @@ org.apache.maven.plugins maven-resources-plugin - 2.6 + 3.1.0 UTF-8 diff --git a/tez-ui/pom.xml b/tez-ui/pom.xml index 500d21af87..e52eb52dfa 100644 --- a/tez-ui/pom.xml +++ b/tez-ui/pom.xml @@ -40,6 +40,173 @@ + + + + default + + + !test + + + + + + exec-maven-plugin + org.codehaus.mojo + + + + ember build + generate-resources + + exec + + + ${webappDir} + ${nodeExecutable} + + ${packageManagerScript} + run + build:mvn + + + + + + + ember test + test + + exec + + + ${skipTests} + ${webappDir} + ${nodeExecutable} + + ${packageManagerScript} + run + test:mvn + + + + + + + + + + + + + ui-tests + + + test + ember + + + + + + exec-maven-plugin + org.codehaus.mojo + + + + ember build + generate-resources + + exec + + + ${webappDir} + ${nodeExecutable} + + ${packageManagerScript} + run + build:mvn + + + + + + + ember test + test + + exec + + + ${skipTests} + ${webappDir} + ${nodeExecutable} + + ${packageManagerScript} + run + test:mvn + + + + + + + + + + + + + no-ui-tests + + + test + .* + + + + + + exec-maven-plugin + org.codehaus.mojo + + + + ember build + generate-resources + + exec + + + ${webappDir} + ${nodeExecutable} + + ${packageManagerScript} + run + build:mvn + + + + + + + ember test + test + + exec + + + ${nodeExecutable} + true + + + + + + + + cleanUICache @@ -86,6 +253,72 @@ --allow-root=true + + noui + + false + + + + + org.apache.rat + apache-rat-plugin + + true + + + + com.github.eirslett + frontend-maven-plugin + + true + + + + exec-maven-plugin + org.codehaus.mojo + + true + + + + ro.isdc.wro4j + wro4j-maven-plugin + + true + + + + org.apache.maven.plugins + maven-war-plugin + + true + + + + org.apache.maven.plugins + maven-jar-plugin + + true + + + + org.apache.maven.plugins + maven-resources-plugin + + true + + + + org.apache.maven.plugins + maven-install-plugin + + true + + + + + @@ -167,50 +400,6 @@ - - exec-maven-plugin - org.codehaus.mojo - - - - - ember build - generate-resources - - exec - - - ${webappDir} - ${nodeExecutable} - - ${packageManagerScript} - run - build:mvn - - - - - - - ember test - test - - exec - - - ${skipTests} - ${webappDir} - ${nodeExecutable} - - ${packageManagerScript} - run - test:mvn - - - - - - ro.isdc.wro4j From fb81c11583ea7c31978e2417e0e31f4a141dd7a7 Mon Sep 17 00:00:00 2001 From: Lewis John McGibbney Date: Fri, 15 Jan 2021 16:08:49 -0600 Subject: [PATCH 282/512] TEZ-4261. Fix Javadoc warnings Signed-off-by: Jonathan Eagles --- .../tez/mapreduce/hadoop/DeprecatedKeys.java | 6 ----- .../tez/mapreduce/hadoop/MRInputHelpers.java | 22 +++++++++---------- 2 files changed, 11 insertions(+), 17 deletions(-) diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java index b8d491a4b0..098057ba17 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java @@ -22,7 +22,6 @@ import java.util.HashMap; import java.util.Map; -import org.apache.hadoop.conf.Configuration; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.library.vertexmanager.ShuffleVertexManager; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; @@ -184,11 +183,6 @@ private static void registerMRToRuntimeKeyTranslation(String mrKey, String tezKey) { mrParamToTezRuntimeParamMap.put(mrKey, tezKey); } - - @SuppressWarnings("unused") - private static void _(String mrKey, String tezKey) { - Configuration.addDeprecation(mrKey, tezKey); - } public static Map getMRToDAGParamMap() { return Collections.unmodifiableMap(mrParamToDAGParamMap); diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java index 9c42cdaf4e..19d64a5abb 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java @@ -782,7 +782,7 @@ private static int getIntProperty(Configuration conf, String propertyName) { } /** - * @see {@link InputContext#getDagIdentifier} + * @see InputContext#getDagIdentifier() * @param conf configuration instance * @return dag index */ @@ -802,7 +802,7 @@ public static String getDagIdString(Configuration conf) { } /** - * * @see {@link InputContext#getTaskVertexIndex} + * @see InputContext#getTaskVertexIndex * @param conf configuration instance * @return vertex index */ @@ -822,7 +822,7 @@ public static String getVertexIdString(Configuration conf) { } /** - * @see {@link InputContext#getTaskIndex} + * @see InputContext#getTaskIndex * @param conf configuration instance * @return task index */ @@ -842,7 +842,7 @@ public static String getTaskIdString(Configuration conf) { } /** - * @see {@link InputContext#getTaskAttemptNumber} + * @see InputContext#getTaskAttemptNumber * @param conf configuration instance * @return task attempt index */ @@ -862,7 +862,7 @@ public static String getTaskAttemptIdString(Configuration conf) { } /** - * @see {@link InputContext#getInputIndex} + * @see InputContext#getInputIndex * @param conf configuration instance * @return input index */ @@ -872,7 +872,7 @@ public static int getInputIndex(Configuration conf) { } /** - * @see {@link InputContext#getDAGName} + * @see InputContext#getDAGName * @param conf configuration instance * @return dag name */ @@ -882,7 +882,7 @@ public static String getDagName(Configuration conf) { } /** - * @see {@link InputContext#getTaskVertexName} + * @see InputContext#getTaskVertexName * @param conf configuration instance * @return vertex name */ @@ -892,7 +892,7 @@ public static String getVertexName(Configuration conf) { } /** - * @see {@link InputContext#getSourceVertexName} + * @see InputContext#getSourceVertexName * @param conf configuration instance * @return source name */ @@ -902,7 +902,7 @@ public static String getInputName(Configuration conf) { } /** - * @see {@link InputContext#getApplicationId} + * @see InputContext#getApplicationId * @param conf configuration instance * @return applicationId as a string */ @@ -912,7 +912,7 @@ public static String getApplicationIdString(Configuration conf) { } /** - * @see {@link InputContext#getUniqueIdentifier} + * @see InputContext#getUniqueIdentifier * @param conf configuration instance * @return unique identifier for the input */ @@ -922,7 +922,7 @@ public static String getUniqueIdentifier(Configuration conf) { } /** - * @see {@link InputContext#getDAGAttemptNumber} + * @see InputContext#getDAGAttemptNumber * @param conf configuration instance * @return attempt number */ From c2d48a688e041e2b5689d26908a7de7d135bff3e Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Wed, 20 Jan 2021 15:53:15 -0500 Subject: [PATCH 283/512] TEZ-4095: Review of Debug Logging (#87) --- .../org/apache/tez/client/TezClientUtils.java | 13 ++---- .../apache/tez/common/JavaOptsChecker.java | 11 ++--- .../org/apache/tez/common/ProgressHelper.java | 12 ++--- .../apache/tez/dag/api/TezConfiguration.java | 4 +- .../tez/dag/api/client/DAGClientImpl.java | 16 ++----- .../dag/api/client/TimelineReaderFactory.java | 6 +-- .../dag/api/client/rpc/DAGClientRPCImpl.java | 8 +--- .../tez/dag/api/client/DAGClientHandler.java | 4 +- .../org/apache/tez/dag/app/DAGAppMaster.java | 36 +++++---------- .../apache/tez/dag/app/RecoveryParser.java | 7 +-- .../tez/dag/app/TaskCommunicatorManager.java | 13 ++---- .../dag/app/TezLocalTaskCommunicatorImpl.java | 4 +- .../tez/dag/app/TezTaskCommunicatorImpl.java | 17 ++----- .../apache/tez/dag/app/dag/impl/DAGImpl.java | 5 +- .../tez/dag/app/dag/impl/TaskAttemptImpl.java | 31 ++++--------- .../apache/tez/dag/app/dag/impl/TaskImpl.java | 12 ++--- .../tez/dag/app/dag/impl/VertexImpl.java | 46 +++++-------------- .../speculation/legacy/LegacySpeculator.java | 4 +- .../launcher/TezContainerLauncherImpl.java | 4 +- .../dag/app/rm/DagAwareYarnTaskScheduler.java | 2 +- .../tez/dag/app/rm/TaskSchedulerManager.java | 8 +--- .../dag/app/rm/YarnTaskSchedulerService.java | 31 ++++--------- .../app/rm/container/AMContainerHelpers.java | 4 +- .../tez/dag/app/web/AMWebController.java | 8 +--- .../apache/tez/dag/app/web/WebUIService.java | 4 +- .../dag/history/recovery/RecoveryService.java | 22 +++------ .../split/TezGroupedSplitsInputFormat.java | 12 ++--- .../split/TezGroupedSplitsInputFormat.java | 12 ++--- .../tez/mapreduce/client/YARNRunner.java | 4 +- .../tez/mapreduce/hadoop/MRInputHelpers.java | 8 +--- .../apache/tez/mapreduce/output/MROutput.java | 4 +- .../tez/mapreduce/processor/MRTask.java | 5 +- .../apache/tez/auxservices/IndexCache.java | 17 ++----- .../tez/auxservices/ShuffleHandler.java | 24 +++------- .../LogicalIOProcessorRuntimeTask.java | 24 +++------- .../runtime/api/impl/TezInputContextImpl.java | 4 +- .../api/impl/TezOutputContextImpl.java | 4 +- .../api/impl/TezProcessorContextImpl.java | 4 +- .../common/resources/ScalingAllocator.java | 9 +--- .../apache/tez/runtime/task/TaskReporter.java | 8 +--- .../org/apache/tez/runtime/task/TezChild.java | 4 +- .../org/apache/tez/http/HttpConnection.java | 8 +--- .../http/async/netty/AsyncHttpConnection.java | 4 +- .../library/common/TezRuntimeUtils.java | 4 +- .../library/common/shuffle/ShuffleUtils.java | 8 +--- .../impl/ShuffleInputEventHandlerImpl.java | 12 ++--- .../common/shuffle/impl/ShuffleManager.java | 20 ++------ .../orderedgrouped/FetcherOrderedGrouped.java | 38 ++++----------- .../shuffle/orderedgrouped/MergeManager.java | 8 +--- ...huffleInputEventHandlerOrderedGrouped.java | 17 ++----- .../orderedgrouped/ShuffleScheduler.java | 12 ++--- .../library/common/sort/impl/IFile.java | 4 +- .../task/local/output/TezTaskOutputFiles.java | 6 +-- .../library/input/OrderedGroupedKVInput.java | 4 +- .../WeightedScalingMemoryDistributor.java | 4 +- .../org/apache/tez/tools/TFileLoader.java | 5 +- 56 files changed, 167 insertions(+), 462 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index d34d31e4cf..1e09489564 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -479,9 +479,7 @@ public static ApplicationSubmissionContext createApplicationSubmissionContext( capability.setVirtualCores( amConfig.getTezConfiguration().getInt(TezConfiguration.TEZ_AM_RESOURCE_CPU_VCORES, TezConfiguration.TEZ_AM_RESOURCE_CPU_VCORES_DEFAULT)); - if (LOG.isDebugEnabled()) { - LOG.debug("AppMaster capability = " + capability); - } + LOG.debug("AppMaster capability = {}", capability); // Setup required Credentials for the AM launch. DAG specific credentials // are handled separately. @@ -531,10 +529,7 @@ public static ApplicationSubmissionContext createApplicationSubmissionContext( } vargsFinal.add(mergedCommand.toString()); - if (LOG.isDebugEnabled()) { - LOG.debug("Command to launch container for ApplicationMaster is : " - + mergedCommand); - } + LOG.debug("Command to launch container for ApplicationMaster is : {}", mergedCommand); Map environment = new TreeMap(); TezYARNUtils.setupDefaultEnv(environment, conf, @@ -968,9 +963,7 @@ public static DAGClientAMProtocolBlockingPB getAMProxy(final Configuration conf, serviceAddr); userUgi.addToken(token); } - if (LOG.isDebugEnabled()) { - LOG.debug("Connecting to Tez AM at " + serviceAddr); - } + LOG.debug("Connecting to Tez AM at {}", serviceAddr); DAGClientAMProtocolBlockingPB proxy = null; try { proxy = userUgi.doAs(new PrivilegedExceptionAction() { diff --git a/tez-api/src/main/java/org/apache/tez/common/JavaOptsChecker.java b/tez-api/src/main/java/org/apache/tez/common/JavaOptsChecker.java index 6de402aae9..3e93446149 100644 --- a/tez-api/src/main/java/org/apache/tez/common/JavaOptsChecker.java +++ b/tez-api/src/main/java/org/apache/tez/common/JavaOptsChecker.java @@ -38,9 +38,8 @@ public class JavaOptsChecker { public void checkOpts(String opts) throws TezException { Set gcOpts = new TreeSet(); - if (LOG.isDebugEnabled()) { - LOG.debug("Checking JVM GC opts: " + opts); - } + LOG.debug("Checking JVM GC opts: {}", opts); + Matcher matcher = pattern.matcher(opts); while (matcher.find()) { if (matcher.groupCount() != 3) { @@ -74,10 +73,8 @@ public void checkOpts(String opts) throws TezException { } } - if (LOG.isDebugEnabled()) { - LOG.debug("Found clashing GC opts" - + ", conflicting GC Values=" + gcOpts); - } + LOG.debug("Found clashing GC opts, conflicting GC Values={}", gcOpts); + throw new TezException("Invalid/conflicting GC options found," + " cmdOpts=\"" + opts + "\""); } diff --git a/tez-api/src/main/java/org/apache/tez/common/ProgressHelper.java b/tez-api/src/main/java/org/apache/tez/common/ProgressHelper.java index 1518ccdaf1..289847a96a 100644 --- a/tez-api/src/main/java/org/apache/tez/common/ProgressHelper.java +++ b/tez-api/src/main/java/org/apache/tez/common/ProgressHelper.java @@ -117,10 +117,8 @@ public void run() { // Report progress as 0.0f when if are errors. processorContext.setProgress(progressVal); } catch (Throwable th) { - if (LOG.isDebugEnabled()) { - LOG.debug("progress update: Encountered InterruptedException during" - + " Processor={}", processorName, th); - } + LOG.debug("progress update: Encountered InterruptedException during" + + " Processor={}", processorName, th); if (th instanceof InterruptedException) { // set interrupt flag to true sand exit Thread.currentThread().interrupt(); @@ -161,10 +159,8 @@ public void shutDownProgressTaskService() { scheduledExecutorService.shutdownNow(); } } catch (InterruptedException e) { - if (LOG.isDebugEnabled()) { - LOG.debug("Interrupted exception while shutting down the " - + "executor service for the processor name={}", processorName); - } + LOG.debug("Interrupted exception while shutting down the " + + "executor service for the processor name={}", processorName); } scheduledExecutorService.shutdownNow(); } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 85f85181ca..179b1957e8 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -1899,9 +1899,7 @@ public TezConfiguration(boolean loadDefaults) { public static void validateProperty(String property, Scope usedScope) { Scope validScope = PropertyScope.get(property); if (validScope == null) { - if (LOG.isDebugEnabled()) { - LOG.debug(property + " is not standard configuration property of tez, can not been validated"); - } + LOG.debug("{} is not standard configuration property of tez, can not been validated", property); } else { if (usedScope.ordinal() > validScope.ordinal()) { throw new IllegalStateException(property + " is set at the scope of " + usedScope diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java index 01a10b269c..b54db324a9 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java @@ -247,9 +247,7 @@ private DAGStatus getDAGStatusInternal(@Nullable Set statusOption LOG.info("Failed to fetch DAG data for completed DAG from YARN Timeline" + " - Application not found by YARN", e); } catch (TezException e) { - if (LOG.isDebugEnabled()) { - LOG.info("DAGStatus fetch failed." + e.getMessage()); - } + LOG.debug("DAGStatus fetch failed", e); } } @@ -302,9 +300,7 @@ public VertexStatus getVertexStatus(String vertexName, Set status + " - Application not found by YARN", e); return null; } catch (TezException e) { - if (LOG.isDebugEnabled()) { - LOG.debug("ERROR fetching vertex data from Yarn Timeline. " + e.getMessage()); - } + LOG.debug("ERROR fetching vertex data from Yarn Timeline", e); } } @@ -425,9 +421,7 @@ private VertexStatus getVertexStatusViaAM(String vertexName, Set */ @VisibleForTesting protected DAGStatus getDAGStatusViaRM() throws TezException, IOException { - if(LOG.isDebugEnabled()) { - LOG.debug("GetDAGStatus via AM for app: " + appId + " dag:" + dagId); - } + LOG.debug("GetDAGStatus via AM for app: {} dag:{}", appId, dagId); ApplicationReport appReport; try { appReport = frameworkClient.getApplicationReport(appId); @@ -638,9 +632,7 @@ private void switchToTimelineClient() throws IOException, TezException { realClient.close(); realClient = new DAGClientTimelineImpl(appId, dagId, conf, frameworkClient, (int) (2 * PRINT_STATUS_INTERVAL_MILLIS)); - if (LOG.isDebugEnabled()) { - LOG.debug("dag completed switching to DAGClientTimelineImpl"); - } + LOG.debug("dag completed switching to DAGClientTimelineImpl"); } @VisibleForTesting diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/TimelineReaderFactory.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/TimelineReaderFactory.java index 40340cc44e..fc1595fa49 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/TimelineReaderFactory.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/TimelineReaderFactory.java @@ -313,10 +313,8 @@ private static ConnectionConfigurator getNewConnectionConf(final Configuration c try { connectionConf = getNewSSLConnectionConf(conf, connTimeout, sslFactory); } catch (IOException e) { - if (LOG.isDebugEnabled()) { - LOG.debug("Cannot load customized ssl related configuration." - + " Falling back to system-generic settings.", e); - } + LOG.debug("Cannot load customized ssl related configuration." + + " Falling back to system-generic settings.", e); } } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java index c54058be8a..5d5752e6e2 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java @@ -146,9 +146,7 @@ public String getSessionIdentifierString() { @Override public void tryKillDAG() throws TezException, IOException { - if(LOG.isDebugEnabled()) { - LOG.debug("TryKill for app: " + appId + " dag:" + dagId); - } + LOG.debug("TryKill for app: {} dag:{}", appId, dagId); try { if (createAMProxyIfNeeded()) { TryKillDAGRequestProto requestProto = @@ -186,9 +184,7 @@ void resetProxy(Exception e) { DAGStatus getDAGStatusViaAM(Set statusOptions, long timeout) throws IOException, TezException { - if(LOG.isDebugEnabled()) { - LOG.debug("GetDAGStatus via AM for app: " + appId + " dag:" + dagId); - } + LOG.debug("GetDAGStatus via AM for app: {} dag:{}", appId, dagId); GetDAGStatusRequestProto.Builder requestProtoBuilder = GetDAGStatusRequestProto.newBuilder() .setDagId(dagId).setTimeout(timeout); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java index 618676d978..4cdd1ec9d1 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java @@ -100,9 +100,7 @@ DAG getDAG(String dagIdStr) throws TezException { final String currentDAGIdStr = currentDAG.getID().toString(); if (!currentDAGIdStr.equals(dagIdStr)) { if (getAllDagIDs().contains(dagIdStr)) { - if (LOG.isDebugEnabled()) { - LOG.debug("Looking for finished dagId " + dagIdStr + " current dag is " + currentDAGIdStr); - } + LOG.debug("Looking for finished dagId {} current dag is {}", dagIdStr, currentDAGIdStr); throw new DAGNotRunningException("DAG " + dagIdStr + " Not running, current dag is " + currentDAGIdStr); } else { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index f4a8923d4a..cde77b3bf6 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -576,9 +576,7 @@ public synchronized void serviceInit(final Configuration conf) throws Exception this.webUIService = new WebUIService(context); addIfService(webUIService, false); } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Web UI Service is not enabled."); - } + LOG.debug("Web UI Service is not enabled."); } this.taskSchedulerManager = createTaskSchedulerManager(taskSchedulerDescriptors); @@ -1335,9 +1333,8 @@ public String submitDAGToAppMaster(DAGPlan dagPlan, // the job user's UGI context LOG.info("Starting DAG submitted via RPC: " + dagPlan.getName()); - if (LOG.isDebugEnabled()) { - LOG.debug("Invoked with additional local resources: " + additionalResources); - } + LOG.debug("Invoked with additional local resources: {}", additionalResources); + if (!dagPlan.getName().startsWith(TezConstants.TEZ_PREWARM_DAG_NAME_PREFIX)) { submittedDAGs.incrementAndGet(); } @@ -1863,9 +1860,8 @@ void startServices() { try { Throwable firstError = null; List threads = new ArrayList(); - if(LOG.isDebugEnabled()) { - LOG.debug("Begin parallel start"); - } + LOG.debug("Begin parallel start"); + for(ServiceWithDependency sd : services.values()) { // start the service. If this fails that service // will be stopped and an exception raised @@ -1889,9 +1885,7 @@ void startServices() { if(firstError != null) { throw ServiceStateException.convert(firstError); } - if(LOG.isDebugEnabled()) { - LOG.debug("End parallel start"); - } + LOG.debug("End parallel start"); } catch (InterruptedException e) { e.printStackTrace(); } @@ -1899,9 +1893,7 @@ void startServices() { void initServices(Configuration conf) { for (ServiceWithDependency sd : services.values()) { - if (LOG.isDebugEnabled()) { - LOG.debug("Initing service : " + sd.service); - } + LOG.debug("Initing service : {}", sd.service); sd.service.init(conf); } } @@ -1919,9 +1911,7 @@ void stopServices() { for (int i = services.size() - 1; i >= 0; i--) { Service service = serviceList.get(i); - if (LOG.isDebugEnabled()) { - LOG.debug("Stopping service : " + service); - } + LOG.debug("Stopping service : {}", service); Exception ex = ServiceOperations.stopQuietly(service); if (ex != null && firstException == null) { LOG.warn("Failed to stop service, name=" + service.getName(), ex); @@ -2163,10 +2153,8 @@ public void serviceStop() throws Exception { boolean deleteTezScratchData = this.amConf.getBoolean( TezConfiguration.TEZ_AM_STAGING_SCRATCH_DATA_AUTO_DELETE, TezConfiguration.TEZ_AM_STAGING_SCRATCH_DATA_AUTO_DELETE_DEFAULT); - if (LOG.isDebugEnabled()) { - LOG.debug("Checking whether tez scratch data dir should be deleted, deleteTezScratchData=" - + deleteTezScratchData); - } + LOG.debug("Checking whether tez scratch data dir should be deleted, deleteTezScratchData={}", + deleteTezScratchData); if (deleteTezScratchData && this.taskSchedulerManager != null && this.taskSchedulerManager.hasUnregistered()) { // Delete tez scratch data dir @@ -2443,9 +2431,7 @@ static class DAGAppMasterShutdownHook implements Runnable { public void run() { LOG.info("DAGAppMasterShutdownHook invoked"); if(appMaster.getServiceState() == STATE.STOPPED) { - if(LOG.isDebugEnabled()) { - LOG.debug("DAGAppMaster already stopped. Ignoring signal"); - } + LOG.debug("DAGAppMaster already stopped. Ignoring signal"); synchronized (appMaster.shutdownHandlerRunning) { try { if (appMaster.shutdownHandlerRunning.get()) { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java index dfb7f61e6e..19c24f300c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java @@ -346,11 +346,8 @@ private static HistoryEvent getNextEvent(CodedInputStream inputStream) } catch (EOFException eof) { return null; } - if (LOG.isDebugEnabled()) { - LOG.debug("Parsed event from input stream" - + ", eventType=" + eventType - + ", event=" + event.toString()); - } + LOG.debug("Parsed event from input stream, eventType={}, event={}", + eventType, event); return event; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java index 55b2d1b021..3a99456ed7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java @@ -220,10 +220,7 @@ public TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request) throws IOException, TezException { ContainerId containerId = ConverterUtils.toContainerId(request .getContainerIdentifier()); - if (LOG.isDebugEnabled()) { - LOG.debug("Received heartbeat from container" - + ", request=" + request); - } + LOG.debug("Received heartbeat from container, request={}", request); if (!registeredContainers.containsKey(containerId)) { LOG.warn("Received task heartbeat from unknown container with id: " + containerId + @@ -488,9 +485,7 @@ public void dagSubmitted() { @Override public void registerRunningContainer(ContainerId containerId, int taskCommId) { - if (LOG.isDebugEnabled()) { - LOG.debug("ContainerId: " + containerId + " registered with TaskAttemptListener"); - } + LOG.debug("ContainerId: {} registered with TaskAttemptListener", containerId); ContainerInfo oldInfo = registeredContainers.put(containerId, NULL_CONTAINER_INFO); if (oldInfo != null) { throw new TezUncheckedException( @@ -515,9 +510,7 @@ public void registerRunningContainer(ContainerId containerId, int taskCommId) { @Override public void unregisterRunningContainer(ContainerId containerId, int taskCommId, ContainerEndReason endReason, String diagnostics) { - if (LOG.isDebugEnabled()) { - LOG.debug("Unregistering Container from TaskAttemptListener: " + containerId); - } + LOG.debug("Unregistering Container from TaskAttemptListener: {}", containerId); ContainerInfo containerInfo = registeredContainers.remove(containerId); if (containerInfo.taskAttemptId != null) { registeredAttempts.remove(containerInfo.taskAttemptId); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezLocalTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezLocalTaskCommunicatorImpl.java index 15d90d3832..b5749591c1 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezLocalTaskCommunicatorImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezLocalTaskCommunicatorImpl.java @@ -39,8 +39,6 @@ protected void startRpcServer() { } catch (UnknownHostException e) { throw new TezUncheckedException(e); } - if (LOG.isDebugEnabled()) { - LOG.debug("Not starting TaskAttemptListener RPC in LocalMode"); - } + LOG.debug("Not starting TaskAttemptListener RPC in LocalMode"); } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java index 9c8fb6c67e..6d69d36014 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java @@ -310,18 +310,14 @@ public ContainerTask getTask(ContainerContext containerContext) throws IOExcepti } else { ContainerId containerId = ConverterUtils.toContainerId(containerContext .getContainerIdentifier()); - if (LOG.isDebugEnabled()) { - LOG.debug("Container with id: " + containerId + " asked for a task"); - } + LOG.debug("Container with id: {} asked for a task", containerId); task = getContainerTask(containerId); if (task != null && !task.shouldDie()) { getContext().taskSubmitted(task.getTaskSpec().getTaskAttemptID(), containerId); getContext().taskStartedRemotely(task.getTaskSpec().getTaskAttemptID()); } } - if (LOG.isDebugEnabled()) { - LOG.debug("getTask returning task: " + task); - } + LOG.debug("getTask returning task: {}", task); return task; } @@ -335,10 +331,7 @@ public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request) throws IOExce TezException { ContainerId containerId = ConverterUtils.toContainerId(request.getContainerIdentifier()); long requestId = request.getRequestId(); - if (LOG.isDebugEnabled()) { - LOG.debug("Received heartbeat from container" - + ", request=" + request); - } + LOG.debug("Received heartbeat from container, request={}", request); ContainerInfo containerInfo = registeredContainers.get(containerId); if (containerInfo == null) { @@ -436,9 +429,7 @@ private ContainerTask getContainerTask(ContainerId containerId) throws IOExcepti } } else { task = null; - if (LOG.isDebugEnabled()) { - LOG.debug("No task assigned yet for running container: " + containerId); - } + LOG.debug("No task assigned yet for running container: {}", containerId); } } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java index bb5c1aa966..09e9e71b92 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java @@ -1651,10 +1651,7 @@ DAGState initializeDAG() { if (!groupInfo.outputs.isEmpty()) { // shared outputs for (String vertexName : groupInfo.groupMembers) { - if (LOG.isDebugEnabled()) { - LOG.debug("Setting shared outputs for group: " + groupName + - " on vertex: " + vertexName); - } + LOG.debug("Setting shared outputs for group: {} on vertex: {}", groupName, vertexName); Vertex v = getVertex(vertexName); v.addSharedOutputs(groupInfo.outputs); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java index 9a5e73de5a..8b16b2e8b7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java @@ -1267,10 +1267,8 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl ta, TaskAttemptEvent TaskAttemptFinishedEvent taFinishedEvent = ta.recoveryData.getTaskAttemptFinishedEvent(); if (taFinishedEvent == null) { - if (LOG.isDebugEnabled()) { - LOG.debug("Only TaskAttemptStartedEvent but no TaskAttemptFinishedEvent, " - + "send out TaskAttemptEventAttemptKilled to move it to KILLED"); - } + LOG.debug("Only TaskAttemptStartedEvent but no TaskAttemptFinishedEvent, " + + "send out TaskAttemptEventAttemptKilled to move it to KILLED"); ta.sendEvent(new TaskAttemptEventAttemptKilled(ta.getID(), "Task Attempt killed in recovery due to can't recover the running task attempt", TaskAttemptTerminationCause.TERMINATED_AT_RECOVERY, true)); @@ -1285,30 +1283,21 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl ta, TaskAttemptEvent + "taskAttemptId=" + ta.getID()); switch (taFinishedEvent.getState()) { case FAILED: - if (LOG.isDebugEnabled()) { - LOG.debug("TaskAttemptFinishedEvent is seen with state of FAILED" - + ", send TA_FAILED to itself" - + ", attemptId=" + ta.attemptId); - } + LOG.debug("TaskAttemptFinishedEvent is seen with state of FAILED, " + + "send TA_FAILED to itself, attemptId={}", ta.attemptId); ta.sendEvent(new TaskAttemptEventAttemptFailed(ta.getID(), TaskAttemptEventType.TA_FAILED, taFinishedEvent.getTaskFailureType(), taFinishedEvent.getDiagnostics(), taFinishedEvent.getTaskAttemptError(), true)); break; case KILLED: - if (LOG.isDebugEnabled()) { - LOG.debug("TaskAttemptFinishedEvent is seen with state of KILLED" - + ", send TA_KILLED to itself" - + ", attemptId=" + ta.attemptId); - } + LOG.debug("TaskAttemptFinishedEvent is seen with state of KILLED, " + + "send TA_KILLED to itself, attemptId={}", ta.attemptId); ta.sendEvent(new TaskAttemptEventAttemptKilled(ta.getID(), taFinishedEvent.getDiagnostics(), taFinishedEvent.getTaskAttemptError(), true)); break; case SUCCEEDED: - if (LOG.isDebugEnabled()) { - LOG.debug("TaskAttemptFinishedEvent is seen with state of SUCCEEDED" - + ", send TA_DONE to itself" - + ", attemptId=" + ta.attemptId); - } + LOG.debug("TaskAttemptFinishedEvent is seen with state of SUCCEEDED, " + + "send TA_DONE to itself, attemptId={}", ta.attemptId); ta.sendEvent(new TaskAttemptEvent(ta.getID(), TaskAttemptEventType.TA_DONE)); break; default: @@ -1671,9 +1660,7 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) { if (ta.recoveryData != null && ta.recoveryData.isTaskAttemptSucceeded()) { TaskAttemptFinishedEvent taFinishedEvent = ta.recoveryData .getTaskAttemptFinishedEvent(); - if (LOG.isDebugEnabled()) { - LOG.debug("TaskAttempt is recovered to SUCCEEDED, attemptId=" + ta.attemptId); - } + LOG.debug("TaskAttempt is recovered to SUCCEEDED, attemptId={}", ta.attemptId); ta.reportedStatus.counters = taFinishedEvent.getCounters(); List tezEvents = taFinishedEvent.getTAGeneratedEvents(); if (tezEvents != null && !tezEvents.isEmpty()) { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java index 0b4b1160f2..cb8545f8ca 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java @@ -690,9 +690,7 @@ private TaskAttempt selectBestAttempt() { public boolean canCommit(TezTaskAttemptID taskAttemptID) { writeLock.lock(); try { - if (LOG.isDebugEnabled()) { - LOG.debug("Commit go/no-go request from " + taskAttemptID); - } + LOG.debug("Commit go/no-go request from {}", taskAttemptID); TaskState state = getState(); if (state == TaskState.SCHEDULED) { // the actual running task ran and is done and asking for commit. we are still stuck @@ -730,9 +728,7 @@ public boolean canCommit(TezTaskAttemptID taskAttemptID) { } } else { if (commitAttempt.equals(taskAttemptID)) { - if (LOG.isDebugEnabled()) { - LOG.debug(taskAttemptID + " already given a go for committing the task output."); - } + LOG.debug("{} already given a go for committing the task output.", taskAttemptID); return true; } // Don't think this can be a pluggable decision, so simply raise an @@ -740,9 +736,7 @@ public boolean canCommit(TezTaskAttemptID taskAttemptID) { // Wait for commit attempt to succeed. Dont kill this. If commit // attempt fails then choose a different committer. When commit attempt // succeeds then this and others will be killed - if (LOG.isDebugEnabled()) { - LOG.debug(commitAttempt + " is current committer. Commit waiting for: " + taskAttemptID); - } + LOG.debug("{} is current committer. Commit waiting for: {}", commitAttempt, taskAttemptID); return false; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index 6ae3ba55a8..e21add0e3b 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -880,10 +880,8 @@ void resetCompletedTaskStatsCache(boolean recompute) { @Override public void initServices() { if (servicesInited.get()) { - if (LOG.isDebugEnabled()) { - LOG.debug("Skipping Initing services for vertex because already" - + " Initialized, name=" + this.vertexName); - } + LOG.debug("Skipping Initing services for vertex because already" + + " Initialized, name={}", this.vertexName); return; } writeLock.lock(); @@ -891,10 +889,7 @@ public void initServices() { List servicesToAdd = new ArrayList<>(); if (isSpeculationEnabled()) { // Initialize the speculator - if (LOG.isDebugEnabled()) { - LOG.debug( - "Initing service vertex speculator, name=" + this.vertexName); - } + LOG.debug("Initing service vertex speculator, name={}", this.vertexName); speculator = new LegacySpeculator(vertexConf, getAppContext(), this); speculator.init(vertexConf); servicesToAdd.add(speculator); @@ -904,9 +899,7 @@ public void initServices() { } finally { writeLock.unlock(); } - if (LOG.isDebugEnabled()) { - LOG.debug("Initing service vertex, name=" + this.vertexName); - } + LOG.debug("Initing service vertex, name={}", this.vertexName); } @Override @@ -936,9 +929,7 @@ public void stopServices() { try { if (servicesInited.get()) { for (AbstractService srvc : services) { - if (LOG.isDebugEnabled()) { - LOG.debug("Stopping service : " + srvc); - } + LOG.debug("Stopping service : {}", srvc); Exception ex = ServiceOperations.stopQuietly(srvc); if (ex != null && firstException == null) { LOG.warn(String.format( @@ -1591,7 +1582,7 @@ private void computeProgress() { if (LOG.isDebugEnabled()) { if (!ProgressHelper.isProgressWithinRange(taskProg)) { LOG.debug("progress update: vertex={}, task={} incorrect; range={}", - getName(), task.getTaskId().toString(), taskProg); + getName(), task.getTaskId(), taskProg); } } accProg += ProgressHelper.processProgress(taskProg); @@ -2523,11 +2514,8 @@ private void initializeCommitters() throws Exception { final RootInputLeafOutput od = entry.getValue(); if (od.getControllerDescriptor() == null || od.getControllerDescriptor().getClassName() == null) { - if (LOG.isDebugEnabled()) { - LOG.debug("Ignoring committer as none specified for output=" - + outputName - + ", vertexId=" + logIdentifier); - } + LOG.debug("Ignoring committer as none specified for output={}, vertexId={}", + outputName, logIdentifier); continue; } LOG.info("Instantiating committer for output=" + outputName @@ -2548,19 +2536,13 @@ public Void run() throws Exception { .createClazzInstance(od.getControllerDescriptor().getClassName(), new Class[]{OutputCommitterContext.class}, new Object[]{outputCommitterContext}); - if (LOG.isDebugEnabled()) { - LOG.debug("Invoking committer init for output=" + outputName - + ", vertex=" + logIdentifier); - } + LOG.debug("Invoking committer init for output={}, vertex={}", outputName, logIdentifier); try { TezUtilsInternal.setHadoopCallerContext(appContext.getHadoopShim(), vertexId); outputCommitter.initialize(); outputCommitters.put(outputName, outputCommitter); - if (LOG.isDebugEnabled()) { - LOG.debug("Invoking committer setup for output=" + outputName - + ", vertex=" + logIdentifier); - } + LOG.debug("Invoking committer setup for output={}, vertex={}", outputName, logIdentifier); outputCommitter.setupOutput(); } finally { appContext.getHadoopShim().clearHadoopCallerContext(); @@ -4741,9 +4723,7 @@ public NoOpVertexManager(VertexManagerPluginContext context) { @Override public void initialize() throws Exception { - if (LOG.isDebugEnabled()) { - LOG.debug("initialize NoOpVertexManager"); - } + LOG.debug("initialize NoOpVertexManager"); configurationDoneEvent = new VertexConfigurationDoneEvent(); configurationDoneEvent.fromProtoStream(CodedInputStream.newInstance(getContext().getUserPayload().deepCopyAsArray())); String vertexName = getContext().getVertexName(); @@ -4769,9 +4749,7 @@ public void onVertexStarted(List completions) } getContext().doneReconfiguringVertex(); int numTasks = getContext().getVertexNumTasks(getContext().getVertexName()); - if (LOG.isDebugEnabled()) { - LOG.debug("Schedule all the tasks, numTask=" + numTasks); - } + LOG.debug("Schedule all the tasks, numTask={}", numTasks); List tasks = new ArrayList(); for (int i=0;i statuses) { // being released // completion of a container we had released earlier // an allocated container completed. notify app - if (LOG.isDebugEnabled()) { - LOG.debug("Released container completed:" + completedId + - " last allocated to task: " + task); - } + LOG.debug("Released container completed:{} last allocated to task: {}", + completedId, task); appContainerStatus.put(task, containerStatus); continue; } @@ -1216,12 +1214,9 @@ boolean preemptIfNeeded() { if(!preemptionWaitDeadlineCrossed && fitsIn(highestPriRequest.getCapability(), freeResources)) { - if (LOG.isDebugEnabled()) { - LOG.debug(highestPriRequest + " fits in free resources"); - } else { - if (numHeartbeats % 50 == 1) { - LOG.info(highestPriRequest + " fits in free resources"); - } + LOG.debug("{} fits in free resources", highestPriRequest); + if (numHeartbeats % 50 == 1) { + LOG.info(highestPriRequest + " fits in free resources"); } return true; } @@ -1509,10 +1504,8 @@ private CookieContainerRequest getMatchingRequestWithoutPriority( if (container.getId().equals( cookieContainerRequest.getAffinitizedContainer())) { // container level match - if (LOG.isDebugEnabled()) { - LOG.debug("Matching with affinity for request: " - + cookieContainerRequest + " container: " + affCId); - } + LOG.debug("Matching with affinity for request: {} container: {}", + cookieContainerRequest, affCId); return cookieContainerRequest; } if (LOG.isDebugEnabled()) { @@ -2025,10 +2018,7 @@ private void mainLoop() { if (delayedContainer == null) { continue; } - if (LOG.isDebugEnabled()) { - LOG.debug("Considering HeldContainer: " - + delayedContainer + " for assignment"); - } + LOG.debug("Considering HeldContainer: {} for assignment", delayedContainer); long currentTs = System.currentTimeMillis(); long nextScheduleTs = delayedContainer.getNextScheduleTime(); if (currentTs >= nextScheduleTs) { @@ -2091,10 +2081,7 @@ private void doAssignAll() { // honor reuse-locality flags (container not timed out yet), Don't queue // (already in queue), don't release (release happens when containers // time-out) - if (LOG.isDebugEnabled()) { - LOG.debug("Trying to assign all delayed containers to newly received" - + " tasks"); - } + LOG.debug("Trying to assign all delayed containers to newly received tasks"); Iterator iter = delayedContainers.iterator(); while(iter.hasNext()) { HeldContainer delayedContainer = iter.next(); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java index ee322655af..19cf5b7e21 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java @@ -126,9 +126,7 @@ private static ContainerLaunchContext createCommonContainerLaunchContext( containerTokens_dob.getLength()); // Add shuffle token - if (LOG.isDebugEnabled()) { - LOG.debug("Putting shuffle token in serviceData in common CLC"); - } + LOG.debug("Putting shuffle token in serviceData in common CLC"); serviceData.put(auxiliaryService, TezCommonUtils.serializeServiceData(TokenCache.getSessionToken(containerCredentials))); } catch (IOException e) { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java b/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java index 2115dac247..08d754d8a0 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java @@ -144,9 +144,7 @@ public void setCorsHeaders() { URL url = new URL(historyUrlBase); origin = url.getProtocol() + "://" + url.getAuthority(); } catch (MalformedURLException e) { - if (LOG.isDebugEnabled()) { - LOG.debug("Invalid url set for tez history url base: " + historyUrlBase, e); - } + LOG.debug("Invalid url set for tez history url base: {}", historyUrlBase, e); } } @@ -161,9 +159,7 @@ public void setCorsHeaders() { } void sendErrorResponse(int sc, String msg, Exception e) { - if (LOG.isDebugEnabled()) { - LOG.debug(msg, e); - } + LOG.debug(msg, e); try { response().sendError(sc, msg); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java index b1560a5ead..1670370187 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java @@ -119,9 +119,7 @@ protected void serviceStart() throws Exception { @Override protected void serviceStop() throws Exception { if (this.webApp != null) { - if (LOG.isDebugEnabled()) { - LOG.debug("Stopping WebApp"); - } + LOG.debug("Stopping WebApp"); this.webApp.stop(); } super.serviceStop(); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java index a0a152c81c..45e7d2fc9c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java @@ -405,11 +405,8 @@ private void createFatalErrorFlagDir() throws IOException { protected void handleSummaryEvent(TezDAGID dagID, HistoryEventType eventType, SummaryEvent summaryEvent) throws IOException { - if (LOG.isDebugEnabled()) { - LOG.debug("Handling summary event" - + ", dagID=" + dagID - + ", eventType=" + eventType); - } + LOG.debug("Handling summary event, dagID={}, eventType={}", dagID, eventType); + if (summaryStream == null) { Path summaryPath = TezCommonUtils.getSummaryRecoveryPath(recoveryPath); if (LOG.isDebugEnabled()) { @@ -470,11 +467,8 @@ protected void handleRecoveryEvent(DAGHistoryEvent event) throws IOException { } - if (LOG.isDebugEnabled()) { - LOG.debug("Writing recovery event to output stream" - + ", dagId=" + dagID - + ", eventType=" + eventType); - } + LOG.debug("Writing recovery event to output stream, dagId={}, eventType={}", + dagID, eventType); ++unflushedEventsCount; recoveryStream.codedOutputStream.writeFixed32NoTag(event.getHistoryEvent().getEventType().ordinal()); event.getHistoryEvent().toProtoStream(recoveryStream.codedOutputStream); @@ -489,11 +483,9 @@ private void maybeFlush(RecoveryStream recoveryStream) throws IOException { boolean doFlush = false; if (maxUnflushedEvents >=0 && unflushedEventsCount >= maxUnflushedEvents) { - if (LOG.isDebugEnabled()) { - LOG.debug("Max unflushed events count reached. Flushing recovery data" - + ", unflushedEventsCount=" + unflushedEventsCount - + ", maxUnflushedEvents=" + maxUnflushedEvents); - } + LOG.debug("Max unflushed events count reached. Flushing recovery data, " + + "unflushedEventsCount={}, maxUnflushedEvents={}", unflushedEventsCount, + maxUnflushedEvents); doFlush = true; } else if (flushInterval >= 0 && ((currentTime - lastFlushTime) >= (flushInterval*1000))) { diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java index bce16eee56..61e1f6c431 100644 --- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java +++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java @@ -68,25 +68,19 @@ public void setInputFormat(InputFormat wrappedInputFormat) { public void setSplitSizeEstimator(SplitSizeEstimator estimator) { Preconditions.checkArgument(estimator != null); this.estimator = estimator; - if (LOG.isDebugEnabled()) { - LOG.debug("Split size estimator : " + estimator); - } + LOG.debug("Split size estimator : {}", estimator); } public void setSplitLocationProvider(SplitLocationProvider locationProvider) { Preconditions.checkArgument(locationProvider != null); this.locationProvider = locationProvider; - if (LOG.isDebugEnabled()) { - LOG.debug("Split size location provider: " + locationProvider); - } + LOG.debug("Split size location provider: {}", locationProvider); } public void setDesiredNumberOfSplits(int num) { Preconditions.checkArgument(num >= 0); this.desiredNumSplits = num; - if (LOG.isDebugEnabled()) { - LOG.debug("desiredNumSplits: " + desiredNumSplits); - } + LOG.debug("desiredNumSplits: {}", desiredNumSplits); } @Override diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezGroupedSplitsInputFormat.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezGroupedSplitsInputFormat.java index 226425cd57..863f9aa792 100644 --- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezGroupedSplitsInputFormat.java +++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezGroupedSplitsInputFormat.java @@ -70,25 +70,19 @@ public void setInputFormat(InputFormat wrappedInputFormat) { public void setDesiredNumberOfSplits(int num) { Preconditions.checkArgument(num >= 0); this.desiredNumSplits = num; - if (LOG.isDebugEnabled()) { - LOG.debug("desiredNumSplits: " + desiredNumSplits); - } + LOG.debug("desiredNumSplits: {}", desiredNumSplits); } public void setSplitSizeEstimator(SplitSizeEstimator estimator) { Preconditions.checkArgument(estimator != null); this.estimator = estimator; - if (LOG.isDebugEnabled()) { - LOG.debug("Split size estimator : " + estimator); - } + LOG.debug("Split size estimator : {}", estimator); } public void setSplitLocationProvider(SplitLocationProvider locationProvider) { Preconditions.checkArgument(locationProvider != null); this.locationProvider = locationProvider; - if (LOG.isDebugEnabled()) { - LOG.debug("Split location provider : " + locationProvider); - } + LOG.debug("Split location provider : {}", locationProvider); } @Override diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java index 5a7d754ba5..9dba357951 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java @@ -616,9 +616,7 @@ public JobStatus submitJob(JobID jobId, String jobSubmitDir, Credentials ts) dagAMConf.set(TezConfiguration.TEZ_AM_LAUNCH_CMD_OPTS, javaOpts.toString()); if (envStr.length() > 0) { dagAMConf.set(TezConfiguration.TEZ_AM_LAUNCH_ENV, envStr); - if (LOG.isDebugEnabled()) { - LOG.debug("Setting MR AM env to : " + envStr); - } + LOG.debug("Setting MR AM env to : {}", envStr); } // Submit to ResourceManager diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java index 19d64a5abb..a8e85a34e4 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java @@ -322,18 +322,14 @@ public static InputSplitInfoMem generateInputSplitsToMem(Configuration conf, InputSplitInfoMem splitInfoMem = null; JobConf jobConf = new JobConf(conf); if (jobConf.getUseNewMapper()) { - if (LOG.isDebugEnabled()) { - LOG.debug("Generating mapreduce api input splits"); - } + LOG.debug("Generating mapreduce api input splits"); Job job = Job.getInstance(conf); org.apache.hadoop.mapreduce.InputSplit[] splits = generateNewSplits(job, groupSplits, sortSplits, targetTasks); splitInfoMem = new InputSplitInfoMem(splits, createTaskLocationHintsFromSplits(splits), splits.length, job.getCredentials(), job.getConfiguration()); } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Generating mapred api input splits"); - } + LOG.debug("Generating mapred api input splits"); org.apache.hadoop.mapred.InputSplit[] splits = generateOldSplits(jobConf, groupSplits, sortSplits, targetTasks); splitInfoMem = new InputSplitInfoMem(splits, createTaskLocationHintsFromSplits(splits), diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java index 950e629907..19ece5a0f6 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java @@ -488,9 +488,7 @@ public void initCommitter(JobConf job, boolean useNewApi) throws IOException, InterruptedException { if (useNewApi) { - if (LOG.isDebugEnabled()) { - LOG.debug("using new api for output committer"); - } + LOG.debug("using new api for output committer"); this.committer = newOutputFormat.getOutputCommitter( newApiTaskAttemptContext); diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java index b79f19cfa0..1a13168cb7 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java @@ -509,10 +509,7 @@ public void close() throws IOException { comparator, keyClass, valueClass); - if (LOG.isDebugEnabled()) { - LOG.debug("Using key class: " + keyClass - + ", valueClass: " + valueClass); - } + LOG.debug("Using key class: {}, valueClass: {}", keyClass, valueClass); org.apache.hadoop.mapreduce.Reducer.Context reducerContext = diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/IndexCache.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/IndexCache.java index 625f7ab8ba..54db975292 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/IndexCache.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/IndexCache.java @@ -84,9 +84,7 @@ public TezSpillRecord getSpillRecord(String mapId, Path fileName, String expecte } } } - if (LOG.isDebugEnabled()) { - LOG.debug("IndexCache HIT: MapId " + mapId + " found"); - } + LOG.debug("IndexCache HIT: MapId {} found", mapId); } if (info.mapSpillRecord.size() == 0) { @@ -125,9 +123,7 @@ public TezIndexRecord getIndexInformation(String mapId, int reduce, } } } - if (LOG.isDebugEnabled()) { - LOG.debug("IndexCache HIT: MapId " + mapId + " found"); - } + LOG.debug("IndexCache HIT: MapId {} found", mapId); } if (info.mapSpillRecord.size() == 0 || @@ -161,14 +157,11 @@ private IndexInformation readIndexFileToCache(Path indexFileName, } } } - if (LOG.isDebugEnabled()) { - LOG.debug("IndexCache HIT: MapId " + mapId + " found"); - } + LOG.debug("IndexCache HIT: MapId {} found", mapId); return info; } - if (LOG.isDebugEnabled()) { - LOG.debug("IndexCache MISS: MapId " + mapId + " not found"); - } + LOG.debug("IndexCache MISS: MapId {} not found", mapId); + TezSpillRecord tmp = null; try { tmp = new TezSpillRecord(indexFileName, fs, expectedIndexOwner); diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java index b67883dfcf..55389ea78e 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java @@ -933,9 +933,7 @@ public AttemptPathInfo load(AttemptPathIdentifier key) throws Path mapOutputFileName = lDirAlloc.getLocalPathToRead( attemptBase + Path.SEPARATOR + DATA_FILE_NAME, conf); - if (LOG.isDebugEnabled()) { - LOG.debug("Loaded : " + key + " via loader"); - } + LOG.debug("Loaded : {} via loader", key); return new AttemptPathInfo(indexFileName, mapOutputFileName); } }); @@ -1011,10 +1009,7 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) boolean keepAliveParam = false; if (keepAliveList != null && keepAliveList.size() == 1) { keepAliveParam = Boolean.parseBoolean(keepAliveList.get(0)); - if (LOG.isDebugEnabled()) { - LOG.debug("KeepAliveParam : " + keepAliveList - + " : " + keepAliveParam); - } + LOG.debug("KeepAliveParam : {} : {}", keepAliveList, keepAliveParam); } final List mapIds = splitMaps(q.get("map")); final Range reduceRange = splitReduces(q.get("reduce")); @@ -1226,11 +1221,8 @@ protected MapOutputInfo getMapOutputInfo(String dagId, String mapId, AttemptPathIdentifier identifier = new AttemptPathIdentifier( jobId, dagId, user, mapId); pathInfo = pathCache.get(identifier); - if (LOG.isDebugEnabled()) { - LOG.debug("Retrieved pathInfo for " + identifier + - " check for corresponding loaded messages to determine whether" + - " it was loaded or cached"); - } + LOG.debug("Retrieved pathInfo for {} check for corresponding loaded " + + "messages to determine whether it was loaded or cached", identifier); } catch (ExecutionException e) { if (e.getCause() instanceof IOException) { throw (IOException) e.getCause(); @@ -1303,13 +1295,9 @@ protected void setResponseHeaders(HttpResponse response, boolean keepAliveParam, response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, String.valueOf(contentLength)); response.headers().set(HttpHeaders.Names.CONNECTION, HttpHeaders.Values.KEEP_ALIVE); response.headers().set(HttpHeaders.Values.KEEP_ALIVE, "timeout=" + connectionKeepAliveTimeOut); - if (LOG.isDebugEnabled()) { - LOG.debug("Content Length in shuffle : " + contentLength); - } + LOG.debug("Content Length in shuffle : {}", contentLength); } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Setting connection close header..."); - } + LOG.debug("Setting connection close header..."); response.headers().set(HttpHeaders.Names.CONNECTION, CONNECTION_CLOSE); } } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java index f8a3de2242..583cc0099a 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java @@ -467,9 +467,7 @@ protected Void callInternal() throws Exception { } protected Void _callInternal() throws Exception { - if (LOG.isDebugEnabled()) { - LOG.debug("Initializing Input using InputSpec: " + inputSpec); - } + LOG.debug("Initializing Input using InputSpec: {}", inputSpec); String edgeName = inputSpec.getSourceVertexName(); InputContext inputContext = createInputContext(inputsMap, inputSpec, inputIndex); LogicalInput input = createInput(inputSpec, inputContext); @@ -483,9 +481,7 @@ protected Void _callInternal() throws Exception { inputContext.getTaskVertexName(), inputContext.getSourceVertexName(), taskSpec.getTaskAttemptID()); initializedInputs.put(edgeName, input); - if (LOG.isDebugEnabled()) { - LOG.debug("Initialized Input with src edge: " + edgeName); - } + LOG.debug("Initialized Input with src edge: {}", edgeName); initializedInputs.put(edgeName, input); return null; } @@ -512,9 +508,7 @@ protected Void callInternal() throws Exception { } protected Void _callInternal() throws Exception { - if (LOG.isDebugEnabled()) { - LOG.debug("Starting Input with src edge: " + srcVertexName); - } + LOG.debug("Starting Input with src edge: {}", srcVertexName); input.start(); LOG.info("Started Input with src edge: " + srcVertexName); @@ -544,9 +538,7 @@ protected Void callInternal() throws Exception { } protected Void _callInternal() throws Exception { - if (LOG.isDebugEnabled()) { - LOG.debug("Initializing Output using OutputSpec: " + outputSpec); - } + LOG.debug("Initializing Output using OutputSpec: {}", outputSpec); String edgeName = outputSpec.getDestinationVertexName(); OutputContext outputContext = createOutputContext(outputSpec, outputIndex); LogicalOutput output = createOutput(outputSpec, outputContext); @@ -559,9 +551,7 @@ protected Void _callInternal() throws Exception { outputContext.getTaskVertexName(), outputContext.getDestinationVertexName(), taskSpec.getTaskAttemptID()); initializedOutputs.put(edgeName, output); - if (LOG.isDebugEnabled()) { - LOG.debug("Initialized Output with dest edge: " + edgeName); - } + LOG.debug("Initialized Output with dest edge: {}", edgeName); initializedOutputs.put(edgeName, output); return null; } @@ -579,9 +569,7 @@ private void initializeGroupInputs() throws TezException { if (groupInputSpecs != null && !groupInputSpecs.isEmpty()) { groupInputsMap = new ConcurrentHashMap(groupInputSpecs.size()); for (GroupInputSpec groupInputSpec : groupInputSpecs) { - if (LOG.isDebugEnabled()) { - LOG.debug("Initializing GroupInput using GroupInputSpec: " + groupInputSpec); - } + LOG.debug("Initializing GroupInput using GroupInputSpec: {}", groupInputSpec); MergedInputContext mergedInputContext = new TezMergedInputContextImpl(groupInputSpec.getMergedInputDescriptor().getUserPayload(), groupInputSpec.getGroupName(), groupInputsMap, inputReadyTracker, localDirs, this); diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java index 9ff284d5bf..f28573a649 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java @@ -187,8 +187,6 @@ public void close() throws IOException { super.close(); this.userPayload = null; this.inputReadyTracker = null; - if (LOG.isDebugEnabled()) { - LOG.debug("Cleared TezInputContextImpl related information"); - } + LOG.debug("Cleared TezInputContextImpl related information"); } } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java index db3212280f..ec8280a239 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java @@ -167,8 +167,6 @@ public OutputStatisticsReporter getStatisticsReporter() { public void close() throws IOException { super.close(); this.userPayload = null; - if (LOG.isDebugEnabled()) { - LOG.debug("Cleared TezOutputContextImpl related information"); - } + LOG.debug("Cleared TezOutputContextImpl related information"); } } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java index 71ed077c50..e09aa8377e 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java @@ -143,9 +143,7 @@ public void close() throws IOException { super.close(); this.userPayload = null; this.inputReadyTracker = null; - if (LOG.isDebugEnabled()) { - LOG.debug("Cleared TezProcessorContextImpl related information"); - } + LOG.debug("Cleared TezProcessorContextImpl related information"); } } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/resources/ScalingAllocator.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/resources/ScalingAllocator.java index 872632e1ea..e045abd9d2 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/resources/ScalingAllocator.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/resources/ScalingAllocator.java @@ -80,16 +80,11 @@ public Long apply(InitialMemoryRequestContext requestContext) { long requestedSize = request.getRequestedSize(); if (requestedSize == 0) { allocations.add(0l); - if (LOG.isDebugEnabled()) { - LOG.debug("Scaling requested: 0 to allocated: 0"); - } + LOG.debug("Scaling requested: 0 to allocated: 0"); } else { long allocated = (long) ((requestedSize / (double) totalRequested) * availableForAllocation); allocations.add(allocated); - if (LOG.isDebugEnabled()) { - LOG.debug("Scaling requested: " + requestedSize + " to allocated: " + allocated); - } - + LOG.debug("Scaling requested: {} to allocated: {}", requestedSize, allocated); } } return allocations; diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskReporter.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskReporter.java index fb066fd2bd..978942d4e7 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskReporter.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskReporter.java @@ -264,16 +264,12 @@ private synchronized ResponseWrapper heartbeat(Collection eventsArg) t int maxEvents = Math.min(maxEventsToGet, task.getMaxEventsToHandle()); TezHeartbeatRequest request = new TezHeartbeatRequest(requestId, events, fromPreRoutedEventId, containerIdStr, task.getTaskAttemptID(), fromEventId, maxEvents); - if (LOG.isDebugEnabled()) { - LOG.debug("Sending heartbeat to AM, request=" + request); - } + LOG.debug("Sending heartbeat to AM, request={}", request); maybeLogCounters(); TezHeartbeatResponse response = umbilical.heartbeat(request); - if (LOG.isDebugEnabled()) { - LOG.debug("Received heartbeat response from AM, response=" + response); - } + LOG.debug("Received heartbeat response from AM, response={}", response); if (response.shouldDie()) { LOG.info("Received should die response from AM"); diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java index 86ceb12d7d..c82355a9fa 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java @@ -337,9 +337,7 @@ private void handleNewTaskLocalResources(ContainerTask containerTask, UserGroupInformation ugi) throws IOException, TezException { final Map additionalResources = containerTask.getAdditionalResources(); - if (LOG.isDebugEnabled()) { - LOG.debug("Additional Resources added to container: " + additionalResources); - } + LOG.debug("Additional Resources added to container: {}", additionalResources); if (additionalResources != null && !additionalResources.isEmpty()) { LOG.info("Localizing additional local resources for Task : " + additionalResources); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnection.java b/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnection.java index 3b45cdd709..e12331c250 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnection.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnection.java @@ -77,9 +77,7 @@ public HttpConnection(URL url, HttpConnectionParams connParams, this.url = url; this.stopWatch = new StopWatch(); this.urlLogCount = new AtomicLong(); - if (LOG.isDebugEnabled()) { - LOG.debug("MapOutput URL :" + url.toString()); - } + LOG.debug("MapOutput URL :{}", url); } @VisibleForTesting @@ -278,9 +276,7 @@ public void cleanup(boolean disconnect) throws IOException { stopWatch.reset().start(); try { if (input != null) { - if (LOG.isDebugEnabled()) { - LOG.debug("Closing input on " + logIdentifier); - } + LOG.debug("Closing input on {}", logIdentifier); input.close(); input = null; } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java b/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java index 43f64b82b3..63b8934821 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java @@ -113,9 +113,7 @@ public AsyncHttpConnection(URL url, HttpConnectionParams connParams, this.httpConnParams = connParams; this.url = url; this.stopWatch = new StopWatch(); - if (LOG.isDebugEnabled()) { - LOG.debug("MapOutput URL :" + url.toString()); - } + LOG.debug("MapOutput URL :{}", url); initClient(httpConnParams); pos = new PipedOutputStream(); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java index daeafbc6fe..9ff3d1c1e5 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java @@ -80,9 +80,7 @@ public static Combiner instantiateCombiner(Configuration conf, TaskContext taskC if (className == null) { return null; } - if (LOG.isDebugEnabled()) { - LOG.debug("Using Combiner class: " + className); - } + LOG.debug("Using Combiner class: {}", className); try { clazz = (Class) conf.getClassByName(className); } catch (ClassNotFoundException e) { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java index 40909d4715..6a61474a5a 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java @@ -121,10 +121,7 @@ public static void shuffleToMemory(byte[] shuffleData, IFile.Reader.readToMemory(shuffleData, input, compressedLength, codec, ifileReadAhead, ifileReadAheadLength); // metrics.inputBytes(shuffleData.length); - if (LOG.isDebugEnabled()) { - LOG.debug("Read " + shuffleData.length + " bytes from input for " - + identifier); - } + LOG.debug("Read {} bytes from input for {}", shuffleData.length, identifier); } catch (InternalError | Exception e) { // Close the streams LOG.info("Failed to read data to memory for " + identifier + ". len=" + compressedLength + @@ -200,8 +197,7 @@ public static void ioCleanup(Closeable... closeables) { try { c.close(); } catch (IOException e) { - if (LOG.isDebugEnabled()) - LOG.debug("Exception in closing " + c, e); + LOG.debug("Exception in closing {}", c, e); } } } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java index e924876628..bcb7bb58ea 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java @@ -183,10 +183,8 @@ private void processDataMovementEvent(DataMovementEvent dme, DataMovementEventPa if (emptyPartitionsBitSet.get(srcIndex)) { CompositeInputAttemptIdentifier srcAttemptIdentifier = constructInputAttemptIdentifier(dme.getTargetIndex(), 1, dme.getVersion(), shufflePayload, false); - if (LOG.isDebugEnabled()) { - LOG.debug("Source partition: " + srcIndex + " did not generate any data. SrcAttempt: [" - + srcAttemptIdentifier + "]. Not fetching."); - } + LOG.debug("Source partition: {} did not generate any data. SrcAttempt: [{}]. Not fetching.", + srcIndex, srcAttemptIdentifier); numDmeEventsNoData.getAndIncrement(); shuffleManager.addCompletedInputWithNoData(srcAttemptIdentifier.expand(0)); return; @@ -261,10 +259,8 @@ private void processCompositeRoutedDataMovementEvent(CompositeRoutedDataMovement allPartitionsEmpty &= emptyPartitionsBitSet.get(srcPartitionId); if (emptyPartitionsBitSet.get(srcPartitionId)) { InputAttemptIdentifier srcAttemptIdentifier = compositeInputAttemptIdentifier.expand(i); - if (LOG.isDebugEnabled()) { - LOG.debug("Source partition: " + srcPartitionId + " did not generate any data. SrcAttempt: [" - + srcAttemptIdentifier + "]. Not fetching."); - } + LOG.debug("Source partition: {} did not generate any data. SrcAttempt: [{}]. Not fetching.", + srcPartitionId, srcAttemptIdentifier); numDmeEventsNoData.getAndIncrement(); shuffleManager.addCompletedInputWithNoData(srcAttemptIdentifier); } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java index 901ee08a7a..56195a8641 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java @@ -441,9 +441,7 @@ protected Void callInternal() throws Exception { break; } - if (LOG.isDebugEnabled()) { - LOG.debug(srcNameTrimmed + ": " + "NumCompletedInputs: " + numCompletedInputs); - } + LOG.debug("{}: NumCompletedInputs: {}", srcNameTrimmed, numCompletedInputs); if (numCompletedInputs.get() < numInputs && !isShutdown.get()) { lock.lock(); try { @@ -661,9 +659,7 @@ public void addKnownInput(String hostName, int port, public void addCompletedInputWithNoData( InputAttemptIdentifier srcAttemptIdentifier) { int inputIdentifier = srcAttemptIdentifier.getInputIdentifier(); - if (LOG.isDebugEnabled()) { - LOG.debug("No input data exists for SrcTask: " + inputIdentifier + ". Marking as complete."); - } + LOG.debug("No input data exists for SrcTask: {}. Marking as complete.", inputIdentifier); lock.lock(); try { if (!completedInputSet.get(inputIdentifier)) { @@ -1144,9 +1140,7 @@ public void onSuccess(Void result) { @Override public void onFailure(Throwable t) { if (isShutdown.get()) { - if (LOG.isDebugEnabled()) { - LOG.debug(srcNameTrimmed + ": " + "Already shutdown. Ignoring error: " + t); - } + LOG.debug("{}: Already shutdown. Ignoring error.", srcNameTrimmed, t); } else { LOG.error(srcNameTrimmed + ": " + "Scheduler failed with error: ", t); inputContext.reportFailure(TaskFailureType.NON_FATAL, t, "Shuffle Scheduler Failed"); @@ -1177,9 +1171,7 @@ private void doBookKeepingForFetcherComplete() { public void onSuccess(FetchResult result) { fetcher.shutdown(); if (isShutdown.get()) { - if (LOG.isDebugEnabled()) { - LOG.debug(srcNameTrimmed + ": " + "Already shutdown. Ignoring event from fetcher"); - } + LOG.debug("{}: Already shutdown. Ignoring event from fetcher", srcNameTrimmed); } else { Iterable pendingInputs = result.getPendingInputs(); if (pendingInputs != null && pendingInputs.iterator().hasNext()) { @@ -1202,9 +1194,7 @@ public void onFailure(Throwable t) { // Unsuccessful - the fetcher may not have shutdown correctly. Try shutting it down. fetcher.shutdown(); if (isShutdown.get()) { - if (LOG.isDebugEnabled()) { - LOG.debug(srcNameTrimmed + ": " + "Already shutdown. Ignoring error from fetcher: " + t); - } + LOG.debug("{}: Already shutdown. Ignoring error from fetcher.", srcNameTrimmed, t); } else { LOG.error(srcNameTrimmed + ": " + "Fetcher failed with error: ", t); shuffleError = t; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java index 327232710b..c9bd092f05 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java @@ -208,9 +208,7 @@ public Void callInternal() { public void shutDown() { if (!stopped) { - if (LOG.isDebugEnabled()) { - LOG.debug("Fetcher stopped for host " + mapHost); - } + LOG.debug("Fetcher stopped for host {}", mapHost); stopped = true; // An interrupt will come in while shutting down the thread. cleanupCurrentConnection(false); @@ -288,19 +286,14 @@ protected void copyFromHost(MapHost host) throws IOException { // Setup connection again if disconnected cleanupCurrentConnection(true); if (stopped) { - if (LOG.isDebugEnabled()) { - LOG.debug("Not re-establishing connection since Fetcher has been stopped"); - } + LOG.debug("Not re-establishing connection since Fetcher has been stopped"); return; } // Connect with retry if (!setupConnection(host, remaining.values())) { if (stopped) { cleanupCurrentConnection(true); - if (LOG.isDebugEnabled()) { - LOG.debug( - "Not reporting connection re-establishment failure since fetcher is stopped"); - } + LOG.debug("Not reporting connection re-establishment failure since fetcher is stopped"); return; } failedTasks = new InputAttemptFetchFailure[] { @@ -354,9 +347,7 @@ boolean setupConnection(MapHost host, Collection attempt connectSucceeded = httpConnection.connect(); if (stopped) { - if (LOG.isDebugEnabled()) { - LOG.debug("Detected fetcher has been shutdown after connection establishment. Returning"); - } + LOG.debug("Detected fetcher has been shutdown after connection establishment. Returning"); return false; } input = httpConnection.getInputStream(); @@ -367,9 +358,7 @@ boolean setupConnection(MapHost host, Collection attempt Thread.currentThread().interrupt(); //reset status } if (stopped) { - if (LOG.isDebugEnabled()) { - LOG.debug("Not reporting fetch failure, since an Exception was caught after shutdown"); - } + LOG.debug("Not reporting fetch failure, since an Exception was caught after shutdown"); return false; } ioErrs.increment(1); @@ -471,9 +460,7 @@ protected InputAttemptFetchFailure[] copyMapOutput(MapHost host, DataInputStream return new InputAttemptFetchFailure[] { InputAttemptFetchFailure.fromAttempt(getNextRemainingAttempt()) }; } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Already shutdown. Ignoring invalid map id error"); - } + LOG.debug("Already shutdown. Ignoring invalid map id error"); return EMPTY_ATTEMPT_ID_ARRAY; } } @@ -518,9 +505,7 @@ protected InputAttemptFetchFailure[] copyMapOutput(MapHost host, DataInputStream return new InputAttemptFetchFailure[] { new InputAttemptFetchFailure(getNextRemainingAttempt()) }; } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Already stopped. Ignoring verification failure."); - } + LOG.debug("Already stopped. Ignoring verification failure."); return EMPTY_ATTEMPT_ID_ARRAY; } } @@ -544,9 +529,7 @@ protected InputAttemptFetchFailure[] copyMapOutput(MapHost host, DataInputStream ioErrs.increment(1); scheduler.reportLocalError(e); } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Already stopped. Ignoring error from merger.reserve"); - } + LOG.debug("Already stopped. Ignoring error from merger.reserve"); } return EMPTY_ATTEMPT_ID_ARRAY; } @@ -760,10 +743,7 @@ protected void setupLocalDiskFetch(MapHost host) throws InterruptedException { LOG.warn("Failed to read local disk output of " + srcAttemptId + " from " + host.getHostIdentifier(), e); } else { - if (LOG.isDebugEnabled()) { - LOG.debug( - "Ignoring fetch error during local disk copy since fetcher has already been stopped"); - } + LOG.debug("Ignoring fetch error during local disk copy since fetcher has already been stopped"); return; } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java index 59ff577fed..46360e1287 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java @@ -388,9 +388,7 @@ public void waitForInMemoryMerge() throws InterruptedException { } if (triggerAdditionalMerge) { inMemoryMerger.waitForMerge(); - if (LOG.isDebugEnabled()) { - LOG.debug("Additional in-memory merge triggered"); - } + LOG.debug("Additional in-memory merge triggered"); } } @@ -700,9 +698,7 @@ static void cleanup(FileSystem fs, Path path) { } try { - if (LOG.isDebugEnabled()) { - LOG.debug("Deleting " + path); - } + LOG.debug("Deleting {}", path); fs.delete(path, true); } catch (IOException e) { LOG.info("Error in deleting " + path); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java index 116098fe26..0c55a3a388 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java @@ -156,11 +156,8 @@ private void processDataMovementEvent(DataMovementEvent dmEvent, DataMovementEve if (shufflePayload.hasEmptyPartitions()) { try { if (emptyPartitionsBitSet.get(partitionId)) { - if (LOG.isDebugEnabled()) { - LOG.debug( - "Source partition: " + partitionId + " did not generate any data. SrcAttempt: [" - + srcAttemptIdentifier + "]. Not fetching."); - } + LOG.debug("Source partition: {} did not generate any data. SrcAttempt: [{}]. Not fetching.", + partitionId, srcAttemptIdentifier); numDmeEventsNoData.getAndIncrement(); scheduler.copySucceeded(srcAttemptIdentifier.expand(0), null, 0, 0, 0, null, true); return; @@ -191,10 +188,8 @@ private void processCompositeRoutedDataMovementEvent(CompositeRoutedDataMovement allPartitionsEmpty &= emptyPartitionsBitSet.get(srcPartitionId); if (emptyPartitionsBitSet.get(srcPartitionId)) { InputAttemptIdentifier srcInputAttemptIdentifier = compositeInputAttemptIdentifier.expand(i); - if (LOG.isDebugEnabled()) { - LOG.debug("Source partition: " + srcPartitionId + " did not generate any data. SrcAttempt: [" - + srcInputAttemptIdentifier + "]. Not fetching."); - } + LOG.debug("Source partition: {} did not generate any data. SrcAttempt: [{}]. Not fetching.", + srcPartitionId, srcInputAttemptIdentifier); numDmeEventsNoData.getAndIncrement(); scheduler.copySucceeded(srcInputAttemptIdentifier, null, 0, 0, 0, null, true); } @@ -212,9 +207,7 @@ private void processCompositeRoutedDataMovementEvent(CompositeRoutedDataMovement private void processTaskFailedEvent(InputFailedEvent ifEvent) { InputAttemptIdentifier taIdentifier = new InputAttemptIdentifier(ifEvent.getTargetIndex(), ifEvent.getVersion()); scheduler.obsoleteInput(taIdentifier); - if (LOG.isDebugEnabled()) { - LOG.debug("Obsoleting output of src-task: " + taIdentifier); - } + LOG.debug("Obsoleting output of src-task: {}", taIdentifier); } /** diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java index 416041e005..67681cedc7 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java @@ -1104,9 +1104,7 @@ public void obsoleteInput(InputAttemptIdentifier srcAttempt) { if (eventInfo.eventsProcessed.isEmpty() && !eventInfo.scheduledForDownload) { // obsoleted anyways; no point tracking if nothing is started pipelinedShuffleInfoEventsMap.remove(srcAttempt.getInputIdentifier()); - if (LOG.isDebugEnabled()) { - LOG.debug("Removing " + eventInfo + " from tracking"); - } + LOG.debug("Removing {} from tracking", eventInfo); return; } IOException exception = new IOException(srcAttempt + " is marked as obsoleteInput, but it " @@ -1128,9 +1126,7 @@ public synchronized void putBackKnownMapOutput(MapHost host, public synchronized MapHost getHost() throws InterruptedException { while (pendingHosts.isEmpty() && remainingMaps.get() > 0) { - if (LOG.isDebugEnabled()) { - LOG.debug("PendingHosts=" + pendingHosts); - } + LOG.debug("PendingHosts={}", pendingHosts); waitAndNotifyProgress(); } @@ -1436,9 +1432,7 @@ protected Void callInternal() throws InterruptedException { if (mapHost == null) { break; // Check for the exit condition. } - if (LOG.isDebugEnabled()) { - LOG.debug(srcNameTrimmed + ": " + "Processing pending host: " + mapHost.toString()); - } + LOG.debug("{}: Processing pending host: {}", srcNameTrimmed, mapHost); if (!isShutdown.get()) { count++; if (LOG.isDebugEnabled()) { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java index 1b2aefff41..a4bbf5aabf 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java @@ -844,9 +844,7 @@ public static void readToMemory(byte[] buffer, InputStream in, int compressedLen try { in.close(); } catch(IOException e) { - if(LOG.isDebugEnabled()) { - LOG.debug("Exception in closing " + in, e); - } + LOG.debug("Exception in closing {}", in, e); } } throw ioe; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutputFiles.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutputFiles.java index 88474f99f4..3fb90865d1 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutputFiles.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutputFiles.java @@ -65,11 +65,7 @@ public TezTaskOutputFiles(Configuration conf, String uniqueId, int dagID) { * if service_id = tez_shuffle then "${appDir}/dagId/output/${uniqueId}" */ private Path getAttemptOutputDir() { - if (LOG.isDebugEnabled()) { - LOG.debug("getAttemptOutputDir: " - + Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + "/" - + uniqueId); - } + LOG.debug("getAttemptOutputDir: {}/{}", Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR, uniqueId); String dagPath = getDagOutputDir(Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR); return new Path(dagPath, uniqueId); } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java index 2b405bb343..313c13d188 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java @@ -130,9 +130,7 @@ public synchronized void start() throws IOException { // Start the shuffle - copy and merge shuffle = createShuffle(); shuffle.run(); - if (LOG.isDebugEnabled()) { - LOG.debug("Initialized the handlers in shuffle..Safe to start processing.."); - } + LOG.debug("Initialized the handlers in shuffle..Safe to start processing.."); List pending = new LinkedList(); pendingEvents.drainTo(pending); if (pending.size() > 0) { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/resources/WeightedScalingMemoryDistributor.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/resources/WeightedScalingMemoryDistributor.java index b82e6d3d8f..52f3d44b4f 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/resources/WeightedScalingMemoryDistributor.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/resources/WeightedScalingMemoryDistributor.java @@ -244,9 +244,7 @@ private RequestType getRequestTypeForClass(String className) { requestType = RequestType.PARTITIONED_UNSORTED_OUTPUT; } else { requestType = RequestType.OTHER; - if (LOG.isDebugEnabled()) { - LOG.debug("Falling back to RequestType.OTHER for class: " + className); - } + LOG.debug("Falling back to RequestType.OTHER for class: {}", className); } return requestType; } diff --git a/tez-tools/tez-tfile-parser/src/main/java/org/apache/tez/tools/TFileLoader.java b/tez-tools/tez-tfile-parser/src/main/java/org/apache/tez/tools/TFileLoader.java index 18e9940d1d..320428b8cd 100644 --- a/tez-tools/tez-tfile-parser/src/main/java/org/apache/tez/tools/TFileLoader.java +++ b/tez-tools/tez-tfile-parser/src/main/java/org/apache/tez/tools/TFileLoader.java @@ -66,10 +66,7 @@ public Tuple getNext() throws IOException { currentKey = recReader.getCurrentKey(); String line = recReader.getCurrentValue().toString(); - if (LOG.isDebugEnabled()) { - LOG.debug("currentKey: " + currentKey - + ", line=" + line); - } + LOG.debug("currentKey: {}, line={}", currentKey, line); //Tuple would be of format: machine, key, line Tuple tuple = tupleFactory.newTuple(3); if (currentKey != null) { From f481d3899dceda668b74d993cd37f72da3009127 Mon Sep 17 00:00:00 2001 From: Eric Badger Date: Fri, 22 Jan 2021 16:48:15 -0600 Subject: [PATCH 284/512] TEZ-4265. Tez UI doesn't show Load Counters button if any of the Dags have them Signed-off-by: Jonathan Eagles --- tez-ui/src/main/webapp/app/components/home-table-controls.js | 4 ++-- .../tests/integration/components/home-table-controls-test.js | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tez-ui/src/main/webapp/app/components/home-table-controls.js b/tez-ui/src/main/webapp/app/components/home-table-controls.js index 70c3863416..0804ede998 100644 --- a/tez-ui/src/main/webapp/app/components/home-table-controls.js +++ b/tez-ui/src/main/webapp/app/components/home-table-controls.js @@ -25,8 +25,8 @@ export default Ember.Component.extend({ var processedRows = this.get("dataProcessor.processedRows"), countersLoaded = true; if(processedRows) { - countersLoaded = processedRows.some(function (row) { - return Object.keys(row.get("counterGroupsHash")).length !== 0; + countersLoaded = !processedRows.any(function (row) { + return Object.keys(row.get("counterGroupsHash")).length === 0; }); } return countersLoaded; diff --git a/tez-ui/src/main/webapp/tests/integration/components/home-table-controls-test.js b/tez-ui/src/main/webapp/tests/integration/components/home-table-controls-test.js index ccce2dbe29..c35c339e3c 100644 --- a/tez-ui/src/main/webapp/tests/integration/components/home-table-controls-test.js +++ b/tez-ui/src/main/webapp/tests/integration/components/home-table-controls-test.js @@ -66,7 +66,7 @@ test('countersLoaded test', function(assert) { })] }); this.render(hbs`{{home-table-controls dataProcessor=dataProcessor}}`); - assert.equal(this.$().find("button").attr("class").split(" ").indexOf("no-visible"), 2); + assert.equal(this.$().find("button").attr("class").split(" ").indexOf("no-visible"), -1); this.set("dataProcessor", { processedRows: [Ember.Object.create({ From 7c71780ae6546ea5a1703d25675c1f8f94f384e6 Mon Sep 17 00:00:00 2001 From: Marton Bod Date: Mon, 25 Jan 2021 11:27:24 +0100 Subject: [PATCH 285/512] =?UTF-8?q?TEZ-4264:=20Make=20vertex=20id=20availa?= =?UTF-8?q?ble=20from=20MROutputCommitter=20(Marton=20Bod=20via=20L=C3=A1s?= =?UTF-8?q?zl=C3=B3=20Bodor)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../org/apache/tez/mapreduce/committer/MROutputCommitter.java | 1 + .../main/java/org/apache/tez/mapreduce/hadoop/MRJobConfig.java | 1 + 2 files changed, 2 insertions(+) diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/committer/MROutputCommitter.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/committer/MROutputCommitter.java index b7ebc4c05d..71e5681cbf 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/committer/MROutputCommitter.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/committer/MROutputCommitter.java @@ -78,6 +78,7 @@ public void initialize() throws IOException { jobConf.getCredentials().mergeAll(UserGroupInformation.getCurrentUser().getCredentials()); jobConf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, getContext().getDAGAttemptNumber()); + jobConf.setInt(MRJobConfig.VERTEX_ID, getContext().getVertexIndex()); committer = getOutputCommitter(getContext()); jobContext = getJobContextFromVertexContext(getContext()); initialized = true; diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRJobConfig.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRJobConfig.java index ca954d92ff..e162460773 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRJobConfig.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRJobConfig.java @@ -645,6 +645,7 @@ public interface MRJobConfig { "mrr.vertex."; public static final String VERTEX_NAME = "mapreduce.task.vertex.name"; + public static final String VERTEX_ID = "mapreduce.task.vertex.id"; public static final String MR_TEZ_SPLITS_VIA_EVENTS = MR_TEZ_PREFIX + "splits.via.events"; public static final boolean MR_TEZ_SPLITS_VIA_EVENTS_DEFAULT = true; From 7374b69ed6c4af4e7c9127f1b05473c2166abf6d Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Mon, 25 Jan 2021 17:44:11 -0500 Subject: [PATCH 286/512] TEZ-4268: Buffer File Read of TEZ AM Local Resources PB File (#91) --- .../org/apache/tez/dag/app/DAGAppMaster.java | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index cde77b3bf6..ba072a9b5e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -20,6 +20,7 @@ +import java.io.BufferedInputStream; import java.io.File; import java.io.FileInputStream; import java.io.FileNotFoundException; @@ -604,17 +605,14 @@ public synchronized void serviceInit(final Configuration conf) throws Exception if (!versionMismatch) { if (isSession) { - FileInputStream sessionResourcesStream = null; - try { - sessionResourcesStream = new FileInputStream( - new File(workingDirectory, TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME)); + try (BufferedInputStream sessionResourcesStream = + new BufferedInputStream( + new FileInputStream(new File(workingDirectory, + TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME)))) { PlanLocalResourcesProto amLocalResourceProto = PlanLocalResourcesProto .parseDelimitedFrom(sessionResourcesStream); - amResources.putAll(DagTypeConverters.convertFromPlanLocalResources(amLocalResourceProto)); - } finally { - if (sessionResourcesStream != null) { - sessionResourcesStream.close(); - } + amResources.putAll(DagTypeConverters + .convertFromPlanLocalResources(amLocalResourceProto)); } } } From 04597c7b591917bbe995a9e182c3be60e33816c9 Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Thu, 28 Jan 2021 05:20:25 +0530 Subject: [PATCH 287/512] TEZ-3985: Correctness: Throw a clear exception for DMEs sent during cleanup (Contributed by Jaume M, reviewed by Rajesh Balamohan, Ashutosh Chauhan) --- .../apache/tez/runtime/api/OutputContext.java | 10 ++ .../api/impl/TezOutputContextImpl.java | 30 +++++- .../runtime/internals/api/TezTrapEvent.java | 52 ++++++++++ .../internals/api/TezTrapEventType.java | 29 ++++++ .../tez/runtime/task/TaskRunner2Callable.java | 12 ++- .../tez/runtime/task/TezTaskRunner2.java | 3 +- .../tez/runtime/task/TezTrapEventHandler.java | 92 ++++++++++++++++++ .../TestLogicalIOProcessorRuntimeTask.java | 97 ++++++++++++++++++- 8 files changed, 318 insertions(+), 7 deletions(-) create mode 100644 tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TezTrapEvent.java create mode 100644 tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TezTrapEventType.java create mode 100644 tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTrapEventHandler.java diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/OutputContext.java b/tez-api/src/main/java/org/apache/tez/runtime/api/OutputContext.java index 882eb4be2a..33fe772f85 100644 --- a/tez-api/src/main/java/org/apache/tez/runtime/api/OutputContext.java +++ b/tez-api/src/main/java/org/apache/tez/runtime/api/OutputContext.java @@ -19,6 +19,7 @@ package org.apache.tez.runtime.api; import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.yarn.event.EventHandler; /** * Context handle for the Output to initialize itself. @@ -48,4 +49,13 @@ public interface OutputContext extends TaskContext { */ public OutputStatisticsReporter getStatisticsReporter(); + /** + * Notify the context that at this point no more events should be sent. + * This is used as a safety measure to prevent events being sent after close + * or in cleanup. After this is called events being queued to be sent to the + * AM will instead be passed to the event handler. + * @param eventHandler should handle the events after the call. + */ + void trapEvents(EventHandler eventHandler); + } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java index ec8280a239..20ec0622c7 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java @@ -29,7 +29,9 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.event.EventHandler; import org.apache.tez.common.TezExecutors; +import org.apache.tez.runtime.internals.api.TezTrapEvent; import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.OutputDescriptor; import org.apache.tez.dag.api.TezConfiguration; @@ -47,17 +49,28 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +@SuppressWarnings("unchecked") public class TezOutputContextImpl extends TezTaskContextImpl implements OutputContext { private static final Logger LOG = LoggerFactory.getLogger(TezOutputContextImpl.class); private volatile UserPayload userPayload; + + /** + * Holds whether we can accept more events to send to the AM. + */ + private volatile boolean trapEvents; private final String destinationVertexName; private final EventMetaData sourceInfo; private final int outputIndex; private final OutputStatisticsReporterImpl statsReporter; + /** + * Handler for the events after the trap flag is set. + */ + private EventHandler trapEventHandler; + class OutputStatisticsReporterImpl implements OutputStatisticsReporter { @Override @@ -71,7 +84,7 @@ public synchronized void reportDataSize(long size) { public void reportItemsProcessed(long items) { // this is a concurrent map. Plus we are not adding/deleting entries runtimeTask.getTaskStatistics().getIOStatistics().get(destinationVertexName) - .setItemsProcessed(items);; + .setItemsProcessed(items); } } @@ -124,7 +137,11 @@ public void sendEvents(List events) { TezEvent tEvt = new TezEvent(e, sourceInfo); tezEvents.add(tEvt); } - tezUmbilical.addEvents(tezEvents); + if (trapEvents) { + trapEventHandler.handle(new TezTrapEvent(tezEvents)); + } else { + tezUmbilical.addEvents(tezEvents); + } } @Override @@ -163,6 +180,15 @@ public OutputStatisticsReporter getStatisticsReporter() { return statsReporter; } + /** + * This will monitor some of the events that will be sent. + */ + @Override + public final void trapEvents(final EventHandler eventHandler) { + trapEvents = true; + this.trapEventHandler = eventHandler; + } + @Override public void close() throws IOException { super.close(); diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TezTrapEvent.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TezTrapEvent.java new file mode 100644 index 0000000000..8c63b74007 --- /dev/null +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TezTrapEvent.java @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.runtime.internals.api; + +import org.apache.tez.common.TezAbstractEvent; +import org.apache.tez.runtime.api.impl.TezEvent; + +import java.util.List; + +import static org.apache.tez.runtime.internals.api.TezTrapEventType.TRAP_EVENT_TYPE; + +/** + * Event sent when no more events should be sent to the AM. + */ +public class TezTrapEvent extends TezAbstractEvent { + /** + * Events that were reported. + */ + private final List tezEvents; + + /** + * Create a tez trap event. + * @param events events tried to be sent to the AM. + */ + public TezTrapEvent(final List events) { + super(TRAP_EVENT_TYPE); + this.tezEvents = events; + } + + /** + * @return events. + */ + public final List getTezEvents() { + return tezEvents; + } +} \ No newline at end of file diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TezTrapEventType.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TezTrapEventType.java new file mode 100644 index 0000000000..89cb78ea06 --- /dev/null +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TezTrapEventType.java @@ -0,0 +1,29 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.runtime.internals.api; + +/** + * Enum type with only one value representing this event. + */ +public enum TezTrapEventType { + /** + * Single value for this event type. + */ + TRAP_EVENT_TYPE +} diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java index 0e6dfda080..810a806228 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java @@ -24,6 +24,7 @@ import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.common.counters.TezCounters; import org.apache.tez.runtime.LogicalIOProcessorRuntimeTask; +import org.apache.tez.runtime.api.impl.TezUmbilical; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,10 +47,16 @@ public class TaskRunner2Callable extends CallableWithNdc outputContext.trapEvents(new TezTrapEventHandler(outputContext, + this.tezUmbilical))); task.cleanup(); } } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java index ae81769aca..bbf037b8bf 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java @@ -177,7 +177,8 @@ public TaskRunner2Result run() { // Safe to do this within a synchronized block because we're providing // the handler on which the Reporter will communicate back. Assuming // the register call doesn't end up hanging. - taskRunnerCallable = new TaskRunner2Callable(task, ugi); + taskRunnerCallable = new TaskRunner2Callable(task, ugi, + umbilicalAndErrorHandler); taskReporter.registerTask(task, umbilicalAndErrorHandler); future = executor.submit(taskRunnerCallable); } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTrapEventHandler.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTrapEventHandler.java new file mode 100644 index 0000000000..b35dbb0162 --- /dev/null +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTrapEventHandler.java @@ -0,0 +1,92 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package org.apache.tez.runtime.task; + +import com.google.common.base.Preconditions; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tez.runtime.api.OutputContext; +import org.apache.tez.runtime.api.impl.TezEvent; +import org.apache.tez.runtime.api.impl.TezOutputContextImpl; +import org.apache.tez.runtime.api.impl.TezUmbilical; +import org.apache.tez.runtime.internals.api.TezTrapEvent; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; + +/** + * Class that handles the events after the trap has been activated. At + * this point no more events of some types shouldn't be sent and it's + * a bug to do so. If the events arrive here probably the task will be + * restarted because it has failed. + */ +public class TezTrapEventHandler implements EventHandler { + /** + * logger. + */ + private static final Logger + LOG = LoggerFactory.getLogger(TezOutputContextImpl.class); + + /** + * Output context that will report the events. + */ + private final OutputContext outputContext; + + /** + * Protocol to send the events. + */ + private final TezUmbilical tezUmbilical; + + /** + * @param output context that will report the events. + * @param umbilical used to send the events to the AM. + */ + TezTrapEventHandler(final OutputContext output, + final TezUmbilical umbilical) { + this.outputContext = output; + this.tezUmbilical = umbilical; + } + + /** + * Decide what to do with the events. + * @param tezTrapEvent event holding the tez events. + */ + @Override + public final void handle(final TezTrapEvent tezTrapEvent) { + Preconditions.checkArgument(tezTrapEvent.getTezEvents() != null); + List tezEvents = new ArrayList( + tezTrapEvent.getTezEvents().size()); + for (TezEvent tezEvent: tezTrapEvent.getTezEvents()) { + switch (tezEvent.getEventType()) { + case COMPOSITE_DATA_MOVEMENT_EVENT: + case DATA_MOVEMENT_EVENT: + String errorMsg = "Some events won't be sent to the AM because all" + + " the events should have been sent at this point. Most likely" + + " this would result in a bug. " + + " event:" + tezEvent.toString(); + Throwable throwable = new Throwable(errorMsg); + LOG.error(errorMsg, throwable); + break; + default: + LOG.info("Event of type " + tezEvent.getEventType() + " will be sent" + + " to the AM after the task was closed "); + tezEvents.add(tezEvent); + } + } + tezUmbilical.addEvents(tezEvents); + } +} \ No newline at end of file diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/TestLogicalIOProcessorRuntimeTask.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/TestLogicalIOProcessorRuntimeTask.java index 599f98f666..ba9a66d0f1 100644 --- a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/TestLogicalIOProcessorRuntimeTask.java +++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/TestLogicalIOProcessorRuntimeTask.java @@ -24,13 +24,18 @@ import static org.junit.Assert.fail; import static org.mockito.Mockito.mock; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.tez.common.TezExecutors; import org.apache.tez.common.TezSharedExecutor; import org.apache.tez.dag.api.InputDescriptor; import org.apache.tez.dag.api.OutputDescriptor; @@ -41,12 +46,15 @@ import org.apache.tez.dag.records.TezTaskID; import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.hadoop.shim.DefaultHadoopShim; +import org.apache.tez.hadoop.shim.HadoopShim; import org.apache.tez.runtime.api.AbstractLogicalIOProcessor; import org.apache.tez.runtime.api.AbstractLogicalInput; import org.apache.tez.runtime.api.AbstractLogicalOutput; import org.apache.tez.runtime.api.Event; +import org.apache.tez.runtime.api.ExecutionContext; import org.apache.tez.runtime.api.LogicalInput; import org.apache.tez.runtime.api.LogicalOutput; +import org.apache.tez.runtime.api.ObjectRegistry; import org.apache.tez.runtime.api.Reader; import org.apache.tez.runtime.api.InputContext; import org.apache.tez.runtime.api.OutputContext; @@ -57,13 +65,16 @@ import org.apache.tez.runtime.api.impl.InputSpec; import org.apache.tez.runtime.api.impl.OutputSpec; import org.apache.tez.runtime.api.impl.TaskSpec; +import org.apache.tez.runtime.api.impl.TezEvent; import org.apache.tez.runtime.api.impl.TezUmbilical; import org.apache.tez.runtime.common.resources.ScalingAllocator; +import org.apache.tez.runtime.task.TaskRunner2Callable; import org.junit.Test; import com.google.common.collect.HashMultimap; import com.google.common.collect.Lists; import com.google.common.collect.Multimap; + import org.mockito.Mockito; public class TestLogicalIOProcessorRuntimeTask { @@ -149,6 +160,39 @@ public void testAutoStart() throws Exception { } + @Test + public void testEventsCantBeSentInCleanup() throws Exception { + TezDAGID dagId = createTezDagId(); + TezVertexID vertexId = createTezVertexId(dagId); + Map serviceConsumerMetadata = new HashMap<>(); + Multimap startedInputsMap = HashMultimap.create(); + TezUmbilical umbilical = mock(TezUmbilical.class); + TezConfiguration tezConf = new TezConfiguration(); + tezConf.set(TezConfiguration.TEZ_TASK_SCALE_MEMORY_ALLOCATOR_CLASS, + ScalingAllocator.class.getName()); + + TezTaskAttemptID taId1 = createTaskAttemptID(vertexId, 1); + TaskSpec task1 = createTaskSpec(taId1, "dag1", "vertex1", 30, + RunExceptionProcessor.class.getName(), + TestOutputWithEvents.class.getName()); + + TezSharedExecutor sharedExecutor = new TezSharedExecutor(tezConf); + LogicalIOProcessorRuntimeTask lio = + new CleanupLogicalIOProcessorRuntimeTask(task1, 0, tezConf, null, + umbilical, serviceConsumerMetadata, new HashMap(), + startedInputsMap, null, "", new ExecutionContextImpl("localhost"), + Runtime.getRuntime().maxMemory(), true, new DefaultHadoopShim(), + sharedExecutor); + + TaskRunner2Callable runner = + new TaskRunner2Callable(lio, UserGroupInformation.getCurrentUser(), umbilical); + + runner.call(); + + // We verify that no events were sent + Mockito.verify(umbilical, Mockito.only()).addEvents(Collections. emptyList()); + } + /** * We should expect no events being sent to the AM if an * exception happens in the close method of the processor @@ -167,7 +211,7 @@ public void testExceptionHappensInClose() throws Exception { TezTaskAttemptID taId1 = createTaskAttemptID(vertexId, 1); TaskSpec task1 = createTaskSpec(taId1, "dag1", "vertex1", 30, - FaultyTestProcessor.class.getName(), + CloseExceptionProcessor.class.getName(), TestOutputWithEvents.class.getName()); TezSharedExecutor sharedExecutor = new TezSharedExecutor(tezConf); @@ -281,6 +325,31 @@ private TezDAGID createTezDagId() { return TezDAGID.getInstance("2000", 100, 1); } + private static class CleanupLogicalIOProcessorRuntimeTask + extends LogicalIOProcessorRuntimeTask { + CleanupLogicalIOProcessorRuntimeTask(TaskSpec taskSpec, + int appAttemptNumber, Configuration tezConf, String[] localDirs, + TezUmbilical tezUmbilical, + Map serviceConsumerMetadata, + Map envMap, Multimap startedInputsMap, + ObjectRegistry objectRegistry, String pid, + org.apache.tez.runtime.api.ExecutionContext ExecutionContext, + long memAvailable, boolean updateSysCounters, HadoopShim hadoopShim, + TezExecutors sharedExecutor) throws IOException { + super(taskSpec, appAttemptNumber, tezConf, localDirs, tezUmbilical, + serviceConsumerMetadata, envMap, startedInputsMap, objectRegistry, + pid, ExecutionContext, memAvailable, updateSysCounters, hadoopShim, + sharedExecutor); + } + + @Override public void cleanup() throws InterruptedException { + getOutputContexts().forEach(context + -> context.sendEvents(Arrays.asList( + CompositeDataMovementEvent.create(0, 0, null) + ))); + } + } + public static class TestProcessor extends AbstractLogicalIOProcessor { public static volatile int runCount = 0; @@ -310,8 +379,30 @@ public void close() throws Exception { } - public static class FaultyTestProcessor extends TestProcessor { - public FaultyTestProcessor(ProcessorContext context) { + public static class RunExceptionProcessor + extends TestProcessor { + + public RunExceptionProcessor(ProcessorContext context) { + super(context); + } + + public void run(Map inputs, + Map outputs) + throws Exception { + // This exception is thrown in purpose because we want to test this + throw new RuntimeException(); + } + + @Override + public void close() throws Exception { + // This exception is thrown because this method shouldn't be called + // if run has thrown an exception. + throw new RuntimeException(); + } + } + + public static class CloseExceptionProcessor extends TestProcessor { + public CloseExceptionProcessor(ProcessorContext context) { super(context); } From 033ba916099acd46e51a99bc839469864a950f45 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Thu, 28 Jan 2021 10:16:39 +0100 Subject: [PATCH 288/512] =?UTF-8?q?TEZ-4240:=20Remove=20SHA-256=20from=20T?= =?UTF-8?q?ez=20(L=C3=A1szl=C3=B3=20Bodor=20reviewed=20by=20Jonathan=20Tur?= =?UTF-8?q?ner=20Eagles)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../src/main/java/org/apache/tez/client/TezClientUtils.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index 1e09489564..8e7ccaf550 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -1124,7 +1124,7 @@ public static byte[] getLocalSha(Path path, Configuration conf) throws IOExcepti InputStream is = null; try { is = FileSystem.getLocal(conf).open(path); - return DigestUtils.sha256(is); + return DigestUtils.sha384(is); } finally { if (is != null) { is.close(); @@ -1136,7 +1136,7 @@ public static byte[] getResourceSha(URI uri, Configuration conf) throws IOExcept InputStream is = null; try { is = FileSystem.get(uri, conf).open(new Path(uri)); - return DigestUtils.sha256(is); + return DigestUtils.sha384(is); } finally { if (is != null) { is.close(); From 41ff94583e6433402e2a7cc183269d2582f0302f Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Thu, 28 Jan 2021 15:33:42 -0500 Subject: [PATCH 289/512] TEZ-4269: Re-Work Threadpool in DAGAppMaster (#92) --- .../java/org/apache/tez/dag/app/AppContext.java | 5 +---- .../org/apache/tez/dag/app/DAGAppMaster.java | 17 ++++------------- 2 files changed, 5 insertions(+), 17 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java index fc4ddcfcdc..c9a7083c1d 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java @@ -20,7 +20,7 @@ import java.util.Map; import java.util.Set; -import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.ExecutorService; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; @@ -74,9 +74,6 @@ public interface AppContext { DAG getCurrentDAG(); - // For testing only! - ThreadPoolExecutor getThreadPool(); - ListeningExecutorService getExecService(); void setDAG(DAG dag); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index ba072a9b5e..dbcefe97a5 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -48,10 +48,10 @@ import java.util.Set; import java.util.Timer; import java.util.TimerTask; +import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -303,7 +303,6 @@ public class DAGAppMaster extends AbstractService { private Path tezSystemStagingDir; private FileSystem recoveryFS; - private ThreadPoolExecutor rawExecutor; private ListeningExecutorService execService; // TODO May not need to be a bidi map @@ -621,9 +620,9 @@ public synchronized void serviceInit(final Configuration conf) throws Exception TezConfiguration.TEZ_AM_DAG_APPCONTEXT_THREAD_COUNT_LIMIT_DEFAULT); // NOTE: LinkedBlockingQueue does not have a capacity Limit and can thus // occupy large memory chunks when numerous Runables are pending for execution - rawExecutor = new ThreadPoolExecutor(threadCount, threadCount, - 60L, TimeUnit.SECONDS, new LinkedBlockingQueue(), - new ThreadFactoryBuilder().setDaemon(true).setNameFormat("App Shared Pool - " + "#%d").build()); + ExecutorService rawExecutor = + Executors.newFixedThreadPool(threadCount, new ThreadFactoryBuilder() + .setDaemon(true).setNameFormat("App Shared Pool - #%d").build()); execService = MoreExecutors.listeningDecorator(rawExecutor); initServices(conf); @@ -1503,14 +1502,6 @@ public DAG getCurrentDAG() { return dag; } - @Override - // For Testing only! - public ThreadPoolExecutor getThreadPool() { - synchronized (DAGAppMaster.this) { - return rawExecutor; - } - } - @Override public ListeningExecutorService getExecService() { return execService; From 61a28c75c3885f0a043d304091c2a303dde5b617 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Thu, 28 Jan 2021 22:17:32 +0100 Subject: [PATCH 290/512] =?UTF-8?q?TEZ-4236:=20DAGClientServer=20is=20not?= =?UTF-8?q?=20really=20needed=20to=20be=20started/used=20in=20local=20mode?= =?UTF-8?q?=20(L=C3=A1szl=C3=B3=20Bodor=20reviewed=20by=20Ashutosh=20Chauh?= =?UTF-8?q?an)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../apache/tez/client/FrameworkClient.java | 109 +++++++++++++ .../java/org/apache/tez/client/TezClient.java | 143 +++++------------- .../org/apache/tez/client/TezClientUtils.java | 14 +- .../apache/tez/dag/api/TezConfiguration.java | 13 ++ .../tez/dag/api/client/DAGClientImpl.java | 2 +- .../dag/api/client/DAGClientImplLocal.java | 53 +++++++ .../org/apache/tez/client/TestTezClient.java | 55 ++++--- .../org/apache/tez/client/LocalClient.java | 79 +++++++++- .../org/apache/tez/dag/app/DAGAppMaster.java | 8 +- .../apache/tez/dag/app/LocalDAGAppMaster.java | 51 +++++++ .../tez/dag/app/rm/TaskSchedulerManager.java | 4 +- .../apache/tez/dag/app/MockLocalClient.java | 4 +- .../org/apache/tez/test/TestLocalMode.java | 10 +- 13 files changed, 394 insertions(+), 151 deletions(-) create mode 100644 tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImplLocal.java create mode 100644 tez-dag/src/main/java/org/apache/tez/dag/app/LocalDAGAppMaster.java diff --git a/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java b/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java index 7c60ec100d..2ec6d2864d 100644 --- a/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java @@ -21,20 +21,39 @@ import java.io.IOException; import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.client.api.YarnClientApplication; import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.tez.common.RPCUtil; import org.apache.tez.common.ReflectionUtils; +import org.apache.tez.dag.api.DAG; +import org.apache.tez.dag.api.DAGSubmissionTimedOut; +import org.apache.tez.dag.api.DagTypeConverters; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.api.TezReflectionException; import org.apache.tez.dag.api.TezUncheckedException; +import org.apache.tez.dag.api.client.DAGClient; +import org.apache.tez.dag.api.client.DAGClientImpl; +import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolBlockingPB; +import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetAMStatusRequestProto; +import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetAMStatusResponseProto; +import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.ShutdownSessionRequestProto; +import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto; +import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGResponseProto; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.protobuf.ServiceException; @Private public abstract class FrameworkClient { + protected static final Logger LOG = LoggerFactory.getLogger(FrameworkClient.class); public static FrameworkClient createFrameworkClient(TezConfiguration tezConf) { @@ -77,4 +96,94 @@ public abstract ApplicationId submitApplication(ApplicationSubmissionContext app public abstract boolean isRunning() throws IOException; + public TezAppMasterStatus getAMStatus(Configuration conf, ApplicationId appId, + UserGroupInformation ugi) throws TezException, ServiceException, IOException { + DAGClientAMProtocolBlockingPB proxy = getProxy(conf, appId, ugi); + + if (proxy == null) { + return TezAppMasterStatus.INITIALIZING; + } + GetAMStatusResponseProto response = + proxy.getAMStatus(null, GetAMStatusRequestProto.newBuilder().build()); + return DagTypeConverters.convertTezAppMasterStatusFromProto(response.getStatus()); + } + + public DAGClient submitDag(DAG dag, SubmitDAGRequestProto request, String clientName, + ApplicationId sessionAppId, long clientTimeout, UserGroupInformation ugi, TezConfiguration tezConf) + throws IOException, TezException, DAGSubmissionTimedOut { + DAGClientAMProtocolBlockingPB proxy = null; + try { + proxy = waitForProxy(clientTimeout, tezConf, sessionAppId, ugi); + } catch (InterruptedException e) { + throw new IOException("Interrupted while trying to create a connection to the AM", e); + } + if (proxy == null) { + try { + LOG.warn("DAG submission to session timed out, stopping session"); + stop(); + } catch (Throwable t) { + LOG.info("Got an exception when trying to stop session", t); + } + throw new DAGSubmissionTimedOut("Could not submit DAG to Tez Session" + + ", timed out after " + clientTimeout + " seconds"); + } + + String dagId = null; + try { + SubmitDAGResponseProto response = proxy.submitDAG(null, request); + // the following check is only for testing since the final class + // SubmitDAGResponseProto cannot be mocked + if (response != null) { + dagId = response.getDagId(); + } + } catch (ServiceException e) { + RPCUtil.unwrapAndThrowException(e); + } + + LOG.info("Submitted dag to TezSession" + + ", sessionName=" + clientName + + ", applicationId=" + sessionAppId + + ", dagId=" + dagId + + ", dagName=" + dag.getName()); + return new DAGClientImpl(sessionAppId, dagId, tezConf, this, ugi); + } + + protected DAGClientAMProtocolBlockingPB waitForProxy(long clientTimeout, Configuration conf, + ApplicationId sessionAppId, UserGroupInformation ugi) + throws IOException, TezException, InterruptedException { + long startTime = System.currentTimeMillis(); + long endTime = startTime + (clientTimeout * 1000); + DAGClientAMProtocolBlockingPB proxy = null; + while (true) { + proxy = TezClientUtils.getAMProxy(this, conf, sessionAppId, ugi); + if (proxy != null) { + break; + } + Thread.sleep(100L); + if (clientTimeout != -1 && System.currentTimeMillis() > endTime) { + break; + } + } + return proxy; + } + + /** + * Shuts down session and returns a boolean=true if a proxy was successfully created and through + * that proxy a shutdownSession was called. + */ + public boolean shutdownSession(Configuration conf, ApplicationId sessionAppId, + UserGroupInformation ugi) throws TezException, IOException, ServiceException { + DAGClientAMProtocolBlockingPB proxy = getProxy(conf, sessionAppId, ugi); + if (proxy != null) { + ShutdownSessionRequestProto request = ShutdownSessionRequestProto.newBuilder().build(); + proxy.shutdownSession(null, request); + return true; + } + return false; + } + + protected DAGClientAMProtocolBlockingPB getProxy(Configuration conf, ApplicationId sessionAppId, + UserGroupInformation ugi) throws TezException, IOException { + return TezClientUtils.getAMProxy(this, conf, sessionAppId, ugi); + } } diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java index fbe35094f0..da213b8564 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java @@ -39,7 +39,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.security.UserGroupInformation; import org.apache.tez.common.JavaOptsChecker; -import org.apache.tez.common.RPCUtil; import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.counters.Limits; import org.apache.tez.dag.api.TezConfigurationConstants; @@ -75,10 +74,7 @@ import org.apache.tez.dag.api.client.DAGClient; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolBlockingPB; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetAMStatusRequestProto; -import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetAMStatusResponseProto; -import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.ShutdownSessionRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto; -import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGResponseProto; import org.apache.tez.dag.api.client.DAGClientImpl; import org.apache.tez.dag.api.records.DAGProtos.DAGPlan; @@ -123,7 +119,8 @@ public class TezClient { private ApplicationId lastSubmittedAppId; @VisibleForTesting final AMConfiguration amConfig; - private FrameworkClient frameworkClient; + @VisibleForTesting + FrameworkClient frameworkClient; private String diagnostics; @VisibleForTesting final boolean isSession; @@ -158,7 +155,7 @@ public class TezClient { private TezClient(String name, TezConfiguration tezConf) { this(name, tezConf, tezConf.getBoolean( - TezConfiguration.TEZ_AM_SESSION_MODE, TezConfiguration.TEZ_AM_SESSION_MODE_DEFAULT)); + TezConfiguration.TEZ_AM_SESSION_MODE, TezConfiguration.TEZ_AM_SESSION_MODE_DEFAULT)); } @Private @@ -549,7 +546,8 @@ public DAGClientAMProtocolBlockingPB sendAMHeartbeat(DAGClientAMProtocolBlocking try { if (proxy == null) { try { - proxy = waitForProxy(); + proxy = frameworkClient.waitForProxy(clientTimeout, amConfig.getTezConfiguration(), + sessionAppId, getUgi()); } catch (InterruptedException e) { LOG.debug("Interrupted while trying to create a connection to the AM", e); } catch (SessionNotRunning e) { @@ -629,7 +627,6 @@ private DAGClient submitDAGSession(DAG dag) throws TezException, IOException { verifySessionStateForSubmission(); - String dagId = null; String callerContextStr = ""; if (dag.getCallerContext() != null) { callerContextStr = ", callerContext=" + dag.getCallerContext().contextAsSimpleString(); @@ -678,42 +675,8 @@ private DAGClient submitDAGSession(DAG dag) throws TezException, IOException { } } - DAGClientAMProtocolBlockingPB proxy = null; - try { - proxy = waitForProxy(); - } catch (InterruptedException e) { - throw new IOException("Interrupted while trying to create a connection to the AM", e); - } - if (proxy == null) { - try { - LOG.warn("DAG submission to session timed out, stopping session"); - stop(); - } catch (Throwable t) { - LOG.info("Got an exception when trying to stop session", t); - } - throw new DAGSubmissionTimedOut("Could not submit DAG to Tez Session" - + ", timed out after " + clientTimeout + " seconds"); - } - - try { - SubmitDAGResponseProto response = proxy.submitDAG(null, request); - // the following check is only for testing since the final class - // SubmitDAGResponseProto cannot be mocked - if (response != null) { - dagId = response.getDagId(); - } - } catch (ServiceException e) { - RPCUtil.unwrapAndThrowException(e); - } - - LOG.info("Submitted dag to TezSession" - + ", sessionName=" + clientName - + ", applicationId=" + sessionAppId - + ", dagId=" + dagId - + ", dagName=" + dag.getName()); - return new DAGClientImpl(sessionAppId, dagId, - amConfig.getTezConfiguration(), - frameworkClient, getUgi()); + return frameworkClient.submitDag(dag, request, clientName, sessionAppId, clientTimeout, + getUgi(), amConfig.getTezConfiguration()); } private UserGroupInformation getUgi() throws IOException { @@ -746,39 +709,34 @@ public synchronized void stop() throws TezException, IOException { sessionStopped.set(true); boolean sessionShutdownSuccessful = false; try { - DAGClientAMProtocolBlockingPB proxy = getAMProxy(sessionAppId); - if (proxy != null) { - ShutdownSessionRequestProto request = - ShutdownSessionRequestProto.newBuilder().build(); - proxy.shutdownSession(null, request); - sessionShutdownSuccessful = true; - boolean asynchronousStop = amConfig.getTezConfiguration().getBoolean( - TezConfiguration.TEZ_CLIENT_ASYNCHRONOUS_STOP, - TezConfiguration.TEZ_CLIENT_ASYNCHRONOUS_STOP_DEFAULT); - if (!asynchronousStop) { - LOG.info("Waiting until application is in a final state"); - long currentTimeMillis = System.currentTimeMillis(); - long timeKillIssued = currentTimeMillis; - long killTimeOut = amConfig.getTezConfiguration().getLong( - TezConfiguration.TEZ_CLIENT_HARD_KILL_TIMEOUT_MS, - TezConfiguration.TEZ_CLIENT_HARD_KILL_TIMEOUT_MS_DEFAULT); - ApplicationReport appReport = frameworkClient - .getApplicationReport(sessionAppId); - while ((currentTimeMillis < timeKillIssued + killTimeOut) - && !isJobInTerminalState(appReport.getYarnApplicationState())) { - try { - Thread.sleep(1000L); - } catch (InterruptedException ie) { - /** interrupted, just break */ - break; - } - currentTimeMillis = System.currentTimeMillis(); - appReport = frameworkClient.getApplicationReport(sessionAppId); + sessionShutdownSuccessful = frameworkClient + .shutdownSession(amConfig.getTezConfiguration(), sessionAppId, getUgi()); + boolean asynchronousStop = amConfig.getTezConfiguration().getBoolean( + TezConfiguration.TEZ_CLIENT_ASYNCHRONOUS_STOP, + TezConfiguration.TEZ_CLIENT_ASYNCHRONOUS_STOP_DEFAULT); + if (!asynchronousStop && sessionShutdownSuccessful) { + LOG.info("Waiting until application is in a final state"); + long currentTimeMillis = System.currentTimeMillis(); + long timeKillIssued = currentTimeMillis; + long killTimeOut = amConfig.getTezConfiguration().getLong( + TezConfiguration.TEZ_CLIENT_HARD_KILL_TIMEOUT_MS, + TezConfiguration.TEZ_CLIENT_HARD_KILL_TIMEOUT_MS_DEFAULT); + ApplicationReport appReport = frameworkClient + .getApplicationReport(sessionAppId); + while ((currentTimeMillis < timeKillIssued + killTimeOut) + && !isJobInTerminalState(appReport.getYarnApplicationState())) { + try { + Thread.sleep(1000L); + } catch (InterruptedException ie) { + /** interrupted, just break */ + break; } + currentTimeMillis = System.currentTimeMillis(); + appReport = frameworkClient.getApplicationReport(sessionAppId); + } - if (!isJobInTerminalState(appReport.getYarnApplicationState())) { - frameworkClient.killApplication(sessionAppId); - } + if (!isJobInTerminalState(appReport.getYarnApplicationState())) { + frameworkClient.killApplication(sessionAppId); } } } catch (TezException e) { @@ -873,14 +831,7 @@ public synchronized TezAppMasterStatus getAppMasterStatus() throws TezException, return TezAppMasterStatus.SHUTDOWN; case RUNNING: try { - DAGClientAMProtocolBlockingPB proxy = getAMProxy(appId); - if (proxy == null) { - return TezAppMasterStatus.INITIALIZING; - } - GetAMStatusResponseProto response = proxy.getAMStatus(null, - GetAMStatusRequestProto.newBuilder().build()); - return DagTypeConverters.convertTezAppMasterStatusFromProto( - response.getStatus()); + return frameworkClient.getAMStatus(amConfig.getTezConfiguration(), appId, getUgi()); } catch (TezException e) { LOG.info("Failed to retrieve AM Status via proxy", e); } catch (ServiceException e) { @@ -1059,32 +1010,6 @@ protected FrameworkClient createFrameworkClient() { return FrameworkClient.createFrameworkClient(amConfig.getTezConfiguration()); } - @VisibleForTesting - // for testing - protected DAGClientAMProtocolBlockingPB getAMProxy(ApplicationId appId) - throws TezException, IOException { - return TezClientUtils.getAMProxy( - frameworkClient, amConfig.getTezConfiguration(), appId, getUgi()); - } - - private DAGClientAMProtocolBlockingPB waitForProxy() - throws IOException, TezException, InterruptedException { - long startTime = System.currentTimeMillis(); - long endTime = startTime + (clientTimeout * 1000); - DAGClientAMProtocolBlockingPB proxy = null; - while (true) { - proxy = getAMProxy(sessionAppId); - if (proxy != null) { - break; - } - Thread.sleep(100l); - if (clientTimeout != -1 && System.currentTimeMillis() > endTime) { - break; - } - } - return proxy; - } - private void verifySessionStateForSubmission() throws SessionNotRunning { Preconditions.checkState(isSession, "Invalid without session mode"); if (!sessionStarted.get()) { diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index 8e7ccaf550..79069ede7d 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -912,13 +912,12 @@ private static Path localizeDagPlanAsText(DAGPlan dagPB, FileSystem fs, AMConfig return textPath; } - static DAGClientAMProtocolBlockingPB getAMProxy(FrameworkClient yarnClient, - Configuration conf, - ApplicationId applicationId, UserGroupInformation ugi) throws TezException, IOException { + static DAGClientAMProtocolBlockingPB getAMProxy(FrameworkClient frameworkClient, + Configuration conf, ApplicationId applicationId, UserGroupInformation ugi) + throws TezException, IOException { ApplicationReport appReport; try { - appReport = yarnClient.getApplicationReport( - applicationId); + appReport = frameworkClient.getApplicationReport(applicationId); if(appReport == null) { throw new TezUncheckedException("Could not retrieve application report" @@ -948,8 +947,9 @@ static DAGClientAMProtocolBlockingPB getAMProxy(FrameworkClient yarnClient, } catch (YarnException e) { throw new TezException(e); } - return getAMProxy(conf, appReport.getHost(), - appReport.getRpcPort(), appReport.getClientToAMToken(), ugi); + + return getAMProxy(conf, appReport.getHost(), appReport.getRpcPort(), + appReport.getClientToAMToken(), ugi); } @Private diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 179b1957e8..05eb4b286c 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -1748,6 +1748,19 @@ public TezConfiguration(boolean loadDefaults) { public static final boolean TEZ_LOCAL_MODE_DEFAULT = false; + /** + * Boolean value. Enable local mode execution in Tez without using network for communicating with + * DAGAppMaster. This option only makes sense when {@link #TEZ_LOCAL_MODE} is true. When + * TEZ_LOCAL_MODE_WITHOUT_NETWORK is turned on, LocalClient will call DAGAppMaster's methods + * directly. + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty(type = "boolean") + public static final String TEZ_LOCAL_MODE_WITHOUT_NETWORK = + TEZ_PREFIX + "local.mode.without.network"; + + public static final boolean TEZ_LOCAL_MODE_WITHOUT_NETWORK_DEFAULT = false; + /** * String value. TezLocalCacheManager uses this folder as a root for temp and localized files. */ diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java index b54db324a9..e58863f614 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java @@ -209,7 +209,7 @@ public DAGStatus getDAGStatus(@Nullable Set statusOptions, } } - private DAGStatus getDAGStatusInternal(@Nullable Set statusOptions, + protected DAGStatus getDAGStatusInternal(@Nullable Set statusOptions, long timeout) throws TezException, IOException { if (!dagCompleted) { diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImplLocal.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImplLocal.java new file mode 100644 index 0000000000..a0509cdc5f --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImplLocal.java @@ -0,0 +1,53 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.dag.api.client; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Set; +import java.util.function.BiFunction; + +import javax.annotation.Nullable; + +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.tez.client.FrameworkClient; +import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.api.TezException; + +/** + * A DAGClientImpl which is typically used for tez.local.mode.without.network=true. + */ +public class DAGClientImplLocal extends DAGClientImpl { + + private BiFunction, Long, DAGStatus> dagStatusFunction; + + public DAGClientImplLocal(ApplicationId appId, String dagId, TezConfiguration conf, + FrameworkClient frameworkClient, UserGroupInformation ugi, + BiFunction, Long, DAGStatus> dagStatusFunction) { + super(appId, dagId, conf, frameworkClient, ugi); + this.dagStatusFunction = dagStatusFunction; + } + + @Override + protected DAGStatus getDAGStatusInternal(@Nullable Set statusOptions, long timeout) + throws TezException, IOException { + return dagStatusFunction.apply(statusOptions == null ? new HashSet<>() : statusOptions, + timeout); + } +} diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java index 7316452df3..67c4a604b6 100644 --- a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java +++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java @@ -56,6 +56,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Time; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; @@ -93,7 +94,6 @@ import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.ShutdownSessionRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.TezAppMasterStatusProto; -import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto; import org.apache.tez.dag.api.records.DAGProtos.DAGStatusProto; import org.apache.tez.dag.api.records.DAGProtos.DAGStatusStateProto; import org.apache.tez.dag.api.records.DAGProtos.ProgressProto; @@ -112,7 +112,6 @@ public class TestTezClient { TestTezClient.class.getName()).getAbsoluteFile(); class TezClientForTest extends TezClient { - TezYarnClient mockTezYarnClient; DAGClientAMProtocolBlockingPB sessionAmProxy; YarnClient mockYarnClient; ApplicationId mockAppId; @@ -120,23 +119,13 @@ class TezClientForTest extends TezClient { Long prewarmTimeoutMs; public TezClientForTest(String name, TezConfiguration tezConf, - @Nullable Map localResources, - @Nullable Credentials credentials) { + @Nullable Map localResources, @Nullable Credentials credentials) { super(name, tezConf, localResources, credentials); } - + @Override protected FrameworkClient createFrameworkClient() { - return mockTezYarnClient; - } - - @Override - protected DAGClientAMProtocolBlockingPB getAMProxy(ApplicationId appId) - throws TezException, IOException { - if (!callRealGetSessionAMProxy) { - return sessionAmProxy; - } - return super.getAMProxy(appId); + return frameworkClient; // already initialized } public void setPrewarmTimeoutMs(Long prewarmTimeoutMs) { @@ -148,7 +137,34 @@ protected long getPrewarmWaitTimeMs() { return prewarmTimeoutMs == null ? super.getPrewarmWaitTimeMs() : prewarmTimeoutMs; } } - + + class TezYarnClientForTest extends TezYarnClient { + private TezClientForTest client; + + protected TezYarnClientForTest(YarnClient yarnClient, TezClientForTest client) { + super(yarnClient); + this.client = client; + } + + @Override + protected DAGClientAMProtocolBlockingPB waitForProxy(long clientTimeout, Configuration conf, + ApplicationId sessionAppId, UserGroupInformation ugi) throws TezException, IOException { + if (!client.callRealGetSessionAMProxy) { + return client.sessionAmProxy; + } + return super.getProxy(conf, sessionAppId, ugi); + } + + @Override + protected DAGClientAMProtocolBlockingPB getProxy(Configuration conf, ApplicationId sessionAppId, + UserGroupInformation ugi) throws TezException, IOException { + if (!client.callRealGetSessionAMProxy) { + return client.sessionAmProxy; + } + return super.getProxy(conf, sessionAppId, ugi); + } + } + TezClientForTest configureAndCreateTezClient() throws YarnException, IOException, ServiceException { return configureAndCreateTezClient(null); } @@ -179,11 +195,11 @@ TezClientForTest configureAndCreateTezClient(Map lrs, boo .thenReturn(GetAMStatusResponseProto.newBuilder().setStatus(TezAppMasterStatusProto.RUNNING).build()); client.sessionAmProxy = sessionAmProxy; - client.mockTezYarnClient = new TezYarnClient(yarnClient); + client.frameworkClient = new TezYarnClientForTest(yarnClient, client); client.mockYarnClient = yarnClient; client.mockAppId = appId1; - - return client; + + return client; } @Test (timeout = 5000) @@ -987,7 +1003,6 @@ public void testYarnZkDeprecatedConf() { String val = "hostname:2181"; conf.set("yarn.resourcemanager.zk-address", val); - ConfigurationProto confProto = null; //Test that Exception is not thrown by createFinalConfProtoForApp TezClientUtils.createFinalConfProtoForApp(conf, null); } diff --git a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java index 5a6bb9a22c..c76bd6bace 100644 --- a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java +++ b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java @@ -22,11 +22,14 @@ import java.net.InetAddress; import java.nio.ByteBuffer; import java.util.List; - +import java.util.Map; +import java.util.Set; +import java.util.function.BiFunction; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.client.api.YarnClientApplication; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; @@ -46,18 +49,28 @@ import org.apache.hadoop.yarn.util.SystemClock; import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.TezUtilsInternal; +import org.apache.tez.dag.api.DAGSubmissionTimedOut; +import org.apache.tez.dag.api.DagTypeConverters; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.TezException; +import org.apache.tez.dag.api.client.DAGClient; import org.apache.tez.dag.api.client.DAGClientHandler; +import org.apache.tez.dag.api.client.DAGClientImpl; +import org.apache.tez.dag.api.client.DAGClientImplLocal; +import org.apache.tez.dag.api.client.DAGStatus; +import org.apache.tez.dag.api.client.StatusGetOpts; +import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto; import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto; import org.apache.tez.dag.app.AppContext; import org.apache.tez.dag.app.DAGAppMaster; import org.apache.tez.dag.app.DAGAppMasterState; +import org.apache.tez.dag.app.LocalDAGAppMaster; import org.apache.tez.dag.app.dag.DAG; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import com.google.common.annotations.VisibleForTesting; +import com.google.protobuf.ServiceException; public class LocalClient extends FrameworkClient { public static final Logger LOG = LoggerFactory.getLogger(LocalClient.class); @@ -72,6 +85,8 @@ public class LocalClient extends FrameworkClient { private boolean isSession; private TezApiVersionInfo versionInfo = new TezApiVersionInfo(); private volatile Throwable amFailException = null; + private boolean isLocalWithoutNetwork; + private static final String localModeDAGSchedulerClassName = "org.apache.tez.dag.app.dag.impl.DAGSchedulerNaturalOrderControlled"; @@ -89,6 +104,10 @@ public void init(TezConfiguration tezConf) { // disable web service for local mode. this.conf.setBoolean(TezConfiguration.TEZ_AM_WEBSERVICE_ENABLE, false); + + this.isLocalWithoutNetwork = + tezConf.getBoolean(TezConfiguration.TEZ_LOCAL_MODE_WITHOUT_NETWORK, + TezConfiguration.TEZ_LOCAL_MODE_WITHOUT_NETWORK_DEFAULT); } @@ -170,7 +189,6 @@ public ApplicationReport getApplicationReport(ApplicationId appId) { report.setYarnApplicationState(convertDAGAppMasterState(dagAppMaster.getState())); report.setFinalApplicationStatus(convertDAGAppMasterStateToFinalYARNState(dagAppMaster.getState())); - List diagnostics = dagAppMaster.getDiagnostics(); if (diagnostics != null) { report.setDiagnostics(diagnostics.toString()); @@ -333,7 +351,7 @@ public void run() { dagAppMaster = createDAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort, - new SystemClock(), appSubmitTime, isSession, userDir.toUri().getPath(), + SystemClock.getInstance(), appSubmitTime, isSession, userDir.toUri().getPath(), new String[] {localDir.toUri().getPath()}, new String[] {logDir.toUri().getPath()}, amCredentials, UserGroupInformation.getCurrentUser().getShortUserName()); DAGAppMaster.initAndStartAppMaster(dagAppMaster, conf); @@ -372,8 +390,57 @@ protected DAGAppMaster createDAGAppMaster(ApplicationAttemptId applicationAttemp TezUtilsInternal.readUserSpecifiedTezConfiguration(userDir) .getAmPluginDescriptor(); - return new DAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort, - new SystemClock(), appSubmitTime, isSession, userDir, localDirs, logDirs, - versionInfo.getVersion(), credentials, jobUserName, amPluginDescriptorProto); + return isLocalWithoutNetwork + ? new LocalDAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort, + SystemClock.getInstance(), appSubmitTime, isSession, userDir, localDirs, logDirs, + versionInfo.getVersion(), credentials, jobUserName, amPluginDescriptorProto) + : new DAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort, + SystemClock.getInstance(), appSubmitTime, isSession, userDir, localDirs, logDirs, + versionInfo.getVersion(), credentials, jobUserName, amPluginDescriptorProto); + } + + @Override + public TezAppMasterStatus getAMStatus(Configuration configuration, ApplicationId appId, + UserGroupInformation ugi) throws TezException, ServiceException, IOException { + return clientHandler.getTezAppMasterStatus(); + } + + @Override + public DAGClient submitDag(org.apache.tez.dag.api.DAG dag, SubmitDAGRequestProto request, + String clientName, ApplicationId sessionAppId, long clientTimeout, UserGroupInformation ugi, + TezConfiguration tezConf) throws IOException, TezException, DAGSubmissionTimedOut { + + Map additionalResources = null; + if (request.hasAdditionalAmResources()) { + additionalResources = + DagTypeConverters.convertFromPlanLocalResources(request.getAdditionalAmResources()); + } + + String dagId = dagAppMaster.submitDAGToAppMaster(request.getDAGPlan(), additionalResources); + + return isLocalWithoutNetwork + ? new DAGClientImplLocal(sessionAppId, dagId, tezConf, this, + ugi, new BiFunction, Long, DAGStatus>() { + @Override + public DAGStatus apply(Set statusOpts, Long timeout) { + try { + return clientHandler.getDAGStatus(dagId, statusOpts, timeout); + } catch (TezException e) { + throw new RuntimeException(e); + } + } + }) + : new DAGClientImpl(sessionAppId, dagId, tezConf, this, ugi); + } + + @Override + public boolean shutdownSession(Configuration configuration, ApplicationId sessionAppId, + UserGroupInformation ugi) throws TezException, IOException, ServiceException { + if (isLocalWithoutNetwork) { + clientHandler.shutdownAM(); + return true; + } else { + return super.shutdownSession(configuration, sessionAppId, ugi); + } } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index dbcefe97a5..395e84a105 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -508,8 +508,7 @@ public synchronized void serviceInit(final Configuration conf) throws Exception recoveryEnabled = conf.getBoolean(TezConfiguration.DAG_RECOVERY_ENABLED, TezConfiguration.DAG_RECOVERY_ENABLED_DEFAULT); - clientRpcServer = new DAGClientServer(clientHandler, appAttemptID, recoveryFS); - addIfService(clientRpcServer, true); + initClientRpcServer(); taskHeartbeatHandler = createTaskHeartbeatHandler(context, conf); addIfService(taskHeartbeatHandler, true); @@ -647,6 +646,11 @@ public synchronized void serviceInit(final Configuration conf) throws Exception } } + protected void initClientRpcServer() { + clientRpcServer = new DAGClientServer(clientHandler, appAttemptID, recoveryFS); + addIfService(clientRpcServer, true); + } + @VisibleForTesting protected DAGAppMasterShutdownHandler createShutdownHandler() { return new DAGAppMasterShutdownHandler(); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/LocalDAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/LocalDAGAppMaster.java new file mode 100644 index 0000000000..e0c8443577 --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/LocalDAGAppMaster.java @@ -0,0 +1,51 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.dag.app; + +import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.util.Clock; +import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto; + +/** + * A DAGAppMaster implementation which is really local in a sense that it doesn't start an RPC + * server for handling dag requests. It is typically used by LocalClient, which already has an + * embedded DAGAppMaster, but by default, it calls RPC methods. With + * tez.local.mode.without.network=true, LocalClient will call the DAGAppMaster's methods directly. + */ +public class LocalDAGAppMaster extends DAGAppMaster { + + public LocalDAGAppMaster(ApplicationAttemptId applicationAttemptId, ContainerId containerId, + String nmHost, int nmPort, int nmHttpPort, Clock clock, long appSubmitTime, boolean isSession, + String workingDirectory, String[] localDirs, String[] logDirs, String clientVersion, + Credentials credentials, String jobUserName, AMPluginDescriptorProto pluginDescriptorProto) { + super(applicationAttemptId, containerId, nmHost, nmPort, nmHttpPort, clock, appSubmitTime, + isSession, workingDirectory, localDirs, logDirs, clientVersion, credentials, jobUserName, + pluginDescriptorProto); + } + + @Override + protected void initClientRpcServer() { + // nothing to do, in case of LocalDAGAppMaster clientRpcServer is not supposed to be used by clients + } + + public int getRpcPort() { + return 0; + } +} diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java index cc2e163720..8e6bfe79a5 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java @@ -641,7 +641,9 @@ protected void instantiateSchedulers(String host, int port, String trackingUrl, @Override public synchronized void serviceStart() throws Exception { - InetSocketAddress serviceAddr = clientService.getBindAddress(); + // clientService is null in case of LocalDAGAppMaster + InetSocketAddress serviceAddr = clientService == null ? new InetSocketAddress("127.0.0.1", 0) + : clientService.getBindAddress(); dagAppMaster = appContext.getAppMaster(); // if web service is enabled then set tracking url. else disable it (value = ""). // the actual url set on the rm web ui will be the proxy url set by WebAppProxyServlet, which diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockLocalClient.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockLocalClient.java index 552651616e..c3355473d1 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockLocalClient.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockLocalClient.java @@ -38,7 +38,7 @@ public class MockLocalClient extends LocalClient { public MockLocalClient(AtomicBoolean mockAppLauncherGoFlag, Clock clock) { this(mockAppLauncherGoFlag, clock, false, false, 1, 1); } - + public MockLocalClient(AtomicBoolean mockAppLauncherGoFlag, Clock clock, boolean initFailFlag, boolean startFailFlag, int concurrency, int containers) { this.mockAppLauncherGoFlag = mockAppLauncherGoFlag; @@ -60,7 +60,7 @@ protected DAGAppMaster createDAGAppMaster(ApplicationAttemptId applicationAttemp concurrency, containers); return mockApp; } - + public MockDAGAppMaster getMockApp() { return mockApp; } diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java b/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java index e6ef8c9cb7..bdb71ad177 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java @@ -68,14 +68,16 @@ public class TestLocalMode { private static FileSystem remoteFs; private final boolean useDfs; + private final boolean useLocalModeWithoutNetwork; - @Parameterized.Parameters(name = "useDFS:{0}") + @Parameterized.Parameters(name = "useDFS:{0} useLocalModeWithoutNetwork:{1}") public static Collection params() { - return Arrays.asList(new Object[][]{{false}, {true}}); + return Arrays.asList(new Object[][]{{false, false}, {true, false}, {false, true}, {true, true}}); } - public TestLocalMode(boolean useDfs) { + public TestLocalMode(boolean useDfs, boolean useLocalModeWithoutNetwork) { this.useDfs = useDfs; + this.useLocalModeWithoutNetwork = useLocalModeWithoutNetwork; } @BeforeClass @@ -105,6 +107,8 @@ public static void afterClass() throws InterruptedException { private TezConfiguration createConf() { TezConfiguration conf = new TezConfiguration(); conf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); + conf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE_WITHOUT_NETWORK, useLocalModeWithoutNetwork); + if (useDfs) { conf.set("fs.defaultFS", remoteFs.getUri().toString()); } else { From 0d3b428ecbd5b3c240191a6d1907010aa549c189 Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Thu, 28 Jan 2021 16:53:24 -0500 Subject: [PATCH 291/512] TEZ-4266: Make Tez Inherit From Apache POM (#89) --- pom.xml | 121 ++++-------------- .../src/main/assembly/tez-dist-minimal.xml | 1 + 2 files changed, 25 insertions(+), 97 deletions(-) diff --git a/pom.xml b/pom.xml index 59ec193dfb..892a8a2d66 100644 --- a/pom.xml +++ b/pom.xml @@ -16,6 +16,11 @@ 4.0.0 + + org.apache + apache + 23 + org.apache.tez tez pom @@ -35,33 +40,35 @@ + + 1.8 + 1.8 true + 3.0.0-M4 + false + UTF-8 + ${maven.build.timestamp} + + apache.snapshots.https + Apache Development Snapshot Repository + https://repository.apache.org/content/repositories/snapshots + apache.staging.https + Apache Release Distribution Repository + https://repository.apache.org/service/local/staging/deploy/maven2 + ${user.home}/clover.license 27.0-jre 3.1.3 3.10.5.Final 0.13.0 1.19 - 1.8 1.7.10 - [${javac.version},) - apache.snapshots.https - Apache Development Snapshot Repository - https://repository.apache.org/content/repositories/snapshots - apache.staging.https - Apache Release Distribution Repository - https://repository.apache.org/service/local/staging/deploy/maven2 - false 2.5.0 ${env.PROTOC_PATH} - UTF-8 scm:git:https://gitbox.apache.org/repos/asf/tez.git - ${maven.build.timestamp} 1.4 3.0.5 - 2.10.4 - 2.4.3 - 3.1.1 + 3.1.1 8.35 1.3.6 ${project.build.directory}/tmp @@ -744,24 +751,9 @@ - - org.apache.maven.plugins - maven-compiler-plugin - 3.1 - - ${javac.version} - ${javac.version} - - - - org.apache.maven.plugins - maven-enforcer-plugin - 1.4 - org.apache.maven.plugins maven-jar-plugin - 2.4 @@ -789,17 +781,11 @@ - - org.apache.maven.plugins - maven-source-plugin - 2.2.1 - org.apache.maven.plugins maven-javadoc-plugin - ${javadoc-maven-plugin.version} - -Xdoclint:none + none @@ -807,11 +793,6 @@ hadoop-maven-plugins ${hadoop.version} - - org.apache.maven.plugins - maven-assembly-plugin - 2.4 - com.github.eirslett frontend-maven-plugin @@ -822,20 +803,9 @@ org.codehaus.mojo 1.3.2 - - org.apache.maven.plugins - maven-war-plugin - 3.2.3 - - - org.apache.maven.plugins - maven-shade-plugin - ${shade-maven-plugin.version} - org.apache.rat apache-rat-plugin - 0.10 CHANGES.txt @@ -865,7 +835,6 @@ org.apache.maven.plugins maven-surefire-plugin - 3.0.0-M4 1 false @@ -918,29 +887,6 @@ findbugs-maven-plugin ${findbugs-maven-plugin.version} - - org.apache.maven.plugins - maven-resources-plugin - 3.1.0 - - UTF-8 - - - - org.apache.maven.plugins - maven-site-plugin - 3.4 - - - org.apache.maven.plugins - maven-deploy-plugin - 2.8.1 - - - org.apache.maven.plugins - maven-gpg-plugin - 1.4 - org.codehaus.mojo build-helper-maven-plugin @@ -996,20 +942,7 @@ - - org.apache.maven.plugins - maven-project-info-reports-plugin - 2.9 - - false - - - - org.apache.maven.plugins - maven-clean-plugin - 3.0.0 - - + ro.isdc.wro4j wro4j-maven-plugin 1.7.9 @@ -1029,7 +962,6 @@ org.apache.maven.plugins maven-enforcer-plugin - 1.4 false @@ -1043,7 +975,7 @@ [3.0.2,) - ${enforced.java.version} + [${maven.compiler.target},) @@ -1090,7 +1022,6 @@ --> org.owasp dependency-check-maven - ${dependency-check-maven.version} @@ -1242,7 +1173,6 @@ org.apache.maven.plugins maven-javadoc-plugin - ${javadoc-maven-plugin.version} true @@ -1270,8 +1200,6 @@ maven-compiler-plugin true - ${javac.version} - ${javac.version} 9999 @@ -1350,7 +1278,6 @@ org.apache.maven.plugins maven-javadoc-plugin - ${javadoc-maven-plugin.version} aggregate diff --git a/tez-dist/src/main/assembly/tez-dist-minimal.xml b/tez-dist/src/main/assembly/tez-dist-minimal.xml index 35f9be76e8..4c95d37efa 100644 --- a/tez-dist/src/main/assembly/tez-dist-minimal.xml +++ b/tez-dist/src/main/assembly/tez-dist-minimal.xml @@ -26,6 +26,7 @@ org.apache.tez:tez-aux-services org.apache.tez:tez-ext-service-tests org.apache.tez:tez-ui + org.apache.tez:tez-docs / From 73bcabd2bca2536bf4f3673443a8dcdaaf79a4eb Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Mon, 1 Feb 2021 11:33:19 +0100 Subject: [PATCH 292/512] TEZ-4277: AsyncDispatcher can hang on serviceStop if the eventhandling thread is in BLOCKED state (#97) (Laszlo Bodor reviewed by Rajesh Balamohan) TEZ-4277: AsyncDispatcher can hang on serviceStop if the eventhandling thread is in BLOCKED state --- .../apache/tez/dag/api/TezConfiguration.java | 11 ++++++++ .../apache/tez/common/AsyncDispatcher.java | 28 ++++++++++++++++--- .../org/apache/tez/client/LocalClient.java | 3 +- 3 files changed, 37 insertions(+), 5 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 05eb4b286c..eef0d65424 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -193,6 +193,17 @@ public TezConfiguration(boolean loadDefaults) { @Private public static final int TEZ_AM_CONCURRENT_DISPATCHER_CONCURRENCY_DEFAULT = 10; + /** + * Integer value. Milliseconds while AsyncDispatcher should wait for events to be processed on + * serviceStop. The idea is borrowed from YARN-3999. + */ + @Private + @ConfigurationScope(Scope.AM) + public static final String TEZ_AM_DISPATCHER_DRAIN_EVENTS_TIMEOUT = TEZ_AM_PREFIX + + "dispatcher.drain-events.timeout"; + @Private + public static final int TEZ_AM_DISPATCHER_DRAIN_EVENTS_TIMEOUT_DEFAULT = 10000; + /** * Boolean value. Execution mode for the Tez application. True implies session mode. If the client * code is written according to best practices then the same code can execute in either mode based diff --git a/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcher.java b/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcher.java index c197f1d32e..f9f21ca313 100644 --- a/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcher.java +++ b/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcher.java @@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.event.Event; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; +import org.apache.tez.dag.api.TezConfiguration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -142,19 +143,34 @@ protected void serviceStop() throws Exception { if (drainEventsOnStop) { blockNewEvents = true; LOG.info("AsyncDispatcher is draining to stop, ignoring any new events."); + long endTime = System.currentTimeMillis() + getConfig() + .getInt(TezConfiguration.TEZ_AM_DISPATCHER_DRAIN_EVENTS_TIMEOUT, + TezConfiguration.TEZ_AM_DISPATCHER_DRAIN_EVENTS_TIMEOUT_DEFAULT); + synchronized (waitForDrained) { - while (!drained && eventHandlingThread.isAlive()) { + while (!drained && eventHandlingThread.isAlive() && System.currentTimeMillis() < endTime) { waitForDrained.wait(1000); - LOG.info("Waiting for AsyncDispatcher to drain."); + LOG.info( + "Waiting for AsyncDispatcher to drain. Current queue size: {}, handler thread state: {}", + eventQueue.size(), eventHandlingThread.getState()); } } - } stopped = true; if (eventHandlingThread != null) { eventHandlingThread.interrupt(); try { - eventHandlingThread.join(); + /* + * The event handling thread can be alive at this point, but in BLOCKED state, which leads + * to app hang, as a BLOCKED thread might never finish under some circumstances + */ + if (eventHandlingThread.getState() == Thread.State.BLOCKED) { + LOG.warn( + "eventHandlingThread is in BLOCKED state, let's not wait for it in order to prevent app hang"); + } else { + eventHandlingThread.join(); + LOG.info("joined event handling thread, state: {}", eventHandlingThread.getState()); + } } catch (InterruptedException ie) { LOG.warn("Interrupted Exception while stopping", ie); } @@ -181,6 +197,10 @@ protected void dispatch(Event event) { throw new Exception("No handler for registered for " + type); } } catch (Throwable t) { + if (t instanceof InterruptedException) { + LOG.warn("Interrupted Exception while handling event: " + event.getType(), t); + Thread.currentThread().interrupt(); + } LOG.error("Error in dispatcher thread", t); // If serviceStop is called, we should exit this thread gracefully. if (exitOnDispatchException diff --git a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java index c76bd6bace..f6d9587734 100644 --- a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java +++ b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java @@ -47,6 +47,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.util.SystemClock; +import org.apache.tez.common.AsyncDispatcher; import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.dag.api.DAGSubmissionTimedOut; @@ -356,7 +357,7 @@ public void run() { amCredentials, UserGroupInformation.getCurrentUser().getShortUserName()); DAGAppMaster.initAndStartAppMaster(dagAppMaster, conf); clientHandler = new DAGClientHandler(dagAppMaster); - + ((AsyncDispatcher)dagAppMaster.getDispatcher()).setDrainEventsOnStop(); } catch (Throwable t) { LOG.error("Error starting DAGAppMaster", t); if (dagAppMaster != null) { From 26759e06bd3a9317fe439d7ad81b60e1dabc932e Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Mon, 8 Feb 2021 09:44:22 +0100 Subject: [PATCH 293/512] =?UTF-8?q?TEZ-4273:=20Clear=20off=20staging=20fil?= =?UTF-8?q?es=20when=20TezYarnClient=20is=20unable=20to=20submit=20applica?= =?UTF-8?q?tions=20(Rajesh=20Balamohan=20via=20L=C3=A1szl=C3=B3=20Bodor)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../java/org/apache/tez/client/TezClient.java | 24 +++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java index da213b8564..378017b364 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java @@ -33,6 +33,7 @@ import javax.annotation.Nullable; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -403,6 +404,10 @@ public synchronized void start() throws TezException, IOException { LOG.info("The url to track the Tez Session: " + appReport.getTrackingUrl()); sessionStarted.set(true); } catch (YarnException e) { + cleanStagingDir(); + throw new TezException(e); + } catch (IOException e) { + cleanStagingDir(); throw new TezException(e); } @@ -411,6 +416,21 @@ public synchronized void start() throws TezException, IOException { } } + private void cleanStagingDir() { + Configuration conf = amConfig.getTezConfiguration(); + String appId = sessionAppId.toString(); + Path stagingDir = TezCommonUtils.getTezSystemStagingPath(conf, appId); + boolean isStgDeleted = false; + try { + FileSystem fs = stagingDir.getFileSystem(conf); + isStgDeleted = fs.delete(stagingDir, true); + } catch (IOException ioe) { + LOG.error("Error deleting staging dir " + stagingDir, ioe); + } finally { + LOG.info("Staging dir {}, deleted:{} ", stagingDir, isStgDeleted); + } + } + public synchronized TezClient getClient(String appIdStr) throws IOException, TezException { return getClient(appIdfromString(appIdStr)); } @@ -450,6 +470,10 @@ public synchronized TezClient getClient(ApplicationId appId) throws TezException LOG.info("The url to track the Tez Session: " + appReport.getTrackingUrl()); sessionStarted.set(true); } catch (YarnException e) { + cleanStagingDir(); + throw new TezException(e); + } catch (IOException e) { + cleanStagingDir(); throw new TezException(e); } From 601d1f3578e50d7212477cd3919e9ccc7cf117cf Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Mon, 8 Feb 2021 19:34:34 +0100 Subject: [PATCH 294/512] TEZ-4283: Docker fails to build on master - upgrade to nodejs10 (#104) --- build-tools/docker/Dockerfile | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/build-tools/docker/Dockerfile b/build-tools/docker/Dockerfile index 143861d25a..ae26e41934 100644 --- a/build-tools/docker/Dockerfile +++ b/build-tools/docker/Dockerfile @@ -171,12 +171,13 @@ RUN pip2 install pylint==1.9.2 RUN pip2 install python-dateutil==2.7.3 ### -# Install node.js for web UI framework (4.2.6 ships with Xenial) +# Install node.js for web UI framework (8.10.0 ships with Bionic, let's override with 10.x) ### -# hadolint ignore=DL3008, DL3016 -RUN apt-get -q update \ - && apt-get install -y --no-install-recommends nodejs npm \ - && apt-get clean \ +RUN curl -sL https://deb.nodesource.com/setup_10.x | bash - +# hadolint ignore=DL3008, DL3015 +RUN apt-get install -y nodejs +# hadolint ignore=DL3016 +RUN apt-get clean \ && rm -rf /var/lib/apt/lists/* \ && npm install npm@latest -g \ && npm install -g jshint From cd0548631dd10be5f0d6c13d295f2726d180d239 Mon Sep 17 00:00:00 2001 From: Laszlo Bodor Date: Mon, 8 Feb 2021 20:21:39 +0100 Subject: [PATCH 295/512] =?UTF-8?q?TEZ-4273:=20Clear=20off=20staging=20fil?= =?UTF-8?q?es=20when=20TezYarnClient=20is=20unable=20to=20submit=20applica?= =?UTF-8?q?tions=20(Rajesh=20Balamohan=20via=20L=C3=A1szl=C3=B3=20Bodor)?= =?UTF-8?q?=20-=20addendum=20test=20fix?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/org/apache/tez/dag/app/TestMockDAGAppMaster.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java index 1a4659593f..5dca6940ea 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java @@ -958,7 +958,7 @@ public void testInitFailed() throws Exception { tezClient.start(); } catch (Exception e) { e.printStackTrace(); - Assert.assertEquals("FailInit", e.getCause().getCause().getMessage()); + Assert.assertEquals("FailInit", e.getCause().getCause().getCause().getMessage()); MockDAGAppMaster mockApp = tezClient.getLocalClient().getMockApp(); // will timeout if DAGAppMasterShutdownHook is not invoked mockApp.waitForServiceToStop(Integer.MAX_VALUE); @@ -974,7 +974,7 @@ public void testStartFailed() { tezClient.start(); } catch (Exception e) { e.printStackTrace(); - Assert.assertEquals("FailStart", e.getCause().getCause().getMessage()); + Assert.assertEquals("FailStart", e.getCause().getCause().getCause().getMessage()); MockDAGAppMaster mockApp = tezClient.getLocalClient().getMockApp(); // will timeout if DAGAppMasterShutdownHook is not invoked mockApp.waitForServiceToStop(Integer.MAX_VALUE); From 1847817c2a92f26b5e9aa71c250f4559a564bf0a Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Mon, 8 Feb 2021 21:29:49 +0100 Subject: [PATCH 296/512] TEZ-4281: dag_*_priority.dot files should go into a valid log directory (#102) (Laszlo Bodor reviewed by Rajesh Balamohan) --- tez-dag/findbugs-exclude.xml | 12 ++++++++++++ .../src/main/java/org/apache/tez/Utils.java | 18 +++--------------- .../org/apache/tez/dag/app/DAGAppMaster.java | 2 +- .../apache/tez/dag/app/dag/impl/DAGImpl.java | 18 +++++++++++++++++- 4 files changed, 33 insertions(+), 17 deletions(-) diff --git a/tez-dag/findbugs-exclude.xml b/tez-dag/findbugs-exclude.xml index a6ce38053b..50422ff0e0 100644 --- a/tez-dag/findbugs-exclude.xml +++ b/tez-dag/findbugs-exclude.xml @@ -252,4 +252,16 @@ + + + + + + + + + + + + diff --git a/tez-dag/src/main/java/org/apache/tez/Utils.java b/tez-dag/src/main/java/org/apache/tez/Utils.java index dbde327107..b352334f44 100644 --- a/tez-dag/src/main/java/org/apache/tez/Utils.java +++ b/tez-dag/src/main/java/org/apache/tez/Utils.java @@ -16,9 +16,7 @@ import javax.annotation.Nullable; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.event.Event; -import org.apache.tez.common.TezCommonUtils; import org.apache.tez.dag.api.records.DAGProtos; import org.apache.tez.dag.app.AppContext; import org.apache.tez.dag.app.dag.DAG; @@ -109,19 +107,6 @@ public static void processNonFatalServiceErrorReport(String entityString, } } - /** - * Generate a visualization file. - * @param dag DAG. - * @param dagPB DAG plan. - * @param scheduler scheduler that provide the priorities of the vertexes. - */ - public static void generateDAGVizFile(final DAG dag, - final DAGProtos.DAGPlan dagPB, @Nullable final DAGScheduler scheduler) { - generateDAGVizFile(dag, dagPB, TezCommonUtils.getTrimmedStrings( - System.getenv(ApplicationConstants.Environment.LOG_DIRS.name())), - scheduler); - } - /** * Generate a visualization file. * @param dag DAG. @@ -217,6 +202,9 @@ public static void generateDAGVizFile(final DAG dag, if (logDirs != null && logDirs.length != 0) { outputFile += logDirs[0]; outputFile += File.separator; + } else { + LOG.warn("DAGVizFile will be created under current (.) directory: {}," + + " which is not expected and recommended", new File(".").getAbsolutePath()); } outputFile += dagId.toString(); // Means we have set the priorities diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index 395e84a105..1352b68f26 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -1030,7 +1030,7 @@ DAGImpl createDAG(DAGPlan dagPB, TezDAGID dagId) { new DAGImpl(dagId, amConf, dagPB, dispatcher.getEventHandler(), taskCommunicatorManager, dagCredentials, clock, appMasterUgi.getShortUserName(), - taskHeartbeatHandler, context); + taskHeartbeatHandler, context).setLogDirs(logDirs); try { if (LOG.isDebugEnabled()) { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java index 09e9e71b92..026ca29b7a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java @@ -57,6 +57,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.state.InvalidStateTransitonException; @@ -67,6 +68,7 @@ import org.apache.hadoop.yarn.util.Clock; import org.apache.tez.common.ATSConstants; import org.apache.tez.common.ReflectionUtils; +import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.counters.AggregateTezCounters; import org.apache.tez.common.counters.DAGCounter; import org.apache.tez.common.counters.TezCounters; @@ -225,6 +227,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG, private TaskSpecificLaunchCmdOption taskSpecificLaunchCmdOption; private static final DagStateChangedCallback STATE_CHANGED_CALLBACK = new DagStateChangedCallback(); + private String[] logDirs; @VisibleForTesting Map> commitFutures @@ -1662,7 +1665,7 @@ DAGState initializeDAG() { // which didn't have the priorities if (getConf().getBoolean(TezConfiguration.TEZ_GENERATE_DEBUG_ARTIFACTS, TezConfiguration.TEZ_GENERATE_DEBUG_ARTIFACTS_DEFAULT)) { - Utils.generateDAGVizFile(this, jobPlan, dagScheduler); + Utils.generateDAGVizFile(this, jobPlan, logDirs, dagScheduler); } return DAGState.INITED; } @@ -2510,4 +2513,17 @@ public void onFailure(Throwable t) { eventHandler.handle(new DAGEventCommitCompleted(dagId, outputKey, false, t)); } } + + public String[] getLogDirs() { + if (logDirs == null) { + logDirs = TezCommonUtils + .getTrimmedStrings(System.getenv(ApplicationConstants.Environment.LOG_DIRS.name())); + } + return logDirs; + } + + public DAGImpl setLogDirs(String[] logDirs) { + this.logDirs = logDirs; + return this; + } } From 84b5bd6e7dfb554ed3a4e9a3642f6cfbb0bb9df7 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Tue, 9 Feb 2021 10:51:56 +0100 Subject: [PATCH 297/512] TEZ-4282: Possible NPE in LocalClient after TEZ-4236 (#103) (Laszlo Bodor reviewed by Panagiotis Garefalakis, Ashutosh Chauhan) --- .../org/apache/tez/client/LocalClient.java | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java index f6d9587734..d0580bbae9 100644 --- a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java +++ b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java @@ -157,7 +157,9 @@ public ApplicationId submitApplication(ApplicationSubmissionContext appContext) @Override public void killApplication(ApplicationId appId) { try { - clientHandler.shutdownAM(); + if (clientHandler != null){ + clientHandler.shutdownAM(); + } } catch (TezException e) { throw new RuntimeException(e); } @@ -403,7 +405,13 @@ protected DAGAppMaster createDAGAppMaster(ApplicationAttemptId applicationAttemp @Override public TezAppMasterStatus getAMStatus(Configuration configuration, ApplicationId appId, UserGroupInformation ugi) throws TezException, ServiceException, IOException { - return clientHandler.getTezAppMasterStatus(); + if (isLocalWithoutNetwork) { + if (clientHandler == null) { + return TezAppMasterStatus.INITIALIZING; + } + return clientHandler.getTezAppMasterStatus(); + } + return super.getAMStatus(configuration, appId, ugi); } @Override @@ -438,10 +446,11 @@ public DAGStatus apply(Set statusOpts, Long timeout) { public boolean shutdownSession(Configuration configuration, ApplicationId sessionAppId, UserGroupInformation ugi) throws TezException, IOException, ServiceException { if (isLocalWithoutNetwork) { - clientHandler.shutdownAM(); + if (clientHandler != null){ + clientHandler.shutdownAM(); + } return true; - } else { - return super.shutdownSession(configuration, sessionAppId, ugi); } + return super.shutdownSession(configuration, sessionAppId, ugi); } } From 851e9d8b5efa962dc879cf6507de7cb495cfdad5 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Tue, 9 Feb 2021 18:55:32 +0100 Subject: [PATCH 298/512] TEZ-4278: Show PR link in Jira (#99) --- .asf.yaml | 38 ++++++++++++++++++++++++++++++++++++++ 1 file changed, 38 insertions(+) create mode 100644 .asf.yaml diff --git a/.asf.yaml b/.asf.yaml new file mode 100644 index 0000000000..4fac240422 --- /dev/null +++ b/.asf.yaml @@ -0,0 +1,38 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +github: + description: "Apache Tez" + homepage: https://tez.apache.org/ + labels: + - tez + - java + - apache + - big-data + - hadoop + features: + wiki: false + issues: false + projects: false + enabled_merge_buttons: + squash: true + merge: false + rebase: false +notifications: + commits: commits@tez.apache.org + issues: dev@tez.apache.org + pullrequests: dev@tez.apache.org + jira_options: link label worklog From a42e1700e6020eab43d42abbfc649a50798a1e92 Mon Sep 17 00:00:00 2001 From: Roman Date: Wed, 10 Feb 2021 22:35:13 +0300 Subject: [PATCH 299/512] TEZ-3966: Fix config file mime type (#26) * Fix file mime type, to avoid error when "X-Content-Type-Options: nosniff" header enabled * Update ember-cli-build.js to include only 'configs.js' --- tez-ui/README.md | 4 ++-- tez-ui/src/main/webapp/README.md | 2 +- tez-ui/src/main/webapp/app/index.html | 2 +- tez-ui/src/main/webapp/config/{configs.env => configs.js} | 0 tez-ui/src/main/webapp/ember-cli-build.js | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) rename tez-ui/src/main/webapp/config/{configs.env => configs.js} (100%) diff --git a/tez-ui/README.md b/tez-ui/README.md index cf62cdcd38..2716ca2823 100644 --- a/tez-ui/README.md +++ b/tez-ui/README.md @@ -32,8 +32,8 @@ For more information on Tez and the Tez UI - Check the [Tez homepage](http://tez * `yarn.timeline-service.enabled` : Enabled the timeline server for logging details * `yarn.timeline-service.webapp.address` : Value must be the IP:PORT on which timeline server is running -### In configs.env - This environment configuration file can be found at `./src/main/webapp/config/configs.env` +### In configs.js + This environment configuration file can be found at `./src/main/webapp/config/configs.js` * `ENV.hosts.timeline` : Timeline Server Address. By default TEZ UI looks for timeline server at http://localhost:8188. * `ENV.hosts.rm` : Resource Manager Address. By default RM REST APIs are expected to be at http://localhost:8088. diff --git a/tez-ui/src/main/webapp/README.md b/tez-ui/src/main/webapp/README.md index 0443df49ad..0ee7b46c10 100644 --- a/tez-ui/src/main/webapp/README.md +++ b/tez-ui/src/main/webapp/README.md @@ -24,7 +24,7 @@ For more information on Tez and the Tez UI - Check the [Tez homepage](http://tez ## Configurations * By default timeline is expected at localhost:8188 & RM at localhost:8088 -* You can point the UI to custom locations by setting the environment variables in `src/main/webapp/config/configs.env` +* You can point the UI to custom locations by setting the environment variables in `src/main/webapp/config/configs.js` ## Development diff --git a/tez-ui/src/main/webapp/app/index.html b/tez-ui/src/main/webapp/app/index.html index 757d66330f..f90c5c4cf3 100644 --- a/tez-ui/src/main/webapp/app/index.html +++ b/tez-ui/src/main/webapp/app/index.html @@ -37,7 +37,7 @@ {{content-for 'body'}} - + diff --git a/tez-ui/src/main/webapp/config/configs.env b/tez-ui/src/main/webapp/config/configs.js similarity index 100% rename from tez-ui/src/main/webapp/config/configs.env rename to tez-ui/src/main/webapp/config/configs.js diff --git a/tez-ui/src/main/webapp/ember-cli-build.js b/tez-ui/src/main/webapp/ember-cli-build.js index 9b52210af9..7bbc77d334 100644 --- a/tez-ui/src/main/webapp/ember-cli-build.js +++ b/tez-ui/src/main/webapp/ember-cli-build.js @@ -44,7 +44,7 @@ module.exports = function(defaults) { var configEnv = new Funnel('config', { srcDir: '/', - include: ['*.env'], + include: ['configs.js'], destDir: '/config' }); var zipWorker = new Funnel('bower_components/zip-js', { From f88aa6f21980ba85160b6338515efc54e974c267 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Fri, 12 Feb 2021 16:28:48 +0100 Subject: [PATCH 300/512] TEZ-4290: Dockerfile improvements (#111) (Laszlo Bodor reviewed by Jonathan Turner Eagles) --- build-tools/docker/Dockerfile | 368 +++++++++++++++------------ build-tools/docker/tez_env_checks.sh | 117 --------- 2 files changed, 205 insertions(+), 280 deletions(-) delete mode 100644 build-tools/docker/tez_env_checks.sh diff --git a/build-tools/docker/Dockerfile b/build-tools/docker/Dockerfile index ae26e41934..f51cc3bd2b 100644 --- a/build-tools/docker/Dockerfile +++ b/build-tools/docker/Dockerfile @@ -1,4 +1,4 @@ - +# # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -15,219 +15,261 @@ # See the License for the specific language governing permissions and # limitations under the License. -# Dockerfile for installing the necessary dependencies for building Hadoop. -# See BUILDING.txt. +############### +# +# Apache Yetus Dockerfile for Apache Tez +# NOTE: This file is compatible with Docker BuildKit. It will work +# with standard docker build, but it is a lot faster +# if BuildKit is enabled. +# +############### -FROM ubuntu:bionic +FROM ubuntu:focal AS tezbase WORKDIR /root - SHELL ["/bin/bash", "-o", "pipefail", "-c"] -##### -# Disable suggests/recommends -##### -RUN echo APT::Install-Recommends "0"\; > /etc/apt/apt.conf.d/10disableextras -RUN echo APT::Install-Suggests "0"\; >> /etc/apt/apt.conf.d/10disableextras - ENV DEBIAN_FRONTEND noninteractive ENV DEBCONF_TERSE true ###### -# Install common dependencies from packages. Versions here are either -# sufficient or irrelevant. -# -# WARNING: DO NOT PUT JAVA APPS HERE! Otherwise they will install default -# Ubuntu Java. See Java section below! +# Install some basic Apache Yetus requirements +# some git repos need ssh-client so do it too +# Adding libffi-dev for all the programming languages +# that take advantage of it. ###### # hadolint ignore=DL3008 -RUN apt-get -q update \ - && apt-get -q install -y --no-install-recommends \ - apt-utils \ - build-essential \ - bzip2 \ - clang \ - curl \ - doxygen \ - fuse \ - g++ \ - gcc \ - git \ - gnupg-agent \ - libbz2-dev \ - libfuse-dev \ - libprotobuf-dev \ - libprotoc-dev \ - libsasl2-dev \ - libsnappy-dev \ - libssl-dev \ - libtool \ - locales \ - make \ - pinentry-curses \ - pkg-config \ - python \ - python2.7 \ - python-pip \ - python-pkg-resources \ - python-setuptools \ - python-wheel \ - rsync \ - software-properties-common \ - sudo \ - valgrind \ - zlib1g-dev \ +RUN apt-get -q update && apt-get -q install --no-install-recommends -y \ + apt-transport-https \ + apt-utils \ + ca-certificates \ + curl \ + dirmngr \ + git \ + gpg \ + gpg-agent \ + libffi-dev \ + locales \ + make \ + pkg-config \ + rsync \ + software-properties-common \ + ssh-client \ + xz-utils \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* +### +# Set the locale +### +RUN locale-gen en_US.UTF-8 +ENV LANG en_US.UTF-8 +ENV LANGUAGE en_US:en +ENV LC_ALL en_US.UTF-8 -####### -# OpenJDK 8 -####### +#### +# Install GNU C/C++ (everything generally needs this) +#### # hadolint ignore=DL3008 -RUN apt-get -q update \ - && apt-get -q install -y --no-install-recommends openjdk-8-jdk libbcprov-java \ +RUN apt-get -q update && apt-get -q install --no-install-recommends -y \ + g++ \ + gcc \ + libc-dev \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* +### +# Install golang as part of base so we can do each +# helper utility in parallel. go bins are typically +# statically linked, so this is perfectly safe. +### +# hadolint ignore=DL3008 +RUN add-apt-repository -y ppa:longsleep/golang-backports \ + && apt-get -q update \ + && apt-get -q install --no-install-recommends -y golang-go \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* -###### -# Install cmake 3.1.0 (3.5.1 ships with Xenial) -###### -RUN mkdir -p /opt/cmake \ - && curl -L -s -S \ - https://cmake.org/files/v3.1/cmake-3.1.0-Linux-x86_64.tar.gz \ - -o /opt/cmake.tar.gz \ - && tar xzf /opt/cmake.tar.gz --strip-components 1 -C /opt/cmake -ENV CMAKE_HOME /opt/cmake -ENV PATH "${PATH}:/opt/cmake/bin" +############ +# Fetch all of the non-conflicting bits in parallel +############# ###### -# Install Google Protobuf 2.5.0 (2.6.0 ships with Xenial) +# Install Google Protobuf 2.5.0 ###### -# hadolint ignore=DL3003 +FROM tezbase AS protobuf250 +SHELL ["/bin/bash", "-o", "pipefail", "-c"] RUN mkdir -p /opt/protobuf-src \ && curl -L -s -S \ https://github.com/google/protobuf/releases/download/v2.5.0/protobuf-2.5.0.tar.gz \ -o /opt/protobuf.tar.gz \ - && tar xzf /opt/protobuf.tar.gz --strip-components 1 -C /opt/protobuf-src \ - && cd /opt/protobuf-src \ - && ./configure --prefix=/opt/protobuf \ - && make install \ - && cd /root \ - && rm -rf /opt/protobuf-src -ENV PROTOBUF_HOME /opt/protobuf -ENV PATH "${PATH}:/opt/protobuf/bin" + && tar xzf /opt/protobuf.tar.gz --strip-components 1 -C /opt/protobuf-src +WORKDIR /opt/protobuf-src +RUN ./configure --prefix=/opt/protobuf \ + && make install +WORKDIR /root +RUN rm -rf /opt/protobuf-src -###### -# Install Apache Maven 3.3.9 (3.3.9 ships with Xenial) -###### +#### +# Install shellcheck (shell script lint) +#### +FROM tezbase AS shellcheck +SHELL ["/bin/bash", "-o", "pipefail", "-c"] +RUN curl -sSL \ + https://github.com/koalaman/shellcheck/releases/download/v0.7.1/shellcheck-v0.7.1.linux.x86_64.tar.xz \ + | tar --strip-components 1 --wildcards -xJf - '*/shellcheck' \ + && chmod a+rx shellcheck \ + && mv shellcheck /bin/shellcheck \ + && shasum -a 512 /bin/shellcheck \ + | awk '$1!="aae813283d49f18f95a205dca1c5184267d07534a08abc952ebea1958fee06f8a0207373b6770a083079ba875458ea9da443f2b9910a50dcd93b935048bc14f5" {exit(1)}' + +#### +# Install hadolint (dockerfile lint) +#### +FROM tezbase AS hadolint +SHELL ["/bin/bash", "-o", "pipefail", "-c"] +RUN curl -sSL \ + https://github.com/hadolint/hadolint/releases/download/v1.18.0/hadolint-Linux-x86_64 \ + -o /bin/hadolint \ + && chmod a+rx /bin/hadolint \ + && shasum -a 512 /bin/hadolint \ + | awk '$1!="df27253d374c143a606483b07a26234ac7b4bca40b4eba53e79609c81aa70146e7d5c145f90dcec71d6d1aad1048b7d9d2de68d92284f48a735d04d19c5c5559" {exit(1)}' + +#### +# Install buf (protobuf lint) +#### +FROM tezbase AS buf +SHELL ["/bin/bash", "-o", "pipefail", "-c"] +RUN curl -sSL \ + https://github.com/bufbuild/buf/releases/download/v0.21.0/buf-Linux-x86_64.tar.gz \ + -o buf.tar.gz \ + && shasum -a 256 buf.tar.gz \ + | awk '$1!="95aba62ac0ecc5a9120cc58c65cdcc85038633a816bddfe8398c5ae3b32803f1" {exit(1)}' \ + && tar -xzf buf.tar.gz -C /usr/local --strip-components 1 \ + && rm buf.tar.gz + +######## +# +# +# Content that needs to be installed in order due to packages... +# +# +######## + +FROM tezbase +SHELL ["/bin/bash", "-o", "pipefail", "-c"] + +#### +# Install java (first, since we want to dicate what form of Java) +#### + +#### +# OpenJDK 8 +#### # hadolint ignore=DL3008 -RUN apt-get -q update \ - && apt-get -q install -y --no-install-recommends maven \ +RUN apt-get -q update && apt-get -q install --no-install-recommends -y openjdk-8-jdk-headless \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* -ENV MAVEN_HOME /usr + +#### +# OpenJDK 11 (but keeps default to JDK8) +# NOTE: This default only works when Apache Yetus is launched +# _in_ the container and not outside of it! +#### +# hadolint ignore=DL3008 +RUN apt-get -q update && apt-get -q install --no-install-recommends -y default-jre-headless openjdk-11-jdk-headless \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* \ + && update-java-alternatives -s java-1.8.0-openjdk-amd64 \ + && rm -f /usr/lib/jvm/default-java \ + && ln -s java-8-openjdk-amd64 /usr/lib/jvm/default-java +ENV JAVA_HOME /usr/lib/jvm/java-8-openjdk-amd64 ###### -# Install findbugs 3.0.1 (3.0.1 ships with Xenial) -# Ant is needed for findbugs +# Install findbugs ###### # hadolint ignore=DL3008 -RUN apt-get -q update \ - && apt-get -q install -y --no-install-recommends findbugs ant \ +RUN apt-get -q update && apt-get -q install --no-install-recommends -y findbugs \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* ENV FINDBUGS_HOME /usr -#### -# Install shellcheck (0.4.6, the latest as of 2017-09-26) -#### +###### +# Install maven +###### # hadolint ignore=DL3008 -RUN add-apt-repository -y ppa:hvr/ghc \ - && apt-get -q update \ - && apt-get -q install -y --no-install-recommends shellcheck ghc-8.0.2 \ +RUN apt-get -q update && apt-get -q install --no-install-recommends -y maven \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* -#### -# Install bats (0.4.0, the latest as of 2017-09-26, ships with Xenial) -#### -# hadolint ignore=DL3008 -RUN apt-get -q update \ - && apt-get -q install -y --no-install-recommends bats \ +###### +# Install python3 and pylint3 +# astroid and pylint go hand-in-hand. Upgrade both at the same time. +###### +# hadolint ignore=DL3008,DL3013 +RUN apt-get -q update && apt-get -q install --no-install-recommends -y \ + python3 \ + python3-bcrypt \ + python3-cffi \ + python3-cryptography \ + python3-dateutil \ + python3-dev \ + python3-dev \ + python3-isort \ + python3-dockerpty \ + python3-nacl \ + python3-pyrsistent \ + python3-setuptools \ + python3-setuptools \ + python3-singledispatch \ + python3-six \ + python3-wheel \ + python3-wrapt \ + python3-yaml \ && apt-get clean \ - && rm -rf /var/lib/apt/lists/* - -#### -# Install pylint at fixed version (2.0.0 removed python2 support) -# https://github.com/PyCQA/pylint/issues/2294 -#### -RUN pip2 install pylint==1.9.2 - -#### -# Install dateutil.parser -#### -RUN pip2 install python-dateutil==2.7.3 + && rm -rf /var/lib/apt/lists/* \ + && curl -sSL https://bootstrap.pypa.io/get-pip.py -o /tmp/get-pip.py \ + && python3 /tmp/get-pip.py \ + && rm /usr/local/bin/pip /tmp/get-pip.py \ + && pip3 install -v \ + astroid==2.4.2 \ + codespell==2.0 \ + pylint==2.5.3 \ + yamllint==1.24.2 \ + && rm -rf /root/.cache \ + && mv /usr/local/bin/pylint /usr/local/bin/pylint3 +RUN ln -s /usr/local/bin/pylint3 /usr/local/bin/pylint +RUN ln -s /usr/local/bin/pip3 /usr/local/bin/pip ### -# Install node.js for web UI framework (8.10.0 ships with Bionic, let's override with 10.x) +# Install npm and JSHint ### -RUN curl -sL https://deb.nodesource.com/setup_10.x | bash - -# hadolint ignore=DL3008, DL3015 -RUN apt-get install -y nodejs -# hadolint ignore=DL3016 -RUN apt-get clean \ +# hadolint ignore=DL3008 +RUN curl -sSL https://deb.nodesource.com/setup_14.x | bash - \ + && apt-get -q install --no-install-recommends -y nodejs \ + && apt-get clean \ && rm -rf /var/lib/apt/lists/* \ - && npm install npm@latest -g \ - && npm install -g jshint + && npm install -g \ + jshint@2.12.0 \ + markdownlint-cli@0.23.2 \ + && rm -rf /root/.npm -### -# Install hadolint -#### -RUN curl -L -s -S \ - https://github.com/hadolint/hadolint/releases/download/v1.11.1/hadolint-Linux-x86_64 \ - -o /bin/hadolint \ - && chmod a+rx /bin/hadolint \ - && shasum -a 512 /bin/hadolint | \ - awk '$1!="734e37c1f6619cbbd86b9b249e69c9af8ee1ea87a2b1ff71dccda412e9dac35e63425225a95d71572091a3f0a11e9a04c2fc25d9e91b840530c26af32b9891ca" {exit(1)}' - -### -# Avoid out of memory errors in builds -### -ENV MAVEN_OPTS -Xms256m -Xmx1536m +##### +# Now all the stuff that was built in parallel +##### +COPY --from=shellcheck /bin/shellcheck /bin/shellcheck +COPY --from=hadolint /bin/hadolint /bin/hadolint +COPY --from=buf /usr/local/bin/buf /usr/local/bin/buf +COPY --from=protobuf250 /opt/protobuf /opt/protobuf -### -# Everything past this point is either not needed for testing or breaks Yetus. -# So tell Yetus not to read the rest of the file: -# YETUS CUT HERE -### +ENV PROTOBUF_HOME /opt/protobuf +ENV PROTOC_PATH /opt/protobuf/bin/protoc +ENV PATH "${PATH}:/opt/protobuf/bin" #### -# Install svn & Forrest (for Apache Hadoop website) +# YETUS CUT HERE +# Magic text above! Everything from here on is ignored +# by Yetus, so could include anything not needed +# by your testing environment ### -# hadolint ignore=DL3008 -RUN apt-get -q update \ - && apt-get -q install -y --no-install-recommends subversion \ - && apt-get clean \ - && rm -rf /var/lib/apt/lists/* - -RUN mkdir -p /opt/apache-forrest \ - && curl -L -s -S \ - https://archive.apache.org/dist/forrest/0.8/apache-forrest-0.8.tar.gz \ - -o /opt/forrest.tar.gz \ - && tar xzf /opt/forrest.tar.gz --strip-components 1 -C /opt/apache-forrest -RUN echo 'forrest.home=/opt/apache-forrest' > build.properties -ENV FORREST_HOME=/opt/apache-forrest - -# Hugo static website generator (for new tez site and Ozone docs) -RUN curl -L -o hugo.deb https://github.com/gohugoio/hugo/releases/download/v0.30.2/hugo_0.30.2_Linux-64bit.deb \ - && dpkg --install hugo.deb \ - && rm hugo.deb - -# Add a welcome message and environment checks. -COPY tez_env_checks.sh /root/tez_env_checks.sh -RUN chmod 755 /root/tez_env_checks.sh -# hadolint ignore=SC2016 -RUN echo '${HOME}/tez_env_checks.sh' >> /root/.bashrc diff --git a/build-tools/docker/tez_env_checks.sh b/build-tools/docker/tez_env_checks.sh deleted file mode 100644 index c9420e79e9..0000000000 --- a/build-tools/docker/tez_env_checks.sh +++ /dev/null @@ -1,117 +0,0 @@ -#!/usr/bin/env bash - -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -# SHELLDOC-IGNORE - -# ------------------------------------------------------- -function showWelcome { -cat < Date: Fri, 12 Feb 2021 17:21:36 +0100 Subject: [PATCH 301/512] TEZ-4259: Create Jenkinsfile for use with tez-multibranch (github yetus integration) (Jonathan Turner Eagles reviewed by Laszlo Bodor) --- Jenkinsfile | 216 ++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 216 insertions(+) create mode 100644 Jenkinsfile diff --git a/Jenkinsfile b/Jenkinsfile new file mode 100644 index 0000000000..1d6264658b --- /dev/null +++ b/Jenkinsfile @@ -0,0 +1,216 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +pipeline { + + agent { + label 'Hadoop' + } + + options { + buildDiscarder(logRotator(numToKeepStr: '5')) + timeout (time: 20, unit: 'HOURS') + timestamps() + checkoutToSubdirectory('src') + } + + environment { + SOURCEDIR = 'src' + // will also need to change notification section below + PATCHDIR = 'out' + DOCKERFILE = "${SOURCEDIR}/build-tools/docker/Dockerfile" + YETUS='yetus' + // Branch or tag name. Yetus release tags are 'rel/X.Y.Z' + YETUS_VERSION='rel/0.12.0' + + } + + parameters { + string(name: 'JIRA_ISSUE_KEY', + defaultValue: '', + description: 'The JIRA issue that has a patch needing pre-commit testing. Example: HADOOP-1234') + } + + stages { + stage ('install yetus') { + steps { + dir("${WORKSPACE}/${YETUS}") { + checkout([ + $class: 'GitSCM', + branches: [[name: "${env.YETUS_VERSION}"]], + userRemoteConfigs: [[ url: 'https://github.com/jteagles/yetus']]] + ) + } + } + } + + stage ('precommit-run') { + steps { + withCredentials( + [usernamePassword(credentialsId: 'apache-tez-at-github.com', + passwordVariable: 'GITHUB_PASSWD', + usernameVariable: 'GITHUB_USER'), + usernamePassword(credentialsId: 'tez-ci', + passwordVariable: 'JIRA_PASSWORD', + usernameVariable: 'JIRA_USER')]) { + sh '''#!/usr/bin/env bash + + set -e + + TESTPATCHBIN="${WORKSPACE}/${YETUS}/precommit/src/main/shell/test-patch.sh" + + # this must be clean for every run + if [[ -d "${WORKSPACE}/${PATCHDIR}" ]]; then + rm -rf "${WORKSPACE}/${PATCHDIR}" + fi + mkdir -p "${WORKSPACE}/${PATCHDIR}" + + # if given a JIRA issue, process it. If CHANGE_URL is set + # (e.g., Github Branch Source plugin), process it. + # otherwise exit, because we don't want Hadoop to do a + # full build. We wouldn't normally do this check for smaller + # projects. :) + if [[ -n "${JIRA_ISSUE_KEY}" ]]; then + YETUS_ARGS+=("${JIRA_ISSUE_KEY}") + elif [[ -z "${CHANGE_URL}" ]]; then + echo "Full build skipped" > "${WORKSPACE}/${PATCHDIR}/report.html" + exit 0 + fi + + YETUS_ARGS+=("--patch-dir=${WORKSPACE}/${PATCHDIR}") + + # where the source is located + YETUS_ARGS+=("--basedir=${WORKSPACE}/${SOURCEDIR}") + + # our project defaults come from a personality file + YETUS_ARGS+=("--project=tez") + + # lots of different output formats + YETUS_ARGS+=("--brief-report-file=${WORKSPACE}/${PATCHDIR}/brief.txt") + YETUS_ARGS+=("--console-report-file=${WORKSPACE}/${PATCHDIR}/console.txt") + YETUS_ARGS+=("--html-report-file=${WORKSPACE}/${PATCHDIR}/report.html") + + # enable writing back to Github + YETUS_ARGS+=(--github-user="${GITHUB_USER}") + YETUS_ARGS+=(--github-password="${GITHUB_PASSWD}") + + # enable writing back to ASF JIRA + YETUS_ARGS+=(--jira-password="${JIRA_PASSWORD}") + YETUS_ARGS+=(--jira-user="${JIRA_USER}") + + # auto-kill any surefire stragglers during unit test runs + YETUS_ARGS+=("--reapermode=kill") + + # set relatively high limits for ASF machines + # changing these to higher values may cause problems + # with other jobs on systemd-enabled machines + YETUS_ARGS+=("--proclimit=5500") + YETUS_ARGS+=("--dockermemlimit=20g") + + # -1 findbugs issues that show up prior to the patch being applied + # YETUS_ARGS+=("--findbugs-strict-precheck") + + # rsync these files back into the archive dir + YETUS_ARGS+=("--archive-list=checkstyle-errors.xml,findbugsXml.xml") + + # URL for user-side presentation in reports and such to our artifacts + # (needs to match the archive bits below) + YETUS_ARGS+=("--build-url-artifacts=artifact/out") + + # plugins to enable + YETUS_ARGS+=("--plugins=all") + + # use Hadoop's bundled shelldocs + YETUS_ARGS+=("--shelldocs=${WORKSPACE}/${SOURCEDIR}/dev-support/bin/shelldocs") + + # don't let these tests cause -1s because we aren't really paying that + # much attention to them + YETUS_ARGS+=("--tests-filter=checkstyle") + + # run in docker mode and specifically point to our + # Dockerfile since we don't want to use the auto-pulled version. + YETUS_ARGS+=("--docker") + YETUS_ARGS+=("--dockerfile=${DOCKERFILE}") + YETUS_ARGS+=("--mvn-custom-repos") + + # effectively treat dev-suport as a custom maven module + YETUS_ARGS+=("--skip-dirs=dev-support") + + # help keep the ASF boxes clean + YETUS_ARGS+=("--sentinel") + + # test with Java 8 and 11 + YETUS_ARGS+=("--java-home=/usr/lib/jvm/java-8-openjdk-amd64") + YETUS_ARGS+=("--multijdkdirs=/usr/lib/jvm/java-11-openjdk-amd64") + YETUS_ARGS+=("--multijdktests=compile") + YETUS_ARGS+=("--debug") + + "${TESTPATCHBIN}" "${YETUS_ARGS[@]}" + ''' + } + } + } + + } + + post { + always { + script { + // Yetus output + archiveArtifacts "${env.PATCHDIR}/**" + // Publish the HTML report so that it can be looked at + // Has to be relative to WORKSPACE. + publishHTML (target: [ + allowMissing: true, + keepAll: true, + alwaysLinkToLastBuild: true, + // Has to be relative to WORKSPACE + reportDir: "${env.PATCHDIR}", + reportFiles: 'report.html', + reportName: 'Yetus Report' + ]) + // Publish JUnit results + try { + junit "${env.SOURCEDIR}/**/target/surefire-reports/*.xml" + } catch(e) { + echo 'junit processing: ' + e.toString() + } + } + } + + // Jenkins pipeline jobs fill slaves on PRs without this :( + cleanup() { + script { + sh ''' + # See YETUS-764 + if [ -f "${WORKSPACE}/${PATCHDIR}/pidfile.txt" ]; then + echo "test-patch process appears to still be running: killing" + kill `cat "${WORKSPACE}/${PATCHDIR}/pidfile.txt"` || true + sleep 10 + fi + if [ -f "${WORKSPACE}/${PATCHDIR}/cidfile.txt" ]; then + echo "test-patch container appears to still be running: killing" + docker kill `cat "${WORKSPACE}/${PATCHDIR}/cidfile.txt"` || true + fi + # See HADOOP-13951 + chmod -R u+rxw "${WORKSPACE}" + ''' + deleteDir() + } + } + } +} \ No newline at end of file From f9c96b2be3d1ff0b81ab397984f99b85cdcf7047 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Fri, 12 Feb 2021 10:48:35 -0600 Subject: [PATCH 302/512] TEZ-4241. UI: containerLogs testWrapper test fails (review by Bodor Laszlo) Signed-off-by: Jonathan Eagles --- tez-ui/src/main/webapp/tests/unit/serializers/dag-test.js | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tez-ui/src/main/webapp/tests/unit/serializers/dag-test.js b/tez-ui/src/main/webapp/tests/unit/serializers/dag-test.js index 6334569491..80f41b6bbb 100644 --- a/tez-ui/src/main/webapp/tests/unit/serializers/dag-test.js +++ b/tez-ui/src/main/webapp/tests/unit/serializers/dag-test.js @@ -110,8 +110,8 @@ test('containerLogs test', function(assert) { }), [], "No logs"); assert.deepEqual(mapper({ - otherinfo: {inProgressLogsURL_1: "foo", inProgressLogsURL_2: "bar"}, - }), [{text: "1", href: "http://foo"}, {text: "2", href: "http://bar"}], "2 logs"); + otherinfo: {inProgressLogsURL_1: "http://foo", inProgressLogsURL_2: "https://bar"}, + }), [{text: "1", href: "http://foo"}, {text: "2", href: "https://bar"}], "2 logs"); }); test('vertexIdNameMap test', function(assert) { From b4152fc85316eb95a4f8a6b2e41a151a89f772da Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Fri, 12 Feb 2021 12:55:37 -0500 Subject: [PATCH 303/512] TEZ-4288: Upgrade SLF4J to 1.7.30 (#109) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 892a8a2d66..9fa2cb4723 100644 --- a/pom.xml +++ b/pom.xml @@ -62,7 +62,7 @@ 3.10.5.Final 0.13.0 1.19 - 1.7.10 + 1.7.30 2.5.0 ${env.PROTOC_PATH} scm:git:https://gitbox.apache.org/repos/asf/tez.git From 7d32807d858dad37bb31637eb04a80c8436f605a Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Sat, 13 Feb 2021 22:56:28 -0600 Subject: [PATCH 304/512] TEZ-4025. javadoc compilation is broken in jdk11 (review by Bodor Laszlo) Signed-off-by: Jonathan Eagles --- tez-tools/tez-javadoc-tools/pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tez-tools/tez-javadoc-tools/pom.xml b/tez-tools/tez-javadoc-tools/pom.xml index b17604f207..e0405b685b 100644 --- a/tez-tools/tez-javadoc-tools/pom.xml +++ b/tez-tools/tez-javadoc-tools/pom.xml @@ -94,6 +94,12 @@ true + + maven-javadoc-plugin + + org.apache.tez.tools.javadoc.doclet + + From 3089d478f70f139139fea68028712d5fc32012d1 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Sun, 14 Feb 2021 20:10:05 +0100 Subject: [PATCH 305/512] TEZ-4286: Consider sending fewer emails from github to dev@tez.apache.org (#107) (Laszlo Bodor reviewed by Jonathan Turner Eagles) --- .asf.yaml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.asf.yaml b/.asf.yaml index 4fac240422..776f8a6f7c 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -33,6 +33,6 @@ github: rebase: false notifications: commits: commits@tez.apache.org - issues: dev@tez.apache.org - pullrequests: dev@tez.apache.org + issues: issues@tez.apache.org + pullrequests: issues@tez.apache.org jira_options: link label worklog From 70623f0d8e9de3c59b61c10abe5442bfd8f66e2c Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Wed, 17 Feb 2021 16:47:00 -0500 Subject: [PATCH 306/512] TEZ-4289: Remove Dependency on commons-math (#110) * TEZ-4289: Remove Dependency on commons-math * Remove stray reference to commons-math * Fix checkstyle and missing parenthesis --- pom.xml | 5 ----- .../src/main/java/org/apache/tez/client/TezClientUtils.java | 5 ++--- tez-dag/pom.xml | 4 ---- .../org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java | 5 ++--- .../org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java | 5 ++--- 5 files changed, 6 insertions(+), 18 deletions(-) diff --git a/pom.xml b/pom.xml index 9fa2cb4723..19fd5167cd 100644 --- a/pom.xml +++ b/pom.xml @@ -668,11 +668,6 @@ commons-collections4 4.1 - - org.apache.commons - commons-math3 - 3.1.1 - commons-cli commons-cli diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index 79069ede7d..389b58400b 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -43,7 +43,6 @@ import com.google.common.base.Strings; import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.lang.StringUtils; -import org.apache.commons.math3.util.Precision; import org.apache.tez.common.JavaOptsChecker; import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto; import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor; @@ -1007,11 +1006,11 @@ public static String maybeAddDefaultMemoryJavaOpts(String javaOpts, Resource res return javaOpts; } - if ((maxHeapFactor <= 0 && !Precision.equals(maxHeapFactor, -1, 0.01)) || maxHeapFactor >= 1) { + if ((maxHeapFactor <= 0 && Double.valueOf("-1") != maxHeapFactor) || maxHeapFactor >= 1) { return javaOpts; } - if (Precision.equals(maxHeapFactor, -1, 0.01)) { + if (Double.valueOf("-1") == maxHeapFactor) { maxHeapFactor = resource.getMemory() < TezConstants.TEZ_CONTAINER_SMALL_SLAB_BOUND_MB ? TezConstants.TEZ_CONTAINER_MAX_JAVA_HEAP_FRACTION_SMALL_SLAB : TezConstants.TEZ_CONTAINER_MAX_JAVA_HEAP_FRACTION_LARGE_SLAB; diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml index 89033859c6..c0dccc8642 100644 --- a/tez-dag/pom.xml +++ b/tez-dag/pom.xml @@ -87,10 +87,6 @@ org.apache.hadoop hadoop-yarn-server-web-proxy - - org.apache.commons - commons-math3 - org.slf4j slf4j-api diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java index d8b3397c70..29f350f9d0 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java @@ -22,7 +22,6 @@ import org.apache.tez.common.Preconditions; import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.commons.lang.mutable.MutableInt; -import org.apache.commons.math3.random.RandomDataGenerator; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.Time; @@ -80,6 +79,7 @@ import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; /** @@ -95,7 +95,6 @@ public class DagAwareYarnTaskScheduler extends TaskScheduler private static final Logger LOG = LoggerFactory.getLogger(DagAwareYarnTaskScheduler.class); private static final Comparator PREEMPT_ORDER_COMPARATOR = new PreemptOrderComparator(); - private final RandomDataGenerator random = new RandomDataGenerator(); private AMRMClientAsyncWrapper client; private ScheduledExecutorService reuseExecutor; private ResourceCalculator resourceCalculator; @@ -1544,7 +1543,7 @@ long getIdleExpirationTimestamp(long now) { if (idleExpirationTimestamp == 0) { if (idleContainerTimeoutMin > 0) { idleExpirationTimestamp = now + (idleContainerTimeoutMin == idleContainerTimeoutMax ? idleContainerTimeoutMin - : random.nextLong(idleContainerTimeoutMin, idleContainerTimeoutMax)); + : ThreadLocalRandom.current().nextLong(idleContainerTimeoutMin, idleContainerTimeoutMax)); } else { idleExpirationTimestamp = Long.MAX_VALUE; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java index b4aa561fbe..6f51566f75 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java @@ -32,6 +32,7 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.PriorityBlockingQueue; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -44,7 +45,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.commons.lang.exception.ExceptionUtils; -import org.apache.commons.math3.random.RandomDataGenerator; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.util.ExitUtil; @@ -153,7 +153,6 @@ public class YarnTaskSchedulerService extends TaskScheduler Set sessionMinHeldContainers = Sets.newHashSet(); - RandomDataGenerator random = new RandomDataGenerator(); private final Configuration conf; @VisibleForTesting @@ -593,7 +592,7 @@ long getHeldContainerExpireTime(long startTime) { long expireTime = (startTime + idleContainerTimeoutMin); if (idleContainerTimeoutMin != -1 && idleContainerTimeoutMin < idleContainerTimeoutMax) { long expireTimeMax = startTime + idleContainerTimeoutMax; - expireTime = random.nextLong(expireTime, expireTimeMax); + expireTime = ThreadLocalRandom.current().nextLong(expireTime, expireTimeMax); } return expireTime; From 2fefe4a6b16bea8f02169ec2e2a6e89575440328 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Thu, 18 Feb 2021 17:35:19 +0100 Subject: [PATCH 307/512] TEZ-4291: Fancy precommit report on PR (#112) --- Jenkinsfile | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/Jenkinsfile b/Jenkinsfile index 1d6264658b..835a0523a4 100644 --- a/Jenkinsfile +++ b/Jenkinsfile @@ -52,7 +52,7 @@ pipeline { checkout([ $class: 'GitSCM', branches: [[name: "${env.YETUS_VERSION}"]], - userRemoteConfigs: [[ url: 'https://github.com/jteagles/yetus']]] + userRemoteConfigs: [[ url: 'https://github.com/apache/yetus']]] ) } } @@ -61,8 +61,8 @@ pipeline { stage ('precommit-run') { steps { withCredentials( - [usernamePassword(credentialsId: 'apache-tez-at-github.com', - passwordVariable: 'GITHUB_PASSWD', + [usernamePassword(credentialsId: 'apache-hadoop-at-github.com', + passwordVariable: 'GITHUB_TOKEN', usernameVariable: 'GITHUB_USER'), usernamePassword(credentialsId: 'tez-ci', passwordVariable: 'JIRA_PASSWORD', @@ -106,7 +106,7 @@ pipeline { # enable writing back to Github YETUS_ARGS+=(--github-user="${GITHUB_USER}") - YETUS_ARGS+=(--github-password="${GITHUB_PASSWD}") + YETUS_ARGS+=(--github-token="${GITHUB_TOKEN}") # enable writing back to ASF JIRA YETUS_ARGS+=(--jira-password="${JIRA_PASSWORD}") @@ -153,6 +153,9 @@ pipeline { # help keep the ASF boxes clean YETUS_ARGS+=("--sentinel") + # use emoji vote so it is easier to find the broken line + YETUS_ARGS+=("--github-use-emoji-vote") + # test with Java 8 and 11 YETUS_ARGS+=("--java-home=/usr/lib/jvm/java-8-openjdk-amd64") YETUS_ARGS+=("--multijdkdirs=/usr/lib/jvm/java-11-openjdk-amd64") From 2dcbe0bca22be89a797acd5f2228d91c4c112069 Mon Sep 17 00:00:00 2001 From: jacobtolar Date: Fri, 5 Mar 2021 09:44:09 -0600 Subject: [PATCH 308/512] fix typo: missing space (#59) I'm happy to file a JIRA if necessary; I wasn't sure if that was useful or not for a 1-character change. :) The log message looks like this currently: ``` 2020-02-10 20:12:58,267 [INFO] [Dispatcher thread {Central}] |node.AMNodeImpl|: Attempt failedon node: ... ``` --- .../main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java index f4ad032514..df19534bfc 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java @@ -304,7 +304,7 @@ protected static class TaskAttemptFailedTransition implements @Override public AMNodeState transition(AMNodeImpl node, AMNodeEvent nEvent) { AMNodeEventTaskAttemptEnded event = (AMNodeEventTaskAttemptEnded) nEvent; - LOG.info("Attempt " + (event.failed() ? "failed" : "killed") + "on node: " + node.getNodeId() + LOG.info("Attempt " + (event.failed() ? "failed" : "killed") + " on node: " + node.getNodeId() + " TA: " + event.getTaskAttemptId() + ", container: " + event.getContainerId() + ", numFailedTAs: " + node.numFailedTAs); From 2b671d6211ad8bdebaa16889bf4f1f4121c4f1f9 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Mon, 5 Apr 2021 11:55:20 +0200 Subject: [PATCH 309/512] TEZ-4299: Default java opts cause jdk11 to fail (#116) (Laszlo Bodor reviewed by Jonathan Turner Eagles) --- .../org/apache/tez/common/TezCommonUtils.java | 6 ++++++ .../apache/tez/dag/api/TezConfiguration.java | 21 +++++++++++++++++-- 2 files changed, 25 insertions(+), 2 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java index 27259580d4..3163968908 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java @@ -566,4 +566,10 @@ public static long getDAGSessionTimeout(Configuration conf) { return 1000l * timeoutSecs; } + public static int getJavaVersion() { + String javaVersionString = System.getProperty("java.version"); + return javaVersionString.split("\\.")[0].equals("1") + ? Integer.parseInt(javaVersionString.split("\\.")[1]) // "1.8" -> 8 + : Integer.parseInt(javaVersionString.split("\\.")[0]); // "9.x" -> 9, "11.x" -> 11 + } } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index eef0d65424..17a826e7e7 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.util.Shell; import org.apache.hadoop.yarn.api.records.LocalResource; +import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.annotation.ConfigurationClass; import org.apache.tez.common.annotation.ConfigurationProperty; import org.apache.tez.dag.api.EdgeProperty.ConcurrentEdgeTriggerType; @@ -343,8 +344,11 @@ public TezConfiguration(boolean loadDefaults) { @ConfigurationScope(Scope.AM) @ConfigurationProperty public static final String TEZ_AM_LAUNCH_CMD_OPTS = TEZ_AM_PREFIX + "launch.cmd-opts"; - public static final String TEZ_AM_LAUNCH_CMD_OPTS_DEFAULT = + public static final String TEZ_AM_LAUNCH_CMD_OPTS_JDK8_DEFAULT = "-XX:+PrintGCDetails -verbose:gc -XX:+PrintGCTimeStamps -XX:+UseNUMA -XX:+UseParallelGC"; + public static final String TEZ_AM_LAUNCH_CMD_OPTS_JDK9_DEFAULT = + "-verbose:gc -Xlog:gc*,safepoint::time,uptime -XX:+UseNUMA -XX:+UseParallelGC"; + public static final String TEZ_AM_LAUNCH_CMD_OPTS_DEFAULT; /** * String value. Command line options which will be prepended to {@link @@ -368,8 +372,21 @@ public TezConfiguration(boolean loadDefaults) { @ConfigurationProperty public static final String TEZ_TASK_LAUNCH_CMD_OPTS = TEZ_TASK_PREFIX + "launch.cmd-opts"; - public static final String TEZ_TASK_LAUNCH_CMD_OPTS_DEFAULT = + public static final String TEZ_TASK_LAUNCH_CMD_OPTS_JDK8_DEFAULT = "-XX:+PrintGCDetails -verbose:gc -XX:+PrintGCTimeStamps -XX:+UseNUMA -XX:+UseParallelGC"; + public static final String TEZ_TASK_LAUNCH_CMD_OPTS_JDK9_DEFAULT = + "-verbose:gc -Xlog:gc*,safepoint::time,uptime -XX:+UseNUMA -XX:+UseParallelGC"; + public static final String TEZ_TASK_LAUNCH_CMD_OPTS_DEFAULT; + + static { + if (TezCommonUtils.getJavaVersion() >= 9) { + TEZ_AM_LAUNCH_CMD_OPTS_DEFAULT = TEZ_AM_LAUNCH_CMD_OPTS_JDK9_DEFAULT; + TEZ_TASK_LAUNCH_CMD_OPTS_DEFAULT = TEZ_TASK_LAUNCH_CMD_OPTS_JDK9_DEFAULT; + } else { + TEZ_AM_LAUNCH_CMD_OPTS_DEFAULT = TEZ_AM_LAUNCH_CMD_OPTS_JDK8_DEFAULT; + TEZ_TASK_LAUNCH_CMD_OPTS_DEFAULT = TEZ_TASK_LAUNCH_CMD_OPTS_JDK8_DEFAULT; + } + } /** * Double value. Tez automatically determines the Xmx for the JVMs used to run From b934644871027722b40d83822bf6a0978141d6bf Mon Sep 17 00:00:00 2001 From: jshmchenxi Date: Sat, 10 Apr 2021 16:11:48 +0800 Subject: [PATCH 310/512] TEZ-4302: NullPointerException in CodecUtils with GzipCodec (#117) (Xi Chen reviewed by Laszlo Bodor) --- .../apache/tez/runtime/library/common/TezRuntimeUtils.java | 5 ++++- .../org/apache/tez/runtime/library/utils/CodecUtils.java | 3 ++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java index 9ff3d1c1e5..a1df131121 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java @@ -38,6 +38,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.tez.dag.api.TezUncheckedException; import org.apache.tez.runtime.api.OutputContext; import org.apache.tez.runtime.api.TaskContext; @@ -269,7 +270,9 @@ public static String getBufferSizeProperty(CompressionCodec codec) { public static String getBufferSizeProperty(String className) { switch (className) { case "org.apache.hadoop.io.compress.DefaultCodec": - return "io.file.buffer.size"; + case "org.apache.hadoop.io.compress.BZip2Codec": + case "org.apache.hadoop.io.compress.GzipCodec": + return CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY; case "org.apache.hadoop.io.compress.SnappyCodec": return CommonConfigurationKeys.IO_COMPRESSION_CODEC_SNAPPY_BUFFERSIZE_KEY; case "org.apache.hadoop.io.compress.ZStandardCodec": diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/CodecUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/CodecUtils.java index 99d22c5dcc..8e5154f3b0 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/CodecUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/CodecUtils.java @@ -78,7 +78,8 @@ public static InputStream getDecompressedInputStreamWithBufferSize(CompressionCo throws IOException { String bufferSizeProp = TezRuntimeUtils.getBufferSizeProperty(codec); Configurable configurableCodec = (Configurable) codec; - int originalSize = configurableCodec.getConf().getInt(bufferSizeProp, DEFAULT_BUFFER_SIZE); + int originalSize = bufferSizeProp == null ? DEFAULT_BUFFER_SIZE : + configurableCodec.getConf().getInt(bufferSizeProp, DEFAULT_BUFFER_SIZE); CompressionInputStream in = null; From aea070714cd464a7403bf13164eb841f3f791f20 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Thu, 15 Apr 2021 07:58:33 +0200 Subject: [PATCH 311/512] TEZ-4279: Add vertexId into vertex status for dag clients (#101) (Laszlo Bodor reviewed by Ashutosh Chauhan) --- .../apache/tez/dag/api/DagTypeConverters.java | 19 ++++++++----------- .../tez/dag/api/client/VertexStatus.java | 4 ++++ tez-api/src/main/proto/DAGApiRecords.proto | 9 +++++---- .../tez/dag/api/TestDagTypeConverters.java | 19 +++++++++++++++++++ .../tez/dag/api/client/rpc/TestDAGClient.java | 1 + .../dag/api/client/VertexStatusBuilder.java | 5 +++++ .../tez/dag/app/dag/impl/VertexImpl.java | 1 + 7 files changed, 43 insertions(+), 15 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java index acc5f12933..5a2cb64cc3 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java @@ -624,23 +624,20 @@ public static TezCountersProto convertTezCountersToProto( } public static DAGProtos.StatusGetOptsProto convertStatusGetOptsToProto( - StatusGetOpts statusGetOpts) { + StatusGetOpts statusGetOpts) { switch (statusGetOpts) { - case GET_COUNTERS: - return DAGProtos.StatusGetOptsProto.GET_COUNTERS; + case GET_COUNTERS: + return DAGProtos.StatusGetOptsProto.GET_COUNTERS; } - throw new TezUncheckedException("Could not convert StatusGetOpts to" - + " proto"); + throw new TezUncheckedException("Could not convert StatusGetOpts to" + " proto"); } - public static StatusGetOpts convertStatusGetOptsFromProto( - DAGProtos.StatusGetOptsProto proto) { + public static StatusGetOpts convertStatusGetOptsFromProto(DAGProtos.StatusGetOptsProto proto) { switch (proto) { - case GET_COUNTERS: - return StatusGetOpts.GET_COUNTERS; + case GET_COUNTERS: + return StatusGetOpts.GET_COUNTERS; } - throw new TezUncheckedException("Could not convert to StatusGetOpts from" - + " proto"); + throw new TezUncheckedException("Could not convert to StatusGetOpts from" + " proto"); } public static List convertStatusGetOptsToProto( diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/VertexStatus.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/VertexStatus.java index dfb9bbe8cd..9efb12d612 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/VertexStatus.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/VertexStatus.java @@ -59,6 +59,10 @@ public VertexStatus(VertexStatusProtoOrBuilder proxy) { this.proxy = proxy; } + public String getId() { + return proxy.getId(); + } + public State getState() { return getState(proxy.getState()); } diff --git a/tez-api/src/main/proto/DAGApiRecords.proto b/tez-api/src/main/proto/DAGApiRecords.proto index 34c369d430..4c8c7f63a9 100644 --- a/tez-api/src/main/proto/DAGApiRecords.proto +++ b/tez-api/src/main/proto/DAGApiRecords.proto @@ -245,10 +245,11 @@ enum VertexStatusStateProto { } message VertexStatusProto { - optional VertexStatusStateProto state = 1; - repeated string diagnostics = 2; - optional ProgressProto progress = 3; - optional TezCountersProto vertexCounters = 4; + required string id = 1; + optional VertexStatusStateProto state = 2; + repeated string diagnostics = 3; + optional ProgressProto progress = 4; + optional TezCountersProto vertexCounters = 5; } enum DAGStatusStateProto { diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/TestDagTypeConverters.java b/tez-api/src/test/java/org/apache/tez/dag/api/TestDagTypeConverters.java index 265fce9d43..edb7fd8445 100644 --- a/tez-api/src/test/java/org/apache/tez/dag/api/TestDagTypeConverters.java +++ b/tez-api/src/test/java/org/apache/tez/dag/api/TestDagTypeConverters.java @@ -34,8 +34,10 @@ import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.security.DAGAccessControls; import org.apache.tez.dag.api.Vertex.VertexExecutionContext; +import org.apache.tez.dag.api.client.StatusGetOpts; import org.apache.tez.dag.api.records.DAGProtos.ACLInfo; import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto; +import org.apache.tez.dag.api.records.DAGProtos.StatusGetOptsProto; import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto; import org.apache.tez.dag.api.records.DAGProtos.TezNamedEntityDescriptorProto; import org.apache.tez.dag.api.records.DAGProtos.VertexExecutionContextProto; @@ -235,6 +237,23 @@ public void testAclConversions() { assertSame(DagTypeConverters.convertDAGAccessControlsFromProto(aclInfo), aclInfo); } + /* + * This unit test can catch if a StatusGetOpts <-> StatusGetOptsProto value is not defined at any + * side. + */ + @Test + public void testConvertStatusGetOptsToProtoCoverage() { + StatusGetOpts[] opts = StatusGetOpts.values(); + for (StatusGetOpts opt : opts) { + DagTypeConverters.convertStatusGetOptsToProto(opt); + } + + StatusGetOptsProto[] optProtos = StatusGetOptsProto.values(); + for (StatusGetOptsProto proto : optProtos) { + DagTypeConverters.convertStatusGetOptsFromProto(proto); + } + } + private void assertSame(DAGAccessControls dagAccessControls, ACLInfo aclInfo) { assertEquals(dagAccessControls.getUsersWithViewACLs(), Sets.newHashSet(aclInfo.getUsersWithViewAccessList())); diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java index 6a5e817d44..50c9a6061c 100644 --- a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java +++ b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java @@ -152,6 +152,7 @@ private void setUpData(){ .build(); vertexStatusProtoWithoutCounters = VertexStatusProto.newBuilder() + .setId("vertex_1") .addDiagnostics("V_Diagnostics_0") .setProgress(vertexProgressProto) .setState(VertexStatusStateProto.VERTEX_SUCCEEDED) // make sure the waitForCompletion be able to finish diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/VertexStatusBuilder.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/VertexStatusBuilder.java index 4de321cf2c..0304fc9d6f 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/VertexStatusBuilder.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/VertexStatusBuilder.java @@ -28,6 +28,7 @@ import org.apache.tez.dag.api.records.DAGProtos.VertexStatusStateProto; import org.apache.tez.dag.api.TezUncheckedException; import org.apache.tez.dag.app.dag.VertexState; +import org.apache.tez.dag.records.TezVertexID; public class VertexStatusBuilder extends VertexStatus { @@ -35,6 +36,10 @@ public VertexStatusBuilder() { super(VertexStatusProto.newBuilder()); } + public void setId(TezVertexID vertexId) { + getBuilder().setId(vertexId.toString()); + } + public void setState(VertexState state) { getBuilder().setState(getProtoState(state)); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index e21add0e3b..452dae5366 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -1536,6 +1536,7 @@ public VertexStatusBuilder getVertexStatus( this.readLock.lock(); try { VertexStatusBuilder status = new VertexStatusBuilder(); + status.setId(getVertexId()); status.setState(getInternalState()); status.setDiagnostics(diagnostics); status.setProgress(getVertexProgress()); From 01847814d32aac6b77a6810ea25d6780fb94b257 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Fri, 23 Apr 2021 11:43:31 +0200 Subject: [PATCH 312/512] TEZ-4157: ShuffleHandler: upgrade to Netty4 and remove Netty3 dependency from tez (#118) (Laszlo Bodor reviewed by Ashutosh Chauhan, Jonathan Turner Eagles) --- pom.xml | 36 +- tez-ext-service-tests/pom.xml | 2 +- .../tez/shufflehandler/ShuffleHandler.java | 269 ++++++------- tez-plugins/tez-aux-services/pom.xml | 4 +- .../tez/auxservices/FadvisedChunkedFile.java | 16 +- .../tez/auxservices/FadvisedFileRegion.java | 55 ++- .../tez/auxservices/ShuffleHandler.java | 380 +++++++++--------- .../tez/auxservices/TestShuffleHandler.java | 177 ++++---- 8 files changed, 479 insertions(+), 460 deletions(-) diff --git a/pom.xml b/pom.xml index 19fd5167cd..c1f0682df1 100644 --- a/pom.xml +++ b/pom.xml @@ -59,7 +59,7 @@ ${user.home}/clover.license 27.0-jre 3.1.3 - 3.10.5.Final + 4.0.52.Final 0.13.0 1.19 1.7.30 @@ -261,7 +261,7 @@ io.netty - netty + netty-all compile ${netty.version} @@ -340,12 +340,22 @@ commons-el commons-el + + io.netty + netty + org.apache.hadoop hadoop-auth ${hadoop.version} + + + io.netty + netty + + org.apache.hadoop @@ -562,6 +572,10 @@ org.apache.hadoop hadoop-yarn-server-common + + io.netty + netty + @@ -578,6 +592,10 @@ org.apache.hadoop hadoop-mapreduce-client-shuffle + + io.netty + netty + @@ -603,6 +621,10 @@ org.apache.hadoop hadoop-mapreduce-client-common + + io.netty + netty + @@ -611,6 +633,12 @@ test test-jar ${hadoop.version} + + + io.netty + netty + + org.apache.hadoop @@ -649,6 +677,10 @@ tomcat jasper-runtime + + io.netty + netty + diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml index c806bf7124..8dfad0d74e 100644 --- a/tez-ext-service-tests/pom.xml +++ b/tez-ext-service-tests/pom.xml @@ -32,7 +32,7 @@ io.netty - netty + netty-all org.slf4j diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java index 1d122be025..43f24ba654 100644 --- a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java +++ b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java @@ -14,17 +14,17 @@ package org.apache.tez.shufflehandler; -import static org.jboss.netty.buffer.ChannelBuffers.wrappedBuffer; -import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE; -import static org.jboss.netty.handler.codec.http.HttpMethod.GET; -import static org.jboss.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST; -import static org.jboss.netty.handler.codec.http.HttpResponseStatus.FORBIDDEN; -import static org.jboss.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR; -import static org.jboss.netty.handler.codec.http.HttpResponseStatus.METHOD_NOT_ALLOWED; -import static org.jboss.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND; -import static org.jboss.netty.handler.codec.http.HttpResponseStatus.OK; -import static org.jboss.netty.handler.codec.http.HttpResponseStatus.UNAUTHORIZED; -import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1; +import static io.netty.buffer.Unpooled.wrappedBuffer; +import static io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE; +import static io.netty.handler.codec.http.HttpMethod.GET; +import static io.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST; +import static io.netty.handler.codec.http.HttpResponseStatus.FORBIDDEN; +import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR; +import static io.netty.handler.codec.http.HttpResponseStatus.METHOD_NOT_ALLOWED; +import static io.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND; +import static io.netty.handler.codec.http.HttpResponseStatus.OK; +import static io.netty.handler.codec.http.HttpResponseStatus.UNAUTHORIZED; +import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1; import javax.crypto.SecretKey; import java.io.File; @@ -41,15 +41,15 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.Executors; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.regex.Pattern; import com.google.common.base.Charsets; + import org.apache.tez.common.Preconditions; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.Path; @@ -64,36 +64,38 @@ import org.apache.tez.runtime.library.common.security.SecureShuffleUtils; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.ShuffleHeader; import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord; -import org.jboss.netty.bootstrap.ServerBootstrap; -import org.jboss.netty.buffer.ChannelBuffers; -import org.jboss.netty.channel.Channel; -import org.jboss.netty.channel.ChannelFactory; -import org.jboss.netty.channel.ChannelFuture; -import org.jboss.netty.channel.ChannelFutureListener; -import org.jboss.netty.channel.ChannelHandlerContext; -import org.jboss.netty.channel.ChannelPipeline; -import org.jboss.netty.channel.ChannelPipelineFactory; -import org.jboss.netty.channel.ChannelStateEvent; -import org.jboss.netty.channel.Channels; -import org.jboss.netty.channel.DefaultFileRegion; -import org.jboss.netty.channel.ExceptionEvent; -import org.jboss.netty.channel.MessageEvent; -import org.jboss.netty.channel.SimpleChannelUpstreamHandler; -import org.jboss.netty.channel.group.ChannelGroup; -import org.jboss.netty.channel.group.DefaultChannelGroup; -import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory; -import org.jboss.netty.handler.codec.frame.TooLongFrameException; -import org.jboss.netty.handler.codec.http.DefaultHttpResponse; -import org.jboss.netty.handler.codec.http.HttpChunkAggregator; -import org.jboss.netty.handler.codec.http.HttpHeaders; -import org.jboss.netty.handler.codec.http.HttpRequest; -import org.jboss.netty.handler.codec.http.HttpRequestDecoder; -import org.jboss.netty.handler.codec.http.HttpResponse; -import org.jboss.netty.handler.codec.http.HttpResponseEncoder; -import org.jboss.netty.handler.codec.http.HttpResponseStatus; -import org.jboss.netty.handler.codec.http.QueryStringDecoder; -import org.jboss.netty.handler.stream.ChunkedWriteHandler; -import org.jboss.netty.util.CharsetUtil; +import io.netty.bootstrap.ServerBootstrap; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelFutureListener; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelPipeline; +import io.netty.channel.DefaultFileRegion; +import io.netty.channel.ChannelHandler.Sharable; +import io.netty.channel.group.ChannelGroup; +import io.netty.channel.group.DefaultChannelGroup; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.nio.NioServerSocketChannel; +import io.netty.channel.socket.nio.NioSocketChannel; +import io.netty.handler.codec.TooLongFrameException; +import io.netty.handler.codec.http.DefaultFullHttpResponse; +import io.netty.handler.codec.http.DefaultHttpResponse; +import io.netty.handler.codec.http.FullHttpResponse; +import io.netty.handler.codec.http.HttpHeaders; +import io.netty.handler.codec.http.HttpObjectAggregator; +import io.netty.handler.codec.http.HttpRequest; +import io.netty.handler.codec.http.HttpRequestDecoder; +import io.netty.handler.codec.http.HttpResponse; +import io.netty.handler.codec.http.HttpResponseEncoder; +import io.netty.handler.codec.http.HttpResponseStatus; +import io.netty.handler.codec.http.QueryStringDecoder; +import io.netty.handler.stream.ChunkedWriteHandler; +import io.netty.util.CharsetUtil; +import io.netty.util.concurrent.GlobalEventExecutor; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -110,9 +112,13 @@ public class ShuffleHandler { Pattern.CASE_INSENSITIVE); private int port; - private final ChannelFactory selector; - private final ChannelGroup accepted = new DefaultChannelGroup(); - protected HttpPipelineFactory pipelineFact; + + // pipeline items + private Shuffle SHUFFLE; + + private NioEventLoopGroup bossGroup; + private NioEventLoopGroup workerGroup; + private final ChannelGroup accepted = new DefaultChannelGroup(GlobalEventExecutor.INSTANCE); private final Configuration conf; private final ConcurrentMap registeredApps = new ConcurrentHashMap(); @@ -171,17 +177,23 @@ public ShuffleHandler(Configuration conf) { maxShuffleThreads = 2 * Runtime.getRuntime().availableProcessors(); } - ThreadFactory bossFactory = new ThreadFactoryBuilder() - .setNameFormat("ShuffleHandler Netty Boss #%d") - .build(); - ThreadFactory workerFactory = new ThreadFactoryBuilder() - .setNameFormat("ShuffleHandler Netty Worker #%d") - .build(); - - selector = new NioServerSocketChannelFactory( - Executors.newCachedThreadPool(bossFactory), - Executors.newCachedThreadPool(workerFactory), - maxShuffleThreads); + final String BOSS_THREAD_NAME_PREFIX = "ShuffleHandler Netty Boss #"; + AtomicInteger bossThreadCounter = new AtomicInteger(0); + bossGroup = new NioEventLoopGroup(maxShuffleThreads, new ThreadFactory() { + @Override + public Thread newThread(Runnable r) { + return new Thread(r, BOSS_THREAD_NAME_PREFIX + bossThreadCounter.incrementAndGet()); + } + }); + + final String WORKER_THREAD_NAME_PREFIX = "ShuffleHandler Netty Worker #"; + AtomicInteger workerThreadCounter = new AtomicInteger(0); + workerGroup = new NioEventLoopGroup(maxShuffleThreads, new ThreadFactory() { + @Override + public Thread newThread(Runnable r) { + return new Thread(r, WORKER_THREAD_NAME_PREFIX + workerThreadCounter.incrementAndGet()); + } + }); connectionKeepAliveEnabled = conf.getBoolean(SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, @@ -199,22 +211,44 @@ public ShuffleHandler(Configuration conf) { public void start() throws Exception { - ServerBootstrap bootstrap = new ServerBootstrap(selector); - try { - pipelineFact = new HttpPipelineFactory(conf); - } catch (Exception ex) { - throw new RuntimeException(ex); - } - bootstrap.setPipelineFactory(pipelineFact); + ServerBootstrap bootstrap = new ServerBootstrap() + .channel(NioServerSocketChannel.class) + .group(bossGroup, workerGroup) + .localAddress(port); + initPipeline(bootstrap, conf); port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT); - Channel ch = bootstrap.bind(new InetSocketAddress(port)); + Channel ch = bootstrap.bind().sync().channel(); accepted.add(ch); - port = ((InetSocketAddress)ch.getLocalAddress()).getPort(); + port = ((InetSocketAddress)ch.localAddress()).getPort(); conf.set(SHUFFLE_PORT_CONFIG_KEY, Integer.toString(port)); - pipelineFact.SHUFFLE.setPort(port); + SHUFFLE.setPort(port); LOG.info("TezShuffleHandler" + " listening on port " + port); } + private void initPipeline(ServerBootstrap bootstrap, Configuration conf) throws Exception { + SHUFFLE = getShuffle(conf); + + if (conf.getBoolean(MRConfig.SHUFFLE_SSL_ENABLED_KEY, + MRConfig.SHUFFLE_SSL_ENABLED_DEFAULT)) { + throw new UnsupportedOperationException( + "SSL Shuffle is not currently supported for the test shuffle handler"); + } + + ChannelInitializer channelInitializer = + new ChannelInitializer() { + @Override + public void initChannel(NioSocketChannel ch) throws Exception { + ChannelPipeline pipeline = ch.pipeline(); + pipeline.addLast("decoder", new HttpRequestDecoder()); + pipeline.addLast("aggregator", new HttpObjectAggregator(1 << 16)); + pipeline.addLast("encoder", new HttpResponseEncoder()); + pipeline.addLast("chunking", new ChunkedWriteHandler()); + pipeline.addLast("shuffle", SHUFFLE); + } + }; + bootstrap.childHandler(channelInitializer); + } + public static void initializeAndStart(Configuration conf) throws Exception { if (!initing.getAndSet(true)) { INSTANCE = new ShuffleHandler(conf); @@ -245,15 +279,13 @@ public void unregisterApplication(String applicationIdString) { removeJobShuffleInfo(applicationIdString); } - public void stop() throws Exception { accepted.close().awaitUninterruptibly(10, TimeUnit.SECONDS); - if (selector != null) { - ServerBootstrap bootstrap = new ServerBootstrap(selector); - bootstrap.releaseExternalResources(); + if (bossGroup != null) { + bossGroup.shutdownGracefully(); } - if (pipelineFact != null) { - pipelineFact.destroy(); + if (workerGroup != null) { + workerGroup.shutdownGracefully(); } } @@ -261,7 +293,6 @@ protected Shuffle getShuffle(Configuration conf) { return new Shuffle(conf); } - private void addJobToken(String appIdString, String user, Token jobToken) { String jobIdString = appIdString.replace("application", "job"); @@ -280,40 +311,8 @@ private void removeJobShuffleInfo(String appIdString) { userRsrc.remove(appIdString); } - class HttpPipelineFactory implements ChannelPipelineFactory { - - final Shuffle SHUFFLE; - - public HttpPipelineFactory(Configuration conf) throws Exception { - SHUFFLE = getShuffle(conf); - // TODO Setup SSL Shuffle - if (conf.getBoolean(MRConfig.SHUFFLE_SSL_ENABLED_KEY, - MRConfig.SHUFFLE_SSL_ENABLED_DEFAULT)) { - throw new UnsupportedOperationException( - "SSL Shuffle is not currently supported for the test shuffle handler"); - } - } - - public void destroy() { - } - - @Override - public ChannelPipeline getPipeline() throws Exception { - ChannelPipeline pipeline = Channels.pipeline(); - pipeline.addLast("decoder", new HttpRequestDecoder()); - pipeline.addLast("aggregator", new HttpChunkAggregator(1 << 16)); - pipeline.addLast("encoder", new HttpResponseEncoder()); - pipeline.addLast("chunking", new ChunkedWriteHandler()); - pipeline.addLast("shuffle", SHUFFLE); - return pipeline; - // TODO factor security manager into pipeline - // TODO factor out encode/decode to permit binary shuffle - // TODO factor out decode of index to permit alt. models - } - - } - - class Shuffle extends SimpleChannelUpstreamHandler { + @Sharable + class Shuffle extends ChannelInboundHandlerAdapter { private final Configuration conf; private final IndexCache indexCache; @@ -343,24 +342,24 @@ private List splitMaps(List mapq) { } @Override - public void channelOpen(ChannelHandlerContext ctx, ChannelStateEvent evt) + public void channelActive(ChannelHandlerContext ctx) throws Exception { + if ((maxShuffleConnections > 0) && (accepted.size() >= maxShuffleConnections)) { - LOG.info(String.format("Current number of shuffle connections (%d) is " + - "greater than or equal to the max allowed shuffle connections (%d)", + LOG.info(String.format("Current number of shuffle connections (%d) is " + + "greater than or equal to the max allowed shuffle connections (%d)", accepted.size(), maxShuffleConnections)); - evt.getChannel().close(); + ctx.channel().close(); return; } - accepted.add(evt.getChannel()); - super.channelOpen(ctx, evt); - + accepted.add(ctx.channel()); + super.channelActive(ctx); } @Override - public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) + public void channelRead(ChannelHandlerContext ctx, Object message) throws Exception { - HttpRequest request = (HttpRequest) evt.getMessage(); + HttpRequest request = (HttpRequest) message; if (request.getMethod() != GET) { sendError(ctx, METHOD_NOT_ALLOWED); return; @@ -372,8 +371,7 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) request.headers().get(ShuffleHeader.HTTP_HEADER_VERSION))) { sendError(ctx, "Incompatible shuffle request version", BAD_REQUEST); } - final Map> q = - new QueryStringDecoder(request.getUri()).getParameters(); + final Map> q = new QueryStringDecoder(request.getUri()).parameters(); final List keepAliveList = q.get("keepAlive"); boolean keepAliveParam = false; if (keepAliveList != null && keepAliveList.size() == 1) { @@ -432,7 +430,7 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) Map mapOutputInfoMap = new HashMap(); - Channel ch = evt.getChannel(); + Channel ch = ctx.channel(); String user = userRsrc.get(jobId); // $x/$user/appcache/$appId/output/$mapId @@ -444,13 +442,13 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) populateHeaders(mapIds, outputBasePathStr, user, reduceId, request, response, keepAliveParam, mapOutputInfoMap); } catch(IOException e) { - ch.write(response); + ch.writeAndFlush(response); LOG.error("Shuffle error in populating headers :", e); String errorMessage = getErrorMessage(e); sendError(ctx,errorMessage , INTERNAL_SERVER_ERROR); return; } - ch.write(response); + ch.writeAndFlush(response); // TODO refactor the following into the pipeline ChannelFuture lastMap = null; for (String mapId : mapIds) { @@ -621,7 +619,7 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch, new ShuffleHeader(mapId, info.getPartLength(), info.getRawLength(), reduce); final DataOutputBuffer dob = new DataOutputBuffer(); header.write(dob); - ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength())); + ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); final File spillfile = new File(mapOutputInfo.mapOutputFileName.toString()); RandomAccessFile spill; @@ -634,15 +632,7 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch, ChannelFuture writeFuture; final DefaultFileRegion partition = new DefaultFileRegion(spill.getChannel(), info.getStartOffset(), info.getPartLength()); - writeFuture = ch.write(partition); - writeFuture.addListener(new ChannelFutureListener() { - // TODO error handling; distinguish IO/connection failures, - // attribute to appropriate spill output - @Override - public void operationComplete(ChannelFuture future) { - partition.releaseExternalResources(); - } - }); + writeFuture = ch.writeAndFlush(partition); return writeFuture; } @@ -653,25 +643,22 @@ protected void sendError(ChannelHandlerContext ctx, protected void sendError(ChannelHandlerContext ctx, String message, HttpResponseStatus status) { - HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status); + FullHttpResponse response = new DefaultFullHttpResponse(HTTP_1_1, status); response.headers().set(CONTENT_TYPE, "text/plain; charset=UTF-8"); // Put shuffle version into http header response.headers().set(ShuffleHeader.HTTP_HEADER_NAME, ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); response.headers().set(ShuffleHeader.HTTP_HEADER_VERSION, ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); - response.setContent( - ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8)); + response.content().writeBytes(Unpooled.copiedBuffer(message, CharsetUtil.UTF_8)); // Close the connection as soon as the error message is sent. - ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE); + ctx.channel().writeAndFlush(response).addListener(ChannelFutureListener.CLOSE); } @Override - public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { - Channel ch = e.getChannel(); - Throwable cause = e.getCause(); if (cause instanceof TooLongFrameException) { sendError(ctx, BAD_REQUEST); return; @@ -688,8 +675,8 @@ public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) } LOG.error("Shuffle error: ", cause); - if (ch.isConnected()) { - LOG.error("Shuffle error " + e); + if (ctx.channel().isActive()) { + LOG.error("Shuffle error", cause); sendError(ctx, INTERNAL_SERVER_ERROR); } } diff --git a/tez-plugins/tez-aux-services/pom.xml b/tez-plugins/tez-aux-services/pom.xml index 97096c8332..7279eaf684 100644 --- a/tez-plugins/tez-aux-services/pom.xml +++ b/tez-plugins/tez-aux-services/pom.xml @@ -112,7 +112,7 @@ io.netty - netty + netty-all com.google.guava @@ -237,7 +237,7 @@ org.apache.tez.shaded.$0 - org.jboss.netty + io.netty org.apache.tez.shaded.$0 diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/FadvisedChunkedFile.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/FadvisedChunkedFile.java index cc3f762f9d..162feb9801 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/FadvisedChunkedFile.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/FadvisedChunkedFile.java @@ -31,7 +31,9 @@ import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.POSIX_FADV_DONTNEED; -import org.jboss.netty.handler.stream.ChunkedFile; +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.stream.ChunkedFile; public class FadvisedChunkedFile extends ChunkedFile { @@ -57,13 +59,13 @@ public FadvisedChunkedFile(RandomAccessFile file, long position, long count, } @Override - public Object nextChunk() throws Exception { + public ByteBuf readChunk(ChannelHandlerContext ctx) throws Exception { if (manageOsCache && readaheadPool != null) { readaheadRequest = readaheadPool - .readaheadStream(identifier, fd, getCurrentOffset(), readaheadLength, - getEndOffset(), readaheadRequest); + .readaheadStream(identifier, fd, currentOffset(), readaheadLength, + endOffset(), readaheadRequest); } - return super.nextChunk(); + return super.readChunk(ctx); } @Override @@ -71,11 +73,11 @@ public void close() throws Exception { if (readaheadRequest != null) { readaheadRequest.cancel(); } - if (manageOsCache && getEndOffset() - getStartOffset() > 0) { + if (manageOsCache && endOffset() - startOffset() > 0) { try { NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(identifier, fd, - getStartOffset(), getEndOffset() - getStartOffset(), + startOffset(), endOffset() - startOffset(), POSIX_FADV_DONTNEED); } catch (Throwable t) { LOG.warn("Failed to manage OS cache for " + identifier, t); diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/FadvisedFileRegion.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/FadvisedFileRegion.java index 40789d806d..23663636e5 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/FadvisedFileRegion.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/FadvisedFileRegion.java @@ -34,7 +34,7 @@ import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.POSIX_FADV_DONTNEED; -import org.jboss.netty.channel.DefaultFileRegion; +import io.netty.channel.DefaultFileRegion; import com.google.common.annotations.VisibleForTesting; @@ -54,6 +54,7 @@ public class FadvisedFileRegion extends DefaultFileRegion { private final FileChannel fileChannel; private ReadaheadRequest readaheadRequest; + private boolean transferred = false; public FadvisedFileRegion(RandomAccessFile file, long position, long count, boolean manageOsCache, int readaheadLength, ReadaheadPool readaheadPool, @@ -77,15 +78,40 @@ public long transferTo(WritableByteChannel target, long position) throws IOException { if (readaheadPool != null && readaheadLength > 0) { readaheadRequest = readaheadPool.readaheadStream(identifier, fd, - getPosition() + position, readaheadLength, - getPosition() + getCount(), readaheadRequest); + position() + position, readaheadLength, + position() + count(), readaheadRequest); } + long written = 0; if(this.shuffleTransferToAllowed) { - return super.transferTo(target, position); + written = super.transferTo(target, position); } else { - return customShuffleTransfer(target, position); + written = customShuffleTransfer(target, position); } + /* + * At this point, we can assume that the transfer was successful. + */ + transferred = true; + return written; + } + + /** + * Since Netty4, deallocate() is called automatically during cleanup, but before the + * ChannelFutureListeners. Deallocate calls FileChannel.close() and makes the file descriptor + * invalid, so every OS cache operation (e.g. posix_fadvice) with the original file descriptor + * will fail after this operation, so we need to take care of cleanup operations here (before + * deallocating) instead of listeners outside. + */ + @Override + protected void deallocate() { + if (readaheadRequest != null) { + readaheadRequest.cancel(); + } + + if (transferred) { + transferSuccessful(); + } + super.deallocate(); } /** @@ -142,24 +168,19 @@ long customShuffleTransfer(WritableByteChannel target, long position) return actualCount - trans; } - - @Override - public void releaseExternalResources() { - if (readaheadRequest != null) { - readaheadRequest.cancel(); - } - super.releaseExternalResources(); - } - /** * Call when the transfer completes successfully so we can advise the OS that * we don't need the region to be cached anymore. */ public void transferSuccessful() { - if (manageOsCache && getCount() > 0) { + if (manageOsCache && count() > 0) { try { - NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(identifier, - fd, getPosition(), getCount(), POSIX_FADV_DONTNEED); + if (fd.valid()) { + NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(identifier, fd, position(), + count(), POSIX_FADV_DONTNEED); + } else { + LOG.debug("File descriptor is not valid anymore, skipping posix_fadvise: " + identifier); + } } catch (Throwable t) { LOG.warn("Failed to manage OS cache for " + identifier, t); } diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java index 55389ea78e..e73805a1f0 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java @@ -21,17 +21,17 @@ import org.apache.hadoop.util.DiskChecker; import static org.fusesource.leveldbjni.JniDBFactory.asString; import static org.fusesource.leveldbjni.JniDBFactory.bytes; -import static org.jboss.netty.buffer.ChannelBuffers.wrappedBuffer; -import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE; -import static org.jboss.netty.handler.codec.http.HttpMethod.GET; -import static org.jboss.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST; -import static org.jboss.netty.handler.codec.http.HttpResponseStatus.FORBIDDEN; -import static org.jboss.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR; -import static org.jboss.netty.handler.codec.http.HttpResponseStatus.METHOD_NOT_ALLOWED; -import static org.jboss.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND; -import static org.jboss.netty.handler.codec.http.HttpResponseStatus.OK; -import static org.jboss.netty.handler.codec.http.HttpResponseStatus.UNAUTHORIZED; -import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1; +import static io.netty.buffer.Unpooled.wrappedBuffer; +import static io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE; +import static io.netty.handler.codec.http.HttpMethod.GET; +import static io.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST; +import static io.netty.handler.codec.http.HttpResponseStatus.FORBIDDEN; +import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR; +import static io.netty.handler.codec.http.HttpResponseStatus.METHOD_NOT_ALLOWED; +import static io.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND; +import static io.netty.handler.codec.http.HttpResponseStatus.OK; +import static io.netty.handler.codec.http.HttpResponseStatus.UNAUTHORIZED; +import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1; import java.io.File; import java.io.FileNotFoundException; @@ -48,7 +48,7 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.regex.Pattern; @@ -102,47 +102,47 @@ import org.iq80.leveldb.DBException; import org.iq80.leveldb.Logger; import org.iq80.leveldb.Options; -import org.jboss.netty.bootstrap.ServerBootstrap; -import org.jboss.netty.buffer.ChannelBuffer; -import org.jboss.netty.buffer.ChannelBuffers; -import org.jboss.netty.channel.Channel; -import org.jboss.netty.channel.ChannelFactory; -import org.jboss.netty.channel.ChannelFuture; -import org.jboss.netty.channel.ChannelFutureListener; -import org.jboss.netty.channel.ChannelHandler; -import org.jboss.netty.channel.ChannelHandlerContext; -import org.jboss.netty.channel.ChannelPipeline; -import org.jboss.netty.channel.ChannelPipelineFactory; -import org.jboss.netty.channel.ChannelStateEvent; -import org.jboss.netty.channel.Channels; -import org.jboss.netty.channel.ExceptionEvent; -import org.jboss.netty.channel.MessageEvent; -import org.jboss.netty.channel.SimpleChannelUpstreamHandler; -import org.jboss.netty.channel.group.ChannelGroup; -import org.jboss.netty.channel.group.DefaultChannelGroup; -import org.jboss.netty.channel.socket.nio.NioServerBossPool; -import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory; -import org.jboss.netty.channel.socket.nio.NioWorkerPool; -import org.jboss.netty.handler.codec.frame.TooLongFrameException; -import org.jboss.netty.handler.codec.http.DefaultHttpResponse; -import org.jboss.netty.handler.codec.http.HttpChunkAggregator; -import org.jboss.netty.handler.codec.http.HttpHeaders; -import org.jboss.netty.handler.codec.http.HttpRequest; -import org.jboss.netty.handler.codec.http.HttpRequestDecoder; -import org.jboss.netty.handler.codec.http.HttpResponse; -import org.jboss.netty.handler.codec.http.HttpResponseEncoder; -import org.jboss.netty.handler.codec.http.HttpResponseStatus; -import org.jboss.netty.handler.codec.http.QueryStringDecoder; -import org.jboss.netty.handler.ssl.SslHandler; -import org.jboss.netty.handler.stream.ChunkedWriteHandler; -import org.jboss.netty.handler.timeout.IdleState; -import org.jboss.netty.handler.timeout.IdleStateAwareChannelHandler; -import org.jboss.netty.handler.timeout.IdleStateEvent; -import org.jboss.netty.handler.timeout.IdleStateHandler; -import org.jboss.netty.util.CharsetUtil; -import org.jboss.netty.util.HashedWheelTimer; -import org.jboss.netty.util.ThreadNameDeterminer; -import org.jboss.netty.util.Timer; + +import io.netty.bootstrap.ServerBootstrap; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import io.netty.channel.ChannelHandler.Sharable; +import io.netty.channel.ChannelDuplexHandler; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelFutureListener; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelOption; +import io.netty.channel.ChannelPipeline; +import io.netty.channel.group.ChannelGroup; +import io.netty.channel.group.DefaultChannelGroup; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.nio.NioServerSocketChannel; +import io.netty.channel.socket.nio.NioSocketChannel; +import io.netty.handler.codec.TooLongFrameException; +import io.netty.handler.codec.http.DefaultFullHttpResponse; +import io.netty.handler.codec.http.DefaultHttpResponse; +import io.netty.handler.codec.http.FullHttpRequest; +import io.netty.handler.codec.http.FullHttpResponse; +import io.netty.handler.codec.http.HttpHeaders; +import io.netty.handler.codec.http.HttpObjectAggregator; +import io.netty.handler.codec.http.HttpRequest; +import io.netty.handler.codec.http.HttpRequestDecoder; +import io.netty.handler.codec.http.HttpResponse; +import io.netty.handler.codec.http.HttpResponseEncoder; +import io.netty.handler.codec.http.HttpResponseStatus; +import io.netty.handler.codec.http.LastHttpContent; +import io.netty.handler.codec.http.QueryStringDecoder; +import io.netty.handler.ssl.SslHandler; +import io.netty.handler.stream.ChunkedWriteHandler; +import io.netty.handler.timeout.IdleState; +import io.netty.handler.timeout.IdleStateEvent; +import io.netty.handler.timeout.IdleStateHandler; +import io.netty.util.CharsetUtil; +import io.netty.util.concurrent.GlobalEventExecutor; + import org.slf4j.LoggerFactory; import com.google.common.annotations.VisibleForTesting; @@ -183,11 +183,15 @@ public class ShuffleHandler extends AuxiliaryService { private static final String INDEX_FILE_NAME = "file.out.index"; private int port; - private ChannelFactory selector; - private final ChannelGroup accepted = new DefaultChannelGroup(); - protected HttpPipelineFactory pipelineFact; + private NioEventLoopGroup bossGroup; + private NioEventLoopGroup workerGroup; + private final ChannelGroup accepted = new DefaultChannelGroup(GlobalEventExecutor.INSTANCE); private int sslFileBufferSize; + // pipeline items + private Shuffle SHUFFLE; + private SSLFactory sslFactory; + /** * Should the shuffle use posix_fadvise calls to manage the OS cache during * sendfile @@ -263,7 +267,6 @@ public class ShuffleHandler extends AuxiliaryService { boolean connectionKeepAliveEnabled = false; private int connectionKeepAliveTimeOut; private int mapOutputMetaInfoCacheSize; - private Timer timer; @Metrics(about="Shuffle output metrics", context="mapred", name="tez") static class ShuffleMetrics implements ChannelFutureListener { @@ -300,7 +303,7 @@ public ReduceMapFileCount(ReduceContext rc) { @Override public void operationComplete(ChannelFuture future) throws Exception { if (!future.isSuccess()) { - future.getChannel().close(); + future.channel().close(); return; } int waitCount = this.reduceContext.getMapsToWait().decrementAndGet(); @@ -308,21 +311,21 @@ public void operationComplete(ChannelFuture future) throws Exception { metrics.operationComplete(future); // Let the idle timer handler close keep-alive connections if (reduceContext.getKeepAlive()) { - ChannelPipeline pipeline = future.getChannel().getPipeline(); + ChannelPipeline pipeline = future.channel().pipeline(); TimeoutHandler timeoutHandler = (TimeoutHandler) pipeline.get(TIMEOUT_HANDLER); timeoutHandler.setEnabledTimeout(true); } else { - future.getChannel().close(); + future.channel().close(); } } else { - pipelineFact.getSHUFFLE().sendMap(reduceContext); + SHUFFLE.sendMap(reduceContext); } } } /** - * Maintain parameters per messageReceived() Netty context. + * Maintain parameters per channelRead() Netty context. * Allows sendMapOutput calls from operationComplete() */ private static class ReduceContext { @@ -419,9 +422,11 @@ public ShuffleHandler() { */ public static ByteBuffer serializeMetaData(int port) throws IOException { //TODO these bytes should be versioned - DataOutputBuffer port_dob = new DataOutputBuffer(); - port_dob.writeInt(port); - return ByteBuffer.wrap(port_dob.getData(), 0, port_dob.getLength()); + DataOutputBuffer portDob = new DataOutputBuffer(); + portDob.writeInt(port); + ByteBuffer buf = ByteBuffer.wrap(portDob.getData(), 0, portDob.getLength()); + portDob.close(); + return buf; } /** @@ -434,6 +439,7 @@ public static int deserializeMetaData(ByteBuffer meta) throws IOException { DataInputByteBuffer in = new DataInputByteBuffer(); in.reset(meta); int port = in.readInt(); + in.close(); return port; } @@ -516,22 +522,23 @@ protected void serviceInit(Configuration conf) throws Exception { DEFAULT_SHUFFLE_MAX_SESSION_OPEN_FILES); final String BOSS_THREAD_NAME_PREFIX = "Tez Shuffle Handler Boss #"; - NioServerBossPool bossPool = new NioServerBossPool(Executors.newCachedThreadPool(), 1, new ThreadNameDeterminer() { + AtomicInteger bossThreadCounter = new AtomicInteger(0); + bossGroup = new NioEventLoopGroup(maxShuffleThreads, new ThreadFactory() { @Override - public String determineThreadName(String currentThreadName, String proposedThreadName) throws Exception { - return BOSS_THREAD_NAME_PREFIX + currentThreadName.substring(currentThreadName.lastIndexOf('-') + 1); + public Thread newThread(Runnable r) { + return new Thread(r, BOSS_THREAD_NAME_PREFIX + bossThreadCounter.incrementAndGet()); } }); final String WORKER_THREAD_NAME_PREFIX = "Tez Shuffle Handler Worker #"; - NioWorkerPool workerPool = new NioWorkerPool(Executors.newCachedThreadPool(), maxShuffleThreads, new ThreadNameDeterminer() { + AtomicInteger workerThreadCounter = new AtomicInteger(0); + workerGroup = new NioEventLoopGroup(maxShuffleThreads, new ThreadFactory() { @Override - public String determineThreadName(String currentThreadName, String proposedThreadName) throws Exception { - return WORKER_THREAD_NAME_PREFIX + currentThreadName.substring(currentThreadName.lastIndexOf('-') + 1); + public Thread newThread(Runnable r) { + return new Thread(r, WORKER_THREAD_NAME_PREFIX + workerThreadCounter.incrementAndGet()); } }); - selector = new NioServerSocketChannelFactory(bossPool, workerPool); super.serviceInit(new YarnConfiguration(conf)); } @@ -542,25 +549,24 @@ protected void serviceStart() throws Exception { userRsrc = new ConcurrentHashMap(); secretManager = new JobTokenSecretManager(); recoverState(conf); - ServerBootstrap bootstrap = new ServerBootstrap(selector); - // Timer is shared across entire factory and must be released separately - timer = new HashedWheelTimer(); - try { - pipelineFact = new HttpPipelineFactory(conf, timer); - } catch (Exception ex) { - throw new RuntimeException(ex); - } - bootstrap.setOption("backlog", conf.getInt(SHUFFLE_LISTEN_QUEUE_SIZE, - DEFAULT_SHUFFLE_LISTEN_QUEUE_SIZE)); - bootstrap.setOption("child.keepAlive", true); - bootstrap.setPipelineFactory(pipelineFact); + ServerBootstrap bootstrap = new ServerBootstrap() + .channel(NioServerSocketChannel.class) + .group(bossGroup, workerGroup) + .localAddress(port) + .option(ChannelOption.SO_BACKLOG, + conf.getInt(SHUFFLE_LISTEN_QUEUE_SIZE, DEFAULT_SHUFFLE_LISTEN_QUEUE_SIZE)) + .childOption(ChannelOption.SO_KEEPALIVE, true); + initPipeline(bootstrap, conf); port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT); - Channel ch = bootstrap.bind(new InetSocketAddress(port)); + Channel ch = bootstrap.bind().sync().channel(); accepted.add(ch); - port = ((InetSocketAddress)ch.getLocalAddress()).getPort(); + + // setup port + port = ((InetSocketAddress)ch.localAddress()).getPort(); conf.set(SHUFFLE_PORT_CONFIG_KEY, Integer.toString(port)); - pipelineFact.SHUFFLE.setPort(port); + SHUFFLE.setPort(port); LOG.info(getName() + " listening on port " + port); + super.serviceStart(); sslFileBufferSize = conf.getInt(SUFFLE_SSL_FILE_BUFFER_SIZE_KEY, @@ -576,20 +582,50 @@ protected void serviceStart() throws Exception { DEFAULT_SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE)); } + private void initPipeline(ServerBootstrap bootstrap, Configuration conf) throws Exception { + SHUFFLE = getShuffle(conf); + if (conf.getBoolean(SHUFFLE_SSL_ENABLED_KEY, SHUFFLE_SSL_ENABLED_DEFAULT)) { + LOG.info("Encrypted shuffle is enabled."); + sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf); + sslFactory.init(); + } + + ChannelInitializer channelInitializer = + new ChannelInitializer() { + @Override + public void initChannel(NioSocketChannel ch) throws Exception { + ChannelPipeline pipeline = ch.pipeline(); + if (sslFactory != null) { + pipeline.addLast("ssl", new SslHandler(sslFactory.createSSLEngine())); + } + pipeline.addLast("decoder", new HttpRequestDecoder()); + pipeline.addLast("aggregator", new HttpObjectAggregator(1 << 16)); + pipeline.addLast("encoder", new HttpResponseEncoder()); + pipeline.addLast("chunking", new ChunkedWriteHandler()); + pipeline.addLast("shuffle", SHUFFLE); + pipeline.addLast("idle", new IdleStateHandler(0, connectionKeepAliveTimeOut, 0)); + pipeline.addLast(TIMEOUT_HANDLER, new TimeoutHandler()); + } + }; + bootstrap.childHandler(channelInitializer); + } + + private void destroyPipeline() { + if (sslFactory != null) { + sslFactory.destroy(); + } + } + @Override protected void serviceStop() throws Exception { accepted.close().awaitUninterruptibly(10, TimeUnit.SECONDS); - if (selector != null) { - ServerBootstrap bootstrap = new ServerBootstrap(selector); - bootstrap.releaseExternalResources(); - } - if (pipelineFact != null) { - pipelineFact.destroy(); + if (bossGroup != null) { + bossGroup.shutdownGracefully(); } - if (timer != null) { - // Release this shared timer resource - timer.stop(); + if (workerGroup != null) { + workerGroup.shutdownGracefully(); } + destroyPipeline(); if (stateDb != null) { stateDb.close(); } @@ -800,7 +836,7 @@ public void log(String message) { } } - static class TimeoutHandler extends IdleStateAwareChannelHandler { + static class TimeoutHandler extends ChannelDuplexHandler { private boolean enabledTimeout; @@ -809,59 +845,14 @@ void setEnabledTimeout(boolean enabledTimeout) { } @Override - public void channelIdle(ChannelHandlerContext ctx, IdleStateEvent e) { - if (e.getState() == IdleState.WRITER_IDLE && enabledTimeout) { - e.getChannel().close(); - } - } - } - - class HttpPipelineFactory implements ChannelPipelineFactory { - - final Shuffle SHUFFLE; - private SSLFactory sslFactory; - private final ChannelHandler idleStateHandler; - - public HttpPipelineFactory(Configuration conf, Timer timer) throws Exception { - SHUFFLE = getShuffle(conf); - if (conf.getBoolean(SHUFFLE_SSL_ENABLED_KEY, - SHUFFLE_SSL_ENABLED_DEFAULT)) { - LOG.info("Encrypted shuffle is enabled."); - sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf); - sslFactory.init(); - } - this.idleStateHandler = new IdleStateHandler(timer, 0, connectionKeepAliveTimeOut, 0); - } - - public Shuffle getSHUFFLE() { - return SHUFFLE; - } - - public void destroy() { - if (sslFactory != null) { - sslFactory.destroy(); - } - } - - @Override - public ChannelPipeline getPipeline() throws Exception { - ChannelPipeline pipeline = Channels.pipeline(); - if (sslFactory != null) { - pipeline.addLast("ssl", new SslHandler(sslFactory.createSSLEngine())); + public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { + if (evt instanceof IdleStateEvent) { + IdleStateEvent e = (IdleStateEvent) evt; + if (e.state() == IdleState.WRITER_IDLE && enabledTimeout) { + ctx.channel().close(); + } } - pipeline.addLast("decoder", new HttpRequestDecoder()); - pipeline.addLast("aggregator", new HttpChunkAggregator(1 << 16)); - pipeline.addLast("encoder", new HttpResponseEncoder()); - pipeline.addLast("chunking", new ChunkedWriteHandler()); - pipeline.addLast("shuffle", SHUFFLE); - pipeline.addLast("idle", idleStateHandler); - pipeline.addLast(TIMEOUT_HANDLER, new TimeoutHandler()); - return pipeline; - // TODO factor security manager into pipeline - // TODO factor out encode/decode to permit binary shuffle - // TODO factor out decode of index to permit alt. models } - } protected static class Range { @@ -887,7 +878,8 @@ public String toString() { } } - class Shuffle extends SimpleChannelUpstreamHandler { + @Sharable + class Shuffle extends ChannelInboundHandlerAdapter { private static final int MAX_WEIGHT = 10 * 1024 * 1024; private static final int EXPIRE_AFTER_ACCESS_MINUTES = 5; @@ -973,24 +965,30 @@ private Range splitReduces(List reduceq) { } @Override - public void channelOpen(ChannelHandlerContext ctx, ChannelStateEvent evt) + public void channelActive(ChannelHandlerContext ctx) throws Exception { if ((maxShuffleConnections > 0) && (accepted.size() >= maxShuffleConnections)) { LOG.info(String.format("Current number of shuffle connections (%d) is " + "greater than or equal to the max allowed shuffle connections (%d)", accepted.size(), maxShuffleConnections)); - evt.getChannel().close(); + ctx.channel().close(); return; } - accepted.add(evt.getChannel()); - super.channelOpen(ctx, evt); + accepted.add(ctx.channel()); + super.channelActive(ctx); } @Override - public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) + public void channelRead(ChannelHandlerContext ctx, Object message) throws Exception { - HttpRequest request = (HttpRequest) evt.getMessage(); + FullHttpRequest request = (FullHttpRequest) message; + handleRequest(ctx, request); + request.release(); + } + + private void handleRequest(ChannelHandlerContext ctx, FullHttpRequest request) + throws IOException, Exception { if (request.getMethod() != GET) { sendError(ctx, METHOD_NOT_ALLOWED); return; @@ -1001,9 +999,9 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) || !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION.equals( request.headers().get(ShuffleHeader.HTTP_HEADER_VERSION))) { sendError(ctx, "Incompatible shuffle request version", BAD_REQUEST); + return; } - final Map> q = - new QueryStringDecoder(request.getUri()).getParameters(); + final Map> q = new QueryStringDecoder(request.getUri()).parameters(); final List keepAliveList = q.get("keepAlive"); final List dagCompletedQ = q.get("dagAction"); boolean keepAliveParam = false; @@ -1024,7 +1022,7 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) "\n keepAlive: " + keepAliveParam); } // If the request is for Dag Deletion, process the request and send OK. - if (deleteDagDirectories(evt, dagCompletedQ, jobQ, dagIdQ)) { + if (deleteDagDirectories(ctx.channel(), dagCompletedQ, jobQ, dagIdQ)) { return; } if (mapIds == null || reduceRange == null || jobQ == null || dagIdQ == null) { @@ -1073,8 +1071,8 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) Map mapOutputInfoMap = new HashMap(); - Channel ch = evt.getChannel(); - ChannelPipeline pipeline = ch.getPipeline(); + Channel ch = ctx.channel(); + ChannelPipeline pipeline = ch.pipeline(); TimeoutHandler timeoutHandler = (TimeoutHandler)pipeline.get(TIMEOUT_HANDLER); timeoutHandler.setEnabledTimeout(false); String user = userRsrc.get(jobId); @@ -1098,19 +1096,23 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) return; } ch.write(response); - //Initialize one ReduceContext object per messageReceived call + //Initialize one ReduceContext object per channelRead call boolean keepAlive = keepAliveParam || connectionKeepAliveEnabled; ReduceContext reduceContext = new ReduceContext(mapIds, reduceRange, ctx, user, mapOutputInfoMap, jobId, dagId, keepAlive); for (int i = 0; i < Math.min(maxSessionOpenFiles, mapIds.size()); i++) { ChannelFuture nextMap = sendMap(reduceContext); if(nextMap == null) { + // by this special message flushed, we can make sure the whole response is finished + ch.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT); return; } } + // by this special message flushed, we can make sure the whole response is finished + ch.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT); } - private boolean deleteDagDirectories(MessageEvent evt, + private boolean deleteDagDirectories(Channel channel, List dagCompletedQ, List jobQ, List dagIdQ) { if (jobQ == null || jobQ.isEmpty()) { @@ -1127,8 +1129,8 @@ private boolean deleteDagDirectories(MessageEvent evt, } catch (IOException e) { LOG.warn("Encountered exception during dag delete "+ e); } - evt.getChannel().write(new DefaultHttpResponse(HTTP_1_1, OK)); - evt.getChannel().close(); + channel.writeAndFlush(new DefaultHttpResponse(HTTP_1_1, OK)) + .addListener(ChannelFutureListener.CLOSE); return true; } return false; @@ -1136,7 +1138,7 @@ private boolean deleteDagDirectories(MessageEvent evt, /** * Calls sendMapOutput for the mapId pointed by ReduceContext.mapsToSend - * and increments it. This method is first called by messageReceived() + * and increments it. This method is first called by channelRead() * maxSessionOpenFiles times and then on the completion of every * sendMapOutput operation. This limits the number of open files on a node, * which can get really large(exhausting file descriptors on the NM) if all @@ -1146,7 +1148,6 @@ private boolean deleteDagDirectories(MessageEvent evt, */ public ChannelFuture sendMap(ReduceContext reduceContext) throws Exception { - ChannelFuture nextMap = null; if (reduceContext.getMapsToSend().get() < reduceContext.getMapIds().size()) { @@ -1155,14 +1156,16 @@ public ChannelFuture sendMap(ReduceContext reduceContext) try { MapOutputInfo info = reduceContext.getInfoMap().get(mapId); + if (info == null) { info = getMapOutputInfo(reduceContext.dagId, mapId, reduceContext.getReduceRange(), reduceContext.getJobId(), reduceContext.getUser()); } + nextMap = sendMapOutput( reduceContext.getCtx(), - reduceContext.getCtx().getChannel(), + reduceContext.getCtx().channel(), reduceContext.getUser(), mapId, reduceContext.getReduceRange(), info); if (null == nextMap) { @@ -1417,23 +1420,12 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch, return null; } ChannelFuture writeFuture; - if (ch.getPipeline().get(SslHandler.class) == null) { + if (ch.pipeline().get(SslHandler.class) == null) { final FadvisedFileRegion partition = new FadvisedFileRegion(spill, rangeOffset, rangePartLength, manageOsCache, readaheadLength, readaheadPool, spillFile.getAbsolutePath(), shuffleBufferSize, shuffleTransferToAllowed); writeFuture = ch.write(partition); - writeFuture.addListener(new ChannelFutureListener() { - // TODO error handling; distinguish IO/connection failures, - // attribute to appropriate spill output - @Override - public void operationComplete(ChannelFuture future) { - if (future.isSuccess()) { - partition.transferSuccessful(); - } - partition.releaseExternalResources(); - } - }); } else { // HTTPS cannot be done with zero copy. final FadvisedChunkedFile chunk = new FadvisedChunkedFile(spill, @@ -1452,42 +1444,46 @@ protected void sendError(ChannelHandlerContext ctx, HttpResponseStatus status) { } protected void sendError(ChannelHandlerContext ctx, String message, HttpResponseStatus status) { - HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status); + FullHttpResponse response = new DefaultFullHttpResponse(HTTP_1_1, status); sendError(ctx, message, response); + response.release(); } - protected void sendError(ChannelHandlerContext ctx, String message, HttpResponse response) { - sendError(ctx, ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8), response); + protected void sendError(ChannelHandlerContext ctx, String message, FullHttpResponse response) { + sendError(ctx, Unpooled.copiedBuffer(message, CharsetUtil.UTF_8), response); } private void sendFakeShuffleHeaderWithError(ChannelHandlerContext ctx, String message, HttpResponse response) throws IOException { + FullHttpResponse fullResponse = + new DefaultFullHttpResponse(response.getProtocolVersion(), response.getStatus()); + fullResponse.headers().set(response.headers()); + ShuffleHeader header = new ShuffleHeader(message, -1, -1, -1); DataOutputBuffer out = new DataOutputBuffer(); header.write(out); - sendError(ctx, wrappedBuffer(out.getData(), 0, out.getLength()), response); + sendError(ctx, wrappedBuffer(out.getData(), 0, out.getLength()), fullResponse); + fullResponse.release(); } - protected void sendError(ChannelHandlerContext ctx, ChannelBuffer content, - HttpResponse response) { + protected void sendError(ChannelHandlerContext ctx, ByteBuf content, + FullHttpResponse response) { response.headers().set(CONTENT_TYPE, "text/plain; charset=UTF-8"); // Put shuffle version into http header response.headers().set(ShuffleHeader.HTTP_HEADER_NAME, ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); response.headers().set(ShuffleHeader.HTTP_HEADER_VERSION, ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); - response.setContent(content); + response.content().writeBytes(content); // Close the connection as soon as the error message is sent. - ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE); + ctx.channel().writeAndFlush(response).addListener(ChannelFutureListener.CLOSE); } @Override - public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { - Channel ch = e.getChannel(); - Throwable cause = e.getCause(); if (cause instanceof TooLongFrameException) { sendError(ctx, BAD_REQUEST); return; @@ -1504,8 +1500,8 @@ public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) } LOG.error("Shuffle error: ", cause); - if (ch.isConnected()) { - LOG.error("Shuffle error " + e); + if (ctx.channel().isActive()) { + LOG.error("Shuffle error", cause); sendError(ctx, INTERNAL_SERVER_ERROR); } } diff --git a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java index 5ca4ed8e26..21addd393e 100644 --- a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java @@ -22,9 +22,7 @@ //import static org.apache.hadoop.test.MetricsAsserts.getMetrics; import org.apache.hadoop.util.DiskChecker.DiskErrorException; import static org.junit.Assert.assertTrue; -import static org.jboss.netty.buffer.ChannelBuffers.wrappedBuffer; -import static org.jboss.netty.handler.codec.http.HttpResponseStatus.OK; -import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1; +import static io.netty.buffer.Unpooled.wrappedBuffer; import static org.junit.Assert.assertEquals; import static org.junit.Assume.assumeTrue; import static org.mockito.Mockito.mock; @@ -44,6 +42,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.zip.Checksum; import org.apache.hadoop.conf.Configuration; @@ -55,8 +54,6 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.io.nativeio.NativeIO; -import org.apache.hadoop.mapred.JobID; -import org.apache.hadoop.mapreduce.TypeConverter; import org.apache.tez.runtime.library.common.security.SecureShuffleUtils; import org.apache.tez.common.security.JobTokenIdentifier; import org.apache.tez.common.security.JobTokenSecretManager; @@ -79,20 +76,21 @@ import org.apache.hadoop.yarn.server.records.Version; import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord; import org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord; -import org.jboss.netty.channel.Channel; -import org.jboss.netty.channel.ChannelFuture; -import org.jboss.netty.channel.ChannelHandlerContext; -import org.jboss.netty.channel.ChannelPipeline; -import org.jboss.netty.channel.socket.SocketChannel; -import org.jboss.netty.channel.MessageEvent; -import org.jboss.netty.channel.AbstractChannel; -import org.jboss.netty.handler.codec.http.DefaultHttpHeaders; -import org.jboss.netty.handler.codec.http.DefaultHttpResponse; -import org.jboss.netty.handler.codec.http.HttpHeaders; -import org.jboss.netty.handler.codec.http.HttpRequest; -import org.jboss.netty.handler.codec.http.HttpResponse; -import org.jboss.netty.handler.codec.http.HttpResponseStatus; -import org.jboss.netty.handler.codec.http.HttpMethod; + +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelPipeline; +import io.netty.channel.socket.SocketChannel; +import io.netty.channel.AbstractChannel; +import io.netty.handler.codec.http.DefaultFullHttpRequest; +import io.netty.handler.codec.http.FullHttpRequest; +import io.netty.handler.codec.http.HttpHeaders; +import io.netty.handler.codec.http.HttpRequest; +import io.netty.handler.codec.http.HttpResponse; +import io.netty.handler.codec.http.HttpResponseStatus; +import io.netty.handler.codec.http.HttpVersion; +import io.netty.handler.codec.http.HttpMethod; import org.junit.Assert; import org.junit.Test; import org.mockito.invocation.InvocationOnMock; @@ -141,12 +139,12 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, new ShuffleHeader("attempt_12345_1_m_1_0", 5678, 5678, 1); DataOutputBuffer dob = new DataOutputBuffer(); header.write(dob); - ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength())); + ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); dob = new DataOutputBuffer(); for (int i = 0; i < 100; ++i) { header.write(dob); } - return ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength())); + return ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); } }; } @@ -162,8 +160,8 @@ protected Shuffle getShuffle(final Configuration conf) { protected void verifyRequest(String appid, ChannelHandlerContext ctx, HttpRequest request, HttpResponse response, URL requestUri) throws IOException { - SocketChannel channel = (SocketChannel)(ctx.getChannel()); - socketKeepAlive = channel.getConfig().isKeepAlive(); + SocketChannel channel = (SocketChannel)(ctx.channel()); + socketKeepAlive = channel.config().isKeepAlive(); } }; } @@ -258,6 +256,7 @@ public void testShuffleMetrics() throws Exception { sh.metrics.operationComplete(cf); checkShuffleMetrics(ms, 3*MiB, 1, 1, 0); + sh.close(); } static void checkShuffleMetrics(MetricsSystem ms, long bytes, int failed, @@ -279,7 +278,7 @@ static void checkShuffleMetrics(MetricsSystem ms, long bytes, int failed, */ @Test (timeout = 10000) public void testClientClosesConnection() throws Exception { - final ArrayList failures = new ArrayList(1); + final AtomicBoolean failureEncountered = new AtomicBoolean(false); Configuration conf = new Configuration(); conf.set(HADOOP_TMP_DIR, TEST_DIR.getAbsolutePath()); conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); @@ -321,27 +320,25 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, new ShuffleHeader("attempt_12345_1_m_1_0", 5678, 5678, 1); DataOutputBuffer dob = new DataOutputBuffer(); header.write(dob); - ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength())); + ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); dob = new DataOutputBuffer(); for (int i = 0; i < 100000; ++i) { header.write(dob); } - return ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength())); + return ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); } @Override protected void sendError(ChannelHandlerContext ctx, HttpResponseStatus status) { - if (failures.size() == 0) { - failures.add(new Error()); - ctx.getChannel().close(); + if (failureEncountered.compareAndSet(false, true)) { + ctx.channel().close(); } } @Override protected void sendError(ChannelHandlerContext ctx, String message, HttpResponseStatus status) { - if (failures.size() == 0) { - failures.add(new Error()); - ctx.getChannel().close(); + if (failureEncountered.compareAndSet(false, true)) { + ctx.channel().close(); } } }; @@ -368,9 +365,9 @@ protected void sendError(ChannelHandlerContext ctx, String message, header.readFields(input); input.close(); - shuffleHandler.stop(); + shuffleHandler.close(); Assert.assertTrue("sendError called when client closed connection", - failures.size() == 0); + !failureEncountered.get()); } static class LastSocketAddress { @@ -378,14 +375,14 @@ static class LastSocketAddress { void setAddress(SocketAddress lastAddress) { this.lastAddress = lastAddress; } - SocketAddress getSocketAddres() { + SocketAddress getSocketAddress() { return lastAddress; } } @Test(timeout = 10000) public void testKeepAlive() throws Exception { - final ArrayList failures = new ArrayList(1); + final AtomicBoolean failureEncountered = new AtomicBoolean(false); Configuration conf = new Configuration(); conf.set(HADOOP_TMP_DIR, TEST_DIR.getAbsolutePath()); conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); @@ -443,8 +440,7 @@ protected void populateHeaders(List mapIds, String jobId, protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch, String user, String mapId, Range reduceRange, MapOutputInfo info) throws IOException { - lastSocketAddress.setAddress(ch.getRemoteAddress()); - HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK); + lastSocketAddress.setAddress(ch.remoteAddress()); // send a shuffle header and a lot of data down the channel // to trigger a broken pipe @@ -452,29 +448,27 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, new ShuffleHeader("attempt_12345_1_m_1_0", 5678, 5678, 1); DataOutputBuffer dob = new DataOutputBuffer(); header.write(dob); - ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength())); + ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); dob = new DataOutputBuffer(); for (int i = 0; i < 100000; ++i) { header.write(dob); } - return ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength())); + return ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); } @Override protected void sendError(ChannelHandlerContext ctx, HttpResponseStatus status) { - if (failures.size() == 0) { - failures.add(new Error()); - ctx.getChannel().close(); + if (failureEncountered.compareAndSet(false, true)) { + ctx.channel().close(); } } @Override protected void sendError(ChannelHandlerContext ctx, String message, HttpResponseStatus status) { - if (failures.size() == 0) { - failures.add(new Error()); - ctx.getChannel().close(); + if (failureEncountered.compareAndSet(false, true)) { + ctx.channel().close(); } } }; @@ -505,7 +499,7 @@ protected void sendError(ChannelHandlerContext ctx, String message, header.readFields(input); byte[] buffer = new byte[1024]; while (input.read(buffer) != -1) {} - SocketAddress firstAddress = lastSocketAddress.getSocketAddres(); + SocketAddress firstAddress = lastSocketAddress.getSocketAddress(); input.close(); // For keepAlive via URL @@ -527,11 +521,12 @@ protected void sendError(ChannelHandlerContext ctx, String message, header = new ShuffleHeader(); header.readFields(input); input.close(); - SocketAddress secondAddress = lastSocketAddress.getSocketAddres(); + SocketAddress secondAddress = lastSocketAddress.getSocketAddress(); Assert.assertNotNull("Initial shuffle address should not be null", firstAddress); Assert.assertNotNull("Keep-Alive shuffle address should not be null", secondAddress); Assert.assertEquals("Initial shuffle address and keep-alive shuffle " + "address should be the same", firstAddress, secondAddress); + shuffleHandler.close(); } @Test @@ -567,7 +562,7 @@ public void testSocketKeepAlive() throws Exception { if (conn != null) { conn.disconnect(); } - shuffleHandler.stop(); + shuffleHandler.close(); } } @@ -603,7 +598,6 @@ public void testIncompatibleShuffleVersion() throws Exception { HttpURLConnection.HTTP_BAD_REQUEST, conn.getResponseCode()); } - shuffleHandler.stop(); shuffleHandler.close(); } @@ -657,12 +651,12 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, new ShuffleHeader("dummy_header", 5678, 5678, 1); DataOutputBuffer dob = new DataOutputBuffer(); header.write(dob); - ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength())); + ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); dob = new DataOutputBuffer(); for (int i=0; i<100000; ++i) { header.write(dob); } - return ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength())); + return ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); } }; } @@ -689,6 +683,10 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, // Try to open numerous connections for (int i = 0; i < connAttempts; i++) { + // connections should be made in a bit relaxed way, otherwise + // non-synced channelActive method will mess them up + Thread.sleep(200); + conns[i].connect(); } @@ -712,7 +710,7 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Assert.fail("Expected a SocketException"); } - shuffleHandler.stop(); + shuffleHandler.close(); } /** @@ -807,7 +805,7 @@ protected void verifyRequest(String appid, ChannelHandlerContext ctx, } } finally { - shuffleHandler.stop(); + shuffleHandler.close(); FileUtil.fullyDelete(absLogDir); } } @@ -900,7 +898,7 @@ protected void verifyRequest(String appid, ChannelHandlerContext ctx, + " did not match expected owner '" + user + "'"; Assert.assertTrue((new String(byteArr)).contains(message)); } finally { - shuffleHandler.stop(); + shuffleHandler.close(); FileUtil.fullyDelete(absLogDir); } } @@ -953,7 +951,6 @@ private static void createIndexFile(File indexFile, Configuration conf) public void testRecovery() throws IOException { final String user = "someuser"; final ApplicationId appId = ApplicationId.newInstance(12345, 1); - final JobID jobId = JobID.downgrade(TypeConverter.fromYarn(appId)); final File tmpDir = new File(System.getProperty("test.build.data", System.getProperty("java.io.tmpdir")), TestShuffleHandler.class.getName()); @@ -1131,7 +1128,7 @@ private static int getShuffleResponseCode(ShuffleHandler shuffle, @Test(timeout = 100000) public void testGetMapOutputInfo() throws Exception { - final ArrayList failures = new ArrayList(1); + final AtomicBoolean failureEncountered = new AtomicBoolean(false); Configuration conf = new Configuration(); conf.set(HADOOP_TMP_DIR, TEST_DIR.getAbsolutePath()); conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); @@ -1173,9 +1170,8 @@ protected void verifyRequest(String appid, @Override protected void sendError(ChannelHandlerContext ctx, String message, HttpResponseStatus status) { - if (failures.size() == 0) { - failures.add(new Error(message)); - ctx.getChannel().close(); + if (failureEncountered.compareAndSet(false, true)) { + ctx.channel().close(); } } @Override @@ -1187,7 +1183,7 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, new ShuffleHeader("attempt_12345_1_m_1_0", 5678, 5678, 1); DataOutputBuffer dob = new DataOutputBuffer(); header.write(dob); - return ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength())); + return ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); } }; } @@ -1227,16 +1223,16 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, // ignore } Assert.assertEquals("sendError called due to shuffle error", - 0, failures.size()); + false, failureEncountered.get()); } finally { - shuffleHandler.stop(); + shuffleHandler.close(); FileUtil.fullyDelete(absLogDir); } } @Test(timeout = 5000) public void testDagDelete() throws Exception { - final ArrayList failures = new ArrayList(1); + final AtomicBoolean failureEncountered = new AtomicBoolean(false); Configuration conf = new Configuration(); conf.set(HADOOP_TMP_DIR, TEST_DIR.getAbsolutePath()); conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, 3); @@ -1261,9 +1257,8 @@ protected Shuffle getShuffle(Configuration conf) { @Override protected void sendError(ChannelHandlerContext ctx, String message, HttpResponseStatus status) { - if (failures.size() == 0) { - failures.add(new Error(message)); - ctx.getChannel().close(); + if (failureEncountered.compareAndSet(false, true)) { + ctx.channel().close(); } } }; @@ -1309,9 +1304,9 @@ protected void sendError(ChannelHandlerContext ctx, String message, // ignore } Assert.assertEquals("sendError called due to shuffle error", - 0, failures.size()); + false, failureEncountered.get()); } finally { - shuffleHandler.stop(); + shuffleHandler.close(); FileUtil.fullyDelete(absLogDir); } } @@ -1323,29 +1318,23 @@ public void testSendMapCount() throws Exception { final ChannelHandlerContext mockCtx = mock(ChannelHandlerContext.class); - final MessageEvent mockEvt = mock(MessageEvent.class); final Channel mockCh = mock(AbstractChannel.class); final ChannelPipeline mockPipeline = Mockito.mock(ChannelPipeline.class); // Mock HttpRequest and ChannelFuture - final HttpRequest mockHttpRequest = createMockHttpRequest(); + final FullHttpRequest httpRequest = createHttpRequest(); final ChannelFuture mockFuture = createMockChannelFuture(mockCh, listenerList); final ShuffleHandler.TimeoutHandler timerHandler = new ShuffleHandler.TimeoutHandler(); // Mock Netty Channel Context and Channel behavior - Mockito.doReturn(mockCh).when(mockCtx).getChannel(); - Mockito.when(mockCh.getPipeline()).thenReturn(mockPipeline); + Mockito.doReturn(mockCh).when(mockCtx).channel(); + Mockito.when(mockCh.pipeline()).thenReturn(mockPipeline); Mockito.when(mockPipeline.get(Mockito.any(String.class))).thenReturn(timerHandler); - when(mockCtx.getChannel()).thenReturn(mockCh); - Mockito.doReturn(mockFuture).when(mockCh).write(Mockito.any(Object.class)); - when(mockCh.write(Object.class)).thenReturn(mockFuture); - - //Mock MessageEvent behavior - Mockito.doReturn(mockCh).when(mockEvt).getChannel(); - when(mockEvt.getChannel()).thenReturn(mockCh); - Mockito.doReturn(mockHttpRequest).when(mockEvt).getMessage(); + when(mockCtx.channel()).thenReturn(mockCh); + Mockito.doReturn(mockFuture).when(mockCh).writeAndFlush(Mockito.any(Object.class)); + when(mockCh.writeAndFlush(Object.class)).thenReturn(mockFuture); final ShuffleHandler sh = new MockShuffleHandler(); Configuration conf = new Configuration(); @@ -1356,7 +1345,7 @@ public void testSendMapCount() throws Exception { sh.start(); int maxOpenFiles =conf.getInt(ShuffleHandler.SHUFFLE_MAX_SESSION_OPEN_FILES, ShuffleHandler.DEFAULT_SHUFFLE_MAX_SESSION_OPEN_FILES); - sh.getShuffle(conf).messageReceived(mockCtx, mockEvt); + sh.getShuffle(conf).channelRead(mockCtx, httpRequest); assertTrue("Number of Open files should not exceed the configured " + "value!-Not Expected", listenerList.size() <= maxOpenFiles); @@ -1419,9 +1408,9 @@ public void testShuffleHandlerSendsDiskError() throws Exception { public ChannelFuture createMockChannelFuture(Channel mockCh, final List listenerList) { final ChannelFuture mockFuture = mock(ChannelFuture.class); - when(mockFuture.getChannel()).thenReturn(mockCh); + when(mockFuture.channel()).thenReturn(mockCh); Mockito.doReturn(true).when(mockFuture).isSuccess(); - Mockito.doAnswer(new Answer() { + Mockito.doAnswer(new Answer() { @Override public Object answer(InvocationOnMock invocation) throws Throwable { //Add ReduceMapFileCount listener to a list @@ -1436,19 +1425,11 @@ public Object answer(InvocationOnMock invocation) throws Throwable { return mockFuture; } - public HttpRequest createMockHttpRequest() { - HttpRequest mockHttpRequest = mock(HttpRequest.class); - Mockito.doReturn(HttpMethod.GET).when(mockHttpRequest).getMethod(); - Mockito.doReturn(new DefaultHttpHeaders()).when(mockHttpRequest).headers(); - Mockito.doAnswer(new Answer() { - @Override - public Object answer(InvocationOnMock invocation) throws Throwable { - String uri = "/mapOutput?job=job_12345_1&dag=1&reduce=1"; - for (int i = 0; i < 100; i++) - uri = uri.concat("&map=attempt_12345_1_m_" + i + "_0"); - return uri; - } - }).when(mockHttpRequest).getUri(); - return mockHttpRequest; + public FullHttpRequest createHttpRequest() { + String uri = "/mapOutput?job=job_12345_1&dag=1&reduce=1"; + for (int i = 0; i < 100; i++) { + uri = uri.concat("&map=attempt_12345_1_m_" + i + "_0"); + } + return new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, uri); } } From 867e4d205f129e198668daf94d0eb000e04c0e8c Mon Sep 17 00:00:00 2001 From: Harish Jaiprakash Date: Wed, 19 May 2021 01:49:07 +0530 Subject: [PATCH 313/512] TEZ-4296 - Use listStatusIterator instead of listStatus in DatePartitionedLogger (#124) Co-authored-by: Harish JP --- .../dag/history/logging/proto/DatePartitionedLogger.java | 9 +++++++-- .../logging/proto/TestProtoHistoryLoggingService.java | 1 + 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DatePartitionedLogger.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DatePartitionedLogger.java index 4ac64c6955..a569567d1e 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DatePartitionedLogger.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DatePartitionedLogger.java @@ -32,6 +32,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.yarn.util.Clock; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -138,7 +139,9 @@ public String getNextDirectory(String currentDir) throws IOException { } // Have to scan the directory to find min date greater than currentDir. String dirName = null; - for (FileStatus status : fileSystem.listStatus(basePath)) { + RemoteIterator iter = fileSystem.listStatusIterator(basePath); + while (iter.hasNext()) { + FileStatus status = iter.next(); String name = status.getPath().getName(); // String comparison is good enough, since its of form date=yyyy-MM-dd if (name.compareTo(currentDir) > 0 && (dirName == null || name.compareTo(dirName) < 0)) { @@ -160,7 +163,9 @@ public List scanForChangedFiles(String subDir, Map cur if (!fileSystem.exists(dirPath)) { return newFiles; } - for (FileStatus status : fileSystem.listStatus(dirPath)) { + RemoteIterator iter = fileSystem.listStatusIterator(dirPath); + while (iter.hasNext()) { + FileStatus status = iter.next(); String fileName = status.getPath().getName(); Long offset = currentOffsets.get(fileName); // If the offset was never added or offset < fileSize. diff --git a/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestProtoHistoryLoggingService.java b/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestProtoHistoryLoggingService.java index 143faed516..fd3154d904 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestProtoHistoryLoggingService.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestProtoHistoryLoggingService.java @@ -277,6 +277,7 @@ private ProtoHistoryLoggingService createService(boolean splitEvents) throws IOE service.setAppContext(appContext); Configuration conf = new Configuration(false); String basePath = tempFolder.newFolder().getAbsolutePath(); + conf.set("fs.file.impl", "org.apache.hadoop.fs.RawLocalFileSystem"); conf.set(TezConfiguration.TEZ_HISTORY_LOGGING_PROTO_BASE_DIR, basePath); conf.setBoolean(TezConfiguration.TEZ_HISTORY_LOGGING_PROTO_SPLIT_DAG_START, splitEvents); service.init(conf); From 4d379149bcdd0c0097f366d9a01e6a9907410ac6 Mon Sep 17 00:00:00 2001 From: Kishen Das Date: Thu, 20 May 2021 07:36:20 -0700 Subject: [PATCH 314/512] TEZ-4305: Check StreamCapabilities before using HFLUSH from ProtoMessageWriter (#120) (Kishen Das reviewed by Laszlo Bodor and Harish JP) Co-authored-by: Kishen Das --- .../org/apache/tez/common/StreamHelper.java | 49 +++++++++++++++++++ .../impl/SimpleHistoryLoggingService.java | 3 +- .../logging/proto/ProtoMessageWriter.java | 3 +- 3 files changed, 53 insertions(+), 2 deletions(-) create mode 100644 tez-common/src/main/java/org/apache/tez/common/StreamHelper.java diff --git a/tez-common/src/main/java/org/apache/tez/common/StreamHelper.java b/tez-common/src/main/java/org/apache/tez/common/StreamHelper.java new file mode 100644 index 0000000000..789d9b4268 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/common/StreamHelper.java @@ -0,0 +1,49 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.common; + +import org.apache.hadoop.fs.StreamCapabilities; +import org.apache.hadoop.fs.Syncable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +public final class StreamHelper { + + private static final Logger LOG = LoggerFactory.getLogger(StreamHelper.class); + + private StreamHelper() { + } + + public static void hflushIfSupported(Syncable syncable) throws IOException { + if (syncable instanceof StreamCapabilities) { + if (((StreamCapabilities) syncable).hasCapability(StreamCapabilities.HFLUSH)) { + syncable.hflush(); + } else { + // it would be no-op, if hflush is not supported by a given writer. + LOG.debug("skipping hflush, since the writer doesn't support it"); + } + } else { + // this is done for backward compatibility in order to make it work with + // older versions of Hadoop. + syncable.hflush(); + } + } +} diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/SimpleHistoryLoggingService.java b/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/SimpleHistoryLoggingService.java index 4372d8ec22..418bc3c5ea 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/SimpleHistoryLoggingService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/SimpleHistoryLoggingService.java @@ -23,6 +23,7 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.tez.common.StreamHelper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -130,7 +131,7 @@ protected void serviceStop() throws Exception { } try { if (outputStream != null) { - outputStream.hflush(); + StreamHelper.hflushIfSupported(outputStream); outputStream.close(); } } catch (IOException ioe) { diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWriter.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWriter.java index 869b603c3e..5b7591b496 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWriter.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWriter.java @@ -30,6 +30,7 @@ import com.google.protobuf.MessageLite; import com.google.protobuf.Parser; +import org.apache.tez.common.StreamHelper; public class ProtoMessageWriter implements Closeable { private final Path filePath; @@ -61,7 +62,7 @@ public void writeProto(T message) throws IOException { } public void hflush() throws IOException { - writer.hflush(); + StreamHelper.hflushIfSupported(writer); } @Override From 0af54dfa5fb114712efc1bdbcca32bb673bc53ef Mon Sep 17 00:00:00 2001 From: Ramesh Kumar Thangarajan Date: Thu, 20 May 2021 16:41:32 +0200 Subject: [PATCH 315/512] =?UTF-8?q?TEZ-4309:=20TezUtils.addToConfFromByteS?= =?UTF-8?q?tring=20throws=20com.google.protobuf.CodedInputStream=20excepti?= =?UTF-8?q?on=20(Ramesh=20Kumar=20Thangarajan=20via=20L=C3=A1szl=C3=B3=20B?= =?UTF-8?q?odor)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../java/org/apache/tez/common/TezUtils.java | 14 ++++++--- .../org/apache/tez/common/TestTezUtils.java | 31 ++++++++++++++++--- 2 files changed, 35 insertions(+), 10 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java index 3f0241834a..1c0be98dcc 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java @@ -101,6 +101,12 @@ public static UserPayload createUserPayloadFromConf(Configuration conf) throws I return UserPayload.create(ByteBuffer.wrap(createByteStringFromConf(conf).toByteArray())); } + private static DAGProtos.ConfigurationProto createConfProto(SnappyInputStream uncompressIs) throws IOException { + CodedInputStream in = CodedInputStream.newInstance(uncompressIs); + in.setSizeLimit(Integer.MAX_VALUE); + return DAGProtos.ConfigurationProto.parseFrom(in); + } + /** * Convert a byte string to a Configuration object * @@ -112,9 +118,7 @@ public static UserPayload createUserPayloadFromConf(Configuration conf) throws I public static Configuration createConfFromByteString(ByteString byteString) throws IOException { Objects.requireNonNull(byteString, "ByteString must be specified"); try(SnappyInputStream uncompressIs = new SnappyInputStream(byteString.newInput());) { - CodedInputStream in = CodedInputStream.newInstance(uncompressIs); - in.setSizeLimit(Integer.MAX_VALUE); - DAGProtos.ConfigurationProto confProto = DAGProtos.ConfigurationProto.parseFrom(in); + DAGProtos.ConfigurationProto confProto = createConfProto(uncompressIs); Configuration conf = new Configuration(false); readConfFromPB(confProto, conf); TezClassLoader.setupForConfiguration(conf); @@ -129,7 +133,7 @@ public static Configuration createConfFromBaseConfAndPayload(TaskContext context UserPayload payload = context.getUserPayload(); ByteString byteString = ByteString.copyFrom(payload.getPayload()); try(SnappyInputStream uncompressIs = new SnappyInputStream(byteString.newInput())) { - DAGProtos.ConfigurationProto confProto = DAGProtos.ConfigurationProto.parseFrom(uncompressIs); + DAGProtos.ConfigurationProto confProto = createConfProto(uncompressIs); readConfFromPB(confProto, configuration); TezClassLoader.setupForConfiguration(configuration); return configuration; @@ -139,7 +143,7 @@ public static Configuration createConfFromBaseConfAndPayload(TaskContext context public static void addToConfFromByteString(Configuration configuration, ByteString byteString) throws IOException { try(SnappyInputStream uncompressIs = new SnappyInputStream(byteString.newInput())) { - DAGProtos.ConfigurationProto confProto = DAGProtos.ConfigurationProto.parseFrom(uncompressIs); + DAGProtos.ConfigurationProto confProto = createConfProto(uncompressIs); readConfFromPB(confProto, configuration); TezClassLoader.setupForConfiguration(configuration); } diff --git a/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java b/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java index f9008b9947..d599cafd76 100644 --- a/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java +++ b/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java @@ -54,13 +54,10 @@ public void testByteStringToAndFromConf() throws IOException { checkConf(conf); } - @Test (timeout=20000) - public void testByteStringToAndFromLargeConf() throws IOException { - Configuration conf = getConf(); + private String constructLargeValue() { int largeSizeMinimum = 64 * 1024 * 1024; final String alphaString = "ABCDEFGHIJKLMNOPQRSTUVWXYZ"; int largeSize = (largeSizeMinimum + alphaString.length() - 1) / alphaString.length(); - largeSize *= alphaString.length(); assertTrue(largeSize >= alphaString.length()); StringBuilder sb = new StringBuilder(largeSize); @@ -71,9 +68,20 @@ public void testByteStringToAndFromLargeConf() throws IOException { String largeValue = sb.toString(); Assert.assertEquals(largeSize, largeValue.length()); + return largeValue; + } + + private ByteString createByteString(Configuration conf, String largeValue) throws IOException { conf.set("testLargeValue", largeValue); Assert.assertEquals(conf.size(), 7); - ByteString bsConf = TezUtils.createByteStringFromConf(conf); + return TezUtils.createByteStringFromConf(conf); + } + + @Test (timeout=20000) + public void testByteStringToAndFromLargeConf() throws IOException { + Configuration conf = getConf(); + String largeValue = constructLargeValue(); + ByteString bsConf = createByteString(conf, largeValue); conf.clear(); Assert.assertEquals(conf.size(), 0); conf = TezUtils.createConfFromByteString(bsConf); @@ -82,6 +90,19 @@ public void testByteStringToAndFromLargeConf() throws IOException { Assert.assertEquals(conf.get("testLargeValue"), largeValue); } + @Test (timeout=20000) + public void testByteStringAddToLargeConf() throws IOException { + Configuration conf = getConf(); + String largeValue = constructLargeValue(); + ByteString bsConf = createByteString(conf, largeValue); + conf.clear(); + Assert.assertEquals(conf.size(), 0); + TezUtils.addToConfFromByteString(conf, bsConf); + Assert.assertEquals(conf.size(), 7); + checkConf(conf); + Assert.assertEquals(conf.get("testLargeValue"), largeValue); + } + @Test (timeout=2000) public void testPayloadToAndFromConf() throws IOException { Configuration conf = getConf(); From b6c7fedfb74e8222f5b12bf2f53967a04f99860c Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Wed, 16 Jun 2021 10:37:09 +0200 Subject: [PATCH 316/512] TEZ-4295: Could not decompress data. Buffer length is too small. (#130) (Laszlo Bodor reviewed by Ashutosh Chauhan) --- .../library/common/TezRuntimeUtils.java | 28 --- .../library/common/sort/impl/IFile.java | 10 +- .../tez/runtime/library/utils/CodecUtils.java | 109 +++++++-- .../orderedgrouped/DummyCompressionCodec.java | 20 +- .../orderedgrouped/TestMergeManager.java | 7 +- .../library/common/sort/impl/TestIFile.java | 10 +- .../runtime/library/utils/TestCodecUtils.java | 224 ++++++++++++++++++ 7 files changed, 351 insertions(+), 57 deletions(-) create mode 100644 tez-runtime-library/src/test/java/org/apache/tez/runtime/library/utils/TestCodecUtils.java diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java index a1df131121..9d9b8c16c0 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java @@ -26,7 +26,6 @@ import java.nio.ByteBuffer; import org.apache.hadoop.io.DataInputByteBuffer; -import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.http.BaseHttpConnection; import org.apache.tez.http.HttpConnection; @@ -37,8 +36,6 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeys; -import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.tez.dag.api.TezUncheckedException; import org.apache.tez.runtime.api.OutputContext; import org.apache.tez.runtime.api.TaskContext; @@ -262,29 +259,4 @@ public static int deserializeShuffleProviderMetaData(ByteBuffer meta) in.close(); } } - - public static String getBufferSizeProperty(CompressionCodec codec) { - return getBufferSizeProperty(codec.getClass().getName()); - } - - public static String getBufferSizeProperty(String className) { - switch (className) { - case "org.apache.hadoop.io.compress.DefaultCodec": - case "org.apache.hadoop.io.compress.BZip2Codec": - case "org.apache.hadoop.io.compress.GzipCodec": - return CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY; - case "org.apache.hadoop.io.compress.SnappyCodec": - return CommonConfigurationKeys.IO_COMPRESSION_CODEC_SNAPPY_BUFFERSIZE_KEY; - case "org.apache.hadoop.io.compress.ZStandardCodec": - return CommonConfigurationKeys.IO_COMPRESSION_CODEC_ZSTD_BUFFER_SIZE_KEY; - case "org.apache.hadoop.io.compress.LzoCodec": - return CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY; - case "com.hadoop.compression.lzo.LzoCodec": - return CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY; - case "org.apache.hadoop.io.compress.Lz4Codec": - return CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZ4_BUFFERSIZE_KEY; - default: - return null; - } - } } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java index a4bbf5aabf..8f673185b0 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java @@ -363,10 +363,10 @@ public Writer(Serialization keySerialization, Serialization valSerialization, FS void setupOutputStream(CompressionCodec codec) throws IOException { this.checksumOut = new IFileOutputStream(this.rawOut); if (codec != null) { - this.compressor = CodecPool.getCompressor(codec); + this.compressor = CodecUtils.getCompressor(codec); if (this.compressor != null) { this.compressor.reset(); - this.compressedOut = codec.createOutputStream(checksumOut, compressor); + this.compressedOut = CodecUtils.createOutputStream(codec, checksumOut, compressor); this.out = new FSDataOutputStream(this.compressedOut, null); this.compressOutput = true; } else { @@ -773,9 +773,9 @@ public Reader(InputStream in, long length, checksumIn = new IFileInputStream(in, length, readAhead, readAheadLength/* , isCompressed */); if (isCompressed && codec != null) { - decompressor = CodecPool.getDecompressor(codec); + decompressor = CodecUtils.getDecompressor(codec); if (decompressor != null) { - this.in = codec.createInputStream(checksumIn, decompressor); + this.in = CodecUtils.createInputStream(codec, checksumIn, decompressor); } else { LOG.warn("Could not obtain decompressor from CodecPool"); this.in = checksumIn; @@ -818,7 +818,7 @@ public static void readToMemory(byte[] buffer, InputStream in, int compressedLen in = checksumIn; Decompressor decompressor = null; if (isCompressed && codec != null) { - decompressor = CodecPool.getDecompressor(codec); + decompressor = CodecUtils.getDecompressor(codec); if (decompressor != null) { decompressor.reset(); in = CodecUtils.getDecompressedInputStreamWithBufferSize(codec, checksumIn, decompressor, diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/CodecUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/CodecUtils.java index 8e5154f3b0..340ecceea8 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/CodecUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/CodecUtils.java @@ -20,27 +20,33 @@ import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.io.compress.CodecPool; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.CompressionInputStream; +import org.apache.hadoop.io.compress.CompressionOutputStream; import org.apache.hadoop.io.compress.Compressor; import org.apache.hadoop.io.compress.Decompressor; import org.apache.hadoop.io.compress.DefaultCodec; import org.apache.hadoop.util.ReflectionUtils; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; import org.apache.tez.runtime.library.common.ConfigUtils; -import org.apache.tez.runtime.library.common.TezRuntimeUtils; -import org.apache.tez.runtime.library.common.sort.impl.IFile; import org.apache.tez.runtime.library.common.sort.impl.IFileInputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.annotations.VisibleForTesting; + public final class CodecUtils { - private static final Logger LOG = LoggerFactory.getLogger(IFile.class); - private static final int DEFAULT_BUFFER_SIZE = 128 * 1024; + private static final Logger LOG = LoggerFactory.getLogger(CodecUtils.class); + @VisibleForTesting + static final int DEFAULT_BUFFER_SIZE = 256 * 1024; private CodecUtils() { } @@ -76,20 +82,21 @@ public static CompressionCodec getCodec(Configuration conf) throws IOException { public static InputStream getDecompressedInputStreamWithBufferSize(CompressionCodec codec, IFileInputStream checksumIn, Decompressor decompressor, int compressedLength) throws IOException { - String bufferSizeProp = TezRuntimeUtils.getBufferSizeProperty(codec); - Configurable configurableCodec = (Configurable) codec; - int originalSize = bufferSizeProp == null ? DEFAULT_BUFFER_SIZE : - configurableCodec.getConf().getInt(bufferSizeProp, DEFAULT_BUFFER_SIZE); - + String bufferSizeProp = getBufferSizeProperty(codec); CompressionInputStream in = null; if (bufferSizeProp != null) { + Configurable configurableCodec = (Configurable) codec; Configuration conf = configurableCodec.getConf(); - int newBufSize = Math.min(compressedLength, DEFAULT_BUFFER_SIZE); - LOG.trace("buffer size was set according to min(compressedLength, {}): {}={}", - DEFAULT_BUFFER_SIZE, bufferSizeProp, newBufSize); - synchronized (codec) { + synchronized (conf) { + int defaultBufferSize = getDefaultBufferSize(conf, codec); + int originalSize = conf.getInt(bufferSizeProp, defaultBufferSize); + + int newBufSize = Math.min(compressedLength, defaultBufferSize); + LOG.debug("buffer size was set according to min({}, {}) => {}={}", compressedLength, + defaultBufferSize, bufferSizeProp, newBufSize); + conf.setInt(bufferSizeProp, newBufSize); in = codec.createInputStream(checksumIn, decompressor); @@ -117,7 +124,7 @@ public static InputStream getDecompressedInputStreamWithBufferSize(CompressionCo * issues above for Compressor instances as well, even when we tried to leverage from * smaller buffer size only on decompression paths. */ - configurableCodec.getConf().setInt(bufferSizeProp, originalSize); + conf.setInt(bufferSizeProp, originalSize); } } else { in = codec.createInputStream(checksumIn, decompressor); @@ -125,4 +132,78 @@ public static InputStream getDecompressedInputStreamWithBufferSize(CompressionCo return in; } + + public static Compressor getCompressor(CompressionCodec codec) { + synchronized (((Configurable) codec).getConf()) { + return CodecPool.getCompressor(codec); + } + } + + public static Decompressor getDecompressor(CompressionCodec codec) { + synchronized (((Configurable) codec).getConf()) { + return CodecPool.getDecompressor(codec); + } + } + + public static CompressionInputStream createInputStream(CompressionCodec codec, + InputStream checksumIn, Decompressor decompressor) throws IOException { + synchronized (((Configurable) codec).getConf()) { + return codec.createInputStream(checksumIn, decompressor); + } + } + + public static CompressionOutputStream createOutputStream(CompressionCodec codec, + OutputStream checksumOut, Compressor compressor) throws IOException { + synchronized (((Configurable) codec).getConf()) { + return codec.createOutputStream(checksumOut, compressor); + } + } + + public static String getBufferSizeProperty(CompressionCodec codec) { + return getBufferSizeProperty(codec.getClass().getName()); + } + + public static String getBufferSizeProperty(String codecClassName) { + switch (codecClassName) { + case "org.apache.hadoop.io.compress.DefaultCodec": + case "org.apache.hadoop.io.compress.BZip2Codec": + case "org.apache.hadoop.io.compress.GzipCodec": + return CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY; + case "org.apache.hadoop.io.compress.SnappyCodec": + return CommonConfigurationKeys.IO_COMPRESSION_CODEC_SNAPPY_BUFFERSIZE_KEY; + case "org.apache.hadoop.io.compress.ZStandardCodec": + return CommonConfigurationKeys.IO_COMPRESSION_CODEC_ZSTD_BUFFER_SIZE_KEY; + case "org.apache.hadoop.io.compress.LzoCodec": + case "com.hadoop.compression.lzo.LzoCodec": + return CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY; + case "org.apache.hadoop.io.compress.Lz4Codec": + return CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZ4_BUFFERSIZE_KEY; + default: + return null; + } + } + + public static int getDefaultBufferSize(Configuration conf, CompressionCodec codec) { + return getDefaultBufferSize(conf, codec.getClass().getName()); + } + + public static int getDefaultBufferSize(Configuration conf, String codecClassName) { + switch (codecClassName) { + case "org.apache.hadoop.io.compress.DefaultCodec": + case "org.apache.hadoop.io.compress.BZip2Codec": + case "org.apache.hadoop.io.compress.GzipCodec": + return CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT; + case "org.apache.hadoop.io.compress.SnappyCodec": + return CommonConfigurationKeys.IO_COMPRESSION_CODEC_SNAPPY_BUFFERSIZE_DEFAULT; + case "org.apache.hadoop.io.compress.ZStandardCodec": + return CommonConfigurationKeys.IO_COMPRESSION_CODEC_ZSTD_BUFFER_SIZE_DEFAULT; + case "org.apache.hadoop.io.compress.LzoCodec": + case "com.hadoop.compression.lzo.LzoCodec": + return CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_DEFAULT; + case "org.apache.hadoop.io.compress.Lz4Codec": + return CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZ4_BUFFERSIZE_DEFAULT; + default: + return DEFAULT_BUFFER_SIZE; + } + } } \ No newline at end of file diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/DummyCompressionCodec.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/DummyCompressionCodec.java index 962a9e0207..530b9a3732 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/DummyCompressionCodec.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/DummyCompressionCodec.java @@ -18,12 +18,16 @@ package org.apache.tez.runtime.library.common.shuffle.orderedgrouped; +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.CompressionInputStream; import org.apache.hadoop.io.compress.CompressionOutputStream; import org.apache.hadoop.io.compress.Compressor; import org.apache.hadoop.io.compress.Decompressor; +import com.google.common.annotations.VisibleForTesting; + import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -33,7 +37,10 @@ /** * A dummy codec. It passes everything to underlying stream */ -public class DummyCompressionCodec implements CompressionCodec { +public class DummyCompressionCodec implements CompressionCodec, Configurable { + @VisibleForTesting + int createInputStreamCalled = 0; + private Configuration conf; @Override public CompressionOutputStream createOutputStream(OutputStream out) throws IOException { @@ -62,6 +69,7 @@ public CompressionInputStream createInputStream(InputStream in) throws IOExcepti @Override public CompressionInputStream createInputStream(InputStream in, Decompressor decompressor) throws IOException { + createInputStreamCalled += 1; return new DummyCompressionInputStream(in); } @@ -128,4 +136,14 @@ public void resetState() throws IOException { //no-op } } + + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } + + @Override + public Configuration getConf() { + return conf; + } } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java index 13f090cffb..dde067beb8 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java @@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -37,7 +36,6 @@ import com.google.common.collect.Sets; -import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.serializer.WritableSerialization; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -273,7 +271,8 @@ public void testDiskMergeWithCodec() throws Throwable { InputContext inputContext = createMockInputContext(UUID.randomUUID().toString()); // Create a mock compressor. We will check if it is used. - CompressionCodec dummyCodec = spy(new DummyCompressionCodec()); + DummyCompressionCodec dummyCodec = new DummyCompressionCodec(); + dummyCodec.setConf(conf); MergeManager mergeManager = new MergeManager(conf, localFs, localDirAllocator, inputContext, null, null, null, null, @@ -312,7 +311,7 @@ public void testDiskMergeWithCodec() throws Throwable { mo4.commit(); mergeManager.close(true); - verify(dummyCodec, atLeastOnce()).createOutputStream(any(), any()); + Assert.assertTrue(dummyCodec.createInputStreamCalled > 0); } @Test(timeout = 60000l) diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java index bf35955625..960aee345a 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java @@ -57,7 +57,6 @@ import org.apache.hadoop.util.NativeCodeLoader; import org.apache.tez.common.TezRuntimeFrameworkConfigs; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; -import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.InMemoryReader; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.InMemoryWriter; import org.apache.tez.runtime.library.common.sort.impl.IFile.Reader; @@ -66,6 +65,7 @@ import org.apache.tez.runtime.library.testutils.KVDataGen; import org.apache.tez.runtime.library.testutils.KVDataGen.KVPair; import org.apache.tez.runtime.library.utils.BufferUtils; +import org.apache.tez.runtime.library.utils.CodecUtils; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -734,7 +734,7 @@ public void testReadToDisk() throws IOException { public void testInMemoryBufferSize() throws IOException { Configurable configurableCodec = (Configurable) codec; int originalCodecBufferSize = - configurableCodec.getConf().getInt(TezRuntimeUtils.getBufferSizeProperty(codec), -1); + configurableCodec.getConf().getInt(CodecUtils.getBufferSizeProperty(codec), -1); // for smaller amount of data, codec buffer should be sized according to compressed data length List data = KVDataGen.generateTestData(false, rnd.nextInt(100)); @@ -742,7 +742,7 @@ public void testInMemoryBufferSize() throws IOException { readAndVerifyData(writer.getRawLength(), writer.getCompressedLength(), data, codec); Assert.assertEquals(originalCodecBufferSize, // original size is repaired - configurableCodec.getConf().getInt(TezRuntimeUtils.getBufferSizeProperty(codec), 0)); + configurableCodec.getConf().getInt(CodecUtils.getBufferSizeProperty(codec), 0)); // buffer size cannot grow infinitely with compressed data size data = KVDataGen.generateTestDataOfKeySize(false, 20000, rnd.nextInt(100)); @@ -750,7 +750,7 @@ public void testInMemoryBufferSize() throws IOException { readAndVerifyData(writer.getRawLength(), writer.getCompressedLength(), data, codec); Assert.assertEquals(originalCodecBufferSize, // original size is repaired - configurableCodec.getConf().getInt(TezRuntimeUtils.getBufferSizeProperty(codec), 0)); + configurableCodec.getConf().getInt(CodecUtils.getBufferSizeProperty(codec), 0)); } @Test(expected = IllegalArgumentException.class) @@ -766,7 +766,7 @@ private void tryWriteFileWithBufferSize(int bufferSize, String codecClassName) Configuration conf = new Configuration(); System.out.println("trying with buffer size: " + bufferSize); - conf.set(TezRuntimeUtils.getBufferSizeProperty(codecClassName), Integer.toString(bufferSize)); + conf.set(CodecUtils.getBufferSizeProperty(codecClassName), Integer.toString(bufferSize)); CompressionCodecFactory codecFactory = new CompressionCodecFactory(conf); CompressionCodec codecToTest = codecFactory.getCodecByClassName(codecClassName); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/utils/TestCodecUtils.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/utils/TestCodecUtils.java new file mode 100644 index 0000000000..afa645974f --- /dev/null +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/utils/TestCodecUtils.java @@ -0,0 +1,224 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.runtime.library.utils; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.lang.reflect.Field; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.io.compress.BZip2Codec; +import org.apache.hadoop.io.compress.CodecPool; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.io.compress.CompressionInputStream; +import org.apache.hadoop.io.compress.CompressionOutputStream; +import org.apache.hadoop.io.compress.Compressor; +import org.apache.hadoop.io.compress.Decompressor; +import org.apache.hadoop.io.compress.DecompressorStream; +import org.apache.hadoop.io.compress.DefaultCodec; +import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.hadoop.io.compress.Lz4Codec; +import org.apache.hadoop.io.compress.SnappyCodec; +import org.apache.hadoop.io.compress.ZStandardCodec; +import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; +import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.DummyCompressionCodec; +import org.apache.tez.runtime.library.common.sort.impl.IFileInputStream; +import org.junit.Assert; +import org.junit.Test; +import org.mockito.Mockito; + +public class TestCodecUtils { + + @Test + public void testConcurrentDecompressorCreationWithModifiedBuffersize() throws Exception { + testConcurrentDecompressorCreationWithModifiedBuffersizeOnCodec(new DefaultCodec()); + } + + private void testConcurrentDecompressorCreationWithModifiedBuffersizeOnCodec( + CompressionCodec codec) throws InterruptedException, ExecutionException { + int modifiedBufferSize = 1000; + int numberOfThreads = 1000; + + ExecutorService service = Executors.newFixedThreadPool(numberOfThreads); + + Configuration conf = new Configuration(); + conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_COMPRESS, true); + ((Configurable) codec).setConf(conf); + + Future[] futures = new Future[numberOfThreads]; + final CountDownLatch latch = new CountDownLatch(1); + + for (int i = 0; i < numberOfThreads; i++) { + futures[i] = service.submit(() -> { + try { + waitForLatch(latch); + + Decompressor decompressor = CodecUtils.getDecompressor(codec); + DecompressorStream stream = + (DecompressorStream) CodecUtils.getDecompressedInputStreamWithBufferSize(codec, + Mockito.mock(IFileInputStream.class), decompressor, modifiedBufferSize); + + Assert.assertEquals("stream buffer size is incorrect", modifiedBufferSize, + getBufferSize(stream)); + + CodecPool.returnDecompressor(decompressor); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + latch.countDown(); + + for (Future f : futures) { + f.get(); + } + } + + @Test + public void testConcurrentCompressorDecompressorCreation() throws Exception { + testConcurrentCompressorDecompressorCreationOnCodec(new DefaultCodec()); + } + + private void testConcurrentCompressorDecompressorCreationOnCodec(CompressionCodec codec) + throws IOException, InterruptedException, ExecutionException { + int modifiedBufferSize = 1000; + int numberOfThreads = 1000; + + ExecutorService service = Executors.newFixedThreadPool(numberOfThreads); + + Configuration conf = new Configuration(); + conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_COMPRESS, true); + ((Configurable) codec).setConf(conf); + + Future[] futures = new Future[numberOfThreads]; + final CountDownLatch latch = new CountDownLatch(1); + + for (int i = 0; i < numberOfThreads; i++) { + // let's "randomly" choose from scenarios and test them concurrently + // 1. getDecompressedInputStreamWithBufferSize + if (i % 3 == 0) { + futures[i] = service.submit(() -> { + try { + waitForLatch(latch); + + Decompressor decompressor = CodecUtils.getDecompressor(codec); + CompressionInputStream stream = + (CompressionInputStream) CodecUtils.getDecompressedInputStreamWithBufferSize(codec, + Mockito.mock(IFileInputStream.class), decompressor, modifiedBufferSize); + + Assert.assertEquals("stream buffer size is incorrect", modifiedBufferSize, + getBufferSize(stream)); + + CodecPool.returnDecompressor(decompressor); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + // 2. getCompressor + } else if (i % 3 == 1) { + futures[i] = service.submit(() -> { + try { + waitForLatch(latch); + + Compressor compressor = CodecUtils.getCompressor(codec); + CompressionOutputStream stream = + CodecUtils.createOutputStream(codec, Mockito.mock(OutputStream.class), compressor); + + Assert.assertEquals("stream buffer size is incorrect", + CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT, getBufferSize(stream)); + + CodecPool.returnCompressor(compressor); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + // 3. getDecompressor + } else if (i % 3 == 2) { + futures[i] = service.submit(() -> { + try { + waitForLatch(latch); + + Decompressor decompressor = CodecUtils.getDecompressor(codec); + CompressionInputStream stream = + CodecUtils.createInputStream(codec, Mockito.mock(InputStream.class), decompressor); + + Assert.assertEquals("stream buffer size is incorrect", + CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT, getBufferSize(stream)); + + CodecPool.returnDecompressor(decompressor); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } + } + latch.countDown(); + + for (Future f : futures) { + f.get(); + } + } + + @Test + public void testDefaultBufferSize() { + Configuration conf = new Configuration(); // config with no buffersize set + + Assert.assertEquals(CodecUtils.DEFAULT_BUFFER_SIZE, + CodecUtils.getDefaultBufferSize(conf, new DummyCompressionCodec())); + Assert.assertEquals(CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT, + CodecUtils.getDefaultBufferSize(conf, new DefaultCodec())); + Assert.assertEquals(CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT, + CodecUtils.getDefaultBufferSize(conf, new BZip2Codec())); + Assert.assertEquals(CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT, + CodecUtils.getDefaultBufferSize(conf, new GzipCodec())); + Assert.assertEquals(CommonConfigurationKeys.IO_COMPRESSION_CODEC_SNAPPY_BUFFERSIZE_DEFAULT, + CodecUtils.getDefaultBufferSize(conf, new SnappyCodec())); + Assert.assertEquals(CommonConfigurationKeys.IO_COMPRESSION_CODEC_ZSTD_BUFFER_SIZE_DEFAULT, + CodecUtils.getDefaultBufferSize(conf, new ZStandardCodec())); + Assert.assertEquals(CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZ4_BUFFERSIZE_DEFAULT, + CodecUtils.getDefaultBufferSize(conf, new Lz4Codec())); + } + + private void waitForLatch(CountDownLatch latch) { + try { + latch.await(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + private int getBufferSize(Object stream) { + try { + Field field = stream.getClass().getDeclaredField("buffer"); + field.setAccessible(true); + byte[] buffer = (byte[]) field.get(stream); + return buffer.length; + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} From d443ae2b30fae8a80c98e9e16ec98313ae14dca1 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Tue, 22 Jun 2021 22:43:08 +0200 Subject: [PATCH 317/512] TEZ-4298: ShuffleHandler is not source compatible with Hadoop 3.3.0 (#129) --- tez-plugins/tez-aux-services/findbugs-exclude.xml | 7 ++++++- .../java/org/apache/tez/auxservices/ShuffleHandler.java | 6 +++--- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/tez-plugins/tez-aux-services/findbugs-exclude.xml b/tez-plugins/tez-aux-services/findbugs-exclude.xml index 5b11308f6d..adfd7041a5 100644 --- a/tez-plugins/tez-aux-services/findbugs-exclude.xml +++ b/tez-plugins/tez-aux-services/findbugs-exclude.xml @@ -12,5 +12,10 @@ limitations under the License. See accompanying LICENSE file. --> - + + + + + + diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java index e73805a1f0..fb28a0f4b1 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java @@ -153,7 +153,6 @@ import com.google.common.cache.RemovalListener; import com.google.common.cache.RemovalNotification; import com.google.common.cache.Weigher; -import com.google.protobuf.ByteString; public class ShuffleHandler extends AuxiliaryService { @@ -796,9 +795,10 @@ private void recoverJobShuffleInfo(String jobIdStr, byte[] data) private void recordJobShuffleInfo(JobID jobId, String user, Token jobToken) throws IOException { if (stateDb != null) { + // Discover type instead of assuming ByteString to allow for shading. TokenProto tokenProto = TokenProto.newBuilder() - .setIdentifier(ByteString.copyFrom(jobToken.getIdentifier())) - .setPassword(ByteString.copyFrom(jobToken.getPassword())) + .setIdentifier(TokenProto.getDefaultInstance().getIdentifier().copyFrom(jobToken.getIdentifier())) + .setPassword(TokenProto.getDefaultInstance().getPassword().copyFrom(jobToken.getPassword())) .setKind(jobToken.getKind().toString()) .setService(jobToken.getService().toString()) .build(); From 26b86e7646021c3bdf1e54a7ee225f46397eba42 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Mon, 28 Jun 2021 10:15:42 -0500 Subject: [PATCH 318/512] TEZ-3918. Setting tez.task.log.level does not work Signed-off-by: Jonathan Eagles --- .../org/apache/tez/client/TezClientUtils.java | 22 +++++++++++-------- .../apache/tez/client/TestTezClientUtils.java | 10 +++++++++ 2 files changed, 23 insertions(+), 9 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index 389b58400b..d0fc374a81 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -760,13 +760,6 @@ static DAGPlan prepareAndCreateDAGPlan(DAG dag, AMConfiguration amConfig, static void maybeAddDefaultLoggingJavaOpts(String logLevel, List vargs) { Objects.requireNonNull(vargs); - if (!vargs.isEmpty()) { - for (String arg : vargs) { - if (arg.contains(TezConstants.TEZ_ROOT_LOGGER_NAME)) { - return; - } - } - } TezClientUtils.addLog4jSystemProperties(logLevel, vargs); } @@ -829,8 +822,19 @@ public static void addLog4jSystemProperties(String logLevel, + TezConstants.TEZ_CONTAINER_LOG4J_PROPERTIES_FILE); vargs.add("-D" + YarnConfiguration.YARN_APP_CONTAINER_LOG_DIR + "=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR); - vargs.add("-D" + TezConstants.TEZ_ROOT_LOGGER_NAME + "=" + logLevel - + "," + TezConstants.TEZ_CONTAINER_LOGGER_NAME); + boolean isRootLoggerPresent = false; + String rootLoggerArg = "-D" + TezConstants.TEZ_ROOT_LOGGER_NAME + "=" + logLevel + + "," + TezConstants.TEZ_CONTAINER_LOGGER_NAME; + for (int i = 0; i < vargs.size(); i++) { + String arg = vargs.get(i); + if (arg.contains(TezConstants.TEZ_ROOT_LOGGER_NAME)) { + vargs.set(i, rootLoggerArg); + isRootLoggerPresent = true; + } + } + if (!isRootLoggerPresent) { + vargs.add(rootLoggerArg); + } } static ConfigurationProto createFinalConfProtoForApp(Configuration amConf, diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java index 29e9210538..29a6769716 100644 --- a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java +++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java @@ -682,6 +682,16 @@ public void testDefaultLoggingJavaOpts() { javaOpts.contains("-Dlog4j.configuratorClass=org.apache.tez.common.TezLog4jConfigurator")); } + @Test + public void testDefaultLoggingJavaOptsWithRootLogger() { + String origJavaOpts = "-D" + TezConstants.TEZ_ROOT_LOGGER_NAME + "=INFO"; + String javaOpts = TezClientUtils.maybeAddDefaultLoggingJavaOpts("FOOBAR", origJavaOpts); + Assert.assertNotNull(javaOpts); + Assert.assertTrue(javaOpts.contains("-D" + TezConstants.TEZ_ROOT_LOGGER_NAME + "=FOOBAR")); + Assert.assertTrue(javaOpts.contains(TezConstants.TEZ_CONTAINER_LOG4J_PROPERTIES_FILE) + && javaOpts.contains("-Dlog4j.configuratorClass=org.apache.tez.common.TezLog4jConfigurator")); + } + @Test (timeout = 5000) public void testConfSerializationForAm() { Configuration conf =new Configuration(false); From 984d09c4d37fe7b73d63bf0ce9d2de7c34bdd735 Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Wed, 30 Jun 2021 10:48:30 -0400 Subject: [PATCH 319/512] TEZ-4275: Use Google Guava Intern Facility (#95) * TEZ-4275: Use Google Guava Intern Facility * Also add hint for running finalization * Use Guava String Intern implementation * Fix checkstyle, white-space issues * Remove GC hint Co-authored-by: David Mollitor --- .../org/apache/tez/dag/records/TezDAGID.java | 15 ++++---- .../org/apache/tez/dag/records/TezID.java | 21 ----------- .../tez/dag/records/TezTaskAttemptID.java | 12 +++---- .../org/apache/tez/dag/records/TezTaskID.java | 12 +++---- .../apache/tez/dag/records/TezVertexID.java | 12 +++---- .../org/apache/tez/util/StringInterner.java | 36 +++++++++++++++++++ .../org/apache/tez/dag/app/DAGAppMaster.java | 6 ---- .../tez/dag/app/dag/impl/TaskAttemptImpl.java | 12 +++---- .../tez/dag/app/dag/impl/VertexImpl.java | 6 ++-- .../tez/history/parser/datamodel/DagInfo.java | 8 ++--- .../parser/datamodel/TaskAttemptInfo.java | 16 ++++----- .../history/parser/datamodel/TaskInfo.java | 10 +++--- .../history/parser/datamodel/VertexInfo.java | 10 +++--- .../tez/history/parser/utils/Utils.java | 4 +-- .../tez/runtime/api/impl/EventMetaData.java | 10 +++--- .../tez/runtime/api/impl/GroupInputSpec.java | 8 ++--- .../tez/runtime/api/impl/InputSpec.java | 6 ++-- .../tez/runtime/api/impl/OutputSpec.java | 6 ++-- .../apache/tez/runtime/api/impl/TaskSpec.java | 14 ++++---- .../tez/runtime/api/impl/TaskStatistics.java | 5 +-- ...huffleInputEventHandlerOrderedGrouped.java | 10 +++--- .../sort/impl/dflt/TestDefaultSorter.java | 4 +-- .../plugins/CriticalPathAnalyzer.java | 9 +++-- 23 files changed, 127 insertions(+), 125 deletions(-) create mode 100644 tez-common/src/main/java/org/apache/tez/util/StringInterner.java diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezDAGID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezDAGID.java index 24365c95e2..68184fc8c0 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezDAGID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezDAGID.java @@ -22,12 +22,14 @@ import java.io.DataOutput; import java.io.IOException; -import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.tez.common.Preconditions; import org.apache.tez.util.FastNumberFormat; +import com.google.common.collect.Interner; +import com.google.common.collect.Interners; + /** * TezDAGID represents the immutable and unique identifier for * a Tez DAG. @@ -40,7 +42,7 @@ */ public class TezDAGID extends TezID { - private static TezIDCache tezDAGIDCache = new TezIDCache<>(); + private static Interner tezDAGIDCache = Interners.newWeakInterner(); private ApplicationId applicationId; /** @@ -53,12 +55,7 @@ public static TezDAGID getInstance(ApplicationId applicationId, int id) { // will be short-lived. // Alternately the cache can be keyed by the hash of the incoming paramters. Preconditions.checkArgument(applicationId != null, "ApplicationID cannot be null"); - return tezDAGIDCache.getInstance(new TezDAGID(applicationId, id)); - } - - @InterfaceAudience.Private - public static void clearCache() { - tezDAGIDCache.clear(); + return tezDAGIDCache.intern(new TezDAGID(applicationId, id)); } /** @@ -72,7 +69,7 @@ public static TezDAGID getInstance(String yarnRMIdentifier, int appId, int id) { // will be short-lived. // Alternately the cache can be keyed by the hash of the incoming paramters. Preconditions.checkArgument(yarnRMIdentifier != null, "yarnRMIdentifier cannot be null"); - return tezDAGIDCache.getInstance(new TezDAGID(yarnRMIdentifier, appId, id)); + return tezDAGIDCache.intern(new TezDAGID(yarnRMIdentifier, appId, id)); } // Public for Writable serialization. Verify if this is actually required. diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezID.java index cd7b27de45..7efbd9a889 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezID.java @@ -21,8 +21,6 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import java.lang.ref.WeakReference; -import java.util.WeakHashMap; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -42,25 +40,6 @@ public abstract class TezID implements WritableComparable { public static final char SEPARATOR = '_'; protected int id; - public static class TezIDCache { - private final WeakHashMap> cache = new WeakHashMap<>(); - - synchronized T getInstance(final T id) { - final WeakReference cached = cache.get(id); - if (cached != null) { - final T value = cached.get(); - if (value != null) - return value; - } - cache.put(id, new WeakReference(id)); - return id; - } - - synchronized void clear() { - cache.clear(); - } - } - /** constructs an ID object from the given int */ public TezID(int id) { this.id = id; diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java index 7aee80f4d6..9ce1b10aa0 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java @@ -25,6 +25,9 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import com.google.common.collect.Interner; +import com.google.common.collect.Interners; + /** * TezTaskAttemptID represents the immutable and unique identifier for * a task attempt. Each task attempt is one particular instance of a Tez Task @@ -46,7 +49,7 @@ public class TezTaskAttemptID extends TezID { public static final String ATTEMPT = "attempt"; private TezTaskID taskId; - private static TezIDCache tezTaskAttemptIDCache = new TezIDCache<>(); + private static Interner tezTaskAttemptIDCache = Interners.newWeakInterner(); // Public for Writable serialization. Verify if this is actually required. public TezTaskAttemptID() { @@ -58,12 +61,7 @@ public TezTaskAttemptID() { * @param id the task attempt number */ public static TezTaskAttemptID getInstance(TezTaskID taskID, int id) { - return tezTaskAttemptIDCache.getInstance(new TezTaskAttemptID(taskID, id)); - } - - @InterfaceAudience.Private - public static void clearCache() { - tezTaskAttemptIDCache.clear(); + return tezTaskAttemptIDCache.intern(new TezTaskAttemptID(taskID, id)); } private TezTaskAttemptID(TezTaskID taskId, int id) { diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java index 15b695c7a5..993df7c87a 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java @@ -28,6 +28,9 @@ import org.apache.tez.common.Preconditions; import org.apache.tez.util.FastNumberFormat; +import com.google.common.collect.Interner; +import com.google.common.collect.Interners; + /** * TaskID represents the immutable and unique identifier for * a Tez Task. Each TaskID encompasses multiple attempts made to @@ -51,7 +54,7 @@ public FastNumberFormat initialValue() { } }; - private static TezIDCache tezTaskIDCache = new TezIDCache<>(); + private static Interner tezTaskIDCache = Interners.newWeakInterner(); private TezVertexID vertexId; /** @@ -61,12 +64,7 @@ public FastNumberFormat initialValue() { */ public static TezTaskID getInstance(TezVertexID vertexID, int id) { Preconditions.checkArgument(vertexID != null, "vertexID cannot be null"); - return tezTaskIDCache.getInstance(new TezTaskID(vertexID, id)); - } - - @InterfaceAudience.Private - public static void clearCache() { - tezTaskIDCache.clear(); + return tezTaskIDCache.intern(new TezTaskID(vertexID, id)); } private TezTaskID(TezVertexID vertexID, int id) { diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java index b5a36abe33..5ecfca6a49 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java @@ -28,6 +28,9 @@ import org.apache.tez.common.Preconditions; import org.apache.tez.util.FastNumberFormat; +import com.google.common.collect.Interner; +import com.google.common.collect.Interners; + /** * TezVertexID represents the immutable and unique identifier for * a Vertex in a Tez DAG. Each TezVertexID encompasses multiple Tez Tasks. @@ -53,7 +56,7 @@ public FastNumberFormat initialValue() { } }; - private static TezIDCache tezVertexIDCache = new TezIDCache<>(); + private static Interner tezVertexIDCache = Interners.newWeakInterner(); private TezDAGID dagId; // Public for Writable serialization. Verify if this is actually required. @@ -67,12 +70,7 @@ public TezVertexID() { */ public static TezVertexID getInstance(TezDAGID dagId, int id) { Preconditions.checkArgument(dagId != null, "DagID cannot be null"); - return tezVertexIDCache.getInstance(new TezVertexID(dagId, id)); - } - - @InterfaceAudience.Private - public static void clearCache() { - tezVertexIDCache.clear(); + return tezVertexIDCache.intern(new TezVertexID(dagId, id)); } private TezVertexID(TezDAGID dagId, int id) { diff --git a/tez-common/src/main/java/org/apache/tez/util/StringInterner.java b/tez-common/src/main/java/org/apache/tez/util/StringInterner.java new file mode 100644 index 0000000000..b8c911307c --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/util/StringInterner.java @@ -0,0 +1,36 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.util; + +import com.google.common.collect.Interner; +import com.google.common.collect.Interners; + +/** + * A class to replace the {@code String.intern()}. The {@code String.intern()} + * has some well-known performance limitations, and should generally be avoided. + * Prefer Google's interner over the JDK's implementation. + */ +public final class StringInterner { + + private static final Interner STRING_INTERNER = + Interners.newWeakInterner(); + + private StringInterner() { + } + + public static String intern(final String str) { + return (str == null) ? null : STRING_INTERNER.intern(str); + } +} diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index 1352b68f26..61e1cb684a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -84,8 +84,6 @@ import org.apache.tez.dag.app.dag.event.DAGEventInternalError; import org.apache.tez.dag.app.dag.event.DAGEventTerminateDag; import org.apache.tez.dag.history.events.DAGRecoveredEvent; -import org.apache.tez.dag.records.TezTaskAttemptID; -import org.apache.tez.dag.records.TezTaskID; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -869,10 +867,6 @@ protected synchronized void handle(DAGAppMasterEvent event) { taskCommunicatorManager.dagComplete(cleanupEvent.getDag()); nodes.dagComplete(cleanupEvent.getDag()); containers.dagComplete(cleanupEvent.getDag()); - TezTaskAttemptID.clearCache(); - TezTaskID.clearCache(); - TezVertexID.clearCache(); - TezDAGID.clearCache(); LOG.info("Completed cleanup for DAG: name=" + cleanupEvent.getDag().getName() + ", with id=" + cleanupEvent.getDag().getID()); synchronized (idleStateLock) { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java index 8b16b2e8b7..593ea6cabf 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java @@ -41,11 +41,11 @@ import org.apache.tez.dag.app.rm.AMSchedulerEventTAStateUpdated; import org.apache.tez.runtime.api.TaskFailureType; import org.apache.tez.serviceplugins.api.TaskScheduler; +import org.apache.tez.util.StringInterner; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.util.StringInterner; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.NodeId; @@ -1399,7 +1399,7 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) { ta.container = container; ta.containerId = tEvent.getContainerId(); ta.containerNodeId = container.getNodeId(); - ta.nodeHttpAddress = StringInterner.weakIntern(container.getNodeHttpAddress()); + ta.nodeHttpAddress = StringInterner.intern(container.getNodeHttpAddress()); } if (event instanceof TaskAttemptEventContainerTerminatedBySystem) { @@ -1411,7 +1411,7 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) { ta.container = container; ta.containerId = tEvent.getContainerId(); ta.containerNodeId = container.getNodeId(); - ta.nodeHttpAddress = StringInterner.weakIntern(container.getNodeHttpAddress()); + ta.nodeHttpAddress = StringInterner.intern(container.getNodeHttpAddress()); } if (ta.recoveryData == null || @@ -1453,8 +1453,8 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent origEvent) { ta.container = container; ta.containerId = event.getContainerId(); ta.containerNodeId = container.getNodeId(); - ta.nodeHttpAddress = StringInterner.weakIntern(container.getNodeHttpAddress()); - ta.nodeRackName = StringInterner.weakIntern(RackResolver.resolve(ta.containerNodeId.getHost()) + ta.nodeHttpAddress = StringInterner.intern(container.getNodeHttpAddress()); + ta.nodeRackName = StringInterner.intern(RackResolver.resolve(ta.containerNodeId.getHost()) .getNetworkLocation()); ta.lastNotifyProgressTimestamp = ta.clock.getTime(); @@ -1463,7 +1463,7 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent origEvent) { // TODO Resolve to host / IP in case of a local address. InetSocketAddress nodeHttpInetAddr = NetUtils .createSocketAddr(ta.nodeHttpAddress); // TODO: Costly? - ta.trackerName = StringInterner.weakIntern(nodeHttpInetAddr.getHostName()); + ta.trackerName = StringInterner.intern(nodeHttpInetAddr.getHostName()); ta.httpPort = nodeHttpInetAddr.getPort(); ta.sendEvent(createDAGCounterUpdateEventTALaunched(ta)); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index 452dae5366..912339e15e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -52,7 +52,6 @@ import org.apache.hadoop.service.AbstractService; import org.apache.hadoop.service.ServiceOperations; import org.apache.hadoop.service.ServiceStateException; -import org.apache.hadoop.util.StringInterner; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.event.EventHandler; @@ -190,6 +189,7 @@ import org.apache.tez.runtime.api.impl.TezEvent; import org.apache.tez.state.OnStateChangedCallback; import org.apache.tez.state.StateMachineTez; +import org.apache.tez.util.StringInterner; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -965,7 +965,7 @@ public VertexImpl(TezVertexID vertexId, VertexPlan vertexPlan, StateChangeNotifier entityStatusTracker, Configuration dagOnlyConf) { this.vertexId = vertexId; this.vertexPlan = vertexPlan; - this.vertexName = StringInterner.weakIntern(vertexName); + this.vertexName = StringInterner.intern(vertexName); this.vertexConf = new Configuration(dagConf); this.vertexOnlyConf = new Configuration(dagOnlyConf); if (vertexPlan.hasVertexConf()) { @@ -4514,7 +4514,7 @@ public Resource getTaskResource() { } void addIO(String name) { - ioIndices.put(StringInterner.weakIntern(name), ioIndices.size()); + ioIndices.put(StringInterner.intern(name), ioIndices.size()); } @VisibleForTesting diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/DagInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/DagInfo.java index 85fcfcfb19..5067ec7aa8 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/DagInfo.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/DagInfo.java @@ -32,10 +32,10 @@ import org.apache.commons.collections4.bidimap.DualHashBidiMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.util.StringInterner; import org.apache.tez.client.CallerContext; import org.apache.tez.dag.api.event.VertexState; import org.apache.tez.dag.history.HistoryEventType; +import org.apache.tez.util.StringInterner; import org.codehaus.jettison.json.JSONArray; import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; @@ -100,7 +100,7 @@ public class DagInfo extends BaseInfo { Preconditions.checkArgument(jsonObject.getString(Constants.ENTITY_TYPE).equalsIgnoreCase (Constants.TEZ_DAG_ID)); - dagId = StringInterner.weakIntern(jsonObject.getString(Constants.ENTITY)); + dagId = StringInterner.intern(jsonObject.getString(Constants.ENTITY)); //Parse additional Info JSONObject otherInfoNode = jsonObject.getJSONObject(Constants.OTHER_INFO); @@ -140,7 +140,7 @@ public class DagInfo extends BaseInfo { diagnostics = otherInfoNode.optString(Constants.DIAGNOSTICS); failedTasks = otherInfoNode.optInt(Constants.NUM_FAILED_TASKS); JSONObject dagPlan = otherInfoNode.optJSONObject(Constants.DAG_PLAN); - name = StringInterner.weakIntern((dagPlan != null) ? (dagPlan.optString(Constants.DAG_NAME)) : null); + name = StringInterner.intern((dagPlan != null) ? (dagPlan.optString(Constants.DAG_NAME)) : null); if (dagPlan != null) { JSONArray vertices = dagPlan.optJSONArray(Constants.VERTICES); if (vertices != null) { @@ -152,7 +152,7 @@ public class DagInfo extends BaseInfo { } else { numVertices = 0; } - status = StringInterner.weakIntern(otherInfoNode.optString(Constants.STATUS)); + status = StringInterner.intern(otherInfoNode.optString(Constants.STATUS)); //parse name id mapping JSONObject vertexIDMappingJson = otherInfoNode.optJSONObject(Constants.VERTEX_NAME_ID_MAPPING); diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java index 3ce39bdccf..c8d400cc35 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java @@ -26,7 +26,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.util.StringInterner; import org.apache.tez.common.ATSConstants; import org.apache.tez.common.counters.DAGCounter; import org.apache.tez.common.counters.TaskCounter; @@ -34,6 +33,7 @@ import org.apache.tez.dag.api.oldrecords.TaskAttemptState; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.history.parser.utils.Utils; +import org.apache.tez.util.StringInterner; import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; @@ -51,7 +51,7 @@ public class TaskAttemptInfo extends BaseInfo { private static final Log LOG = LogFactory.getLog(TaskAttemptInfo.class); - private static final String SUCCEEDED = StringInterner.weakIntern(TaskAttemptState.SUCCEEDED.name()); + private static final String SUCCEEDED = TaskAttemptState.SUCCEEDED.name(); private final String taskAttemptId; private final long startTime; @@ -96,7 +96,7 @@ public String getTaskAttemptId() { jsonObject.getString(Constants.ENTITY_TYPE).equalsIgnoreCase (Constants.TEZ_TASK_ATTEMPT_ID)); - taskAttemptId = StringInterner.weakIntern(jsonObject.optString(Constants.ENTITY)); + taskAttemptId = StringInterner.intern(jsonObject.optString(Constants.ENTITY)); //Parse additional Info final JSONObject otherInfoNode = jsonObject.getJSONObject(Constants.OTHER_INFO); @@ -132,15 +132,15 @@ public String getTaskAttemptId() { diagnostics = otherInfoNode.optString(Constants.DIAGNOSTICS); creationTime = otherInfoNode.optLong(Constants.CREATION_TIME); - creationCausalTA = StringInterner.weakIntern( + creationCausalTA = StringInterner.intern( otherInfoNode.optString(Constants.CREATION_CAUSAL_ATTEMPT)); allocationTime = otherInfoNode.optLong(Constants.ALLOCATION_TIME); - containerId = StringInterner.weakIntern(otherInfoNode.optString(Constants.CONTAINER_ID)); + containerId = StringInterner.intern(otherInfoNode.optString(Constants.CONTAINER_ID)); String id = otherInfoNode.optString(Constants.NODE_ID); - nodeId = StringInterner.weakIntern((id != null) ? (id.split(":")[0]) : ""); + nodeId = StringInterner.intern((id != null) ? (id.split(":")[0]) : ""); logUrl = otherInfoNode.optString(Constants.COMPLETED_LOGS_URL); - status = StringInterner.weakIntern(otherInfoNode.optString(Constants.STATUS)); + status = StringInterner.intern(otherInfoNode.optString(Constants.STATUS)); container = new Container(containerId, nodeId); if (otherInfoNode.has(Constants.LAST_DATA_EVENTS)) { List eventInfo = Utils.parseDataEventDependencyFromJSON( @@ -154,7 +154,7 @@ public String getTaskAttemptId() { } } terminationCause = StringInterner - .weakIntern(otherInfoNode.optString(ATSConstants.TASK_ATTEMPT_ERROR_ENUM)); + .intern(otherInfoNode.optString(ATSConstants.TASK_ATTEMPT_ERROR_ENUM)); executionTimeInterval = (endTime > startTime) ? (endTime - startTime) : 0; } diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskInfo.java index de74ad29ad..43886fa5c6 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskInfo.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskInfo.java @@ -31,9 +31,9 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.util.StringInterner; import org.apache.tez.dag.api.oldrecords.TaskAttemptState; import org.apache.tez.dag.history.HistoryEventType; +import org.apache.tez.util.StringInterner; import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; @@ -71,7 +71,7 @@ public class TaskInfo extends BaseInfo { jsonObject.getString(Constants.ENTITY_TYPE).equalsIgnoreCase (Constants.TEZ_TASK_ID)); - taskId = StringInterner.weakIntern(jsonObject.optString(Constants.ENTITY)); + taskId = StringInterner.intern(jsonObject.optString(Constants.ENTITY)); //Parse additional Info final JSONObject otherInfoNode = jsonObject.getJSONObject(Constants.OTHER_INFO); @@ -106,10 +106,10 @@ public class TaskInfo extends BaseInfo { endTime = eTime; diagnostics = otherInfoNode.optString(Constants.DIAGNOSTICS); - successfulAttemptId = StringInterner.weakIntern( - otherInfoNode.optString(Constants.SUCCESSFUL_ATTEMPT_ID)); + successfulAttemptId = StringInterner + .intern(otherInfoNode.optString(Constants.SUCCESSFUL_ATTEMPT_ID)); scheduledTime = otherInfoNode.optLong(Constants.SCHEDULED_TIME); - status = StringInterner.weakIntern(otherInfoNode.optString(Constants.STATUS)); + status = StringInterner.intern(otherInfoNode.optString(Constants.STATUS)); } @Override diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java index efcce3bf76..c196c48cd4 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java @@ -30,9 +30,9 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.util.StringInterner; import org.apache.tez.dag.api.oldrecords.TaskState; import org.apache.tez.dag.history.HistoryEventType; +import org.apache.tez.util.StringInterner; import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; @@ -91,7 +91,7 @@ public class VertexInfo extends BaseInfo { jsonObject.getString(Constants.ENTITY_TYPE).equalsIgnoreCase (Constants.TEZ_VERTEX_ID)); - vertexId = StringInterner.weakIntern(jsonObject.optString(Constants.ENTITY)); + vertexId = StringInterner.intern(jsonObject.optString(Constants.ENTITY)); taskInfoMap = Maps.newHashMap(); inEdgeList = Lists.newLinkedList(); @@ -149,9 +149,9 @@ public class VertexInfo extends BaseInfo { killedTasks = otherInfoNode.optInt(Constants.NUM_KILLED_TASKS); numFailedTaskAttempts = otherInfoNode.optInt(Constants.NUM_FAILED_TASKS_ATTEMPTS); - vertexName = StringInterner.weakIntern(otherInfoNode.optString(Constants.VERTEX_NAME)); - processorClass = StringInterner.weakIntern(otherInfoNode.optString(Constants.PROCESSOR_CLASS_NAME)); - status = StringInterner.weakIntern(otherInfoNode.optString(Constants.STATUS)); + vertexName = StringInterner.intern(otherInfoNode.optString(Constants.VERTEX_NAME)); + processorClass = StringInterner.intern(otherInfoNode.optString(Constants.PROCESSOR_CLASS_NAME)); + status = StringInterner.intern(otherInfoNode.optString(Constants.STATUS)); } public static VertexInfo create(JSONObject vertexInfoObject) throws diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/utils/Utils.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/utils/Utils.java index aacec8ee12..08eb92b967 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/utils/Utils.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/utils/Utils.java @@ -22,7 +22,6 @@ import com.google.common.collect.Lists; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.util.StringInterner; import org.apache.log4j.ConsoleAppender; import org.apache.log4j.Level; import org.apache.log4j.Logger; @@ -34,6 +33,7 @@ import org.apache.tez.history.parser.datamodel.Constants; import org.apache.tez.history.parser.datamodel.Event; import org.apache.tez.history.parser.datamodel.TaskAttemptInfo.DataDependencyEvent; +import org.apache.tez.util.StringInterner; import org.codehaus.jettison.json.JSONArray; import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; @@ -106,7 +106,7 @@ public static List parseDataEventDependencyFromJSON(JSONObj for (int i=0; i groupVertices, InputDescriptor inputDescriptor) { - this.groupName = StringInterner.weakIntern(groupName); + this.groupName = StringInterner.intern(groupName); this.groupVertices = groupVertices; this.mergedInputDescriptor = inputDescriptor; } @@ -70,11 +70,11 @@ public void write(DataOutput out) throws IOException { @Override public void readFields(DataInput in) throws IOException { - groupName = StringInterner.weakIntern(Text.readString(in)); + groupName = StringInterner.intern(Text.readString(in)); int numMembers = in.readInt(); groupVertices = Lists.newArrayListWithCapacity(numMembers); for (int i=0; i getIOStatistics() { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java index 0c55a3a388..c97cfdf3ce 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java @@ -30,7 +30,6 @@ import org.apache.tez.runtime.library.common.shuffle.ShuffleEventHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.util.StringInterner; import org.apache.hadoop.conf.Configuration; import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.TezUtilsInternal; @@ -42,6 +41,7 @@ import org.apache.tez.runtime.library.common.InputAttemptIdentifier; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto; +import org.apache.tez.util.StringInterner; import com.google.protobuf.InvalidProtocolBufferException; @@ -167,7 +167,7 @@ private void processDataMovementEvent(DataMovementEvent dmEvent, DataMovementEve } } - scheduler.addKnownMapOutput(StringInterner.weakIntern(shufflePayload.getHost()), shufflePayload.getPort(), + scheduler.addKnownMapOutput(StringInterner.intern(shufflePayload.getHost()), shufflePayload.getPort(), partitionId, srcAttemptIdentifier); } @@ -200,7 +200,7 @@ private void processCompositeRoutedDataMovementEvent(CompositeRoutedDataMovement } } - scheduler.addKnownMapOutput(StringInterner.weakIntern(shufflePayload.getHost()), shufflePayload.getPort(), + scheduler.addKnownMapOutput(StringInterner.intern(shufflePayload.getHost()), shufflePayload.getPort(), partitionId, compositeInputAttemptIdentifier); } @@ -221,7 +221,9 @@ private void processTaskFailedEvent(InputFailedEvent ifEvent) { */ private CompositeInputAttemptIdentifier constructInputAttemptIdentifier(int targetIndex, int targetIndexCount, int version, DataMovementEventPayloadProto shufflePayload) { - String pathComponent = (shufflePayload.hasPathComponent()) ? StringInterner.weakIntern(shufflePayload.getPathComponent()) : null; + String pathComponent = (shufflePayload.hasPathComponent()) + ? StringInterner.intern(shufflePayload.getPathComponent()) + : null; int spillEventId = shufflePayload.getSpillId(); CompositeInputAttemptIdentifier srcAttemptIdentifier = null; if (shufflePayload.hasSpillId()) { diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java index e0fb15320c..754fbfa497 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java @@ -52,7 +52,6 @@ import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Text; -import org.apache.hadoop.util.StringInterner; import org.apache.tez.common.TezRuntimeFrameworkConfigs; import org.apache.tez.common.TezUtils; import org.apache.tez.common.counters.TaskCounter; @@ -78,6 +77,7 @@ import org.apache.tez.runtime.library.conf.OrderedPartitionedKVOutputConfig.SorterImpl; import org.apache.tez.runtime.library.partitioner.HashPartitioner; import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads; +import org.apache.tez.util.StringInterner; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; @@ -222,7 +222,7 @@ public void testSortLimitsWithLargeRecords() throws IOException { Text key = new Text(i + ""); //Generate random size between 1 MB to 100 MB. int valSize = ThreadLocalRandom.current().nextInt(1 * 1024 * 1024, 100 * 1024 * 1024); - String val = StringInterner.weakIntern(StringUtils.repeat("v", valSize)); + String val = StringInterner.intern(StringUtils.repeat("v", valSize)); sorter.write(key, new Text(val)); i = (i + 1) % 10; } diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java index 5944870c77..387b0cf99d 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java @@ -26,7 +26,6 @@ import java.util.Map; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.util.StringInterner; import org.apache.hadoop.util.ToolRunner; import org.apache.tez.analyzer.Analyzer; import org.apache.tez.analyzer.CSVResult; @@ -54,8 +53,8 @@ public class CriticalPathAnalyzer extends TezAnalyzerBase implements Analyzer { private static final Logger LOG = LoggerFactory.getLogger(CriticalPathAnalyzer.class); - String succeededState = StringInterner.weakIntern(TaskAttemptState.SUCCEEDED.name()); - String failedState = StringInterner.weakIntern(TaskAttemptState.FAILED.name()); + private static final String SUCCEEDED_STATE = TaskAttemptState.SUCCEEDED.name(); + private static final String FAILED_STATE = TaskAttemptState.FAILED.name(); public enum CriticalPathDependency { DATA_DEPENDENCY, @@ -130,8 +129,8 @@ public void analyze(DagInfo dagInfo) throws TezException { for (TaskInfo task : vertex.getTasks()) { for (TaskAttemptInfo attempt : task.getTaskAttempts()) { attempts.put(attempt.getTaskAttemptId(), attempt); - if (attempt.getStatus().equals(succeededState) || - attempt.getStatus().equals(failedState)) { + if (attempt.getStatus().equals(SUCCEEDED_STATE) || + attempt.getStatus().equals(FAILED_STATE)) { if (lastAttemptFinishTime < attempt.getFinishTime()) { lastAttempt = attempt; lastAttemptFinishTime = attempt.getFinishTime(); From 5eeccf0e318e22cdcbbe202a9f554f93d138c207 Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Wed, 30 Jun 2021 11:11:09 -0400 Subject: [PATCH 320/512] TEZ-4308: Add Whitespace in ShuffleScheduler Error Message (#122) --- .../shuffle/orderedgrouped/ShuffleScheduler.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java index 67681cedc7..540d44f409 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java @@ -1044,12 +1044,13 @@ boolean isShuffleHealthy(InputAttemptIdentifier srcAttempt) { && !fetcherHealthy && (!reducerProgressedEnough || reducerStalled)) { String errorMsg = (srcNameTrimmed + ": " - + "Shuffle failed with too many fetch failures and insufficient progress!" - + "failureCounts=" + failureCounts.size() + + "Shuffle failed with too many fetch failures and insufficient progress: " + + "[failureCounts=" + failureCounts.size() + ", pendingInputs=" + (numInputs - doneMaps) + ", fetcherHealthy=" + fetcherHealthy + ", reducerProgressedEnough=" + reducerProgressedEnough - + ", reducerStalled=" + reducerStalled); + + ", reducerStalled=" + reducerStalled) + + "]"; LOG.error(errorMsg); if (LOG.isDebugEnabled()) { LOG.debug("Host failures=" + hostFailures.keySet()); @@ -1109,7 +1110,7 @@ public void obsoleteInput(InputAttemptIdentifier srcAttempt) { } IOException exception = new IOException(srcAttempt + " is marked as obsoleteInput, but it " + "exists in shuffleInfoEventMap. Some data could have been already merged " - + "to memory/disk outputs. Failing the fetch early. eventInfo:" + eventInfo.toString()); + + "to memory/disk outputs. Failing the fetch early. eventInfo: " + eventInfo); String message = "Got obsolete event. Killing self as attempt's data could have been consumed"; killSelf(exception, message); return; @@ -1379,7 +1380,7 @@ protected Void callInternal() throws InterruptedException { } catch (InterruptedException e) { if (isShutdown.get()) { LOG.info(srcNameTrimmed + ": " + - "Interrupted while waiting for fetchers to complete" + + "Interrupted while waiting for fetchers to complete " + "and hasBeenShutdown. Breaking out of ShuffleSchedulerCallable loop"); Thread.currentThread().interrupt(); break; From 4684d06bef7898c6de4c6286c9f62cf2be3dcf92 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Wed, 30 Jun 2021 17:21:57 +0200 Subject: [PATCH 321/512] TEZ-4313: Apache Tez Release 0.10.1 (#134) --- Tez_DOAP.rdf | 7 +++++ docs/pom.xml | 2 +- .../markdown/releases/apache-tez-0-10-1.md | 30 +++++++++++++++++++ docs/src/site/markdown/releases/index.md | 1 + docs/src/site/site.xml | 3 +- hadoop-shim-impls/hadoop-shim-2.7/pom.xml | 2 +- hadoop-shim-impls/hadoop-shim-2.8/pom.xml | 2 +- hadoop-shim-impls/pom.xml | 2 +- hadoop-shim/pom.xml | 2 +- pom.xml | 2 +- tez-api/pom.xml | 2 +- tez-build-tools/pom.xml | 2 +- tez-common/pom.xml | 2 +- tez-dag/pom.xml | 2 +- tez-dist/pom.xml | 2 +- tez-examples/pom.xml | 2 +- tez-ext-service-tests/pom.xml | 2 +- tez-mapreduce/pom.xml | 2 +- tez-plugins/pom.xml | 2 +- tez-plugins/tez-aux-services/pom.xml | 2 +- tez-plugins/tez-history-parser/pom.xml | 2 +- .../tez-protobuf-history-plugin/pom.xml | 2 +- .../tez-yarn-timeline-cache-plugin/pom.xml | 2 +- .../pom.xml | 2 +- .../tez-yarn-timeline-history-with-fs/pom.xml | 2 +- tez-plugins/tez-yarn-timeline-history/pom.xml | 2 +- tez-runtime-internals/pom.xml | 2 +- tez-runtime-library/pom.xml | 2 +- tez-tests/pom.xml | 2 +- tez-tools/analyzers/job-analyzer/pom.xml | 2 +- tez-tools/analyzers/pom.xml | 2 +- tez-tools/pom.xml | 2 +- tez-tools/tez-javadoc-tools/pom.xml | 2 +- tez-tools/tez-tfile-parser/pom.xml | 2 +- tez-ui/pom.xml | 2 +- 35 files changed, 71 insertions(+), 32 deletions(-) create mode 100644 docs/src/site/markdown/releases/apache-tez-0-10-1.md diff --git a/Tez_DOAP.rdf b/Tez_DOAP.rdf index 055a45cb5f..d4c473d493 100644 --- a/Tez_DOAP.rdf +++ b/Tez_DOAP.rdf @@ -34,6 +34,13 @@ Java + + + Version 0.10.1 + 2021-07-01 + 0.10.1 + + Version 0.10.0 diff --git a/docs/pom.xml b/docs/pom.xml index c618cec4ba..d3bd5cf010 100644 --- a/docs/pom.xml +++ b/docs/pom.xml @@ -27,7 +27,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-docs pom diff --git a/docs/src/site/markdown/releases/apache-tez-0-10-1.md b/docs/src/site/markdown/releases/apache-tez-0-10-1.md new file mode 100644 index 0000000000..efe346aae0 --- /dev/null +++ b/docs/src/site/markdown/releases/apache-tez-0-10-1.md @@ -0,0 +1,30 @@ + + +Apache TEZ® 0.10.1 + +Apache TEZ® 0.10.1 +---------------------- + +- [Download Release Artifacts](http://www.apache.org/dyn/closer.lua/tez/0.10.1/) +- [Release Notes](0.10.1/release-notes.txt) +- Documentation + - [API Javadocs](0.10.1/tez-api-javadocs/index.html) : Documentation for the Tez APIs + - [Runtime Library Javadocs](0.10.1/tez-runtime-library-javadocs/index.html) : Documentation for built-in implementations of useful Inputs, Outputs, Processors etc. written based on the Tez APIs + - [Tez Mapreduce Javadocs](0.10.1/tez-mapreduce-javadocs/index.html) : Documentation for built-in implementations of Mapreduce compatible Inputs, Outputs, Processors etc. written based on the Tez APIs + - [Tez Configuration](0.10.1/tez-api-javadocs/configs/TezConfiguration.html) : Documentation for configurations of Tez. These configurations are typically specified in tez-site.xml. + - [Tez Runtime Configuration](0.10.1/tez-runtime-library-javadocs/configs/TezRuntimeConfiguration.html) : Documentation for runtime configurations of Tez. These configurations are typically specified by job submitters. diff --git a/docs/src/site/markdown/releases/index.md b/docs/src/site/markdown/releases/index.md index 26c18b0017..7d5c94871c 100644 --- a/docs/src/site/markdown/releases/index.md +++ b/docs/src/site/markdown/releases/index.md @@ -19,6 +19,7 @@ Releases ------------ +- [Apache TEZ® 0.10.1](./apache-tez-0-10-1.html) (Jul 01, 2021) - [Apache TEZ® 0.10.0](./apache-tez-0-10-0.html) (Oct 15, 2020) - [Apache TEZ® 0.9.2](./apache-tez-0-9-2.html) (Mar 29, 2019) - [Apache TEZ® 0.9.1](./apache-tez-0-9-1.html) (Jan 04, 2018) diff --git a/docs/src/site/site.xml b/docs/src/site/site.xml index 04187dd3f7..c8e6479a8f 100644 --- a/docs/src/site/site.xml +++ b/docs/src/site/site.xml @@ -129,7 +129,8 @@ - + + diff --git a/hadoop-shim-impls/hadoop-shim-2.7/pom.xml b/hadoop-shim-impls/hadoop-shim-2.7/pom.xml index d75ed460c1..2557e036c3 100644 --- a/hadoop-shim-impls/hadoop-shim-2.7/pom.xml +++ b/hadoop-shim-impls/hadoop-shim-2.7/pom.xml @@ -19,7 +19,7 @@ hadoop-shim-impls org.apache.tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT 4.0.0 hadoop-shim-2.7 diff --git a/hadoop-shim-impls/hadoop-shim-2.8/pom.xml b/hadoop-shim-impls/hadoop-shim-2.8/pom.xml index 9450c75c60..0a149cca04 100644 --- a/hadoop-shim-impls/hadoop-shim-2.8/pom.xml +++ b/hadoop-shim-impls/hadoop-shim-2.8/pom.xml @@ -19,7 +19,7 @@ hadoop-shim-impls org.apache.tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT 4.0.0 hadoop-shim-2.8 diff --git a/hadoop-shim-impls/pom.xml b/hadoop-shim-impls/pom.xml index 77192a016d..25cf3fd25a 100644 --- a/hadoop-shim-impls/pom.xml +++ b/hadoop-shim-impls/pom.xml @@ -20,7 +20,7 @@ tez org.apache.tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT hadoop-shim-impls pom diff --git a/hadoop-shim/pom.xml b/hadoop-shim/pom.xml index 383c5cf2b6..18ddaca3df 100644 --- a/hadoop-shim/pom.xml +++ b/hadoop-shim/pom.xml @@ -20,7 +20,7 @@ tez org.apache.tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT hadoop-shim diff --git a/pom.xml b/pom.xml index c1f0682df1..1cf2bcac93 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ org.apache.tez tez pom - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez diff --git a/tez-api/pom.xml b/tez-api/pom.xml index 95a7610ee4..8cab40c172 100644 --- a/tez-api/pom.xml +++ b/tez-api/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-api diff --git a/tez-build-tools/pom.xml b/tez-build-tools/pom.xml index baf75b18ec..232a20be89 100644 --- a/tez-build-tools/pom.xml +++ b/tez-build-tools/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-build-tools diff --git a/tez-common/pom.xml b/tez-common/pom.xml index 6b82cdc7f8..5fd6910b18 100644 --- a/tez-common/pom.xml +++ b/tez-common/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-common diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml index c0dccc8642..723082c54e 100644 --- a/tez-dag/pom.xml +++ b/tez-dag/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-dag diff --git a/tez-dist/pom.xml b/tez-dist/pom.xml index ea6041d76a..741ed9d1e9 100644 --- a/tez-dist/pom.xml +++ b/tez-dist/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-dist diff --git a/tez-examples/pom.xml b/tez-examples/pom.xml index 3ac8534b6d..f6ab04c020 100644 --- a/tez-examples/pom.xml +++ b/tez-examples/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-examples diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml index 8dfad0d74e..327d457187 100644 --- a/tez-ext-service-tests/pom.xml +++ b/tez-ext-service-tests/pom.xml @@ -20,7 +20,7 @@ tez org.apache.tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-ext-service-tests diff --git a/tez-mapreduce/pom.xml b/tez-mapreduce/pom.xml index a01d028039..ee7c318ff0 100644 --- a/tez-mapreduce/pom.xml +++ b/tez-mapreduce/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-mapreduce diff --git a/tez-plugins/pom.xml b/tez-plugins/pom.xml index 6ce710ea27..27f19289ce 100644 --- a/tez-plugins/pom.xml +++ b/tez-plugins/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-plugins pom diff --git a/tez-plugins/tez-aux-services/pom.xml b/tez-plugins/tez-aux-services/pom.xml index 7279eaf684..965663ce4a 100644 --- a/tez-plugins/tez-aux-services/pom.xml +++ b/tez-plugins/tez-aux-services/pom.xml @@ -20,7 +20,7 @@ tez-plugins org.apache.tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-aux-services diff --git a/tez-plugins/tez-history-parser/pom.xml b/tez-plugins/tez-history-parser/pom.xml index 1b1d4e8285..5240bba1d8 100644 --- a/tez-plugins/tez-history-parser/pom.xml +++ b/tez-plugins/tez-history-parser/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez-plugins - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-history-parser diff --git a/tez-plugins/tez-protobuf-history-plugin/pom.xml b/tez-plugins/tez-protobuf-history-plugin/pom.xml index f84bb23dad..ab06981f3d 100644 --- a/tez-plugins/tez-protobuf-history-plugin/pom.xml +++ b/tez-plugins/tez-protobuf-history-plugin/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-protobuf-history-plugin diff --git a/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml b/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml index c817dcfbd1..942d43658a 100644 --- a/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml +++ b/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-yarn-timeline-cache-plugin diff --git a/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml b/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml index 27288953fa..50ed0bd77d 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-yarn-timeline-history-with-acls diff --git a/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml b/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml index 8939263799..4ba760f1aa 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-yarn-timeline-history-with-fs diff --git a/tez-plugins/tez-yarn-timeline-history/pom.xml b/tez-plugins/tez-yarn-timeline-history/pom.xml index 99406650e6..78b83dd877 100644 --- a/tez-plugins/tez-yarn-timeline-history/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-yarn-timeline-history diff --git a/tez-runtime-internals/pom.xml b/tez-runtime-internals/pom.xml index 8fe7d79e40..f7e14dae59 100644 --- a/tez-runtime-internals/pom.xml +++ b/tez-runtime-internals/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-runtime-internals diff --git a/tez-runtime-library/pom.xml b/tez-runtime-library/pom.xml index ecabf8976a..2caff344ec 100644 --- a/tez-runtime-library/pom.xml +++ b/tez-runtime-library/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-runtime-library diff --git a/tez-tests/pom.xml b/tez-tests/pom.xml index d76f27a1a9..72f95d56ce 100644 --- a/tez-tests/pom.xml +++ b/tez-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-tests diff --git a/tez-tools/analyzers/job-analyzer/pom.xml b/tez-tools/analyzers/job-analyzer/pom.xml index a857bb3a53..89ae9088ff 100644 --- a/tez-tools/analyzers/job-analyzer/pom.xml +++ b/tez-tools/analyzers/job-analyzer/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-perf-analyzer - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-job-analyzer diff --git a/tez-tools/analyzers/pom.xml b/tez-tools/analyzers/pom.xml index 64484d3915..3c989e68ec 100644 --- a/tez-tools/analyzers/pom.xml +++ b/tez-tools/analyzers/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-tools - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-perf-analyzer pom diff --git a/tez-tools/pom.xml b/tez-tools/pom.xml index 34f9f992e0..bffe9abc5d 100644 --- a/tez-tools/pom.xml +++ b/tez-tools/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-tools pom diff --git a/tez-tools/tez-javadoc-tools/pom.xml b/tez-tools/tez-javadoc-tools/pom.xml index e0405b685b..6b0546674b 100644 --- a/tez-tools/tez-javadoc-tools/pom.xml +++ b/tez-tools/tez-javadoc-tools/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-tools - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-javadoc-tools diff --git a/tez-tools/tez-tfile-parser/pom.xml b/tez-tools/tez-tfile-parser/pom.xml index c2f40201a3..a258a5e92b 100644 --- a/tez-tools/tez-tfile-parser/pom.xml +++ b/tez-tools/tez-tfile-parser/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-tools - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-tfile-parser diff --git a/tez-ui/pom.xml b/tez-ui/pom.xml index e52eb52dfa..80cb05944b 100644 --- a/tez-ui/pom.xml +++ b/tez-ui/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-ui war From 47a59abba5a44570eff61639d53689518569cab2 Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Wed, 30 Jun 2021 12:01:06 -0400 Subject: [PATCH 322/512] TEZ-4267: Remove Superfluous Code from DAGAppMaster (#90) * TEZ-4267: Remove Superfluous Code from DAGAppMaster * Remove superfluous variable * Remove superfluous comment Co-authored-by: David Mollitor --- .../org/apache/tez/dag/app/DAGAppMaster.java | 52 ++++++------------- 1 file changed, 17 insertions(+), 35 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index 61e1cb684a..abc10bd86c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -190,7 +190,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; -import com.google.common.base.Joiner; import org.apache.tez.common.Preconditions; import com.google.common.collect.Maps; import com.google.common.util.concurrent.ListeningExecutorService; @@ -224,7 +223,6 @@ public class DAGAppMaster extends AbstractService { * Priority of the DAGAppMaster shutdown hook. */ public static final int SHUTDOWN_HOOK_PRIORITY = 30; - private static final Joiner PATH_JOINER = Joiner.on('/'); @VisibleForTesting static final String INVALID_SESSION_ERR_MSG = "Initial application attempt in session mode failed. " @@ -311,7 +309,6 @@ public class DAGAppMaster extends AbstractService { /** * set of already executed dag names. */ - Set dagNames = new HashSet(); Set dagIDs = new HashSet(); protected boolean isLastAMRetry = false; @@ -371,19 +368,17 @@ public DAGAppMaster(ApplicationAttemptId applicationAttemptId, this.containerID.toString(), this.appMasterUgi.getShortUserName()); LOG.info("Created DAGAppMaster for application " + applicationAttemptId - + ", versionInfo=" + dagVersionInfo.toString()); + + ", versionInfo=" + dagVersionInfo); TezCommonUtils.logCredentials(LOG, this.appMasterUgi.getCredentials(), "am"); } // Pull this WebAppUtils function into Tez until YARN-4186 - public static String getRunningLogURL(String nodeHttpAddress, + private static String getRunningLogURL(String nodeHttpAddress, String containerId, String user) { - if (nodeHttpAddress == null || nodeHttpAddress.isEmpty() - || containerId == null || containerId.isEmpty() || user == null - || user.isEmpty()) { + if (containerId.isEmpty() || user == null | user.isEmpty()) { return null; } - return PATH_JOINER.join(nodeHttpAddress, "node", "containerlogs", + return String.format("%s/node/containerlogs/%s/%s", nodeHttpAddress, containerId, user); } @@ -695,8 +690,7 @@ private void handleInternalError(String errDiagnosticsPrefix, String errDiagDagE state = DAGAppMasterState.ERROR; if (currentDAG != null) { _updateLoggers(currentDAG, "_post"); - String errDiagnostics = errDiagnosticsPrefix + ". Aborting dag: " + currentDAG.getID(); - LOG.info(errDiagnostics); + LOG.info(errDiagnosticsPrefix + ". Aborting dag: " + currentDAG.getID()); // Inform the current DAG about the error sendEvent(new DAGEventInternalError(currentDAG.getID(), errDiagDagEvent)); } else { @@ -758,8 +752,8 @@ protected synchronized void handle(DAGAppMasterEvent event) { DAGAppMasterEventDAGFinished finishEvt = (DAGAppMasterEventDAGFinished) event; String timeStamp = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss").format(Calendar.getInstance().getTime()); - System.err.println(timeStamp + " Completed Dag: " + finishEvt.getDAGId().toString()); - System.out.println(timeStamp + " Completed Dag: " + finishEvt.getDAGId().toString()); + System.err.println(timeStamp + " Completed Dag: " + finishEvt.getDAGId()); + System.out.println(timeStamp + " Completed Dag: " + finishEvt.getDAGId()); // Stop vertex services if any stopVertexServices(currentDAG); if (!isSession) { @@ -767,13 +761,11 @@ protected synchronized void handle(DAGAppMasterEvent event) { this.taskSchedulerManager.setShouldUnregisterFlag(); _updateLoggers(currentDAG, "_post"); setStateOnDAGCompletion(); - LOG.info("Shutting down on completion of dag:" + - finishEvt.getDAGId().toString()); + LOG.info("Shutting down on completion of dag:" + finishEvt.getDAGId()); shutdownHandler.shutdown(); } else { - LOG.info("DAG completed, dagId=" - + finishEvt.getDAGId().toString() - + ", dagState=" + finishEvt.getDAGState()); + LOG.info("DAG completed, dagId=" + finishEvt.getDAGId() + ", dagState=" + + finishEvt.getDAGState()); lastDAGCompletionTime = clock.getTime(); _updateLoggers(currentDAG, "_post"); if (this.historyEventHandler.hasRecoveryFailed()) { @@ -1028,9 +1020,8 @@ DAGImpl createDAG(DAGPlan dagPB, TezDAGID dagId) { try { if (LOG.isDebugEnabled()) { - LOG.debug("JSON dump for submitted DAG, dagId=" + dagId.toString() - + ", json=" - + DAGUtils.generateSimpleJSONPlan(dagPB).toString()); + LOG.debug("JSON dump for submitted DAG, dagId=" + dagId + ", json=" + + DAGUtils.generateSimpleJSONPlan(dagPB)); } } catch (JSONException e) { LOG.warn("Failed to generate json for DAG", e); @@ -1038,7 +1029,7 @@ DAGImpl createDAG(DAGPlan dagPB, TezDAGID dagId) { writeDebugArtifacts(dagPB, newDag); return newDag; - } // end createDag() + } private void writeDebugArtifacts(DAGPlan dagPB, DAGImpl newDag) { boolean debugArtifacts = @@ -1052,7 +1043,7 @@ private void writeDebugArtifacts(DAGPlan dagPB, DAGImpl newDag) { private void writePBTextFile(DAG dag) { String logFile = logDirs[new Random().nextInt(logDirs.length)] + File.separatorChar - + dag.getID().toString() + "-" + TezConstants.TEZ_PB_PLAN_TEXT_NAME; + + dag.getID() + "-" + TezConstants.TEZ_PB_PLAN_TEXT_NAME; LOG.info("Writing DAG plan to: " + logFile); File outFile = new File(logFile); @@ -1061,7 +1052,7 @@ private void writePBTextFile(DAG dag) { printWriter.println(TezUtilsInternal.convertDagPlanToString(dag.getJobPlan())); printWriter.close(); } catch (IOException e) { - LOG.warn("Failed to write TEZ_PLAN to " + outFile.toString(), e); + LOG.warn("Failed to write TEZ_PLAN to " + outFile, e); } } @@ -2256,15 +2247,6 @@ public void handle(VertexEvent event) { } } - private static void validateInputParam(String value, String param) - throws IOException { - if (value == null) { - String msg = param + " is null"; - LOG.error(msg); - throw new IOException(msg); - } - } - private long checkAndHandleDAGClientTimeout() throws TezException { if (EnumSet.of(DAGAppMasterState.NEW, DAGAppMasterState.RECOVERING).contains(this.state) || sessionStopped.get()) { @@ -2333,8 +2315,8 @@ public static void main(String[] args) { clientVersion = VersionInfo.UNKNOWN; } - validateInputParam(appSubmitTimeStr, - ApplicationConstants.APP_SUBMIT_TIME_ENV); + Objects.requireNonNull(appSubmitTimeStr, + ApplicationConstants.APP_SUBMIT_TIME_ENV + " is null"); ContainerId containerId = ConverterUtils.toContainerId(containerIdStr); ApplicationAttemptId applicationAttemptId = From 8f3044c9dc0f46c113c4d404c1b5fd4e317dc91b Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Wed, 30 Jun 2021 21:32:53 +0200 Subject: [PATCH 323/512] TEZ-4314: Fix some plugin versions in site generation (#131) --- docs/pom.xml | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/docs/pom.xml b/docs/pom.xml index d3bd5cf010..0d55177641 100644 --- a/docs/pom.xml +++ b/docs/pom.xml @@ -467,6 +467,7 @@ org.apache.maven.plugins maven-site-plugin + 2.4 ./target @@ -487,6 +488,7 @@ org.apache.maven.plugins maven-project-info-reports-plugin + 3.1.2 ${project.basedir}/src/site/custom/project-info-report.properties false @@ -494,10 +496,10 @@ - project-team - mailing-list - issue-tracking - license + team + mailing-lists + issue-management + licenses From b31501d9b7f2cc8f4cc4283bd285dc48609dc77b Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Thu, 1 Jul 2021 00:38:57 +0200 Subject: [PATCH 324/512] TEZ-4125: Upgrade to Hadoop 3.2.x (#126) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 1cf2bcac93..1d8194ad58 100644 --- a/pom.xml +++ b/pom.xml @@ -58,7 +58,7 @@ ${user.home}/clover.license 27.0-jre - 3.1.3 + 3.2.2 4.0.52.Final 0.13.0 1.19 From 464d86d8ffd5ee8e3bd7a4d48ab64b10e3db36b0 Mon Sep 17 00:00:00 2001 From: Igor Dvorzhak Date: Fri, 9 Jul 2021 09:03:24 -0700 Subject: [PATCH 325/512] TEZ-4214. Upgrade RoaringBitmap version to 0.7.45 (#74) --- pom.xml | 5 +++-- tez-plugins/tez-aux-services/pom.xml | 1 + 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 1d8194ad58..ba65cc6ad6 100644 --- a/pom.xml +++ b/pom.xml @@ -64,11 +64,12 @@ 1.19 1.7.30 2.5.0 + 0.7.45 ${env.PROTOC_PATH} scm:git:https://gitbox.apache.org/repos/asf/tez.git 1.4 3.0.5 - 3.1.1 + 3.1.1 8.35 1.3.6 ${project.build.directory}/tmp @@ -232,7 +233,7 @@ org.roaringbitmap RoaringBitmap - 0.5.21 + ${roaringbitmap.version} org.slf4j diff --git a/tez-plugins/tez-aux-services/pom.xml b/tez-plugins/tez-aux-services/pom.xml index 965663ce4a..2a936d6fa5 100644 --- a/tez-plugins/tez-aux-services/pom.xml +++ b/tez-plugins/tez-aux-services/pom.xml @@ -195,6 +195,7 @@ org.apache.maven.plugins maven-shade-plugin + 3.2.4 package From 3f541d06f4aad0e1ad051bec273a05a678fd7776 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Mon, 9 Aug 2021 08:24:34 +0200 Subject: [PATCH 326/512] TEZ-4231: Fix multiple history parser and event converter issues (#123) (Laszlo Bodor reviewed by Rajesh Balamohan) --- .../history/parser/ProtoHistoryParser.java | 1 - .../history/parser/SimpleHistoryParser.java | 31 +++- .../history/parser/datamodel/BaseInfo.java | 16 +- .../tez/history/parser/utils/Utils.java | 4 +- .../HistoryEventProtoJsonConversion.java | 12 +- .../org/apache/tez/analyzer/Analyzer.java | 8 - .../tez/analyzer/plugins/AnalyzerDriver.java | 6 + .../plugins/ContainerReuseAnalyzer.java | 9 +- .../plugins/CriticalPathAnalyzer.java | 11 +- .../analyzer/plugins/DagOverviewAnalyzer.java | 132 ++++++++++++++++ .../analyzer/plugins/HungTaskAnalyzer.java | 143 ++++++++++++++++++ .../plugins/InputReadErrorAnalyzer.java | 94 ++++++++++++ .../analyzer/plugins/LocalityAnalyzer.java | 10 +- .../plugins/OneOnOneEdgeAnalyzer.java | 9 +- .../analyzer/plugins/ShuffleTimeAnalyzer.java | 9 +- .../tez/analyzer/plugins/SkewAnalyzer.java | 13 +- .../analyzer/plugins/SlowNodeAnalyzer.java | 9 +- .../analyzer/plugins/SlowTaskIdentifier.java | 11 +- .../plugins/SlowestVertexAnalyzer.java | 8 +- .../analyzer/plugins/SpillAnalyzerImpl.java | 9 +- .../plugins/TaskAssignmentAnalyzer.java | 27 ++-- .../TaskAttemptResultStatisticsAnalyzer.java | 11 +- .../plugins/TaskConcurrencyAnalyzer.java | 8 +- .../tez/analyzer/plugins/TezAnalyzerBase.java | 7 +- .../VertexLevelCriticalPathAnalyzer.java | 9 +- 25 files changed, 462 insertions(+), 145 deletions(-) create mode 100644 tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/DagOverviewAnalyzer.java create mode 100644 tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/HungTaskAnalyzer.java create mode 100644 tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/InputReadErrorAnalyzer.java diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ProtoHistoryParser.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ProtoHistoryParser.java index d28fd67d4f..397a46fde9 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ProtoHistoryParser.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ProtoHistoryParser.java @@ -98,7 +98,6 @@ public boolean hasNext() throws IOException { return message != null; } catch (java.io.EOFException e) { reader.close(); - if (!fileIt.hasNext()) { return false; } else { diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java index db3f6488e0..c1711ce2cb 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java @@ -236,11 +236,13 @@ protected void readEventsFromSource(String dagId, JSONObjectSource source, // time etc). if (dagJson == null) { dagJson = jsonObject; - } else if (dagJson.optJSONObject(ATSConstants.OTHER_INFO) - .optJSONObject(ATSConstants.DAG_PLAN) == null) { - // if DAG_PLAN is not filled already, let's try to fetch it from other - dagJson.getJSONObject(ATSConstants.OTHER_INFO).put(ATSConstants.DAG_PLAN, jsonObject - .getJSONObject(ATSConstants.OTHER_INFO).getJSONObject(ATSConstants.DAG_PLAN)); + } else { + if (dagJson.optJSONObject(ATSConstants.OTHER_INFO).optJSONObject(ATSConstants.DAG_PLAN) == null) { + // if DAG_PLAN is not filled already, let's try to fetch it from other + dagJson.getJSONObject(ATSConstants.OTHER_INFO).put(ATSConstants.DAG_PLAN, + jsonObject.getJSONObject(ATSConstants.OTHER_INFO).getJSONObject(ATSConstants.DAG_PLAN)); + } + mergeSubJSONArray(jsonObject, dagJson, Constants.EVENTS); } JSONArray relatedEntities = dagJson.optJSONArray(Constants .RELATED_ENTITIES); @@ -268,6 +270,8 @@ protected void readEventsFromSource(String dagId, JSONObjectSource source, } if (!vertexJsonMap.containsKey(vertexName)) { vertexJsonMap.put(vertexName, jsonObject); + } else { + mergeSubJSONArray(jsonObject, vertexJsonMap.get(vertexName), Constants.EVENTS); } populateOtherInfo(jsonObject.optJSONObject(Constants.OTHER_INFO), vertexName, vertexJsonMap); break; @@ -281,6 +285,8 @@ protected void readEventsFromSource(String dagId, JSONObjectSource source, } if (!taskJsonMap.containsKey(taskName)) { taskJsonMap.put(taskName, jsonObject); + } else { + mergeSubJSONArray(jsonObject, taskJsonMap.get(taskName), Constants.EVENTS); } populateOtherInfo(jsonObject.optJSONObject(Constants.OTHER_INFO), taskName, taskJsonMap); break; @@ -294,6 +300,8 @@ protected void readEventsFromSource(String dagId, JSONObjectSource source, } if (!attemptJsonMap.containsKey(taskAttemptName)) { attemptJsonMap.put(taskAttemptName, jsonObject); + } else { + mergeSubJSONArray(jsonObject, attemptJsonMap.get(taskAttemptName), Constants.EVENTS); } populateOtherInfo(jsonObject.optJSONObject(Constants.OTHER_INFO), taskAttemptName, attemptJsonMap); break; @@ -311,4 +319,17 @@ protected void readEventsFromSource(String dagId, JSONObjectSource source, "Please provide a valid/complete history log file containing " + dagId); } } + + private void mergeSubJSONArray(JSONObject source, JSONObject destination, String key) + throws JSONException { + if (source.optJSONArray(key) == null) { + source.put(key, new JSONArray()); + } + if (destination.optJSONArray(key) == null) { + destination.put(key, new JSONArray()); + } + for (int i = 0; i < source.getJSONArray(key).length(); i++) { + destination.getJSONArray(key).put(source.getJSONArray(key).get(i)); + } + } } \ No newline at end of file diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseInfo.java index 3f9666a950..783f486a15 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseInfo.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseInfo.java @@ -44,8 +44,20 @@ public abstract class BaseInfo { BaseInfo(JSONObject jsonObject) throws JSONException { final JSONObject otherInfoNode = jsonObject.getJSONObject(Constants.OTHER_INFO); //parse tez counters - tezCounters = Utils.parseTezCountersFromJSON( - otherInfoNode.optJSONObject(Constants.COUNTERS)); + JSONObject countersObj = otherInfoNode.optJSONObject(Constants.COUNTERS); + if (countersObj == null) { + /* + * This is a workaround for formatting differences, where a TaskFinishedEvent's + * counter is a correct json object shown as string, but VertexFinishedEvent's + * counter is an encoded json string, so the latter is interpreted as a String + * while parsing. The issue might be somewhere while converting these event objects + * to proto (HistoryEventProtoConverter). Even if should be fixed there, + * already generated events should be parsed correctly, hence this workaround. + * Will be investigated in the scope of TEZ-4324. + */ + countersObj = new JSONObject(otherInfoNode.optString(Constants.COUNTERS)); + } + tezCounters = Utils.parseTezCountersFromJSON(countersObj); //parse events eventList = Lists.newArrayList(); diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/utils/Utils.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/utils/Utils.java index 08eb92b967..94b50a6c5c 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/utils/Utils.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/utils/Utils.java @@ -128,12 +128,12 @@ public static void parseEvents(JSONArray eventNodes, List eventList) thro JSONObject eventNode = eventNodes.optJSONObject(i); final String eventInfo = eventNode.optString(Constants.EVENT_INFO); final String eventType = eventNode.optString(Constants.EVENT_TYPE); - final long time = eventNode.optLong(Constants.EVENT_TIME_STAMP); + final long time = eventNode.optLong(Constants.EVENT_TIME_STAMP) == 0 + ? eventNode.optLong(Constants.TIMESTAMP) : eventNode.optLong(Constants.EVENT_TIME_STAMP); Event event = new Event(eventInfo, eventType, time); eventList.add(event); - } } diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoJsonConversion.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoJsonConversion.java index 26e20abeb3..ef84b2ec61 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoJsonConversion.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoJsonConversion.java @@ -556,12 +556,12 @@ private static JSONObject convertTaskFinishedEvent(HistoryEventProto event) thro events.put(finishEvent); jsonObject.put(ATSConstants.EVENTS, events); - long startTime = getLongDataValueByKey(event, ATSConstants.START_TIME); + long timeTaken = getLongDataValueByKey(event, ATSConstants.TIME_TAKEN); JSONObject otherInfo = new JSONObject(); - otherInfo.put(ATSConstants.START_TIME, startTime); + otherInfo.put(ATSConstants.START_TIME, event.getEventTime() - timeTaken); otherInfo.put(ATSConstants.FINISH_TIME, event.getEventTime()); - otherInfo.put(ATSConstants.TIME_TAKEN, event.getEventTime() - startTime); + otherInfo.put(ATSConstants.TIME_TAKEN, timeTaken); otherInfo.put(ATSConstants.STATUS, getDataValueByKey(event, ATSConstants.STATUS)); otherInfo.put(ATSConstants.DIAGNOSTICS, getDataValueByKey(event, ATSConstants.DIAGNOSTICS)); @@ -620,11 +620,13 @@ private static JSONObject convertVertexFinishedEvent(HistoryEventProto event) events.put(finishEvent); jsonObject.put(ATSConstants.EVENTS, events); - long startTime = getLongDataValueByKey(event, ATSConstants.START_TIME); + long timeTaken = getLongDataValueByKey(event, ATSConstants.TIME_TAKEN); JSONObject otherInfo = new JSONObject(); + otherInfo.put(ATSConstants.START_TIME, event.getEventTime() - timeTaken); otherInfo.put(ATSConstants.FINISH_TIME, event.getEventTime()); - otherInfo.put(ATSConstants.TIME_TAKEN, (event.getEventTime() - startTime)); + otherInfo.put(ATSConstants.TIME_TAKEN, timeTaken); + otherInfo.put(ATSConstants.STATUS, getDataValueByKey(event, ATSConstants.STATUS)); otherInfo.put(ATSConstants.DIAGNOSTICS, getDataValueByKey(event, ATSConstants.DIAGNOSTICS)); otherInfo.put(ATSConstants.COUNTERS, getJSONDataValueByKey(event, ATSConstants.COUNTERS)); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/Analyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/Analyzer.java index 6021c5897e..1f0a7ad62c 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/Analyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/Analyzer.java @@ -18,7 +18,6 @@ package org.apache.tez.analyzer; -import org.apache.hadoop.conf.Configuration; import org.apache.tez.dag.api.TezException; import org.apache.tez.history.parser.datamodel.DagInfo; @@ -54,11 +53,4 @@ public interface Analyzer { * @return description of analyzer */ public String getDescription(); - - /** - * Get config properties related to this analyzer - * - * @return config related to analyzer - */ - public Configuration getConfiguration(); } diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/AnalyzerDriver.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/AnalyzerDriver.java index cad0d98d75..294527cd3e 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/AnalyzerDriver.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/AnalyzerDriver.java @@ -48,12 +48,18 @@ public static void main(String argv[]){ "Print task-to-node assignment details of a DAG"); pgd.addClass("TaskAttemptResultStatisticsAnalyzer", TaskAttemptResultStatisticsAnalyzer.class, "Print vertex:node:status level details of task attempt results"); + pgd.addClass("InputReadErrorAnalyzer", InputReadErrorAnalyzer.class, + "Print INPUT_READ_ERROR sources"); pgd.addClass("TaskConcurrencyAnalyzer", TaskConcurrencyAnalyzer.class, "Print the task concurrency details in a DAG"); pgd.addClass("VertexLevelCriticalPathAnalyzer", VertexLevelCriticalPathAnalyzer.class, "Find critical path at vertex level in a DAG"); pgd.addClass("OneOnOneEdgeAnalyzer", OneOnOneEdgeAnalyzer.class, "Find out schedule misses in 1:1 edges in a DAG"); + pgd.addClass("DagOverviewAnalyzer", DagOverviewAnalyzer.class, + "Print basic dag information (dag/vertex events)"); + pgd.addClass("TaskHangAnalyzer", HungTaskAnalyzer.class, + "Print all vertices/tasks and their last attempts with status/duration/node"); exitCode = pgd.run(argv); } catch(Throwable e){ e.printStackTrace(); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/ContainerReuseAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/ContainerReuseAnalyzer.java index 5b862f87f7..553ff0e2cc 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/ContainerReuseAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/ContainerReuseAnalyzer.java @@ -39,15 +39,13 @@ */ public class ContainerReuseAnalyzer extends TezAnalyzerBase implements Analyzer { - private final Configuration config; - private static final String[] headers = { "vertexName", "taskAttempts", "node", "containerId", "reuseCount" }; private final CSVResult csvResult; public ContainerReuseAnalyzer(Configuration config) { - this.config = config; + super(config); this.csvResult = new CSVResult(headers); } @@ -82,11 +80,6 @@ public String getDescription() { return "Get details on container reuse analysis"; } - @Override - public Configuration getConfiguration() { - return config; - } - public static void main(String[] args) throws Exception { Configuration config = new Configuration(); ContainerReuseAnalyzer analyzer = new ContainerReuseAnalyzer(config); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java index 387b0cf99d..3f5e3004b8 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java @@ -113,10 +113,11 @@ public List getNotes() { ArrayList concurrencyByTime = Lists.newArrayList(); public CriticalPathAnalyzer() { + super(new Configuration()); } public CriticalPathAnalyzer(Configuration conf) { - setConf(conf); + super(conf); } @Override @@ -643,13 +644,9 @@ public String getDescription() { return "Analyze critical path of the DAG"; } - @Override - public Configuration getConfiguration() { - return getConf(); - } - public static void main(String[] args) throws Exception { - int res = ToolRunner.run(new Configuration(), new CriticalPathAnalyzer(), args); + Configuration config = new Configuration(); + int res = ToolRunner.run(config, new CriticalPathAnalyzer(config), args); System.exit(res); } diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/DagOverviewAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/DagOverviewAnalyzer.java new file mode 100644 index 0000000000..b193c30a90 --- /dev/null +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/DagOverviewAnalyzer.java @@ -0,0 +1,132 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.analyzer.plugins; + +import java.text.SimpleDateFormat; +import java.util.Comparator; +import java.util.Date; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.ToolRunner; +import org.apache.tez.analyzer.Analyzer; +import org.apache.tez.analyzer.CSVResult; +import org.apache.tez.analyzer.Result; +import org.apache.tez.dag.api.TezException; +import org.apache.tez.history.parser.datamodel.DagInfo; +import org.apache.tez.history.parser.datamodel.Event; +import org.apache.tez.history.parser.datamodel.TaskAttemptInfo; +import org.apache.tez.history.parser.datamodel.TaskInfo; +import org.apache.tez.history.parser.datamodel.VertexInfo; + +public class DagOverviewAnalyzer extends TezAnalyzerBase implements Analyzer { + private final String[] headers = + { "name", "id", "event_type", "status", "event_time", "event_time_str", "vertex_task_stats", "diagnostics" }; + private final CSVResult csvResult; + private static final SimpleDateFormat FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); + + public DagOverviewAnalyzer(Configuration config) { + super(config); + csvResult = new CSVResult(headers); + } + + @Override + public void analyze(DagInfo dagInfo) throws TezException { + for (Event event : dagInfo.getEvents()) { + csvResult.addRecord(new String[] { dagInfo.getDagId(), dagInfo.getDagId(), event.getType(), + dagInfo.getStatus(), Long.toString(event.getTime()), toDateStr(event.getTime()), "", "" }); + } + for (VertexInfo vertex : dagInfo.getVertices()) { + for (Event event : vertex.getEvents()) { + String vertexFailureInfoIfAny = ""; + for (TaskAttemptInfo attempt : vertex.getTaskAttempts()) { + if (attempt.getStatus().contains("FAILED")) { + vertexFailureInfoIfAny = attempt.getTaskAttemptId() + ": " + + attempt.getDiagnostics().replaceAll(",", " ").replaceAll("\n", " "); + break; + } + } + csvResult.addRecord(new String[] { vertex.getVertexName(), vertex.getVertexId(), + event.getType(), vertex.getStatus(), Long.toString(event.getTime()), + toDateStr(event.getTime()), getTaskStats(vertex), vertexFailureInfoIfAny }); + } + + // a failed task can lead to dag failure, so hopefully holds valuable information + for (TaskInfo failedTask : vertex.getFailedTasks()) { + for (Event failedTaskEvent : failedTask.getEvents()) { + if (failedTaskEvent.getType().equalsIgnoreCase("TASK_FINISHED")) { + csvResult.addRecord(new String[] { vertex.getVertexName(), failedTask.getTaskId(), + failedTaskEvent.getType(), failedTask.getStatus(), Long.toString(failedTaskEvent.getTime()), + toDateStr(failedTaskEvent.getTime()), getTaskStats(vertex), + failedTask.getDiagnostics().replaceAll(",", " ").replaceAll("\n", " ") }); + } + } + // if we already found a failing task, let's scan the failing attempts as well + for (TaskAttemptInfo failedAttempt : failedTask.getFailedTaskAttempts()) { + for (Event failedTaskAttemptEvent : failedAttempt.getEvents()) { + if (failedTaskAttemptEvent.getType().equalsIgnoreCase("TASK_ATTEMPT_FINISHED")) { + csvResult.addRecord(new String[] { vertex.getVertexName(), + failedAttempt.getTaskAttemptId(), failedTaskAttemptEvent.getType(), + failedAttempt.getStatus(), Long.toString(failedTaskAttemptEvent.getTime()), + toDateStr(failedTaskAttemptEvent.getTime()), getTaskStats(vertex), + failedAttempt.getDiagnostics().replaceAll(",", " ").replaceAll("\n", " ") }); + } + } + } + } + } + + csvResult.sort(new Comparator() { + public int compare(String[] first, String[] second) { + return (int) (Long.parseLong(first[4]) - Long.parseLong(second[4])); + } + }); + } + + private String getTaskStats(VertexInfo vertex) { + return String.format("numTasks: %d failedTasks: %d completedTasks: %d", vertex.getNumTasks(), + vertex.getFailedTasksCount(), vertex.getCompletedTasksCount()); + } + + private static synchronized String toDateStr(long time) { + return FORMAT.format(new Date(time)); + } + + @Override + public Result getResult() throws TezException { + return csvResult; + } + + @Override + public String getName() { + return "Dag overview analyzer"; + } + + @Override + public String getDescription() { + return "High level dag events overview (dag, vertex event summary)." + + " Helps understand the overall progress of a dag by simply listing the dag/vertex related events"; + } + + public static void main(String[] args) throws Exception { + Configuration config = new Configuration(); + DagOverviewAnalyzer analyzer = new DagOverviewAnalyzer(config); + int res = ToolRunner.run(config, analyzer, args); + analyzer.printResults(); + System.exit(res); + } +} diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/HungTaskAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/HungTaskAnalyzer.java new file mode 100644 index 0000000000..9a38e28ba2 --- /dev/null +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/HungTaskAnalyzer.java @@ -0,0 +1,143 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.analyzer.plugins; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.ToolRunner; +import org.apache.tez.analyzer.Analyzer; +import org.apache.tez.analyzer.CSVResult; +import org.apache.tez.analyzer.Result; +import org.apache.tez.dag.api.TezException; +import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.history.parser.datamodel.DagInfo; +import org.apache.tez.history.parser.datamodel.TaskAttemptInfo; +import org.apache.tez.history.parser.datamodel.VertexInfo; + +import java.util.Comparator; +import java.util.HashMap; +import java.util.Map; + +/** + * Gives insights about hanging task attempts by providing details about last attempts of all tasks. + */ +public class HungTaskAnalyzer extends TezAnalyzerBase implements Analyzer { + private final String[] headers = { "vertex", "task", " number_of_attempts", "last_attempt_id", + "last_attempt_status", "last_attempt_duration_ms", "last_attempt_node" }; + private final CSVResult csvResult; + + private static final String HEADER_NUM_ATTEMPTS = "num_attempts"; + private static final String HEADER_LAST_ATTEMPT_ID_AND_STATUS = "last_attempt_id_and_status"; + private static final String HEADER_LAST_ATTEMPT_STATUS = "last_attempt_status"; + private static final String HEADER_LAST_ATTEMPT_NODE = "last_attempt_node"; + private static final String HEADER_LAST_ATTEMPT_DURATION_MS = "last_attempt_duration_ms"; + + public HungTaskAnalyzer(Configuration config) { + super(config); + csvResult = new CSVResult(headers); + } + + @Override + public void analyze(DagInfo dagInfo) throws TezException { + Map> taskData = new HashMap<>(); // task attempt count per task + for (VertexInfo vertex : dagInfo.getVertices()) { + taskData.clear(); + for (TaskAttemptInfo attempt : vertex.getTaskAttempts()) { + String taskId = attempt.getTaskInfo().getTaskId(); + + int numAttemptsForTask = attempt.getTaskInfo().getNumberOfTaskAttempts(); + Map thisTaskData = taskData.get(taskId); + + if (thisTaskData == null) { + thisTaskData = new HashMap<>(); + thisTaskData.put(HEADER_NUM_ATTEMPTS, Integer.toString(numAttemptsForTask)); + taskData.put(taskId, thisTaskData); + } + + int attemptNumber = TezTaskAttemptID.fromString(attempt.getTaskAttemptId()).getId(); + if (attemptNumber == numAttemptsForTask - 1) { + thisTaskData.put(HEADER_LAST_ATTEMPT_ID_AND_STATUS, String.format("%s/%s", attempt.getTaskAttemptId(), attempt.getStatus())); + thisTaskData.put(HEADER_LAST_ATTEMPT_STATUS, attempt.getDetailedStatus()); + thisTaskData.put(HEADER_LAST_ATTEMPT_NODE, attempt.getNodeId()); + + thisTaskData.put(HEADER_LAST_ATTEMPT_DURATION_MS, + (attempt.getFinishTime() == 0 || attempt.getStartTime() == 0) ? "-1" + : Long.toString(attempt.getFinishTime() - attempt.getStartTime())); + } + } + for (Map.Entry> task : taskData.entrySet()) { + addARecord(vertex.getVertexName(), task.getKey(), task.getValue().get(HEADER_NUM_ATTEMPTS), + task.getValue().get(HEADER_LAST_ATTEMPT_ID_AND_STATUS), task.getValue().get(HEADER_LAST_ATTEMPT_STATUS), + task.getValue().get(HEADER_LAST_ATTEMPT_DURATION_MS), + task.getValue().get(HEADER_LAST_ATTEMPT_NODE)); + } + } + + csvResult.sort(new Comparator() { + public int compare(String[] first, String[] second) { + int vertexOrder = first[0].compareTo(second[0]); + int lastAttemptStatusOrder = + (first[4] == null || second[4] == null) ? 0 : first[4].compareTo(second[4]); + int attemptNumberOrder = Integer.valueOf(second[2]).compareTo(Integer.valueOf(first[2])); + + return vertexOrder == 0 + ? (lastAttemptStatusOrder == 0 ? attemptNumberOrder : lastAttemptStatusOrder) + : vertexOrder; + } + }); + } + + private void addARecord(String vertexName, String taskId, String numAttempts, + String lastAttemptId, String lastAttemptStatus, String lastAttemptDuration, + String lastAttemptNode) { + String[] record = new String[7]; + record[0] = vertexName; + record[1] = taskId; + record[2] = numAttempts; + record[3] = lastAttemptId; + record[4] = lastAttemptStatus; + record[5] = lastAttemptDuration; + record[6] = lastAttemptNode; + + csvResult.addRecord(record); + } + + @Override + public Result getResult() throws TezException { + return csvResult; + } + + @Override + public String getName() { + return "Hung Task Analyzer"; + } + + @Override + public String getDescription() { + return "TaskHandAnalyzer can give quick insights about hanging task attempts" + + " by giving an overview of all tasks and their last attempts' status, duration, etc."; + } + + public static void main(String[] args) throws Exception { + Configuration config = new Configuration(); + HungTaskAnalyzer analyzer = new HungTaskAnalyzer(config); + int res = ToolRunner.run(config, analyzer, args); + analyzer.printResults(); + System.exit(res); + } +} diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/InputReadErrorAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/InputReadErrorAnalyzer.java new file mode 100644 index 0000000000..3cb523ff9a --- /dev/null +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/InputReadErrorAnalyzer.java @@ -0,0 +1,94 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.analyzer.plugins; + +import java.util.Comparator; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.ToolRunner; +import org.apache.tez.analyzer.Analyzer; +import org.apache.tez.analyzer.CSVResult; +import org.apache.tez.analyzer.Result; +import org.apache.tez.dag.api.TezException; +import org.apache.tez.history.parser.datamodel.DagInfo; +import org.apache.tez.history.parser.datamodel.Event; +import org.apache.tez.history.parser.datamodel.TaskAttemptInfo; +import org.apache.tez.history.parser.datamodel.VertexInfo; + +/** + * Helps finding the root cause of shuffle errors, e.g. which node(s) can be blamed for them. + */ +public class InputReadErrorAnalyzer extends TezAnalyzerBase implements Analyzer { + private final String[] headers = { "vertex:attempt", "status", "time", "node", "diagnostics" }; + private final CSVResult csvResult; + + public InputReadErrorAnalyzer(Configuration config) { + super(config); + csvResult = new CSVResult(headers); + } + + @Override + public void analyze(DagInfo dagInfo) throws TezException { + for (VertexInfo vertex : dagInfo.getVertices()) { + for (TaskAttemptInfo attempt : vertex.getTaskAttempts()) { + String terminationCause = attempt.getTerminationCause(); + if ("INPUT_READ_ERROR".equalsIgnoreCase(terminationCause) + || "OUTPUT_LOST".equalsIgnoreCase(terminationCause) + || "NODE_FAILED".equalsIgnoreCase(terminationCause)) { + for (Event event : attempt.getEvents()) { + if (event.getType().equalsIgnoreCase("TASK_ATTEMPT_FINISHED")) { + csvResult.addRecord(new String[] { + vertex.getVertexName() + ":" + attempt.getTaskAttemptId(), + attempt.getDetailedStatus(), String.valueOf(event.getTime()), attempt.getNodeId(), + attempt.getDiagnostics().replaceAll(",", " ").replaceAll("\n", " ") }); + } + } + } + } + } + + csvResult.sort(new Comparator() { + public int compare(String[] first, String[] second) { + return (int) (Long.parseLong(second[2]) - Long.parseLong(first[2])); + } + }); + } + + @Override + public Result getResult() throws TezException { + return csvResult; + } + + @Override + public String getName() { + return "Input read error analyzer"; + } + + @Override + public String getDescription() { + return "Prints every task attempt (with node) which are related to input read errors"; + } + + public static void main(String[] args) throws Exception { + Configuration config = new Configuration(); + InputReadErrorAnalyzer analyzer = new InputReadErrorAnalyzer(config); + int res = ToolRunner.run(config, analyzer, args); + analyzer.printResults(); + System.exit(res); + } +} diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/LocalityAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/LocalityAnalyzer.java index ec72df17ac..d640704f93 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/LocalityAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/LocalityAnalyzer.java @@ -52,12 +52,10 @@ public class LocalityAnalyzer extends TezAnalyzerBase implements Analyzer { private static final String DATA_LOCAL_RATIO = "tez.locality-analyzer.data.local.ratio"; private static final float DATA_LOCAL_RATIO_DEFAULT = 0.5f; - private final Configuration config; - private final CSVResult csvResult; public LocalityAnalyzer(Configuration config) { - this.config = config; + super(config); csvResult = new CSVResult(headers); } @@ -119,7 +117,7 @@ public void analyze(DagInfo dagInfo) throws TezException { record.add(otherTaskResult.avgHDFSBytesRead + ""); String recommendation = ""; - if (dataLocalRatio < config.getFloat(DATA_LOCAL_RATIO, DATA_LOCAL_RATIO_DEFAULT)) { + if (dataLocalRatio < getConf().getFloat(DATA_LOCAL_RATIO, DATA_LOCAL_RATIO_DEFAULT)) { recommendation = "Data locality is poor for this vertex. Try tuning " + TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS + ", " + TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED + ", " @@ -182,10 +180,6 @@ private TaskAttemptDetails computeAverages(VertexInfo vertexInfo, DAGCounter cou return "Analyze for locality information (data local, rack local, off-rack)"; } - @Override public Configuration getConfiguration() { - return config; - } - /** * Placeholder for task attempt details */ diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/OneOnOneEdgeAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/OneOnOneEdgeAnalyzer.java index 2ba715ed43..a6cb3f1511 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/OneOnOneEdgeAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/OneOnOneEdgeAnalyzer.java @@ -55,12 +55,10 @@ public class OneOnOneEdgeAnalyzer extends TezAnalyzerBase implements Analyzer { // DataMovementType::ONE_TO_ONE private static final String ONE_TO_ONE = "ONE_TO_ONE"; - private final Configuration config; - private final CSVResult csvResult; public OneOnOneEdgeAnalyzer(Configuration config) { - this.config = config; + super(config); csvResult = new CSVResult(headers); } @@ -140,11 +138,6 @@ public String getDescription() { return "To understand the locality miss in 1:1 edge"; } - @Override - public Configuration getConfiguration() { - return config; - } - public static void main(String[] args) throws Exception { Configuration conf = new Configuration(); OneOnOneEdgeAnalyzer analyzer = new OneOnOneEdgeAnalyzer(conf); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/ShuffleTimeAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/ShuffleTimeAnalyzer.java index 57e91c62d3..f8f9112bb7 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/ShuffleTimeAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/ShuffleTimeAnalyzer.java @@ -66,14 +66,12 @@ public class ShuffleTimeAnalyzer extends TezAnalyzerBase implements Analyzer { private final CSVResult csvResult = new CSVResult(headers); - private final Configuration config; - private final float realWorkDoneRatio; private final long minShuffleRecords; public ShuffleTimeAnalyzer(Configuration config) { - this.config = config; + super(config); realWorkDoneRatio = config.getFloat (REAL_WORK_DONE_RATIO, REAL_WORK_DONE_RATIO_DEFAULT); @@ -208,11 +206,6 @@ public String getDescription() { + "and the real work done in the task"; } - @Override - public Configuration getConfiguration() { - return config; - } - public static void main(String[] args) throws Exception { Configuration config = new Configuration(); ShuffleTimeAnalyzer analyzer = new ShuffleTimeAnalyzer(config); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SkewAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SkewAnalyzer.java index 6025541fc9..a7d14fae25 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SkewAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SkewAnalyzer.java @@ -85,14 +85,12 @@ public class SkewAnalyzer extends TezAnalyzerBase implements Analyzer { private final CSVResult csvResult = new CSVResult(headers); - private final Configuration config; - private final float minRatio; private final float maxRatio; private final long maxShuffleBytesPerSource; public SkewAnalyzer(Configuration config) { - this.config = config; + super(config); maxRatio = config.getFloat(ATTEMPT_SHUFFLE_KEY_GROUP_MAX_RATIO, ATTEMPT_SHUFFLE_KEY_GROUP_MAX_RATIO_DEFAULT); minRatio = config.getFloat(ATTEMPT_SHUFFLE_KEY_GROUP_MIN_RATIO, @@ -214,7 +212,7 @@ private void analyzeRecordSkewPerSource(TaskAttemptInfo attemptInfo) { if (vertexNumTasks > 1) { if (ratio > maxRatio) { //input records > 60% of vertex level record count - if (inputRecordsCount > (vertexLevelInputRecordsCount * 0.60)) { + if (inputRecordsCount > (vertexLevelInputRecordsCount * 0.6f)) { List result = Lists.newLinkedList(); result.add(attemptInfo.getTaskInfo().getVertexInfo().getVertexName()); result.add(attemptInfo.getTaskAttemptId()); @@ -305,12 +303,7 @@ public String getName() { @Override public String getDescription() { - return "Analyzer reducer skews by mining reducer task counters"; - } - - @Override - public Configuration getConfiguration() { - return null; + return "Analyze reducer skews by mining reducer task counters"; } public static void main(String[] args) throws Exception { diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowNodeAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowNodeAnalyzer.java index a810a8a645..9e573c2033 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowNodeAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowNodeAnalyzer.java @@ -59,10 +59,8 @@ public class SlowNodeAnalyzer extends TezAnalyzerBase implements Analyzer { private final CSVResult csvResult = new CSVResult(headers); - private final Configuration config; - public SlowNodeAnalyzer(Configuration config) { - this.config = config; + super(config); } @Override @@ -182,11 +180,6 @@ public String getDescription() { return sb.toString(); } - @Override - public Configuration getConfiguration() { - return config; - } - public static void main(String[] args) throws Exception { Configuration config = new Configuration(); SlowNodeAnalyzer analyzer = new SlowNodeAnalyzer(config); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowTaskIdentifier.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowTaskIdentifier.java index d2474ad0f6..7c9958b250 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowTaskIdentifier.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowTaskIdentifier.java @@ -51,10 +51,8 @@ public class SlowTaskIdentifier extends TezAnalyzerBase implements Analyzer { private static final String NO_OF_TASKS = "tez.slow-task-analyzer.task.count"; private static final int NO_OF_TASKS_DEFAULT = 100; - private final Configuration config; - public SlowTaskIdentifier(Configuration config) { - this.config = config; + super(config); this.csvResult = new CSVResult(headers); } @@ -75,7 +73,7 @@ public void analyze(DagInfo dagInfo) throws TezException { }); int limit = Math.min(taskAttempts.size(), - Math.max(0, config.getInt(NO_OF_TASKS, NO_OF_TASKS_DEFAULT))); + Math.max(0, getConf().getInt(NO_OF_TASKS, NO_OF_TASKS_DEFAULT))); if (limit == 0) { return; @@ -111,11 +109,6 @@ public String getDescription() { return "Identifies slow tasks in the DAG"; } - @Override - public Configuration getConfiguration() { - return config; - } - public static void main(String[] args) throws Exception { Configuration config = new Configuration(); SlowTaskIdentifier analyzer = new SlowTaskIdentifier(config); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowestVertexAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowestVertexAnalyzer.java index 33f2421699..efa39a3223 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowestVertexAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowestVertexAnalyzer.java @@ -49,7 +49,6 @@ public class SlowestVertexAnalyzer extends TezAnalyzerBase implements Analyzer { private final CSVResult csvResult = new CSVResult(headers); - private final Configuration config; private final MetricRegistry metrics = new MetricRegistry(); private Histogram taskAttemptRuntimeHistorgram; @@ -59,7 +58,7 @@ public class SlowestVertexAnalyzer extends TezAnalyzerBase implements Analyzer { private final long vertexRuntimeThreshold; public SlowestVertexAnalyzer(Configuration config) { - this.config = config; + super(config); this.vertexRuntimeThreshold = Math.max(1, config.getLong(MAX_VERTEX_RUNTIME, MAX_VERTEX_RUNTIME_DEFAULT)); @@ -204,11 +203,6 @@ public String getDescription() { return "Identify the slowest vertex in the DAG, which needs to be looked into first"; } - @Override - public Configuration getConfiguration() { - return config; - } - public static void main(String[] args) throws Exception { Configuration config = new Configuration(); SlowestVertexAnalyzer analyzer = new SlowestVertexAnalyzer(config); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SpillAnalyzerImpl.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SpillAnalyzerImpl.java index d69ca23b5a..026dd1593f 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SpillAnalyzerImpl.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SpillAnalyzerImpl.java @@ -60,10 +60,8 @@ public class SpillAnalyzerImpl extends TezAnalyzerBase implements Analyzer { private final long minOutputBytesPerTask; - private final Configuration config; - public SpillAnalyzerImpl(Configuration config) { - this.config = config; + super(config); minOutputBytesPerTask = Math.max(0, config.getLong(OUTPUT_BYTES_THRESHOLD, OUTPUT_BYTES_THRESHOLD_DEFAULT)); this.csvResult = new CSVResult(headers); @@ -130,11 +128,6 @@ public String getDescription() { return "Analyze spill details in the task"; } - @Override - public Configuration getConfiguration() { - return config; - } - public static void main(String[] args) throws Exception { Configuration config = new Configuration(); SpillAnalyzerImpl analyzer = new SpillAnalyzerImpl(config); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskAssignmentAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskAssignmentAnalyzer.java index ce6fa417c1..02b821f367 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskAssignmentAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskAssignmentAnalyzer.java @@ -36,29 +36,27 @@ */ public class TaskAssignmentAnalyzer extends TezAnalyzerBase implements Analyzer { - private final String[] headers = { "vertex", "node", "numTasks", "load" }; - private final Configuration config; + private final String[] headers = { "vertex", "node", "numTaskAttempts", "load" }; private final CSVResult csvResult; public TaskAssignmentAnalyzer(Configuration config) { - this.config = config; + super(config); csvResult = new CSVResult(headers); } @Override public void analyze(DagInfo dagInfo) throws TezException { - Map map = new HashMap<>(); + Map taskAttemptsPerNode = new HashMap<>(); for (VertexInfo vertex : dagInfo.getVertices()) { - map.clear(); + taskAttemptsPerNode.clear(); for (TaskAttemptInfo attempt : vertex.getTaskAttempts()) { - Integer previousValue = map.get(attempt.getNodeId()); - map.put(attempt.getNodeId(), - previousValue == null ? 1 : previousValue + 1); + Integer previousValue = taskAttemptsPerNode.get(attempt.getNodeId()); + taskAttemptsPerNode.put(attempt.getNodeId(), previousValue == null ? 1 : previousValue + 1); } - double mean = vertex.getTaskAttempts().size() / Math.max(1.0, map.size()); - for (Map.Entry assignment : map.entrySet()) { - addARecord(vertex.getVertexName(), assignment.getKey(), - assignment.getValue(), assignment.getValue() * 100 / mean); + double mean = vertex.getTaskAttempts().size() / Math.max(1.0, taskAttemptsPerNode.size()); + for (Map.Entry assignment : taskAttemptsPerNode.entrySet()) { + addARecord(vertex.getVertexName(), assignment.getKey(), assignment.getValue(), + assignment.getValue() * 100 / mean); } } } @@ -88,11 +86,6 @@ public String getDescription() { return "Get the Task assignments on different nodes of the cluster"; } - @Override - public Configuration getConfiguration() { - return config; - } - public static void main(String[] args) throws Exception { Configuration config = new Configuration(); TaskAssignmentAnalyzer analyzer = new TaskAssignmentAnalyzer(config); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskAttemptResultStatisticsAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskAttemptResultStatisticsAnalyzer.java index df2f95cdce..cf6b2f0d8e 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskAttemptResultStatisticsAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskAttemptResultStatisticsAnalyzer.java @@ -44,11 +44,10 @@ public class TaskAttemptResultStatisticsAnalyzer extends TezAnalyzerBase implements Analyzer { private final String[] headers = { "vertex (+task stats: all/succeeded/failed/killed)", "node", "status", "numAttempts" }; - private final Configuration config; private final CSVResult csvResult; public TaskAttemptResultStatisticsAnalyzer(Configuration config) { - this.config = config; + super(config); csvResult = new CSVResult(headers); } @@ -71,7 +70,8 @@ public void analyze(DagInfo dagInfo) throws TezException { } map.forEach((key, value) -> { - addARecord(key.split("#")[0], key.split("#")[1], key.split("#")[2], value); + String[] keys = key.split("#"); + addARecord(keys[0], keys[1], keys.length > 2 ? keys[2] : "", value); }); csvResult.sort(new Comparator() { @@ -110,11 +110,6 @@ public String getDescription() { return "Get statistics about task attempts states in vertex:node:status level"; } - @Override - public Configuration getConfiguration() { - return config; - } - public static void main(String[] args) throws Exception { Configuration config = new Configuration(); TaskAttemptResultStatisticsAnalyzer analyzer = new TaskAttemptResultStatisticsAnalyzer(config); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskConcurrencyAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskConcurrencyAnalyzer.java index 72f3b36a5b..91f51b4c21 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskConcurrencyAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskConcurrencyAnalyzer.java @@ -41,11 +41,10 @@ public class TaskConcurrencyAnalyzer extends TezAnalyzerBase implements Analyzer private static final String[] headers = { "time", "vertexName", "concurrentTasksRunning" }; private final CSVResult csvResult; - private final Configuration config; public TaskConcurrencyAnalyzer(Configuration conf) { + super(conf); this.csvResult = new CSVResult(headers); - this.config = conf; } private enum EventType {START, FINISH} @@ -153,11 +152,6 @@ public String getDescription() { + "would be helpful in understanding whether any starvation was there or not."; } - @Override - public Configuration getConfiguration() { - return config; - } - public static void main(String[] args) throws Exception { Configuration config = new Configuration(); TaskConcurrencyAnalyzer analyzer = new TaskConcurrencyAnalyzer(config); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java index 75a55a754d..705c6e9cfb 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java @@ -33,6 +33,7 @@ import org.apache.commons.cli.Options; import org.apache.commons.cli.ParseException; import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.fs.Path; import org.apache.hadoop.util.Tool; @@ -67,7 +68,11 @@ public abstract class TezAnalyzerBase extends Configured implements Tool, Analyz private String outputDir; private boolean saveResults = false; - + + public TezAnalyzerBase(Configuration config) { + setConf(config); + } + @SuppressWarnings("static-access") private static Options buildOptions() { Option dagIdOption = OptionBuilder.withArgName(DAG_ID).withLongOpt(DAG_ID) diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/VertexLevelCriticalPathAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/VertexLevelCriticalPathAnalyzer.java index 06b8983e9d..78a4d41f38 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/VertexLevelCriticalPathAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/VertexLevelCriticalPathAnalyzer.java @@ -44,8 +44,6 @@ * Identify a set of vertices which fall in the critical path in a DAG. */ public class VertexLevelCriticalPathAnalyzer extends TezAnalyzerBase implements Analyzer { - private final Configuration config; - private static final String[] headers = { "CriticalPath", "Score" }; private final CSVResult csvResult; @@ -58,7 +56,7 @@ public class VertexLevelCriticalPathAnalyzer extends TezAnalyzerBase implements private static final String CONNECTOR = "-->"; public VertexLevelCriticalPathAnalyzer(Configuration config) { - this.config = config; + super(config); this.csvResult = new CSVResult(headers); this.dotFileLocation = config.get(DOT_FILE_DIR, DOT_FILE_DIR_DEFAULT); } @@ -105,11 +103,6 @@ public String getDescription() { return "Analyze vertex level critical path of the DAG"; } - @Override - public Configuration getConfiguration() { - return config; - } - private static Map sortByValues(Map result) { //Sort result by time in reverse order final Ordering reversValueOrdering = From f56075c552dd3c41f8d812d595882eed22f98537 Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Mon, 9 Aug 2021 16:05:13 -0400 Subject: [PATCH 327/512] TEZ-4321: Do Not Wrap Singleton Collections (#138) --- .../apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java | 6 ++---- .../apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java | 3 +-- .../tez/dag/app/dag/impl/OneToOneEdgeManagerOnDemand.java | 3 +-- .../java/org/apache/tez/dag/app/TestMockDAGAppMaster.java | 3 +-- 4 files changed, 5 insertions(+), 10 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java index ca510f7fc4..867403fcd5 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java @@ -53,8 +53,7 @@ public int getNumSourceTaskPhysicalOutputs(int sourceTaskIndex) { public void routeDataMovementEventToDestination(DataMovementEvent event, int sourceTaskIndex, int sourceOutputIndex, Map> destinationTaskAndInputIndices) { - List inputIndices = - Collections.unmodifiableList(Collections.singletonList(sourceTaskIndex)); + List inputIndices = Collections.singletonList(sourceTaskIndex); // for each task make the i-th source task as the i-th physical input for (int i=0; i> destinationTaskAndInputIndices) { - List inputIndices = - Collections.unmodifiableList(Collections.singletonList(sourceTaskIndex)); + List inputIndices = Collections.singletonList(sourceTaskIndex); // for each task make the i-th source task as the i-th physical input for (int i=0; i destinationInputIndices = - Collections.unmodifiableList(Collections.singletonList(0)); + final List destinationInputIndices = Collections.singletonList(0); final AtomicBoolean stateChecked = new AtomicBoolean(false); public OneToOneEdgeManager(EdgeManagerPluginContext context) { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManagerOnDemand.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManagerOnDemand.java index e2dbf457cf..464d87df59 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManagerOnDemand.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManagerOnDemand.java @@ -34,8 +34,7 @@ public class OneToOneEdgeManagerOnDemand extends EdgeManagerPluginOnDemand { - final List destinationInputIndices = - Collections.unmodifiableList(Collections.singletonList(0)); + final List destinationInputIndices = Collections.singletonList(0); final AtomicBoolean stateChecked = new AtomicBoolean(false); final EventRouteMetadata commonRouteMeta = diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java index 5dca6940ea..aa5d3b9535 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java @@ -300,8 +300,7 @@ public void testBasicEvents() throws Exception { } public static class LegacyEdgeTestEdgeManager extends EdgeManagerPlugin { - List destinationInputIndices = - Collections.unmodifiableList(Collections.singletonList(0)); + List destinationInputIndices = Collections.singletonList(0); public LegacyEdgeTestEdgeManager(EdgeManagerPluginContext context) { super(context); } From fc63c04039c17ca05107f515023d6d627fc302d8 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Tue, 10 Aug 2021 23:02:40 +0200 Subject: [PATCH 328/512] TEZ-4293: Use Tez credentials for CI (#141) --- Jenkinsfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Jenkinsfile b/Jenkinsfile index 835a0523a4..aadfaa953e 100644 --- a/Jenkinsfile +++ b/Jenkinsfile @@ -61,7 +61,7 @@ pipeline { stage ('precommit-run') { steps { withCredentials( - [usernamePassword(credentialsId: 'apache-hadoop-at-github.com', + [usernamePassword(credentialsId: 'apache-tez-at-github.com', passwordVariable: 'GITHUB_TOKEN', usernameVariable: 'GITHUB_USER'), usernamePassword(credentialsId: 'tez-ci', From 8c97365325b9667178537a454bc150cc5ce9b981 Mon Sep 17 00:00:00 2001 From: Takanobu Asanuma Date: Thu, 19 Aug 2021 19:42:24 +0900 Subject: [PATCH 329/512] TEZ-4323: Jetty jars were removed from dist package with TEZ-4114 (#140) (Takanobu Asanuma reviewed by Jonathan Turner Eagles) --- pom.xml | 4 ---- 1 file changed, 4 deletions(-) diff --git a/pom.xml b/pom.xml index ba65cc6ad6..3785352bf5 100644 --- a/pom.xml +++ b/pom.xml @@ -289,10 +289,6 @@ commons-logging commons-logging-api - - org.eclipse.jetty - * - com.sun.jersey jersey-core From e2c4ee0842e43e67e4d1dadc8308b427b24aadee Mon Sep 17 00:00:00 2001 From: jteagles Date: Tue, 24 Aug 2021 10:13:15 -0500 Subject: [PATCH 330/512] TEZ-4328. Import external tez component em-helpers (#145) Co-authored-by: Sreenath Somarajapuram --- .../src/main/resources/META-INF/LICENSE.txt | 1 - .../webapp/app/components/em-breadcrumbs.js | 69 +++++++++ .../main/webapp/app/components/em-progress.js | 102 ++++++++++++ tez-ui/src/main/webapp/app/helpers/txt.js | 62 ++++++++ tez-ui/src/main/webapp/app/styles/app.less | 5 + .../webapp/app/styles/em-breadcrumbs.less | 29 ++++ .../main/webapp/app/styles/em-progress.less | 52 +++++++ tez-ui/src/main/webapp/app/styles/shared.less | 4 + tez-ui/src/main/webapp/app/styles/txt.less | 24 +++ .../templates/components/em-breadcrumbs.hbs | 45 ++++++ .../app/templates/components/em-progress.hbs | 25 +++ .../src/main/webapp/app/utils/formatters.js | 146 ++++++++++++++++++ tez-ui/src/main/webapp/config/environment.js | 4 + tez-ui/src/main/webapp/package.json | 1 - .../components/em-breadcrumbs-test.js | 107 +++++++++++++ .../components/em-progress-test.js | 73 +++++++++ .../webapp/tests/unit/helpers/txt-test.js | 59 +++++++ .../tests/unit/utils/formatters-test.js | 99 ++++++++++++ tez-ui/src/main/webapp/yarn.lock | 10 -- 19 files changed, 905 insertions(+), 12 deletions(-) create mode 100644 tez-ui/src/main/webapp/app/components/em-breadcrumbs.js create mode 100644 tez-ui/src/main/webapp/app/components/em-progress.js create mode 100644 tez-ui/src/main/webapp/app/helpers/txt.js create mode 100644 tez-ui/src/main/webapp/app/styles/em-breadcrumbs.less create mode 100644 tez-ui/src/main/webapp/app/styles/em-progress.less create mode 100644 tez-ui/src/main/webapp/app/styles/txt.less create mode 100644 tez-ui/src/main/webapp/app/templates/components/em-breadcrumbs.hbs create mode 100644 tez-ui/src/main/webapp/app/templates/components/em-progress.hbs create mode 100644 tez-ui/src/main/webapp/app/utils/formatters.js create mode 100644 tez-ui/src/main/webapp/tests/integration/components/em-breadcrumbs-test.js create mode 100644 tez-ui/src/main/webapp/tests/integration/components/em-progress-test.js create mode 100644 tez-ui/src/main/webapp/tests/unit/helpers/txt-test.js create mode 100644 tez-ui/src/main/webapp/tests/unit/utils/formatters-test.js diff --git a/tez-ui/src/main/resources/META-INF/LICENSE.txt b/tez-ui/src/main/resources/META-INF/LICENSE.txt index ba6f29b344..354d7457a9 100644 --- a/tez-ui/src/main/resources/META-INF/LICENSE.txt +++ b/tez-ui/src/main/resources/META-INF/LICENSE.txt @@ -233,7 +233,6 @@ The Apache TEZ tez-ui bundles the following files under the MIT License: - snippet-ss v1.11.0 (https://github.com/sreenaths/snippet-ss) - em-tgraph v0.0.4 (https://github.com/sreenaths/em-tgraph) - em-table v0.3.12 (https://github.com/sreenaths/em-table) - - em-helpers v0.5.8 (https://github.com/sreenaths/em-helpers) - ember-cli-app-version v1.0.0 (https://github.com/EmberSherpa/ember-cli-app-version) - Authored by Taras Mankovski - ember-cli-auto-register v1.1.0 (https://github.com/williamsbdev/ember-cli-auto-register) - Copyright © 2015 Brandon Williams http://williamsbdev.com - ember-cli-content-security-policy v0.4.0 (https://github.com/rwjblue/ember-cli-content-security-policy) diff --git a/tez-ui/src/main/webapp/app/components/em-breadcrumbs.js b/tez-ui/src/main/webapp/app/components/em-breadcrumbs.js new file mode 100644 index 0000000000..fcf90cfc5c --- /dev/null +++ b/tez-ui/src/main/webapp/app/components/em-breadcrumbs.js @@ -0,0 +1,69 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import layout from '../templates/components/em-breadcrumbs'; + +export default Ember.Component.extend({ + layout: layout, + + itemStyle: Ember.computed("items", function () { + var itemCount = this.get("items.length"); + + if(itemCount) { + let widthPercent = 100 / itemCount; + return new Ember.Handlebars.SafeString(`max-width: ${widthPercent}%`); + } + }), + + normalizedItems: Ember.computed("items", function () { + var items = this.get("items"); + + if(items) { + let lastIndex = items.length - 1; + items = items.map(function (item, index) { + var itemDef = { + text: item.text || "", + classNames: item.classNames || [], + }; + + Ember.assert("classNames must be an array", Array.isArray(itemDef.classNames)); + + if(index === lastIndex) { + itemDef.classNames.push("active"); + } + else { + itemDef.routeName = item.routeName; + itemDef.model = item.model; + itemDef.href = item.href; + if(item.queryParams) { + itemDef.queryParams = { + isQueryParams: true, + values: item.queryParams + }; + } + } + + itemDef.classNames = itemDef.classNames.join(" "); + return itemDef; + }); + } + + return items; + }) +}); diff --git a/tez-ui/src/main/webapp/app/components/em-progress.js b/tez-ui/src/main/webapp/app/components/em-progress.js new file mode 100644 index 0000000000..926764b300 --- /dev/null +++ b/tez-ui/src/main/webapp/app/components/em-progress.js @@ -0,0 +1,102 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import layout from '../templates/components/em-progress'; + +export default Ember.Component.extend({ + layout: layout, + + value: 0, + valueMin: 0, + valueMax: 1, + + classNames: ["em-progress-container"], + classNameBindings: ["animated", "striped"], + + striped: false, + style: null, + + progressBar: null, + + widthPercent: Ember.computed("value", "valueMin", "valueMax", function () { + var value = this.get("value"), + valueMin = this.get("valueMin"), + valueMax = this.get("valueMax"); + + if(value < valueMin) { + value = valueMin; + } + else if(value > valueMax) { + value = valueMax; + } + + value -= valueMin; + valueMax -= valueMin; + + return (value / valueMax) * 100; + }), + + progressText: Ember.computed("widthPercent", function () { + var percent = parseInt(this.get("widthPercent")); + if(isNaN(percent)) { + percent = 0; + } + return percent + "%"; + }), + + animated: Ember.computed("widthPercent", "striped", function () { + return this.get('striped') && this.get('widthPercent') > 0 && this.get('widthPercent') < 100; + }), + + progressBarClasses: Ember.computed("style", "striped", "animated", function () { + var classes = [], + style = this.get("style"); + + if(style) { + classes.push(`progress-bar-${style}`); + } + if(this.get("striped")) { + classes.push("progress-bar-striped"); + } + if(this.get("animated")) { + classes.push("active"); + } + + return classes.join(" "); + }), + + renderProgress: Ember.observer("progressBar", "widthPercent", function () { + var widthPercent = this.get('widthPercent'); + this.get("progressBar").width(widthPercent + "%"); + }), + + didInsertElement: function () { + Ember.run.scheduleOnce('afterRender', this, function() { + this.setProperties({ + progressBar: this.$(".progress-bar") + }); + }); + }, + + willDestroy: function () { + this.setProperties({ + progressBar: null, + }); + } +}); diff --git a/tez-ui/src/main/webapp/app/helpers/txt.js b/tez-ui/src/main/webapp/app/helpers/txt.js new file mode 100644 index 0000000000..81fdbdaee5 --- /dev/null +++ b/tez-ui/src/main/webapp/app/helpers/txt.js @@ -0,0 +1,62 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import formatters from '../utils/formatters'; + +export function txt(value, hash) { + var message, + dataType = hash.type, + formatter = hash.formatter, + titleAttr = ""; + + if(value) { + value = value[0]; + } + + if(value instanceof Error) { + message = value.message; + titleAttr = `title="${value.message}" `; + } + else { + try { + if(value !== undefined && !formatter && dataType) { + formatter = formatters[dataType]; + } + + if(formatter && value !== undefined && value !== null) { + value = formatter(value, hash); + } + + if(value === undefined || value === null) { + message = 'Not Available!'; + } + else { + return Ember.String.htmlSafe(Ember.Handlebars.Utils.escapeExpression(value.toString())); + } + } + catch(error) { + message = "Invalid Data!"; + Ember.Logger.error(error); + } + } + + return Ember.String.htmlSafe(` ${message} `); +} + +export default Ember.Helper.helper(txt); diff --git a/tez-ui/src/main/webapp/app/styles/app.less b/tez-ui/src/main/webapp/app/styles/app.less index 7c662298c9..f8a66e376d 100644 --- a/tez-ui/src/main/webapp/app/styles/app.less +++ b/tez-ui/src/main/webapp/app/styles/app.less @@ -44,6 +44,8 @@ @import "em-table-status-cell"; @import "query-timeline"; @import "home-table-controls"; +@import "em-progress"; +@import "em-breadcrumbs"; // Modals @import "column-selector"; @@ -54,3 +56,6 @@ @import "details-page"; @import "swimlane-page"; @import "vertex-configs-page"; + +// Helpers +@import "txt"; diff --git a/tez-ui/src/main/webapp/app/styles/em-breadcrumbs.less b/tez-ui/src/main/webapp/app/styles/em-breadcrumbs.less new file mode 100644 index 0000000000..05ed77ed27 --- /dev/null +++ b/tez-ui/src/main/webapp/app/styles/em-breadcrumbs.less @@ -0,0 +1,29 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +@import "bower_components/snippet-ss/less/no"; + +.breadcrumb { + .no-wrap; + li { + .no-wrap; + overflow: hidden; + text-overflow: ellipsis; + } +} diff --git a/tez-ui/src/main/webapp/app/styles/em-progress.less b/tez-ui/src/main/webapp/app/styles/em-progress.less new file mode 100644 index 0000000000..416b6966d9 --- /dev/null +++ b/tez-ui/src/main/webapp/app/styles/em-progress.less @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +@import "bower_components/bootstrap/less/variables"; +@import "bower_components/snippet-ss/less/effects"; + +.progress { + .progress-bar { + .progress-text { + padding-left: 10px; + .text-outer-glow(@progress-bar-bg); + } + } + + .progress-bar-success { + .progress-text { + .text-outer-glow(@progress-bar-success-bg); + } + } + .progress-bar-info { + .progress-text { + .text-outer-glow(@progress-bar-info-bg); + } + } + .progress-bar-warning { + .progress-text { + .text-outer-glow(@progress-bar-warning-bg); + } + } + .progress-bar-danger { + .progress-text { + .text-outer-glow(@progress-bar-danger-bg); + } + } +} + diff --git a/tez-ui/src/main/webapp/app/styles/shared.less b/tez-ui/src/main/webapp/app/styles/shared.less index dedac8c91b..b34cfa6045 100644 --- a/tez-ui/src/main/webapp/app/styles/shared.less +++ b/tez-ui/src/main/webapp/app/styles/shared.less @@ -86,3 +86,7 @@ b { } } } + +.em-message { + opacity: .5; +} diff --git a/tez-ui/src/main/webapp/app/styles/txt.less b/tez-ui/src/main/webapp/app/styles/txt.less new file mode 100644 index 0000000000..171c250f73 --- /dev/null +++ b/tez-ui/src/main/webapp/app/styles/txt.less @@ -0,0 +1,24 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +@import "./shared"; + +.txt-message { + .em-message; +} diff --git a/tez-ui/src/main/webapp/app/templates/components/em-breadcrumbs.hbs b/tez-ui/src/main/webapp/app/templates/components/em-breadcrumbs.hbs new file mode 100644 index 0000000000..f50972a01a --- /dev/null +++ b/tez-ui/src/main/webapp/app/templates/components/em-breadcrumbs.hbs @@ -0,0 +1,45 @@ +{{! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +}} + +

diff --git a/tez-ui/src/main/webapp/app/templates/components/em-progress.hbs b/tez-ui/src/main/webapp/app/templates/components/em-progress.hbs new file mode 100644 index 0000000000..bd7fafcda5 --- /dev/null +++ b/tez-ui/src/main/webapp/app/templates/components/em-progress.hbs @@ -0,0 +1,25 @@ +{{! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +}} + +
+
+ + {{progressText}} + +
+
diff --git a/tez-ui/src/main/webapp/app/utils/formatters.js b/tez-ui/src/main/webapp/app/utils/formatters.js new file mode 100644 index 0000000000..d724eb85ba --- /dev/null +++ b/tez-ui/src/main/webapp/app/utils/formatters.js @@ -0,0 +1,146 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; + +import moment from 'moment'; +import numeral from 'numeral'; + +const DEFAULT_DATE_TIMEZONE = "UTC", + DEFAULT_DATE_FORMAT = "DD MMM YYYY HH:mm:ss", + DEFAULT_NUM_FORMAT = '0,0', + DEFAULT_MEM_FORMAT = '0 b'; + +function durationFormatter(arr, value, unit) { + if(value > 0) { + if(value > 1) { + unit += 's'; + } + arr.push(value + unit); + } +} + +const DURATION_FORMATS = { + long: { + collateFunction: durationFormatter, + + year: " year", + month: " month", + day: " day", + hour: " hour", + minute: " minute", + second: " second", + millisecond: " millisecond" + }, + short: { + collateFunction: durationFormatter, + + year: " yr", + month: " mo", + day: " day", + hour: " hr", + minute: " min", + second: " sec", + millisecond: " msec" + }, + xshort: { + collateFunction: function (arr, value, unit) { + if(value > 0) { + arr.push(value + unit); + } + }, + + year: "Y", + month: "M", + day: "D", + hour: "h", + minute: "m", + second: "s", + millisecond: "ms" + } +}; + +function validateNumber(value, message) { + value = parseFloat(value); + + if(isNaN(value)) { + throw new Error(message || "Invalid number!"); + } + + return value; +} + +export default Ember.Controller.create({ + date: function (value, options) { + var date = moment.tz(value, options.valueFormat, options.valueTimeZone || DEFAULT_DATE_TIMEZONE); + + date = options.timeZone ? date.tz(options.timeZone) : date.local(); + date = date.format(options.format || DEFAULT_DATE_FORMAT); + + if(date === "Invalid date") { + throw new Error(date); + } + + return date; + }, + duration: function (value, options) { + var format = DURATION_FORMATS[options.format || "xshort"], + duration, + ret = []; + + value = validateNumber(value, "Invalid duration"); + + if(value === 0) { + return `0${format.millisecond}`; + } + + duration = moment.duration(value, options.valueUnit); + + format.collateFunction(ret, duration.years(), format.year); + format.collateFunction(ret, duration.months(), format.month); + format.collateFunction(ret, duration.days(), format.day); + format.collateFunction(ret, duration.hours(), format.hour); + format.collateFunction(ret, duration.minutes(), format.minute); + format.collateFunction(ret, duration.seconds(), format.second); + format.collateFunction(ret, Math.round(duration.milliseconds()), format.millisecond); + + return ret.join(" "); + }, + number: function (value, options) { + value = validateNumber(value); + return numeral(value).format(options.format || DEFAULT_NUM_FORMAT); + }, + memory: function (value) { + value = validateNumber(value, "Invalid memory"); + if(value === 0) { + return "0 B"; + } + return numeral(value).format(DEFAULT_MEM_FORMAT); + }, + json: function (value, options) { + if(value && typeof value === "object" && value.constructor === Object) { + try { + value = JSON.stringify(value, options.replacer, options.space || 4); + } + catch(err){ + Ember.Logger.error(err); + } + } + return value; + } +}); diff --git a/tez-ui/src/main/webapp/config/environment.js b/tez-ui/src/main/webapp/config/environment.js index 0c755ac700..68a16cf803 100644 --- a/tez-ui/src/main/webapp/config/environment.js +++ b/tez-ui/src/main/webapp/config/environment.js @@ -39,6 +39,10 @@ module.exports = function(environment) { 'child-src': "'self' 'unsafe-inline'", 'style-src': "'self' 'unsafe-inline'", 'script-src': "'self' 'unsafe-inline'" + }, + + moment: { + includeTimezone: '2010-2020' } }; diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json index 92f54d266d..fa80389b94 100644 --- a/tez-ui/src/main/webapp/package.json +++ b/tez-ui/src/main/webapp/package.json @@ -61,7 +61,6 @@ "phantomjs-prebuilt": "2.1.13" }, "dependencies": { - "em-helpers": "0.8.0", "em-table": "0.11.3", "em-tgraph": "0.0.14" } diff --git a/tez-ui/src/main/webapp/tests/integration/components/em-breadcrumbs-test.js b/tez-ui/src/main/webapp/tests/integration/components/em-breadcrumbs-test.js new file mode 100644 index 0000000000..63edbc491a --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/components/em-breadcrumbs-test.js @@ -0,0 +1,107 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +moduleForComponent('em-breadcrumbs', 'Integration | Component | em breadcrumbs', { + integration: true +}); + +test('Basic creation test', function(assert) { + + // Set any properties with this.set('myProperty', 'value'); + // Handle any actions with this.on('myAction', function(val) { ... });" + EOL + EOL + + + this.render(hbs`{{em-breadcrumbs}}`); + + assert.equal(this.$().text().trim(), ''); + + // Template block usage:" + EOL + + this.render(hbs` + {{#em-breadcrumbs}} + template block text + {{/em-breadcrumbs}} + `); + + assert.equal(this.$().text().trim(), ''); +}); + +test('Test with one link-to item', function(assert) { + var testItems = [{ + routeName: "foo", + text: "fooText" + }], + elements; + + this.set("items", testItems); + this.render(hbs`{{em-breadcrumbs items=items}}`); + + elements = this.$("li"); + + assert.equal(elements.length, 1); + assert.equal(Ember.$(elements[0]).text().trim(), testItems[0].text); + assert.equal(elements[0].title, testItems[0].text); + assert.equal(elements[0].style.maxWidth, "100%"); +}); + +test('Test with two link-to item', function(assert) { + var testItems = [{ + routeName: "foo", + text: "fooText" + },{ + routeName: "bar", + text: "barText" + }], + elements; + + this.set("items", testItems); + this.render(hbs`{{em-breadcrumbs items=items}}`); + + elements = this.$("li"); + + assert.equal(elements.length, 2); + + assert.equal(Ember.$(elements[0]).text().trim(), testItems[0].text); + assert.equal(elements[0].title, testItems[0].text); + assert.equal(elements[0].style.maxWidth, "50%"); + + assert.equal(Ember.$(elements[1]).text().trim(), testItems[1].text); + assert.equal(elements[1].title, testItems[1].text); + assert.equal(elements[1].style.maxWidth, "50%"); +}); + +test('Test with one anchor tag item', function(assert) { + var testItems = [{ + href: "foo.bar", + text: "fooText" + }], + elements; + + this.set("items", testItems); + this.render(hbs`{{em-breadcrumbs items=items}}`); + + elements = this.$("li"); + + assert.equal(elements.length, 1); + assert.equal(Ember.$(elements[0]).text().trim(), testItems[0].text); + assert.equal(elements[0].title, testItems[0].text); + assert.equal(elements[0].style.maxWidth, "100%"); +}); diff --git a/tez-ui/src/main/webapp/tests/integration/components/em-progress-test.js b/tez-ui/src/main/webapp/tests/integration/components/em-progress-test.js new file mode 100644 index 0000000000..8fcdfaf1d0 --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/components/em-progress-test.js @@ -0,0 +1,73 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +moduleForComponent('em-progress', 'Integration | Component | em progress', { + integration: true +}); + +test('It renders', function(assert) { + + // Set any properties with this.set('myProperty', 'value'); + // Handle any actions with this.on('myAction', function(val) { ... });" + EOL + EOL + + + this.render(hbs`{{em-progress}}`); + + assert.equal(this.$().text().trim(), '0%'); + + this.render(hbs`{{#em-progress}}{{/em-progress}}`); + assert.equal(this.$().text().trim(), '0%'); +}); + +test('With a specific value', function(assert) { + this.render(hbs`{{em-progress value=0.5}}`); + assert.equal(this.$().text().trim(), '50%'); +}); + +test('Custom valueMin & valueMax', function(assert) { + this.render(hbs`{{em-progress value=15 valueMin=10 valueMax=20}}`); + assert.equal(this.$().text().trim(), '50%'); + + assert.notOk(this.$('.striped')[0], "Striped class added"); +}); + +test('Check for stripes & animation while in progress', function(assert) { + this.render(hbs`{{em-progress value=0.5 striped=true}}`); + + assert.equal(this.$().text().trim(), '50%'); + assert.ok(this.$('.striped')[0], "Striped class added"); + assert.ok(this.$('.animated')[0], "Animated class should be added!"); +}); + +test('Check for stripes & animation while starting', function(assert) { + this.render(hbs`{{em-progress value=0 striped=true}}`); + + assert.equal(this.$().text().trim(), '0%'); + assert.ok(this.$('.striped')[0], "Striped class added"); + assert.ok(!this.$('.animated')[0], "Animated class shouldn't be added!"); +}); + +test('Check for stripes & animation on completion', function(assert) { + this.render(hbs`{{em-progress value=1 striped=true}}`); + + assert.equal(this.$().text().trim(), '100%'); + assert.ok(this.$('.striped')[0], "Striped class added"); + assert.ok(!this.$('.animated')[0], "Animated class shouldn't be added!"); +}); diff --git a/tez-ui/src/main/webapp/tests/unit/helpers/txt-test.js b/tez-ui/src/main/webapp/tests/unit/helpers/txt-test.js new file mode 100644 index 0000000000..18d3de7ace --- /dev/null +++ b/tez-ui/src/main/webapp/tests/unit/helpers/txt-test.js @@ -0,0 +1,59 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { txt } from '../../../helpers/txt'; +import { module, test } from 'qunit'; + +module('Unit | Helper | txt'); + +test('txt: created', function(assert) { + assert.ok(txt); +}); + +test('txt: String', function(assert) { + assert.equal(txt(["Abc"], {}), "Abc"); + assert.equal(txt(null, {}).string, ' Not Available! '); +}); + +test('txt: String - success', function(assert) { + assert.equal(txt(["Abc"], {}), "Abc"); + assert.equal(txt(null, {}).string, ' Not Available! '); + assert.equal(txt([null], {}).string, ' Not Available! '); +}); + +test('txt: String - error', function(assert) { + var obj = {}; + + obj.toString = null; + assert.equal(txt([obj], {}).string, ' Invalid Data! '); +}); + +test('txt: json', function(assert) { + var obj = { + x: 1, + y: 2 + }; + assert.equal(txt([obj], { + type: "json", + }).string, '{\n "x": 1,\n "y": 2\n}'); +}); + +test('txt: error', function(assert) { + var err = new Error("testError"); + assert.equal(txt([err], {}).string, ' testError '); +}); diff --git a/tez-ui/src/main/webapp/tests/unit/utils/formatters-test.js b/tez-ui/src/main/webapp/tests/unit/utils/formatters-test.js new file mode 100644 index 0000000000..4ecc14309c --- /dev/null +++ b/tez-ui/src/main/webapp/tests/unit/utils/formatters-test.js @@ -0,0 +1,99 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import fmts from '../../../utils/formatters'; +import { module, test } from 'qunit'; + +import Ember from 'ember'; + +module('Unit | Utility | formatters'); + +test('Formatter functions created', function(assert) { + assert.ok(fmts); + + assert.ok(fmts.date); + assert.ok(fmts.duration); + assert.ok(fmts.number); + assert.ok(fmts.memory); +}); + +test('duration', function(assert) { + var options = { + format: "long" + }; + assert.equal(fmts.duration(0, options), "0 millisecond"); + assert.equal(fmts.duration(1, options), "1 millisecond"); + assert.equal(fmts.duration(60, options), "60 milliseconds"); + assert.equal(fmts.duration(6000, options), "6 seconds"); + assert.equal(fmts.duration(66000, options), "1 minute 6 seconds"); + assert.equal(fmts.duration(666000, options), "11 minutes 6 seconds"); + assert.equal(fmts.duration(6666000, options), "1 hour 51 minutes 6 seconds"); + assert.equal(fmts.duration(66666000, options), "18 hours 31 minutes 6 seconds"); + + options = { + format: "short" + }; // By default format = short + assert.equal(fmts.duration(0, options), "0 msec"); + assert.equal(fmts.duration(60, options), "60 msecs"); + assert.equal(fmts.duration(6000, options), "6 secs"); + assert.equal(fmts.duration(66000, options), "1 min 6 secs"); + assert.equal(fmts.duration(666000, options), "11 mins 6 secs"); + assert.equal(fmts.duration(6666000, options), "1 hr 51 mins 6 secs"); + assert.equal(fmts.duration(66666000, options), "18 hrs 31 mins 6 secs"); + + assert.equal(fmts.duration(60.4, options), "60 msecs"); + assert.equal(fmts.duration(60.6, options), "61 msecs"); + + options = {}; // By default format = xshort + assert.equal(fmts.duration(0, options), "0ms"); + assert.equal(fmts.duration(60, options), "60ms"); + assert.equal(fmts.duration(6000, options), "6s"); + assert.equal(fmts.duration(66000, options), "1m 6s"); + assert.equal(fmts.duration(666000, options), "11m 6s"); + assert.equal(fmts.duration(6666000, options), "1h 51m 6s"); + assert.equal(fmts.duration(66666000, options), "18h 31m 6s"); +}); + +test('number', function(assert) { + assert.equal(fmts.number(6000, {}), "6,000"); + assert.equal(fmts.number(6000000, {}), "6,000,000"); +}); + +test('memory', function(assert) { + assert.equal(fmts.memory(0, {}), "0 B"); + assert.equal(fmts.memory(600, {}), "600 B"); + assert.equal(fmts.memory(1024, {}), "1 KB"); + assert.equal(fmts.memory(1024 * 1024, {}), "1 MB"); + assert.equal(fmts.memory(1024 * 1024 * 1024, {}), "1 GB"); + assert.equal(fmts.memory(1024 * 1024 * 1024 * 1024, {}), "1 TB"); +}); + +test('json', function(assert) { + var str = "testString", + complexObj = Ember.Object.create(); + + assert.equal(fmts.json(str, {}), str); + assert.equal(fmts.json(complexObj, {}), complexObj); + + assert.equal(fmts.json(null, {}), null); + assert.equal(fmts.json(undefined, {}), undefined); + + assert.equal(fmts.json({x: 1}, {}), '{\n "x": 1\n}'); + assert.equal(fmts.json({x: 1, y: 2}, {space: 1}), '{\n "x": 1,\n "y": 2\n}'); + assert.equal(fmts.json({x: 1, y: {z: 3}}, {space: 1}), '{\n "x": 1,\n "y": {\n "z": 3\n }\n}'); +}); diff --git a/tez-ui/src/main/webapp/yarn.lock b/tez-ui/src/main/webapp/yarn.lock index fbcdd21369..00250e82cf 100644 --- a/tez-ui/src/main/webapp/yarn.lock +++ b/tez-ui/src/main/webapp/yarn.lock @@ -1391,16 +1391,6 @@ ee-first@1.1.1: version "1.1.1" resolved "https://registry.yarnpkg.com/ee-first/-/ee-first-1.1.1.tgz#590c61156b0ae2f4f0255732a158b266bc56b21d" -em-helpers@0.8.0: - version "0.8.0" - resolved "https://registry.yarnpkg.com/em-helpers/-/em-helpers-0.8.0.tgz#01678f3692a61d563cce68e49459e206d14db095" - dependencies: - ember-cli-htmlbars "^1.0.1" - ember-cli-less "^1.4.0" - source-map "^0.5.6" - optionalDependencies: - phantomjs-prebuilt "2.1.13" - em-table@0.11.3: version "0.11.3" resolved "https://registry.yarnpkg.com/em-table/-/em-table-0.11.3.tgz#20e605cc3814214e644199399a2383cee8d23eeb" From 0ccf440e64e73646a14ee7366833eb274a3533c9 Mon Sep 17 00:00:00 2001 From: jteagles Date: Thu, 26 Aug 2021 09:14:29 -0500 Subject: [PATCH 331/512] TEZ-4329. Import external tez component em-table (#146) Co-authored-by: Sreenath Somarajapuram --- .../src/main/resources/META-INF/LICENSE.txt | 1 - .../webapp/app/components/em-table-cell.js | 124 +++++++ .../webapp/app/components/em-table-column.js | 109 ++++++ .../components/em-table-facet-panel-values.js | 199 ++++++++++ .../app/components/em-table-facet-panel.js | 86 +++++ .../app/components/em-table-header-cell.js | 64 ++++ .../app/components/em-table-linked-cell.js | 64 ++++ .../app/components/em-table-pagination-ui.js | 98 +++++ .../app/components/em-table-progress-cell.js | 46 +++ .../app/components/em-table-search-ui.js | 95 +++++ .../app/components/em-table-status-cell.js | 2 + .../main/webapp/app/components/em-table.js | 281 ++++++++++++++ .../webapp/app/controllers/app/configs.js | 2 +- .../main/webapp/app/controllers/app/dags.js | 2 +- .../webapp/app/controllers/counters-table.js | 2 +- .../webapp/app/controllers/dag/attempts.js | 2 +- .../webapp/app/controllers/dag/graphical.js | 2 +- .../webapp/app/controllers/dag/index/index.js | 2 +- .../webapp/app/controllers/dag/swimlane.js | 2 +- .../main/webapp/app/controllers/dag/tasks.js | 2 +- .../webapp/app/controllers/dag/vertices.js | 2 +- .../main/webapp/app/controllers/home/index.js | 4 +- .../webapp/app/controllers/home/queries.js | 4 +- .../webapp/app/controllers/query/configs.js | 2 +- .../webapp/app/controllers/query/timeline.js | 2 +- .../src/main/webapp/app/controllers/table.js | 2 +- .../webapp/app/controllers/task/attempts.js | 2 +- .../webapp/app/controllers/vertex/attempts.js | 2 +- .../webapp/app/controllers/vertex/configs.js | 2 +- .../webapp/app/controllers/vertex/tasks.js | 2 +- tez-ui/src/main/webapp/app/styles/app.less | 1 + .../app/styles/em-table-facet-panel.less | 218 +++++++++++ .../src/main/webapp/app/styles/em-table.less | 344 ++++++++++++++++++ .../main/webapp/app/styles/pagination-ui.less | 88 +++++ .../main/webapp/app/styles/progress-cell.less | 26 ++ .../src/main/webapp/app/styles/search-ui.less | 37 ++ tez-ui/src/main/webapp/app/styles/shared.less | 115 ++++++ .../src/main/webapp/app/styles/variables.less | 28 ++ .../templates/components/em-table-cell.hbs | 41 +++ .../templates/components/em-table-column.hbs | 22 ++ .../em-table-facet-panel-values.hbs | 50 +++ .../components/em-table-facet-panel.hbs | 33 ++ .../components/em-table-header-cell.hbs | 30 ++ .../components/em-table-linked-cell.hbs | 41 +++ .../components/em-table-pagination-ui.hbs | 45 +++ .../components/em-table-progress-cell.hbs | 29 ++ .../components/em-table-search-ui.hbs | 52 +++ .../app/templates/components/em-table.hbs | 106 ++++++ .../webapp/app/utils/column-definition.js | 125 +++++++ .../app/utils/counter-column-definition.js | 2 +- .../main/webapp/app/utils/data-processor.js | 275 ++++++++++++++ .../src/main/webapp/app/utils/facet-types.js | 85 +++++ tez-ui/src/main/webapp/app/utils/sql.js | 94 +++++ .../main/webapp/app/utils/table-definition.js | 61 ++++ tez-ui/src/main/webapp/bower.json | 1 + tez-ui/src/main/webapp/ember-cli-build.js | 1 + tez-ui/src/main/webapp/package.json | 1 - .../components/em-table-cell-test.js | 45 +++ .../components/em-table-column-test.js | 30 ++ .../components/em-table-facet-panel-test.js | 43 +++ .../em-table-facet-panel-values-test.js | 44 +++ .../components/em-table-header-cell-test.js | 30 ++ .../components/em-table-linked-cell-test.js | 30 ++ .../components/em-table-pagination-ui-test.js | 204 +++++++++++ .../components/em-table-progress-cell-test.js | 43 +++ .../components/em-table-search-ui-test.js | 30 ++ .../integration/components/em-table-test.js | 48 +++ .../integration/em-table-status-cell-test.js | 40 ++ .../unit/utils/column-definition-test.js | 104 ++++++ .../tests/unit/utils/data-processor-test.js | 137 +++++++ .../tests/unit/utils/facet-types-test.js | 28 ++ .../main/webapp/tests/unit/utils/sql-test.js | 90 +++++ .../tests/unit/utils/table-definition-test.js | 52 +++ tez-ui/src/main/webapp/yarn.lock | 10 - 74 files changed, 4135 insertions(+), 33 deletions(-) create mode 100644 tez-ui/src/main/webapp/app/components/em-table-cell.js create mode 100644 tez-ui/src/main/webapp/app/components/em-table-column.js create mode 100644 tez-ui/src/main/webapp/app/components/em-table-facet-panel-values.js create mode 100644 tez-ui/src/main/webapp/app/components/em-table-facet-panel.js create mode 100644 tez-ui/src/main/webapp/app/components/em-table-header-cell.js create mode 100644 tez-ui/src/main/webapp/app/components/em-table-linked-cell.js create mode 100644 tez-ui/src/main/webapp/app/components/em-table-pagination-ui.js create mode 100644 tez-ui/src/main/webapp/app/components/em-table-progress-cell.js create mode 100644 tez-ui/src/main/webapp/app/components/em-table-search-ui.js create mode 100644 tez-ui/src/main/webapp/app/components/em-table.js create mode 100644 tez-ui/src/main/webapp/app/styles/em-table-facet-panel.less create mode 100644 tez-ui/src/main/webapp/app/styles/em-table.less create mode 100644 tez-ui/src/main/webapp/app/styles/pagination-ui.less create mode 100644 tez-ui/src/main/webapp/app/styles/progress-cell.less create mode 100644 tez-ui/src/main/webapp/app/styles/search-ui.less create mode 100644 tez-ui/src/main/webapp/app/styles/variables.less create mode 100644 tez-ui/src/main/webapp/app/templates/components/em-table-cell.hbs create mode 100644 tez-ui/src/main/webapp/app/templates/components/em-table-column.hbs create mode 100644 tez-ui/src/main/webapp/app/templates/components/em-table-facet-panel-values.hbs create mode 100644 tez-ui/src/main/webapp/app/templates/components/em-table-facet-panel.hbs create mode 100644 tez-ui/src/main/webapp/app/templates/components/em-table-header-cell.hbs create mode 100644 tez-ui/src/main/webapp/app/templates/components/em-table-linked-cell.hbs create mode 100644 tez-ui/src/main/webapp/app/templates/components/em-table-pagination-ui.hbs create mode 100644 tez-ui/src/main/webapp/app/templates/components/em-table-progress-cell.hbs create mode 100644 tez-ui/src/main/webapp/app/templates/components/em-table-search-ui.hbs create mode 100644 tez-ui/src/main/webapp/app/templates/components/em-table.hbs create mode 100644 tez-ui/src/main/webapp/app/utils/column-definition.js create mode 100644 tez-ui/src/main/webapp/app/utils/data-processor.js create mode 100644 tez-ui/src/main/webapp/app/utils/facet-types.js create mode 100644 tez-ui/src/main/webapp/app/utils/sql.js create mode 100644 tez-ui/src/main/webapp/app/utils/table-definition.js create mode 100644 tez-ui/src/main/webapp/tests/integration/components/em-table-cell-test.js create mode 100644 tez-ui/src/main/webapp/tests/integration/components/em-table-column-test.js create mode 100644 tez-ui/src/main/webapp/tests/integration/components/em-table-facet-panel-test.js create mode 100644 tez-ui/src/main/webapp/tests/integration/components/em-table-facet-panel-values-test.js create mode 100644 tez-ui/src/main/webapp/tests/integration/components/em-table-header-cell-test.js create mode 100644 tez-ui/src/main/webapp/tests/integration/components/em-table-linked-cell-test.js create mode 100644 tez-ui/src/main/webapp/tests/integration/components/em-table-pagination-ui-test.js create mode 100644 tez-ui/src/main/webapp/tests/integration/components/em-table-progress-cell-test.js create mode 100644 tez-ui/src/main/webapp/tests/integration/components/em-table-search-ui-test.js create mode 100644 tez-ui/src/main/webapp/tests/integration/components/em-table-test.js create mode 100644 tez-ui/src/main/webapp/tests/integration/em-table-status-cell-test.js create mode 100644 tez-ui/src/main/webapp/tests/unit/utils/column-definition-test.js create mode 100644 tez-ui/src/main/webapp/tests/unit/utils/data-processor-test.js create mode 100644 tez-ui/src/main/webapp/tests/unit/utils/facet-types-test.js create mode 100644 tez-ui/src/main/webapp/tests/unit/utils/sql-test.js create mode 100644 tez-ui/src/main/webapp/tests/unit/utils/table-definition-test.js diff --git a/tez-ui/src/main/resources/META-INF/LICENSE.txt b/tez-ui/src/main/resources/META-INF/LICENSE.txt index 354d7457a9..608dc614c8 100644 --- a/tez-ui/src/main/resources/META-INF/LICENSE.txt +++ b/tez-ui/src/main/resources/META-INF/LICENSE.txt @@ -232,7 +232,6 @@ The Apache TEZ tez-ui bundles the following files under the MIT License: - more-js v0.8.2 (https://github.com/sreenaths/snippet-ss) - snippet-ss v1.11.0 (https://github.com/sreenaths/snippet-ss) - em-tgraph v0.0.4 (https://github.com/sreenaths/em-tgraph) - - em-table v0.3.12 (https://github.com/sreenaths/em-table) - ember-cli-app-version v1.0.0 (https://github.com/EmberSherpa/ember-cli-app-version) - Authored by Taras Mankovski - ember-cli-auto-register v1.1.0 (https://github.com/williamsbdev/ember-cli-auto-register) - Copyright © 2015 Brandon Williams http://williamsbdev.com - ember-cli-content-security-policy v0.4.0 (https://github.com/rwjblue/ember-cli-content-security-policy) diff --git a/tez-ui/src/main/webapp/app/components/em-table-cell.js b/tez-ui/src/main/webapp/app/components/em-table-cell.js new file mode 100644 index 0000000000..d4e6a54f53 --- /dev/null +++ b/tez-ui/src/main/webapp/app/components/em-table-cell.js @@ -0,0 +1,124 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import layout from '../templates/components/em-table-cell'; + +export default Ember.Component.extend({ + layout: layout, + + classNames: ['table-cell'], + classNameBindings: ['innerCell', 'isWaiting'], + + innerCell: Ember.computed('index', function () { + if(this.get('index')) { + return 'inner'; + } + }), + + row: null, + columnDefinition: null, + + isWaiting: false, + + _value: null, + _observedPath: null, + _comment: null, + _cellContent: Ember.computed({ + set: function (key, value, prevValue) { + if(value !== prevValue) { + this.highlightCell(); + } + return value; + } + }), + + _addObserver: function (path) { + this._removeObserver(); + this.get('row').addObserver(path, this, this._onValueChange); + this.set('_observedPath', path); + }, + + _removeObserver: function () { + var path = this.get('_observedPath'); + if(path) { + this.get('row').removeObserver(path, this, this._onValueChange); + this.set('_observedPath', null); + } + }, + + _pathObserver: Ember.on('init', Ember.observer('row', 'columnDefinition.contentPath', 'columnDefinition.observePath', function () { + var path = this.get('columnDefinition.contentPath'); + if(path && this.get('columnDefinition.observePath')) { + this._addObserver(path); + } + })), + + _onValueChange: function (row, path) { + this.set('_value', row.get(path)); + }, + + setContent: function (content) { + var comment; + + if(content && content.hasOwnProperty("content")) { + comment = content.comment; + content = content.content; + } + + this.setProperties({ + _comment: comment, + _cellContent: content, + isWaiting: false + }); + }, + + _cellContentObserver: Ember.on('init', Ember.observer('row', 'columnDefinition', '_value', function () { + var cellContent = this.get('columnDefinition').getCellContent(this.get('row'), this.get("_value")), + that = this; + + if(cellContent && cellContent.then) { + cellContent.then(function (content) { + that.setContent(content); + }); + this.set('isWaiting', true); + } + else if(cellContent === undefined && this.get('columnDefinition.observePath')) { + this.set('isWaiting', true); + } + else { + this.setContent(cellContent); + } + })), + + highlightCell: function () { + var element = this.$(); + if(element) { + element.removeClass("bg-transition"); + element.addClass("highlight"); + Ember.run.later(function () { + element.addClass("bg-transition"); + element.removeClass("highlight"); + }, 100); + } + }, + + willDestroy: function () { + this._removeObserver(); + } +}); diff --git a/tez-ui/src/main/webapp/app/components/em-table-column.js b/tez-ui/src/main/webapp/app/components/em-table-column.js new file mode 100644 index 0000000000..a3cb5a9072 --- /dev/null +++ b/tez-ui/src/main/webapp/app/components/em-table-column.js @@ -0,0 +1,109 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; + +import layout from '../templates/components/em-table-column'; + +export default Ember.Component.extend({ + layout: layout, + + definition: null, + rows: null, + index: 0, + + tableDefinition: null, + dataProcessor: null, + adjustedWidth: null, + defaultWidth: "", + + classNames: ['table-column'], + classNameBindings: ['inner', 'extraClassNames'], + + inner: Ember.computed('index', function () { + return !!this.get('index'); + }), + + extraClassNames: Ember.computed("definition.classNames", function () { + var classNames = this.get("definition.classNames"); + if(classNames) { + return classNames.join(" "); + } + }), + + didInsertElement: function () { + Ember.run.scheduleOnce('afterRender', this, function() { + this.setWidth(); + this.setMinWidth(); + }); + }, + + setMinWidth: Ember.observer("definition.minWidth", function () { + this.$().css("minWidth", this.get('definition.minWidth')); + }), + + setWidth: Ember.observer("adjustedWidth", "defaultWidth", function () { + var thisElement = this.$(); + thisElement.css("width", this.get('adjustedWidth') || this.get('defaultWidth')); + Ember.run.scheduleOnce('afterRender', this, function() { + this.get('parentView').send('columnWidthChanged', thisElement.width(), this.get("definition"), this.get("index")); + }); + }), + + _onColResize: function (event) { + var data = event.data, + width; + + if(!data.startEvent) { + data.startEvent = event; + } + + width = data.startWidth + event.clientX - data.startEvent.clientX; + data.thisObj.set('adjustedWidth', width); + }, + + _endColResize: function (event) { + var thisObj = event.data.thisObj; + Ember.$(document).off('mousemove', thisObj._onColResize); + Ember.$(document).off('mouseup', thisObj._endColResize); + }, + + actions: { + sort: function () { + var definition = this.get('definition'), + beforeSort = definition.get('beforeSort'); + + if(!beforeSort || beforeSort.call(definition, definition)) { + let columnId = this.get('definition.id'), + sortOrder = this.get('tableDefinition.sortOrder') === 'desc' ? 'asc' : 'desc'; + + this.get('parentView').send('sort', columnId, sortOrder); + } + }, + startColResize: function () { + var mouseTracker = { + thisObj: this, + startWidth: this.$().width(), + startEvent: null + }; + + Ember.$(document).on('mousemove', mouseTracker, this._onColResize); + Ember.$(document).on('mouseup', mouseTracker, this._endColResize); + } + } +}); diff --git a/tez-ui/src/main/webapp/app/components/em-table-facet-panel-values.js b/tez-ui/src/main/webapp/app/components/em-table-facet-panel-values.js new file mode 100644 index 0000000000..ec88181f4d --- /dev/null +++ b/tez-ui/src/main/webapp/app/components/em-table-facet-panel-values.js @@ -0,0 +1,199 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import layout from '../templates/components/em-table-facet-panel-values'; + +const LIST_LIMIT = 7; + +export default Ember.Component.extend({ + layout: layout, + + data: null, + checkedCount: null, + + tableDefinition: null, + dataProcessor: null, + + tmpFacetConditions: null, + + hideValues: true, + + currentPage: 1, + + classNames: ['em-table-facet-panel-values'], + classNameBindings: ['hideValues', 'hideFilter', 'hideSelectAll'], + + filterText: null, + allButtonTitle: Ember.computed("filterText", function () { + let filterText = this.get("filterText"); + return filterText ? `Select all with substring '${filterText}'` : "Select all"; + }), + isVisible: Ember.computed("data.facets.length", "tableDefinition.minValuesToDisplay", function () { + return this.get("data.facets.length") >= this.get("tableDefinition.minValuesToDisplay"); + }), + hideFilter: Ember.computed("allFacets.length", function () { + return this.get("allFacets.length") < LIST_LIMIT; + }), + hideSelectAll: Ember.computed("fieldFacetConditions", "checkedCount", "data.facets", function () { + return this.get("fieldFacetConditions.in.length") === this.get("data.facets.length"); + }), + + fieldFacetConditions: Ember.computed("tmpFacetConditions", "data.column.id", function () { + var columnID = this.get("data.column.id"), + conditions = this.get(`tmpFacetConditions.${columnID}`), + facets = this.get("data.facets") || []; + + if(!conditions) { + conditions = { + in: facets.map(facet => facet.value) + }; + this.set(`tmpFacetConditions.${columnID}`, conditions); + } + + return conditions; + }), + + allFacets: Ember.computed("data.facets", "fieldFacetConditions", function () { + var facets = this.get("data.facets") || [], + + checkedValues = this.get("fieldFacetConditions.in"), + selectionHash = {}; + + if(checkedValues) { + checkedValues.forEach(function (valueText) { + selectionHash[valueText] = 1; + }); + } + + return Ember.A(facets.map(function (facet) { + facet = Ember.Object.create(facet); + facet.set("checked", selectionHash[facet.value]); + + if(!facet.get("displayText")) { + facet.set("displayText", facet.get("value")); + } + + return facet; + })); + }), + + filteredFacets: Ember.computed("allFacets", "filterText", function () { + var allFacets = this.get("allFacets"), + filterText = this.get("filterText"), + filteredFacets; + + if(filterText) { + filteredFacets = allFacets.filter(function (facet) { + return facet.get("value").match(filterText); + }); + } + else { + filteredFacets = allFacets; + } + + return filteredFacets; + }), + + _filterObserver: Ember.observer("filterText", function () { + this.set("currentPage", 1); + }), + + totalPages: Ember.computed("filteredFacets.length", "tableDefinition.facetValuesPageSize", function () { + return Math.ceil(this.get("filteredFacets.length") / this.get("tableDefinition.facetValuesPageSize")); + }), + showPagination: Ember.computed("totalPages", function () { + return this.get("totalPages") > 1; + }), + showPrevious: Ember.computed("currentPage", function () { + return this.get("currentPage") > 1; + }), + showNext: Ember.computed("currentPage", "totalPages", function () { + return this.get("currentPage") < this.get("totalPages"); + }), + + paginatedFacets: Ember.computed("filteredFacets", "currentPage", "tableDefinition.facetValuesPageSize", function () { + let currentPage = this.get("currentPage"), + pageSize = this.get("tableDefinition.facetValuesPageSize"); + return this.get("filteredFacets").slice( + (currentPage - 1) * pageSize, + currentPage * pageSize); + }), + + actions: { + changePage: function (factor) { + var newPage = this.get("currentPage") + factor; + if(newPage > 0 && newPage <= this.get("totalPages")) { + this.set("currentPage", newPage); + } + }, + toggleValueDisplay: function () { + this.toggleProperty("hideValues"); + this.get("parentView").sendAction("toggleValuesDisplayAction", !this.get("hideValues"), this.get("data")); + }, + clickedCheckbox: function (facet) { + var checkedValues = this.get("fieldFacetConditions.in"), + value = facet.get("value"), + valueIndex = checkedValues.indexOf(value); + + facet.toggleProperty("checked"); + + if(facet.get("checked")) { + if(valueIndex === -1) { + checkedValues.push(value); + } + } + else if(valueIndex !== -1) { + checkedValues.splice(valueIndex, 1); + } + + this.set("checkedCount", checkedValues.length); + }, + + selectAll: function () { + var filteredFacets = this.get("filteredFacets"), + checkedValues = this.get("fieldFacetConditions.in"); + + filteredFacets.forEach(function (facet) { + if(!facet.get("checked")) { + checkedValues.push(facet.get("value")); + } + + facet.set("checked", true); + }); + + this.set("fieldFacetConditions.in", checkedValues); + this.set("checkedCount", checkedValues.length); + }, + clickedOnly: function (facet) { + var allFacets = this.get("allFacets"), + checkedValues = []; + + allFacets.forEach(function (facet) { + facet.set("checked", false); + }); + + facet.set("checked", true); + checkedValues.push(facet.get("value")); + + this.set("fieldFacetConditions.in", checkedValues); + this.set("checkedCount", checkedValues.length); + } + } + +}); diff --git a/tez-ui/src/main/webapp/app/components/em-table-facet-panel.js b/tez-ui/src/main/webapp/app/components/em-table-facet-panel.js new file mode 100644 index 0000000000..fdbd8f5b89 --- /dev/null +++ b/tez-ui/src/main/webapp/app/components/em-table-facet-panel.js @@ -0,0 +1,86 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import layout from '../templates/components/em-table-facet-panel'; + +export default Ember.Component.extend({ + layout: layout, + + classNames: ["em-table-facet-panel"], + classNameBindings: ['isEmpty', 'hideFilter'], + + isVisible: Ember.computed.alias('tableDefinition.enableFaceting'), + + tableDefinition: null, + dataProcessor: null, + tmpFacetConditions: {}, + + filterText: null, + isEmpty: Ember.computed("dataProcessor.facetedFields.length", function () { + return this.get("dataProcessor.facetedFields.length") === 0; + }), + hideFilter: Ember.computed("dataProcessor.facetedFields.length", "tableDefinition.minFieldsForFilter", function () { + return this.get("dataProcessor.facetedFields.length") < this.get("tableDefinition.minFieldsForFilter"); + }), + + didInsertElement: Ember.observer("filterText", "dataProcessor.facetedFields", function () { + var fields = this.get("dataProcessor.facetedFields"), + filterText = this.get("filterText"), + filterRegex = new RegExp(filterText, "i"), + elements = Ember.$(this.get("element")).find(".field-list>li"); + + elements.each(function (index, element) { + var foundMatch = !filterText || Ember.get(fields, `${index}.column.headerTitle`).match(filterRegex); + Ember.$(element)[foundMatch ? "show" : "hide"](); + }); + }), + + _facetConditionsObserver: Ember.observer("tableDefinition.facetConditions", "dataProcessor.processedRows.[]", function () { + var facetConditions = Ember.$.extend({}, this.get("tableDefinition.facetConditions")); + this.set("tmpFacetConditions", facetConditions); + }), + + actions: { + applyFilters: function () { + var tmpFacetConditions = this.get("tmpFacetConditions"), + facetedFields = this.get("dataProcessor.facetedFields"), + normalizedTmpFacetConditions = {}; + + facetedFields.forEach(function (field) { + var column = field.column, + columnId = column.get("id"), + facetType = column.get("facetType"), + normalizedConditions; + + if(facetType) { + normalizedConditions = facetType.normaliseConditions(tmpFacetConditions[columnId], field.facets); + if(normalizedConditions) { + normalizedTmpFacetConditions[columnId] = normalizedConditions; + } + } + }); + + this.set("tableDefinition.facetConditions", normalizedTmpFacetConditions); + }, + clearFilters: function () { + this.set("tmpFacetConditions", {}); + this.set("tableDefinition.facetConditions", {}); + }, + } +}); diff --git a/tez-ui/src/main/webapp/app/components/em-table-header-cell.js b/tez-ui/src/main/webapp/app/components/em-table-header-cell.js new file mode 100644 index 0000000000..c0a8e12379 --- /dev/null +++ b/tez-ui/src/main/webapp/app/components/em-table-header-cell.js @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import layout from '../templates/components/em-table-header-cell'; + +export default Ember.Component.extend({ + layout: layout, + + title: null, // Header cell Name + attributeBindings: ['title'], + + definition: null, + tableDefinition: null, + dataProcessor: null, + + classNames: ['table-header-cell'], + classNameBindings: ['isSorting'], + + isSorting: Ember.computed("dataProcessor.isSorting", function () { + return this.get("dataProcessor.isSorting") && this.get('tableDefinition.sortColumnId') === this.get('definition.id'); + }), + + sortIconCSS: Ember.computed('tableDefinition.sortOrder', 'tableDefinition.sortColumnId', function () { + if(this.get('tableDefinition.sortColumnId') === this.get('definition.id')) { + return this.get('tableDefinition.sortOrder'); + } + }), + + sortToggledTitle: Ember.computed('tableDefinition.sortOrder', 'tableDefinition.sortColumnId', function () { + if(this.get('tableDefinition.sortColumnId') === this.get('definition.id')) { + switch(this.get('tableDefinition.sortOrder')) { + case "asc": + return "descending"; + case "desc": + return "ascending"; + } + } + }), + + actions: { + sort: function () { + this.get('parentView').send('sort'); + }, + startColResize: function () { + this.get('parentView').send('startColResize'); + } + } +}); diff --git a/tez-ui/src/main/webapp/app/components/em-table-linked-cell.js b/tez-ui/src/main/webapp/app/components/em-table-linked-cell.js new file mode 100644 index 0000000000..c42c56613b --- /dev/null +++ b/tez-ui/src/main/webapp/app/components/em-table-linked-cell.js @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import layout from '../templates/components/em-table-linked-cell'; + +export default Ember.Component.extend({ + layout: layout, + + definition: null, + content: null, + + normalizedLinks: Ember.computed("content", function () { + var content = this.get("content"), + links; + + if(content) { + if(!Array.isArray(content)) { + content = [content]; + } + + links = content.map(function (link) { + var model, + text = Ember.get(link, "text") || Ember.get(link, "displayText"); + + if(text) { + link = Ember.Object.create(link, { + text: text + }); + + if(link.get("model") === undefined) { + link.set("model", link.get("id")); + } + + model = link.get("model"); + link.set("withModel", model !== undefined); + + return link; + } + }); + + links = links.filter(function (link) { + return link; + }); + } + + return links; + }) +}); diff --git a/tez-ui/src/main/webapp/app/components/em-table-pagination-ui.js b/tez-ui/src/main/webapp/app/components/em-table-pagination-ui.js new file mode 100644 index 0000000000..858928b60b --- /dev/null +++ b/tez-ui/src/main/webapp/app/components/em-table-pagination-ui.js @@ -0,0 +1,98 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import layout from '../templates/components/em-table-pagination-ui'; + +export default Ember.Component.extend({ + layout: layout, + + tableDefinition: null, + dataProcessor: null, + + classNames: ['pagination-ui'], + isVisible: Ember.computed.alias('tableDefinition.enablePagination'), + + showFirst: Ember.computed('_possiblePages', function () { + return this.get("dataProcessor.totalPages") && this.get('_possiblePages.0.pageNum') !== 1; + }), + + showLast: Ember.computed('_possiblePages', 'dataProcessor.totalPages', function () { + var possiblePages = this.get("_possiblePages"); + if(possiblePages.length) { + return possiblePages[possiblePages.length - 1].pageNum !== this.get("dataProcessor.totalPages"); + } + }), + + rowCountOptions: Ember.computed('tableDefinition.rowCountOptions', 'tableDefinition.rowCount', function () { + var options = this.get('tableDefinition.rowCountOptions'), + rowCount = this.get('tableDefinition.rowCount'); + + return options.map(function (option) { + return { + value: option, + selected: option === rowCount + }; + }); + }), + + _possiblePages: Ember.computed('tableDefinition.pageNum', 'dataProcessor.totalPages', function () { + var pageNum = this.get('tableDefinition.pageNum'), + totalPages = this.get('dataProcessor.totalPages'), + possiblePages = [], + startPage = 1, + endPage = totalPages, + delta = 0; + + if(totalPages > 5) { + startPage = pageNum - 2; + endPage = pageNum + 2; + + if(startPage < 1) { + delta = 1 - startPage; + } + else if(endPage > totalPages) { + delta = totalPages - endPage; + } + + startPage += delta; + endPage += delta; + } + + while(startPage <= endPage) { + possiblePages.push({ + isCurrent: startPage === pageNum, + pageNum: startPage++ + }); + } + + return possiblePages; + }), + + actions: { + rowSelected: function (value) { + value = parseInt(value); + if(this.get('tableDefinition.rowCount') !== value) { + this.get('parentView').send('rowChanged', value); + } + }, + changePage: function (value) { + this.get('parentView').send('pageChanged', value); + } + } +}); diff --git a/tez-ui/src/main/webapp/app/components/em-table-progress-cell.js b/tez-ui/src/main/webapp/app/components/em-table-progress-cell.js new file mode 100644 index 0000000000..32f75c473a --- /dev/null +++ b/tez-ui/src/main/webapp/app/components/em-table-progress-cell.js @@ -0,0 +1,46 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import layout from '../templates/components/em-table-progress-cell'; + +export default Ember.Component.extend({ + layout: layout, + + content: null, + + message: Ember.computed("content", function () { + var content = this.get("content"); + + if(content === undefined || content === null) { + return "Not Available!"; + } + else if(isNaN(parseFloat(content))){ + return "Invalid Data!"; + } + }), + + _definition: Ember.computed("definition", function () { + return Ember.Object.extend({ + valueMin: 0, + valueMax: 1, + striped: true, + style: null + }).create(this.get("definition")); + }) +}); diff --git a/tez-ui/src/main/webapp/app/components/em-table-search-ui.js b/tez-ui/src/main/webapp/app/components/em-table-search-ui.js new file mode 100644 index 0000000000..58c4f75cdf --- /dev/null +++ b/tez-ui/src/main/webapp/app/components/em-table-search-ui.js @@ -0,0 +1,95 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import layout from '../templates/components/em-table-search-ui'; + +export default Ember.Component.extend({ + layout: layout, + + tableDefinition: null, + dataProcessor: null, + + classNames: ['search-ui'], + classNameBindings: ['hasError'], + isVisible: Ember.computed.alias('tableDefinition.enableSearch'), + + searchTypes: ["Regex", "SQL"], + actualSearchType: null, + + text: Ember.computed.oneWay('tableDefinition.searchText'), + + _actualSearchTypeDecider: Ember.observer("tableDefinition.searchType", "text", function () { + var searchType = this.get("tableDefinition.searchType"), + actualSearchType = this.get("actualSearchType"); + + switch(searchType) { + case "SQL": + case "Regex": + actualSearchType = searchType; + break; + + case "manual": + if(!actualSearchType) { + actualSearchType = "Regex"; + } + // Will be set from the template + break; + + case "auto": + var text = this.get("text"), + columns = this.get('tableDefinition.columns'); + + if(text) { + actualSearchType = this.get("dataProcessor.sql").validateClause(text, columns) ? "SQL" : "Regex"; + } + else { + actualSearchType = null; + } + break; + } + + this.set("actualSearchType", actualSearchType); + }), + + hasError: Ember.computed("text", "actualSearchType", "tableDefinition.searchType", function () { + var text = this.get("text"), + columns = this.get('tableDefinition.columns'), + actualSearchType = this.get("actualSearchType"); + + if(text) { + switch(actualSearchType) { + case "SQL": + return !this.get("dataProcessor.sql").validateClause(text, columns); + case "Regex": + try { + new RegExp(text); + } + catch(e) { + return true; + } + } + } + }), + + actions: { + search: function () { + this.get('parentView').send('search', this.get('text'), this.get("actualSearchType")); + } + } +}); diff --git a/tez-ui/src/main/webapp/app/components/em-table-status-cell.js b/tez-ui/src/main/webapp/app/components/em-table-status-cell.js index 7751719865..7f1fee8e35 100644 --- a/tez-ui/src/main/webapp/app/components/em-table-status-cell.js +++ b/tez-ui/src/main/webapp/app/components/em-table-status-cell.js @@ -17,8 +17,10 @@ */ import Ember from 'ember'; +import layout from '../templates/components/em-table-status-cell'; export default Ember.Component.extend({ + layout: layout, content: null, diff --git a/tez-ui/src/main/webapp/app/components/em-table.js b/tez-ui/src/main/webapp/app/components/em-table.js new file mode 100644 index 0000000000..79aae3d480 --- /dev/null +++ b/tez-ui/src/main/webapp/app/components/em-table.js @@ -0,0 +1,281 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import Definition from '../utils/table-definition'; +import ColumnDefinition from '../utils/column-definition'; +import DataProcessor from '../utils/data-processor'; + +import layout from '../templates/components/em-table'; + +const DEFAULT_ROW_HIGHLIGHT_COLOR = "#EEE"; + +function createAssigner(targetPath, targetKey, sourcePath) { + return Ember.on("init", Ember.observer(targetPath, sourcePath, function () { + var target = this.get(targetPath), + source = this.get(sourcePath); + if(target && source !== undefined) { + target.set(targetKey, source); + } + })); +} + +const HANDLERS = { + // Mouse handlers + mouseOver: function(event) { + var index = Ember.$(this).index() + 1; + event.data.highlightRow(index); + }, + mouseLeave: function(event) { + event.data.highlightRow(-1); + }, + + // Scroll handler + onScroll: function(event) { + var tableBody = event.currentTarget, + scrollValues = event.data.get("scrollValues"); + + scrollValues.set("left", tableBody.scrollLeft); + scrollValues.set("width", tableBody.scrollWidth); + } +}; + +export default Ember.Component.extend({ + layout: layout, + + classNames: ["em-table"], + classNameBindings: ["showScrollShadow", "showLeftScrollShadow", "showRightScrollShadow"], + + definition: null, + dataProcessor: null, + + highlightRowOnMouse: false, // Could be true or {color: "#XYZ"} + + headerComponentNames: ['em-table-search-ui', 'em-table-pagination-ui'], + footerComponentNames: ['em-table-pagination-ui'], + + leftPanelComponentName: "em-table-facet-panel", + rightPanelComponentName: "", + + columnWidthChangeAction: null, + + scrollChangeAction: null, + scrollValues: null, + _widthTrackerTimer: null, + + init: function() { + this._super(); + this.set("scrollValues", Ember.Object.create({ + left: 0, + width: 0, + viewPortWidth: 0 + })); + }, + + showScrollShadow: false, + showLeftScrollShadow: false, + showRightScrollShadow: false, + + assignDefinitionInProcessor: createAssigner('_dataProcessor', 'tableDefinition', '_definition'), + assignRowsInProcessor: createAssigner('_dataProcessor', 'rows', 'rows'), + assignColumnsInDefinition: createAssigner('_definition', 'columns', 'columns'), + + assignEnableSortInDefinition: createAssigner('_definition', 'enableSort', 'enableSort'), + assignEnableSearchInDefinition: createAssigner('_definition', 'enableSearch', 'enableSearch'), + assignEnablePaginationInDefinition: createAssigner('_definition', 'enablePagination', 'enablePagination'), + assignRowCountInDefinition: createAssigner('_definition', 'rowCount', 'rowCount'), + + _definition: Ember.computed('definition', 'definitionClass', function () { + return this.get('definition') || (this.get('definitionClass') || Definition).create(); + }), + _dataProcessor: Ember.computed('dataProcessor', 'dataProcessorClass', function () { + return this.get('dataProcessor') || (this.get('dataProcessorClass') || DataProcessor).create(); + }), + + displayFooter: Ember.computed("_definition.minRowsForFooter", "_dataProcessor.processedRows.length", function () { + return this.get("_definition.minRowsForFooter") <= this.get("_dataProcessor.processedRows.length"); + }), + + _processedRowsObserver: Ember.observer('_dataProcessor.processedRows', function () { + this.sendAction('rowsChanged', this.get('_dataProcessor.processedRows')); + }), + + _setColumnWidth: function (columns) { + var widthText = (100 / columns.length) + "%"; + columns.forEach(function (column) { + if(!column.width) { + column.width = widthText; + } + }); + }, + + _columns: Ember.computed('_definition.columns', function () { + var rawColumns = this.get('_definition.columns'), + normalisedColumns = { + left: [], + center: [], + right: [], + length: rawColumns.length + }; + + rawColumns.forEach(function (column) { + normalisedColumns[column.get("pin")].push({ + definition: column, + width: column.width + }); + }); + + if(normalisedColumns.center.length === 0) { + normalisedColumns.center = [{ + definition: ColumnDefinition.fillerColumn, + }]; + } + + this._setColumnWidth(normalisedColumns.center); + + return normalisedColumns; + }), + + message: Ember.computed('_dataProcessor.message', '_columns.length', '_dataProcessor.processedRows.length', function () { + var message = this.get("_dataProcessor.message"); + if(message) { + return message; + } + else if(!this.get('_columns.length')) { + return "No columns available!"; + } + else if(!this.get("_dataProcessor.processedRows.length")) { + let identifiers = Ember.String.pluralize(this.get('_definition.recordType') || "record"); + return `No ${identifiers} available!`; + } + }), + + highlightRow: function (index) { + var element = Ember.$(this.get("element")), + sheet = element.find("style")[0].sheet, + elementID = element.attr("id"), + color = this.get("highlightRowOnMouse.color") || DEFAULT_ROW_HIGHLIGHT_COLOR; + + try { + sheet.deleteRule(0); + }catch(e){} + + if(index >= 0) { + sheet.insertRule(`#${elementID} .table-cell:nth-child(${index}){ background-color: ${color}; }`, 0); + } + }, + + didInsertElement: function () { + Ember.run.scheduleOnce('afterRender', this, function() { + this.highlightRowOnMouseObserver(); + this.scrollChangeActionObserver(); + }); + }, + + highlightRowOnMouseObserver: Ember.observer("highlightRowOnMouse", function () { + var highlightRowOnMouse = this.get("highlightRowOnMouse"), + element = this.get("element"); + + if(element) { + element = Ember.$(element).find(".table-mid"); + + if(highlightRowOnMouse) { + element.on('mouseover', '.table-cell', this, HANDLERS.mouseOver); + element.on('mouseleave', this, HANDLERS.mouseLeave); + } + else { + element.off('mouseover', '.table-cell', HANDLERS.mouseOver); + element.off('mouseleave', HANDLERS.mouseLeave); + } + } + }), + + scrollValuesObserver: Ember.observer("scrollValues.left", "scrollValues.width", "scrollValues.viewPortWidth", function () { + var scrollValues = this.get("scrollValues"); + + this.sendAction("scrollChangeAction", scrollValues); + + + this.set("showLeftScrollShadow", scrollValues.left > 1); + this.set("showRightScrollShadow", scrollValues.left < (scrollValues.width - scrollValues.viewPortWidth)); + }), + + scrollChangeActionObserver: Ember.observer("scrollChangeAction", "message", "showScrollShadow", function () { + Ember.run.scheduleOnce('afterRender', this, function() { + var addScrollListener = this.get("scrollChangeAction") || this.get("showScrollShadow"), + element = this.$().find(".table-body"), + scrollValues = this.get("scrollValues"); + + if(addScrollListener && element) { + element = element.get(0); + + clearInterval(this.get("_widthTrackerTimer")); + + if(element) { + if(addScrollListener) { + Ember.$(element).on('scroll', this, HANDLERS.onScroll); + + this.set("_widthTrackerTimer", setInterval(function () { + scrollValues.setProperties({ + width: element.scrollWidth, + viewPortWidth: element.offsetWidth + }); + }, 1000)); + } + else { + element.off('scroll', HANDLERS.onScroll); + } + } + } + }); + }), + + willDestroyElement: function () { + this._super(); + clearInterval(this.get("_widthTrackerTimer")); + Ember.$(this.$().find(".table-body")).off(); + Ember.$(this.$().find(".table-mid")).off(); + Ember.$(this.$()).off(); + }, + + actions: { + search: function (searchText, actualSearchType) { + this.set('_definition.searchText', searchText); + this.set('_definition._actualSearchType', actualSearchType); + this.sendAction("searchAction", searchText); + }, + sort: function (sortColumnId, sortOrder) { + this.get("_definition").setProperties({ + sortColumnId, + sortOrder + }); + this.sendAction("sortAction", sortColumnId, sortOrder); + }, + rowChanged: function (rowCount) { + this.set('_definition.rowCount', rowCount); + this.sendAction("rowAction", rowCount); + }, + pageChanged: function (pageNum) { + this.set('_definition.pageNum', pageNum); + this.sendAction("pageAction", pageNum); + }, + columnWidthChanged: function (width, columnDefinition, index) { + this.sendAction("columnWidthChangeAction", width, columnDefinition, index); + } + } +}); diff --git a/tez-ui/src/main/webapp/app/controllers/app/configs.js b/tez-ui/src/main/webapp/app/controllers/app/configs.js index 838abc1512..e8f13fc0e0 100644 --- a/tez-ui/src/main/webapp/app/controllers/app/configs.js +++ b/tez-ui/src/main/webapp/app/controllers/app/configs.js @@ -20,7 +20,7 @@ import Ember from 'ember'; import TableController from '../table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; var MoreObject = more.Object; diff --git a/tez-ui/src/main/webapp/app/controllers/app/dags.js b/tez-ui/src/main/webapp/app/controllers/app/dags.js index bb4502a9b1..1febc66f97 100644 --- a/tez-ui/src/main/webapp/app/controllers/app/dags.js +++ b/tez-ui/src/main/webapp/app/controllers/app/dags.js @@ -17,7 +17,7 @@ */ import MultiTableController from '../multi-table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; export default MultiTableController.extend({ breadcrumbs: [{ diff --git a/tez-ui/src/main/webapp/app/controllers/counters-table.js b/tez-ui/src/main/webapp/app/controllers/counters-table.js index 42361b4bb5..37bae66d06 100644 --- a/tez-ui/src/main/webapp/app/controllers/counters-table.js +++ b/tez-ui/src/main/webapp/app/controllers/counters-table.js @@ -20,7 +20,7 @@ import Ember from 'ember'; import TableController from './table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../utils/column-definition'; var MoreObject = more.Object; diff --git a/tez-ui/src/main/webapp/app/controllers/dag/attempts.js b/tez-ui/src/main/webapp/app/controllers/dag/attempts.js index 47e95d9aaa..4616638cba 100644 --- a/tez-ui/src/main/webapp/app/controllers/dag/attempts.js +++ b/tez-ui/src/main/webapp/app/controllers/dag/attempts.js @@ -17,7 +17,7 @@ */ import MultiTableController from '../multi-table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; export default MultiTableController.extend({ breadcrumbs: [{ diff --git a/tez-ui/src/main/webapp/app/controllers/dag/graphical.js b/tez-ui/src/main/webapp/app/controllers/dag/graphical.js index c55ab8b3a4..cc4130fd80 100644 --- a/tez-ui/src/main/webapp/app/controllers/dag/graphical.js +++ b/tez-ui/src/main/webapp/app/controllers/dag/graphical.js @@ -19,7 +19,7 @@ import Ember from 'ember'; import MultiTableController from '../multi-table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; export default MultiTableController.extend({ diff --git a/tez-ui/src/main/webapp/app/controllers/dag/index/index.js b/tez-ui/src/main/webapp/app/controllers/dag/index/index.js index c9adde4c6b..6196b9aa87 100644 --- a/tez-ui/src/main/webapp/app/controllers/dag/index/index.js +++ b/tez-ui/src/main/webapp/app/controllers/dag/index/index.js @@ -19,7 +19,7 @@ import Ember from 'ember'; import MultiTableController from '../../multi-table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../../utils/column-definition'; export default MultiTableController.extend({ columns: ColumnDefinition.make([{ diff --git a/tez-ui/src/main/webapp/app/controllers/dag/swimlane.js b/tez-ui/src/main/webapp/app/controllers/dag/swimlane.js index bbac40b26d..1fe2988858 100644 --- a/tez-ui/src/main/webapp/app/controllers/dag/swimlane.js +++ b/tez-ui/src/main/webapp/app/controllers/dag/swimlane.js @@ -19,7 +19,7 @@ import Ember from 'ember'; import MultiTableController from '../multi-table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; import VertexProcess from '../../utils/vertex-process'; import fullscreen from 'em-tgraph/utils/fullscreen'; diff --git a/tez-ui/src/main/webapp/app/controllers/dag/tasks.js b/tez-ui/src/main/webapp/app/controllers/dag/tasks.js index 92f674ad8e..834abf9b64 100644 --- a/tez-ui/src/main/webapp/app/controllers/dag/tasks.js +++ b/tez-ui/src/main/webapp/app/controllers/dag/tasks.js @@ -17,7 +17,7 @@ */ import MultiTableController from '../multi-table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; export default MultiTableController.extend({ breadcrumbs: [{ diff --git a/tez-ui/src/main/webapp/app/controllers/dag/vertices.js b/tez-ui/src/main/webapp/app/controllers/dag/vertices.js index 313a5a9be5..34295e5fde 100644 --- a/tez-ui/src/main/webapp/app/controllers/dag/vertices.js +++ b/tez-ui/src/main/webapp/app/controllers/dag/vertices.js @@ -17,7 +17,7 @@ */ import MultiTableController from '../multi-table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; export default MultiTableController.extend({ breadcrumbs: [{ diff --git a/tez-ui/src/main/webapp/app/controllers/home/index.js b/tez-ui/src/main/webapp/app/controllers/home/index.js index 754e5e6b99..53c4e6d4bf 100644 --- a/tez-ui/src/main/webapp/app/controllers/home/index.js +++ b/tez-ui/src/main/webapp/app/controllers/home/index.js @@ -19,8 +19,8 @@ import Ember from 'ember'; import TableController from '../table'; -import ColumnDefinition from 'em-table/utils/column-definition'; -import TableDefinition from 'em-table/utils/table-definition'; +import ColumnDefinition from '../../utils/column-definition'; +import TableDefinition from '../../utils/table-definition'; export default TableController.extend({ diff --git a/tez-ui/src/main/webapp/app/controllers/home/queries.js b/tez-ui/src/main/webapp/app/controllers/home/queries.js index ba7e6e3776..b5e483de6e 100644 --- a/tez-ui/src/main/webapp/app/controllers/home/queries.js +++ b/tez-ui/src/main/webapp/app/controllers/home/queries.js @@ -19,8 +19,8 @@ import Ember from 'ember'; import TableController from '../table'; -import ColumnDefinition from 'em-table/utils/column-definition'; -import TableDefinition from 'em-table/utils/table-definition'; +import ColumnDefinition from '../../utils/column-definition'; +import TableDefinition from '../../utils/table-definition'; export default TableController.extend({ diff --git a/tez-ui/src/main/webapp/app/controllers/query/configs.js b/tez-ui/src/main/webapp/app/controllers/query/configs.js index 8dcc91cd0d..d828088f2e 100644 --- a/tez-ui/src/main/webapp/app/controllers/query/configs.js +++ b/tez-ui/src/main/webapp/app/controllers/query/configs.js @@ -20,7 +20,7 @@ import Ember from 'ember'; import TableController from '../table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; var MoreObject = more.Object; diff --git a/tez-ui/src/main/webapp/app/controllers/query/timeline.js b/tez-ui/src/main/webapp/app/controllers/query/timeline.js index b52fc26011..a7cd85eca2 100644 --- a/tez-ui/src/main/webapp/app/controllers/query/timeline.js +++ b/tez-ui/src/main/webapp/app/controllers/query/timeline.js @@ -19,7 +19,7 @@ import Ember from 'ember'; import TableController from '../table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; var MoreObject = more.Object; diff --git a/tez-ui/src/main/webapp/app/controllers/table.js b/tez-ui/src/main/webapp/app/controllers/table.js index 57adf00469..01aec40569 100644 --- a/tez-ui/src/main/webapp/app/controllers/table.js +++ b/tez-ui/src/main/webapp/app/controllers/table.js @@ -20,7 +20,7 @@ import Ember from 'ember'; import AbstractController from './abstract'; -import TableDefinition from 'em-table/utils/table-definition'; +import TableDefinition from '../utils/table-definition'; import isIOCounter from '../utils/misc'; import CounterColumnDefinition from '../utils/counter-column-definition'; diff --git a/tez-ui/src/main/webapp/app/controllers/task/attempts.js b/tez-ui/src/main/webapp/app/controllers/task/attempts.js index a6acaecc84..04eb22ab7c 100644 --- a/tez-ui/src/main/webapp/app/controllers/task/attempts.js +++ b/tez-ui/src/main/webapp/app/controllers/task/attempts.js @@ -17,7 +17,7 @@ */ import MultiTableController from '../multi-table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; import AutoCounterColumn from '../../mixins/auto-counter-column'; diff --git a/tez-ui/src/main/webapp/app/controllers/vertex/attempts.js b/tez-ui/src/main/webapp/app/controllers/vertex/attempts.js index b07be92e06..107ecdb82b 100644 --- a/tez-ui/src/main/webapp/app/controllers/vertex/attempts.js +++ b/tez-ui/src/main/webapp/app/controllers/vertex/attempts.js @@ -17,7 +17,7 @@ */ import MultiTableController from '../multi-table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; import AutoCounterColumn from '../../mixins/auto-counter-column'; diff --git a/tez-ui/src/main/webapp/app/controllers/vertex/configs.js b/tez-ui/src/main/webapp/app/controllers/vertex/configs.js index 1cf4a3d31d..2e1d94e647 100644 --- a/tez-ui/src/main/webapp/app/controllers/vertex/configs.js +++ b/tez-ui/src/main/webapp/app/controllers/vertex/configs.js @@ -20,7 +20,7 @@ import Ember from 'ember'; import TableController from '../table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; var MoreObject = more.Object; diff --git a/tez-ui/src/main/webapp/app/controllers/vertex/tasks.js b/tez-ui/src/main/webapp/app/controllers/vertex/tasks.js index 560c8ba0f9..dac000e247 100644 --- a/tez-ui/src/main/webapp/app/controllers/vertex/tasks.js +++ b/tez-ui/src/main/webapp/app/controllers/vertex/tasks.js @@ -17,7 +17,7 @@ */ import MultiTableController from '../multi-table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; import AutoCounterColumn from '../../mixins/auto-counter-column'; diff --git a/tez-ui/src/main/webapp/app/styles/app.less b/tez-ui/src/main/webapp/app/styles/app.less index f8a66e376d..44bfb836e0 100644 --- a/tez-ui/src/main/webapp/app/styles/app.less +++ b/tez-ui/src/main/webapp/app/styles/app.less @@ -41,6 +41,7 @@ @import "em-swimlane"; @import "em-tooltip"; @import "em-swimlane-vertex-name"; +@import "em-table.less"; @import "em-table-status-cell"; @import "query-timeline"; @import "home-table-controls"; diff --git a/tez-ui/src/main/webapp/app/styles/em-table-facet-panel.less b/tez-ui/src/main/webapp/app/styles/em-table-facet-panel.less new file mode 100644 index 0000000000..28be8f0ef1 --- /dev/null +++ b/tez-ui/src/main/webapp/app/styles/em-table-facet-panel.less @@ -0,0 +1,218 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +.em-table-facet-panel { + width: 160px; + + margin: 5px 5px 0 0; + + border: 1px solid @border-color; + border-radius: @border-radius; + + padding: 5px 10px 10px 10px; + + background-color: @table-bg; + + overflow: hidden; + + .field-filter-box { + width: 100%; + } + + &.hide-filter { + .field-filter-box { + display: none; + } + } + + .filter-message { + color: #999; + } + + h4 { + text-align: center; + margin-top: 5px; + margin-bottom: 0px; + } + + ul { + list-style-type: none; + } + + li { + margin: 2px 0; + } + + ul.field-list { + padding-top: 5px; + padding-left: 0px; + + .em-table-facet-panel-values { + + position: relative; + + .field-name { + .no-select; + + padding-right: 20px; + + cursor: pointer; + display: flex; + + &::before { + content: "\25bc"; + font-size: .7em; + color: @text-light; + margin-top: 5px; + } + + .field-title { + overflow-x: hidden; + text-overflow: ellipsis; + white-space: nowrap; + margin: 0 3px; + } + + .field-count { + color: @text-light; + white-space: nowrap; + } + + .all-button { + margin-left: 5px; + position: absolute; + right: 0px; + } + } + + &.hide-select-all { + .field-name { + padding-right: 0px; + + .all-button { + display: none; + } + } + } + + .value-list { + overflow: hidden; + + padding-left: 10px; + + .filter-box { + width: 100%; + } + + li { + display: flex; + + .checkbox-container { + order: 0; + flex: 0 1 auto; + align-self: auto; + + padding-right: 5px; + } + + .facet-value { + order: 0; + flex: 1 1 auto; + align-self: auto; + padding-right: 2px; + + overflow-x: hidden; + text-overflow: ellipsis; + white-space: nowrap; + } + + .only-button { + order: 0; + flex: 0 1 auto; + align-self: auto; + + cursor: pointer; + + display: none; + padding: 0 5px; + } + + .facet-count { + order: 0; + flex: 0 1 auto; + align-self: auto; + + &:hover{ + text-decoration: none; + } + } + + &:hover { + .only-button { + display: inline; + } + } + } + + .pagination-controls { + padding-top: 5px; + + position: relative; + + .arrows { + position: absolute; + top: 5px; + right: 0px; + } + + span { + user-select: none; + color: lightgrey; + + &.active { + cursor: pointer; + color: #3B99FC; + } + } + } + + } + + &.hide-values { + .value-list { + display: none; + } + + .field-name::before { + transform: rotate(-90deg) translate(2px, 2px); + } + + .field-name .all-button { + display: none; + } + } + + &.hide-filter { + .filter-box { + display: none; + } + } + + } + } +} diff --git a/tez-ui/src/main/webapp/app/styles/em-table.less b/tez-ui/src/main/webapp/app/styles/em-table.less new file mode 100644 index 0000000000..d859d78475 --- /dev/null +++ b/tez-ui/src/main/webapp/app/styles/em-table.less @@ -0,0 +1,344 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// Imports +@import (once) "bower_components/bootstrap/less/bootstrap"; + +@import (once) "bower_components/snippet-ss/less/use"; +@import (once) "bower_components/snippet-ss/less/background"; +@import (once) "bower_components/snippet-ss/less/effects"; + +@import "./variables"; +@import "./shared"; + +@import "./search-ui"; +@import "./pagination-ui"; + +@import "./progress-cell"; + +@import "./em-table-facet-panel"; + +.em-table { + font-size: @font-size; + color: @text-color; + + margin: 10px 0px; + overflow: hidden; + + .table-header { + .clear-fix; + } + + .table-mid { + display: flex; + flex-direction: row; + flex-wrap: nowrap; + justify-content: flex-start; + align-content: stretch; + align-items: flex-start; + + .table-panel-left, table-panel-right, .table-body-left, .table-body-right { + order: 0; + flex: 0 1 auto; + align-self: auto; + } + + .table-body, .table-message { + order: 0; + flex: 1 1 auto; + align-self: auto; + border: 1px solid @border-color; + + margin-top: 5px; + } + + &>div:nth-child(2) { + border-top-left-radius: @border-radius; + border-bottom-left-radius: @border-radius; + border-left: 1px solid @border-color; + } + + &>div:nth-last-child(2) { + border-top-right-radius: @border-radius; + border-bottom-right-radius: @border-radius; + border-right: 1px solid @border-color; + } + } + + .table-footer { + .clear-fix; + } + + .table-body-left, .table-body-right, .table-body { + border-top: 1px solid @border-color; + border-bottom: 1px solid @border-color; + background-color: @table-bg; + } + + .table-message { + border-radius: @border-radius; + background-color: @table-bg; + + text-align: center; + padding: 10px; + } + + .table-body-left, .table-body-right, .table-body { + margin: 5px 0px; + } + + .table-body-left, .table-body-right { + white-space: nowrap; + font-size: 0; // If not set, each column will have a space in between + } + + .table-body{ + .force-scrollbar; + + .table-scroll-body { + //Adding this here will keep the column, and table background same white + //making the UI look better when scroll bar is shown + .dotted-bg; + + white-space: nowrap; + font-size: 0; // If not set, each column will have a space in between + } + } + + &.show-scroll-shadow { + .left-scroll-shadow, .right-scroll-shadow { + order: 0; + flex: 0 1 auto; + align-self: stretch; + position: relative; + + opacity: 0; + transition: opacity 0.3s; + + width: 0px; + z-index: 99; + + pointer-events: none; + + .shadow-container { + position: absolute; + overflow: hidden; + + top: 0px; + bottom: 0px; + width: 50px; + + &:before { + content: ""; + position: absolute; + + top: 10px; + bottom: 15px; + width: 50px; + } + } + } + .left-scroll-shadow { + .shadow-container { + &:before { + left: -50px; + box-shadow: 12px 0 40px -4px rgba(0, 0, 0, 0.2); + } + } + } + .right-scroll-shadow { + .shadow-container { + right: 0px; + &:before { + left: 50px; + box-shadow: -12px 0 40px -4px rgba(0, 0, 0, 0.2); + } + } + } + + &.show-left-scroll-shadow { + .left-scroll-shadow { + opacity: 1; + } + } + &.show-right-scroll-shadow { + .right-scroll-shadow { + opacity: 1; + } + } + } + + .table-column { + .use-border-padding-in-width-height; + + background-color: @table-bg; + + vertical-align: top; + overflow: hidden; + display: inline-block; + min-width: 150px; + + &.inner { + border-left: 1px solid @border-color; + } + + // Just the shaded header + .table-header-cell { + background-color: @bg-grey; + border-bottom: 1px solid @border-color; + + &.is-sorting { + .animated-stripes; + } + } + + .header-body, .table-cell { + font-size: @font-size; + white-space: nowrap; + + text-overflow: ellipsis; + overflow: hidden; + + height: 2.1em; + padding: 5px; + + .ember-view { + text-overflow: ellipsis; + overflow: hidden; + } + } + + .header-body { + font-weight: bold; + + padding-right: 1.1em; // To compensate space occupied by sort/resize buttons + position: relative; // So that buttons can be positioned + + .sort-bar { + cursor: pointer; + position: absolute; + + left: 0; + right: .5em; + top: 0; + bottom: 0; + } + + .sort-icon { + cursor: pointer; + position: absolute; + right: .5em; + top: .2em; + + &:before, &:after { + font-size: .7em; + opacity: .5; + position: absolute; + } + + &:before { + content: "\25B2"; + + top: 0em; + right: 0px; + } + + &:after { + content: "\25BC"; + + top: 1em; + right: 0px; + } + + &.asc{ + &:before { + opacity: 1; + } + &:after { + opacity: .5; + } + } + + &.desc { + &:before { + opacity: .5; + } + &:after { + opacity: 1; + } + } + } + + .resize-column:after { + content: "\22EE"; + cursor: col-resize; + opacity: .3; + + position: absolute; + right: 2px; + top: 6px; + } + } + + .table-cell { + position: relative; + + .comment-indicator { + position: absolute; + + color: white; + font-size: 10px; + padding-left: 4px; + + top: -4px; + right: -4px; + + width: 10px; + height: 10px; + background-color: orange; + border-radius: 10px; + + opacity: 0.6; + + &:hover { + top: -2px; + right: -2px; + } + } + + &.bg-transition { + -webkit-transition: box-shadow 500ms ease-out 500ms; + -moz-transition: box-shadow 500ms ease-out 500ms; + -o-transition: box-shadow 500ms ease-out 500ms; + transition: box-shadow 500ms ease-out 500ms; + } + + &.highlight { + box-shadow: 0 0 60px lighten(@brand-primary, 10%) inset; + } + &.is-waiting { + .animated-stripes; + } + &.inner { + border-top: 1px dotted @border-color; + margin-top: -1px; + } + } + } + +} diff --git a/tez-ui/src/main/webapp/app/styles/pagination-ui.less b/tez-ui/src/main/webapp/app/styles/pagination-ui.less new file mode 100644 index 0000000000..df3d7c070f --- /dev/null +++ b/tez-ui/src/main/webapp/app/styles/pagination-ui.less @@ -0,0 +1,88 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +.pagination-ui { + .inline-block; + .align-top; + + float: right; + + .page-list { + .inline-block; + .align-top; + + overflow: hidden; + + border: 1px solid @border-color; + border-radius: 5px; + background-color: @table-bg; + + padding: 0px; + + font-size: 0px; + + li { + .inline-block; + + padding: 6px 12px; + height: 32px; + + font-size: @font-size; + color: @text-light; + + border-left: 1px solid @border-color; + + pointer-events: none; + + &.clickable { + pointer-events: auto; + color: @text-color; + + &:hover { + background-color: @bg-grey; + cursor: pointer; + } + } + } + + .total-page-count { + font-size: .8em; + } + + :first-child { + border-left: none; + } + } + + .row-select { + margin-left: 5px; + + display: inline-block; + text-align: center; + + select { + cursor: pointer; + } + } +} + +.table-footer { + .pagination-ui { + position: static; + } +} diff --git a/tez-ui/src/main/webapp/app/styles/progress-cell.less b/tez-ui/src/main/webapp/app/styles/progress-cell.less new file mode 100644 index 0000000000..9030216a53 --- /dev/null +++ b/tez-ui/src/main/webapp/app/styles/progress-cell.less @@ -0,0 +1,26 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +.table-cell { + .em-progress-container { + .progress { + margin: -1px 0 0 0; + } + } +} diff --git a/tez-ui/src/main/webapp/app/styles/search-ui.less b/tez-ui/src/main/webapp/app/styles/search-ui.less new file mode 100644 index 0000000000..b850031c19 --- /dev/null +++ b/tez-ui/src/main/webapp/app/styles/search-ui.less @@ -0,0 +1,37 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +.search-ui { + .inline-block; + .align-top; + + max-width: 500px; + + margin-bottom: 2px; + + .type-select { + width: 5em; + margin-right: -3px !important; + } + + .search-syntax { + .label; + .label-default; + .no-select; + } +} diff --git a/tez-ui/src/main/webapp/app/styles/shared.less b/tez-ui/src/main/webapp/app/styles/shared.less index b34cfa6045..b448c30685 100644 --- a/tez-ui/src/main/webapp/app/styles/shared.less +++ b/tez-ui/src/main/webapp/app/styles/shared.less @@ -33,6 +33,99 @@ b { padding-left: 10px; } +.no-select { + -webkit-user-select: none; + -moz-user-select: none; + -ms-user-select: none; + + cursor: default; +} + +.no-display { + display: none !important; +} + +.no-visible { + visibility: hidden !important; +} + +.no-margin { + margin: 0px !important; +} + +.no-pointer { + pointer-events: none; +} + +.inactive { + .no-pointer; + opacity: 0.4; +} + +.no-wrap { + white-space: nowrap; +} + +.no-border { + border: none !important; +} + +.align-top { + vertical-align: top; +} + +.align-super { + vertical-align: super; +} + +.inline-block { + display: inline-block; +} + +.dotted-background { + background: + radial-gradient(#EEE 15%, transparent 17%) 0 0, + radial-gradient(#EEE 15%, transparent 17%) 5px -5px, + radial-gradient(#EEE 15%, transparent 17%) 5px 5px; + background-color: #DDD; + background-size: 10px 10px; +} + +.absolute { + position: absolute; +} + +.use-gpu { + -webkit-transform: translateZ(0); + -moz-transform: translateZ(0); + -ms-transform: translateZ(0); + -o-transform: translateZ(0); + transform: translateZ(0); +} + +.force-scrollbar { + overflow: auto; + + &::-webkit-scrollbar { + -webkit-appearance: none; + } + &::-webkit-scrollbar:vertical { + width: 11px; + } + &::-webkit-scrollbar:horizontal { + height: 11px; + } + &::-webkit-scrollbar-thumb { + border-radius: 8px; + border: 2px solid #EEE; + background-color: #BBB; + } + &::-webkit-scrollbar-track { + background-color: #EEE; + border-radius: 8px; + } +} + .align-checknradio { input[type=checkbox], input[type=radio] { vertical-align: middle; @@ -41,6 +134,28 @@ b { } } +.left-divider{ + padding-left: 5px; + border-left: 1px solid lightgrey; + margin-left: 5px; +} + +.clear-fix { + &:after { + content: " "; + visibility: hidden; + display: block; + height: 0; + clear: both; + } +} + +.animated-stripes { + .diagonal-stripes-background(#FFF, #EEE); + .animate; + .white-inner-glow; +} + .diagnostics { padding: 10px; white-space: pre-line; diff --git a/tez-ui/src/main/webapp/app/styles/variables.less b/tez-ui/src/main/webapp/app/styles/variables.less new file mode 100644 index 0000000000..e6dbeca7b2 --- /dev/null +++ b/tez-ui/src/main/webapp/app/styles/variables.less @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +@text-light: #BBBBBB; +@text-color: #222222; + +@bg-grey: #f0f0f0; +@table-bg: white; + +@border-color: #dcdcdc; +@border-radius: 5px; + +@font-size: 14px; diff --git a/tez-ui/src/main/webapp/app/templates/components/em-table-cell.hbs b/tez-ui/src/main/webapp/app/templates/components/em-table-cell.hbs new file mode 100644 index 0000000000..777be936f4 --- /dev/null +++ b/tez-ui/src/main/webapp/app/templates/components/em-table-cell.hbs @@ -0,0 +1,41 @@ +{{! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +}} + +{{#if isWaiting}} + Waiting... +{{else}} + {{#if columnDefinition.cellComponentName}} + {{component columnDefinition.cellComponentName content=_cellContent definition=columnDefinition.cellDefinition}} + {{else}} + {{#unless columnDefinition.cellDefinition}} + {{txt _cellContent}} + {{else}} + {{txt _cellContent + type=columnDefinition.cellDefinition.type + format=columnDefinition.cellDefinition.format + timeZone=columnDefinition.cellDefinition.timeZone + valueFormat=columnDefinition.cellDefinition.valueFormat + valueTimeZone=columnDefinition.cellDefinition.valueTimeZone + valueUnit=columnDefinition.cellDefinition.valueUnit + }} + {{/unless}} + {{/if}} + {{#if _comment}} +
+ {{/if}} +{{/if}} diff --git a/tez-ui/src/main/webapp/app/templates/components/em-table-column.hbs b/tez-ui/src/main/webapp/app/templates/components/em-table-column.hbs new file mode 100644 index 0000000000..a3908db67a --- /dev/null +++ b/tez-ui/src/main/webapp/app/templates/components/em-table-column.hbs @@ -0,0 +1,22 @@ +{{! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +}} + +{{em-table-header-cell title=definition.headerTitle definition=definition tableDefinition=tableDefinition dataProcessor=dataProcessor}} +{{#each rows as |row rowIndex|}} + {{em-table-cell columnDefinition=definition row=row index=rowIndex}} +{{/each}} diff --git a/tez-ui/src/main/webapp/app/templates/components/em-table-facet-panel-values.hbs b/tez-ui/src/main/webapp/app/templates/components/em-table-facet-panel-values.hbs new file mode 100644 index 0000000000..579e9eb2e1 --- /dev/null +++ b/tez-ui/src/main/webapp/app/templates/components/em-table-facet-panel-values.hbs @@ -0,0 +1,50 @@ +{{! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +}} + +
+
{{data.column.headerTitle}}
+
({{data.facets.length}})
+ All +
+ +
    + {{input type="text" class="filter-box" value=filterText placeholder="Filter"}} + + {{#if showPagination}} +
    + {{currentPage}}/{{totalPages}} +
    + + +
    +
    + {{/if}} + + {{#each paginatedFacets key="value" as |facet index|}} +
  • +
    + +
    +
    {{facet.displayText}}
    + only + {{facet.count}} +
  • + {{else}} + No fields! + {{/each}} +
diff --git a/tez-ui/src/main/webapp/app/templates/components/em-table-facet-panel.hbs b/tez-ui/src/main/webapp/app/templates/components/em-table-facet-panel.hbs new file mode 100644 index 0000000000..d513786207 --- /dev/null +++ b/tez-ui/src/main/webapp/app/templates/components/em-table-facet-panel.hbs @@ -0,0 +1,33 @@ +{{! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +}} + +{{#if dataProcessor.facetedFields.length}} +
    + {{input type="text" class="field-filter-box" value=filterText placeholder="Filter"}} + {{#each dataProcessor.facetedFields key="column.id" as |field fieldIndex|}} +
  • {{component field.column.facetType.componentName data=field tableDefinition=tableDefinition dataProcessor=dataProcessor tmpFacetConditions=tmpFacetConditions}}
  • + {{/each}} +
+ +
+ + +
+{{else}} +

Not Available!

+{{/if}} diff --git a/tez-ui/src/main/webapp/app/templates/components/em-table-header-cell.hbs b/tez-ui/src/main/webapp/app/templates/components/em-table-header-cell.hbs new file mode 100644 index 0000000000..c48eb7a12a --- /dev/null +++ b/tez-ui/src/main/webapp/app/templates/components/em-table-header-cell.hbs @@ -0,0 +1,30 @@ +{{! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +}} + +
+ {{title}} + {{#if tableDefinition.enableSort}}{{#if definition.enableSort}} + + {{#if tableDefinition.headerAsSortButton}} + + {{/if}} + {{/if}}{{/if}} + {{#if tableDefinition.enableColumnResize}}{{#if definition.enableColumnResize}} + + {{/if}}{{/if}} +
diff --git a/tez-ui/src/main/webapp/app/templates/components/em-table-linked-cell.hbs b/tez-ui/src/main/webapp/app/templates/components/em-table-linked-cell.hbs new file mode 100644 index 0000000000..8a82631983 --- /dev/null +++ b/tez-ui/src/main/webapp/app/templates/components/em-table-linked-cell.hbs @@ -0,0 +1,41 @@ +{{! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +}} + +{{#if normalizedLinks.length}} + {{#each normalizedLinks as |link|}} + {{#if link.routeName}} + {{#if link.withModel}} + {{#link-to link.routeName link.model target=definition.target}} + {{link.text}} + {{/link-to}} + {{else}} + {{#link-to link.routeName target=definition.target}} + {{link.text}} + {{/link-to}} + {{/if}} + {{else if link.href}} + + {{link.text}} + + {{else}} + {{link.text}} + {{/if}} + {{/each}} +{{else}} + Not Available! +{{/if}} diff --git a/tez-ui/src/main/webapp/app/templates/components/em-table-pagination-ui.hbs b/tez-ui/src/main/webapp/app/templates/components/em-table-pagination-ui.hbs new file mode 100644 index 0000000000..78edcaece4 --- /dev/null +++ b/tez-ui/src/main/webapp/app/templates/components/em-table-pagination-ui.hbs @@ -0,0 +1,45 @@ +{{! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +}} + +
    + {{#if showFirst}} +
  • + First +
  • + {{/if}} + {{#each _possiblePages as |page|}} +
  • + {{page.pageNum}} +
  • + {{/each}} + {{#if showLast}} +
  • + Last - {{dataProcessor.totalPages}} +
  • + {{/if}} +
+ +
+ +
diff --git a/tez-ui/src/main/webapp/app/templates/components/em-table-progress-cell.hbs b/tez-ui/src/main/webapp/app/templates/components/em-table-progress-cell.hbs new file mode 100644 index 0000000000..9df8be0360 --- /dev/null +++ b/tez-ui/src/main/webapp/app/templates/components/em-table-progress-cell.hbs @@ -0,0 +1,29 @@ +{{! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +}} + +{{#if message}} + {{message}} +{{else}} + {{em-progress + value=content + valueMin=_definition.valueMin + valueMax=_definition.valueMax + striped=_definition.striped + style=_definition.style + }} +{{/if}} diff --git a/tez-ui/src/main/webapp/app/templates/components/em-table-search-ui.hbs b/tez-ui/src/main/webapp/app/templates/components/em-table-search-ui.hbs new file mode 100644 index 0000000000..13cb1ddadc --- /dev/null +++ b/tez-ui/src/main/webapp/app/templates/components/em-table-search-ui.hbs @@ -0,0 +1,52 @@ +{{! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +}} + +
+ {{#if (eq tableDefinition.searchType "manual")}} + + + + {{/if}} + + {{input + type="text" + class="form-control" + placeholder="Search..." + enter="search" + value=text + }} + + + + +
diff --git a/tez-ui/src/main/webapp/app/templates/components/em-table.hbs b/tez-ui/src/main/webapp/app/templates/components/em-table.hbs new file mode 100644 index 0000000000..f7be9d41b3 --- /dev/null +++ b/tez-ui/src/main/webapp/app/templates/components/em-table.hbs @@ -0,0 +1,106 @@ +{{! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +}} + +{{!--To add CSS rules at runtime!--}} + + +{{!--Header--}} +
+ {{#each headerComponentNames as |componentName|}} + {{component componentName tableDefinition=_definition dataProcessor=_dataProcessor}} + {{/each}} +
+ +
+
+ {{#if leftPanelComponentName}} + {{component leftPanelComponentName tableDefinition=_definition dataProcessor=_dataProcessor}} + {{/if}} +
+ + {{#if message}} +

{{message}}

+ {{else}} + {{!--Body--}} + {{#if _columns.left.length}} +
+ {{#each _columns.left as |column colIndex|}} + {{em-table-column + rows=_dataProcessor.processedRows + definition=column.definition + defaultWidth=column.width + tableDefinition=_definition + dataProcessor=_dataProcessor + index=colIndex + }} + {{/each}} +
+ {{/if}} + + + + +
+
+ {{#each _columns.center as |column colIndex|}} + {{em-table-column + rows=_dataProcessor.processedRows + definition=column.definition + defaultWidth=column.width + tableDefinition=_definition + dataProcessor=_dataProcessor + index=colIndex + }} + {{/each}} +
+
+ + + + + {{#if _columns.right.length}} +
+ {{#each _columns.right as |column colIndex|}} + {{em-table-column + rows=_dataProcessor.processedRows + definition=column.definition + defaultWidth=column.width + tableDefinition=_definition + dataProcessor=_dataProcessor + index=colIndex + }} + {{/each}} +
+ {{/if}} + {{/if}} + +
+ {{#if rightPanelComponentName}} + {{component rightPanelComponentName tableDefinition=_definition dataProcessor=_dataProcessor}} + {{/if}} +
+
+ +{{!--Footer--}} +{{#if displayFooter}} + +{{/if}} diff --git a/tez-ui/src/main/webapp/app/utils/column-definition.js b/tez-ui/src/main/webapp/app/utils/column-definition.js new file mode 100644 index 0000000000..1316866ab1 --- /dev/null +++ b/tez-ui/src/main/webapp/app/utils/column-definition.js @@ -0,0 +1,125 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; + +import facetTypes from './facet-types'; + +function getContentAtPath(row) { + var contentPath = this.get('contentPath'); + + if(contentPath) { + return Ember.get(row, contentPath); + } + else { + throw new Error("contentPath not set!"); + } +} + +function returnEmptyString() { + return ""; +} + +var ColumnDefinition = Ember.Object.extend({ + id: "", + headerTitle: "Not Available!", + + classNames: [], + + cellComponentName: null, + + enableSearch: true, + enableSort: true, + enableColumnResize: true, + + width: null, + minWidth: "150px", + + contentPath: null, + observePath: false, + + cellDefinition: null, + + pin: "center", + + facetType: facetTypes.VALUES, + + beforeSort: null, + getCellContent: getContentAtPath, + getSearchValue: getContentAtPath, + getSortValue: getContentAtPath, + + init: function () { + if(!this.get("id")) { + throw new Error("ID is not set."); + } + }, +}); + +ColumnDefinition.make = function (rawDefinition) { + if(Array.isArray(rawDefinition)) { + return rawDefinition.map(function (def) { + return ColumnDefinition.create(def); + }); + } + else if(typeof rawDefinition === 'object') { + return ColumnDefinition.create(rawDefinition); + } + else { + throw new Error("rawDefinition must be an Array or an Object."); + } +}; + +ColumnDefinition.makeFromModel = function (ModelClass, columnOptions) { + var attributes = Ember.get(ModelClass, 'attributes'), + columns = []; + if(attributes) { + attributes.forEach(function (meta, name) { + var column = Ember.Object.create({ + id: name, + headerTitle: name.capitalize(), + contentPath: name, + }); + + if(columnOptions) { + column.setProperties(columnOptions); + } + + columns.push(column); + }); + + return ColumnDefinition.make(columns); + } + else { + throw new Error("Value passed is not a model class"); + } +}; + +ColumnDefinition.fillerColumn = ColumnDefinition.create({ + id: "fillerColumn", + headerTitle: "", + getCellContent: returnEmptyString, + getSearchValue: returnEmptyString, + getSortValue: returnEmptyString, + + enableSearch: false, + enableSort: false, + enableColumnResize: false, +}); + +export default ColumnDefinition; diff --git a/tez-ui/src/main/webapp/app/utils/counter-column-definition.js b/tez-ui/src/main/webapp/app/utils/counter-column-definition.js index d66e551eed..5590e10244 100644 --- a/tez-ui/src/main/webapp/app/utils/counter-column-definition.js +++ b/tez-ui/src/main/webapp/app/utils/counter-column-definition.js @@ -19,7 +19,7 @@ import Ember from 'ember'; import isIOCounter from '../utils/misc'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from './column-definition'; /* * Returns a counter value from for a row diff --git a/tez-ui/src/main/webapp/app/utils/data-processor.js b/tez-ui/src/main/webapp/app/utils/data-processor.js new file mode 100644 index 0000000000..07d31c09bf --- /dev/null +++ b/tez-ui/src/main/webapp/app/utils/data-processor.js @@ -0,0 +1,275 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; + +import SQL from './sql'; + +/** + * Handles Sorting, Searching & Pagination + */ +export default Ember.Object.extend({ + isSorting: false, + isSearching: false, + + tableDefinition: null, + + sql: SQL.create(), + + rows: [], + _sortedRows: [], + _searchedRows: [], + _facetFilteredRows: [], + + _searchObserver: Ember.on("init", Ember.observer('tableDefinition.searchText', 'tableDefinition._actualSearchType', '_sortedRows.[]', function () { + Ember.run.once(this, "startSearch"); + })), + + _sortObserver: Ember.on("init", Ember.observer( + 'tableDefinition.sortColumnId', + 'tableDefinition.sortOrder', + 'rows.[]', function () { + Ember.run.once(this, "startSort"); + })), + + _facetedFilterObserver: Ember.on("init", Ember.observer('tableDefinition.facetConditions', '_searchedRows.[]', function () { + Ember.run.once(this, "startFacetedFilter"); + })), + + regexSearch: function (clause, rows, columns) { + var regex; + + try { + regex = new RegExp(clause, "i"); + } + catch(e) { + regex = new RegExp("", "i"); + } + + function checkRow(column) { + var value; + if(!column.get('enableSearch')) { + return false; + } + value = column.getSearchValue(this); + + if(typeof value === 'string') { + value = value.toLowerCase(); + return value.match(regex); + } + + return false; + } + + return rows.filter(function (row) { + return columns.some(checkRow, row); + }); + }, + + startSearch: function () { + var searchText = String(this.get('tableDefinition.searchText')), + rows = this.get('_sortedRows') || [], + columns = this.get('tableDefinition.columns'), + actualSearchType = this.get('tableDefinition._actualSearchType'), + that = this; + + if(searchText) { + this.set("isSearching", true); + + Ember.run.later(function () { + var result; + + switch(actualSearchType) { + case "SQL": + result = that.get("sql").search(searchText, rows, columns); + break; + + //case "Regex": Commenting as default will be called anyways + default: + result = that.regexSearch(searchText, rows, columns); + break; + } + + that.setProperties({ + _searchedRows: result, + isSearching: false + }); + }); + } + else { + this.set("_searchedRows", rows); + } + }, + + compareFunction: function (a, b){ + // Checking for undefined and null to handle some special cases in JavaScript comparison + // Eg: 1 > undefined = false & 1 < undefined = false + // "a1" > null = false & "a1" < null = false + if(a === undefined || a === null) { + return -1; + } + else if(b === undefined || b === null) { + return 1; + } + else if(a < b) { + return -1; + } + else if(a > b) { + return 1; + } + else { + return 0; + } + }, + + startSort: function () { + var rows = this.get('rows'), + tableDefinition = this.get('tableDefinition'), + sortColumnId = this.get('tableDefinition.sortColumnId'), + descending = this.get('tableDefinition.sortOrder') === 'desc', + that = this, + column; + + if(tableDefinition) { + column = tableDefinition.get('columns').find(function (element) { + return element.get('id') === sortColumnId; + }); + } + + if(rows && Array.isArray(rows.content)) { + rows = rows.toArray(); + } + + if(rows && rows.get('length') > 0 && column) { + this.set('isSorting', true); + + Ember.run.later(function () { + /* + * Creating sortArray as calling getSortValue form inside the + * sort function every time would be more costly. + */ + var sortArray = rows.map(function (row) { + return { + value: column.getSortValue(row), + row: row + }; + }), + compareFunction = that.get("compareFunction"); + + sortArray.sort(function (a, b) { + var result = compareFunction(a.value, b.value); + if(descending && result) { + result = -result; + } + return result; + }); + + that.setProperties({ + _sortedRows: sortArray.map(function (record) { + return record.row; + }), + isSorting: false + }); + }); + } + else { + this.set('_sortedRows', rows); + } + }, + + startFacetedFilter: function () { + var clause = this.get("sql").createFacetClause(this.get('tableDefinition.facetConditions'), this.get("tableDefinition.columns")), + rows = this.get('_searchedRows') || [], + columns = this.get('tableDefinition.columns'), + that = this; + + if(clause && columns) { + this.set("isSearching", true); + + Ember.run.later(function () { + var result = that.get("sql").search(clause, rows, columns); + + that.setProperties({ + _facetFilteredRows: result, + isSearching: false + }); + }); + } + else { + this.set("_facetFilteredRows", rows); + } + }, + + facetedFields: Ember.computed('_searchedRows.[]', 'tableDefinition.columns', function () { + var searchedRows = this.get("_searchedRows"), + columns = this.get('tableDefinition.columns'), + fields = []; + + if(columns) { + columns.forEach(function (column) { + var facetedData; + if(column.facetType) { + facetedData = column.facetType.facetRows(column, searchedRows); + if(facetedData) { + fields.push({ + column: column, + facets: facetedData + }); + } + } + }); + } + + return fields; + }), + + pageDetails: Ember.computed("tableDefinition.rowCount", "tableDefinition.pageNum", "_facetFilteredRows.length", function () { + var tableDefinition = this.get("tableDefinition"), + + pageNum = tableDefinition.get('pageNum'), + rowCount = tableDefinition.get('rowCount'), + + startIndex = (pageNum - 1) * rowCount, + + totalRecords = this.get('_facetFilteredRows.length'); + + if(startIndex < 0) { + startIndex = 0; + } + + return { + pageNum: pageNum, + totalPages: Math.ceil(totalRecords / rowCount), + rowCount: rowCount, + + startIndex: startIndex, + + fromRecord: totalRecords ? startIndex + 1 : 0, + toRecord: Math.min(startIndex + rowCount, totalRecords), + totalRecords: totalRecords + }; + }), + totalPages: Ember.computed.alias("pageDetails.totalPages"), // Adding an alias for backward compatibility + + // Paginate + processedRows: Ember.computed('_facetFilteredRows.[]', 'tableDefinition.rowCount', 'tableDefinition.pageNum', function () { + var rowCount = this.get('tableDefinition.rowCount'), + startIndex = (this.get('tableDefinition.pageNum') - 1) * rowCount; + return this.get('_facetFilteredRows').slice(startIndex, startIndex + rowCount); + }), +}); diff --git a/tez-ui/src/main/webapp/app/utils/facet-types.js b/tez-ui/src/main/webapp/app/utils/facet-types.js new file mode 100644 index 0000000000..0a340bbf7d --- /dev/null +++ b/tez-ui/src/main/webapp/app/utils/facet-types.js @@ -0,0 +1,85 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; + +var facetTypes = { + VALUES: { + componentName: "em-table-facet-panel-values", + + toClause: function (column, facetConditions) { + var values, clauses = []; + + if(facetConditions) { + if(Ember.get(facetConditions, "in.length")) { + values = facetConditions.in.map(function (value) { + value = value.replace(/'/g, "''"); + return `'${value}'`; + }); + clauses.push(`${column.id} IN (${values})`); + } + + if(Ember.get(facetConditions, "notIn.length")) { + values = facetConditions.notIn.map(function (value) { + value = value.replace(/'/g, "''"); + return `'${value}'`; + }); + clauses.push(`${column.id} NOT IN (${values})`); + } + + return clauses.join(" AND "); + } + }, + + facetRows: function (column, rows) { + var facetedDataHash = {}, + facetedDataArr = []; + + rows.forEach(function (row) { + var value = column.getSearchValue(row); + + if(typeof value === "string") { + if(!facetedDataHash[value]) { + facetedDataHash[value] = { + count: 0, + value: value + }; + facetedDataArr.push(facetedDataHash[value]); + } + facetedDataHash[value].count++; + } + + }); + + if(facetedDataArr.length) { + facetedDataArr = facetedDataArr.sort(function (a, b) { + return -(a.count - b.count); // Sort in reverse order + }); + return facetedDataArr; + } + }, + + normaliseConditions: function (conditions, data) { + if(Ember.get(conditions, "in.length") < data.length) { + return conditions; + } + } + }, +}; + +export default facetTypes; diff --git a/tez-ui/src/main/webapp/app/utils/sql.js b/tez-ui/src/main/webapp/app/utils/sql.js new file mode 100644 index 0000000000..81db3a07f5 --- /dev/null +++ b/tez-ui/src/main/webapp/app/utils/sql.js @@ -0,0 +1,94 @@ +/*global alasql*/ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +import Ember from 'ember'; + +/* + * A wrapper around AlaSQL + */ +export default Ember.Object.extend({ + + constructQuery: function(clause) { + return `SELECT * FROM ? WHERE ${clause}`; + }, + + validateClause: function (clause, columns) { + clause = clause.toString(); + + var query = this.constructQuery(this.normaliseClause(clause, columns || [])), + valid = false; + + if(clause.match(/\W/g)) { // If it contain special characters including space + try { + alasql(query, [[{}]]); + valid = true; + } + catch(e) {} + } + + return valid; + }, + + createFacetClause: function (conditions, columns) { + if(conditions && columns) { + return columns.map(function (column) { + if(column.get("facetType")) { + return column.get("facetType.toClause")(column, conditions[Ember.get(column, "id")]); + } + }).filter(clause => clause).join(" AND "); + } + }, + + normaliseClause: function (clause, columns) { + clause = clause.toString(); + columns.forEach(function (column) { + var headerTitle = column.get("headerTitle"); + clause = clause.replace(new RegExp(`"${headerTitle}"`, "gi"), column.get("id")); + }); + return clause; + }, + + search: function (clause, rows, columns) { + clause = this.normaliseClause(clause, columns); + + // Convert into a form that alasql can digest easily + var dataSet = rows.map(function (row, index) { + var rowObj = { + _index_: index + }; + + columns.forEach(function (column) { + if(column.get("enableSearch") && row) { + rowObj[column.get("id")] = column.getSearchValue(row); + } + }); + + return rowObj; + }); + + // Search + dataSet = alasql(this.constructQuery(clause), [dataSet]); + + return dataSet.map(function (data) { + return rows[data._index_]; + }); + } + +}); diff --git a/tez-ui/src/main/webapp/app/utils/table-definition.js b/tez-ui/src/main/webapp/app/utils/table-definition.js new file mode 100644 index 0000000000..c304ec4e80 --- /dev/null +++ b/tez-ui/src/main/webapp/app/utils/table-definition.js @@ -0,0 +1,61 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; + +export default Ember.Object.extend({ + + recordType: "", + + // Search + enableSearch: true, + searchText: '', + searchType: 'auto', // Can be either of auto, manual, regex OR sql + _actualSearchType: "Regex", // Set from em-table-search-ui + + // Faceting + enableFaceting: false, + facetConditions: null, + minFieldsForFilter: 15, + minValuesToDisplay: 2, + facetValuesPageSize: 10, + + // Sort + enableSort: true, + sortColumnId: '', + sortOrder: '', + headerAsSortButton: false, + + // Pagination + enablePagination: true, + pageNum: 1, + rowCount: 10, + rowCountOptions: [5, 10, 25, 50, 100], + + enableColumnResize: true, + showScrollShadow: false, + + minRowsForFooter: 25, + + columns: [], + + _pageNumResetObserver: Ember.observer('searchText', 'facetConditions', 'rowCount', function () { + this.set('pageNum', 1); + }), + +}); diff --git a/tez-ui/src/main/webapp/bower.json b/tez-ui/src/main/webapp/bower.json index 56a69f323e..cca56d817a 100644 --- a/tez-ui/src/main/webapp/bower.json +++ b/tez-ui/src/main/webapp/bower.json @@ -1,6 +1,7 @@ { "name": "tez-ui", "dependencies": { + "alasql": "^0.4.0", "ember": "2.2.0", "ember-cli-shims": "0.0.6", "ember-cli-test-loader": "0.2.1", diff --git a/tez-ui/src/main/webapp/ember-cli-build.js b/tez-ui/src/main/webapp/ember-cli-build.js index 7bbc77d334..e4217e9591 100644 --- a/tez-ui/src/main/webapp/ember-cli-build.js +++ b/tez-ui/src/main/webapp/ember-cli-build.js @@ -71,6 +71,7 @@ module.exports = function(defaults) { app.import('bower_components/codemirror/mode/sql/sql.js'); app.import('bower_components/codemirror/mode/pig/pig.js'); app.import('bower_components/codemirror/lib/codemirror.css'); + app.import('bower_components/alasql/dist/alasql.js'); return app.toTree(new MergeTrees([configEnv, zipWorker, copyFonts])); }; diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json index fa80389b94..ad3aa74c5d 100644 --- a/tez-ui/src/main/webapp/package.json +++ b/tez-ui/src/main/webapp/package.json @@ -61,7 +61,6 @@ "phantomjs-prebuilt": "2.1.13" }, "dependencies": { - "em-table": "0.11.3", "em-tgraph": "0.0.14" } } diff --git a/tez-ui/src/main/webapp/tests/integration/components/em-table-cell-test.js b/tez-ui/src/main/webapp/tests/integration/components/em-table-cell-test.js new file mode 100644 index 0000000000..ccf535884e --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/components/em-table-cell-test.js @@ -0,0 +1,45 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +import ColumnDefinition from '../../../utils/column-definition'; + +moduleForComponent('em-table-cell', 'Integration | Component | em table cell', { + integration: true +}); + +test('Basic rendering test', function(assert) { + var columnDefinition = ColumnDefinition.create({ + id: 'id', + contentPath: 'keyA' + }), + row = Ember.Object.create({ + keyA: 'valueA', + keyB: 'valueB' + }); + + this.set('columnDefinition', columnDefinition); + this.set('row', row); + this.render(hbs`{{em-table-cell columnDefinition=columnDefinition row=row}}`); + + assert.equal(this.$().text().trim(), 'valueA'); +}); diff --git a/tez-ui/src/main/webapp/tests/integration/components/em-table-column-test.js b/tez-ui/src/main/webapp/tests/integration/components/em-table-column-test.js new file mode 100644 index 0000000000..96eff7af2e --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/components/em-table-column-test.js @@ -0,0 +1,30 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +moduleForComponent('em-table-column', 'Integration | Component | em table column', { + integration: true +}); + +test('Basic rendering test', function(assert) { + this.render(hbs`{{em-table-column}}`); + + assert.equal(this.$().text().trim(), ''); +}); diff --git a/tez-ui/src/main/webapp/tests/integration/components/em-table-facet-panel-test.js b/tez-ui/src/main/webapp/tests/integration/components/em-table-facet-panel-test.js new file mode 100644 index 0000000000..cc0f1f0741 --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/components/em-table-facet-panel-test.js @@ -0,0 +1,43 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +moduleForComponent('em-table-facet-panel', 'Integration | Component | em table facet panel', { + integration: true +}); + +test('Basic renders', function(assert) { + + // Set any properties with this.set('myProperty', 'value'); + // Handle any actions with this.on('myAction', function(val) { ... });" + EOL + EOL + + + this.render(hbs`{{em-table-facet-panel}}`); + + assert.equal(this.$().text().replace(/\n|\r\n|\r| /g, '').trim(), 'NotAvailable!'); + + // Template block usage:" + EOL + + this.render(hbs` + {{#em-table-facet-panel}} + template block text + {{/em-table-facet-panel}} + `); + + assert.equal(this.$().text().replace(/\n|\r\n|\r| /g, '').trim(), 'NotAvailable!'); +}); diff --git a/tez-ui/src/main/webapp/tests/integration/components/em-table-facet-panel-values-test.js b/tez-ui/src/main/webapp/tests/integration/components/em-table-facet-panel-values-test.js new file mode 100644 index 0000000000..f401a7da6e --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/components/em-table-facet-panel-values-test.js @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +moduleForComponent('em-table-facet-panel-values', 'Integration | Component | em table facet panel values', { + integration: true +}); + +test('Basic render test', function(assert) { + + // Set any properties with this.set('myProperty', 'value'); + // Handle any actions with this.on('myAction', function(val) { ... });" + EOL + EOL + + + this.set("tmpFacetConditions", {}); + this.render(hbs`{{em-table-facet-panel-values tmpFacetConditions=tmpFacetConditions}}`); + + assert.ok(this.$().text().trim()); + + // Template block usage:" + EOL + + this.render(hbs` + {{#em-table-facet-panel-values tmpFacetConditions=tmpFacetConditions}} + template block text + {{/em-table-facet-panel-values}} + `); + + assert.ok(this.$().text().trim()); +}); diff --git a/tez-ui/src/main/webapp/tests/integration/components/em-table-header-cell-test.js b/tez-ui/src/main/webapp/tests/integration/components/em-table-header-cell-test.js new file mode 100644 index 0000000000..0c502ce9df --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/components/em-table-header-cell-test.js @@ -0,0 +1,30 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +moduleForComponent('em-table-header-cell', 'Integration | Component | em table header cell', { + integration: true +}); + +test('Basic rendering test', function(assert) { + this.render(hbs`{{em-table-header-cell}}`); + + assert.equal(this.$().text().trim(), ''); +}); diff --git a/tez-ui/src/main/webapp/tests/integration/components/em-table-linked-cell-test.js b/tez-ui/src/main/webapp/tests/integration/components/em-table-linked-cell-test.js new file mode 100644 index 0000000000..7553c41014 --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/components/em-table-linked-cell-test.js @@ -0,0 +1,30 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +moduleForComponent('em-table-linked-cell', 'Integration | Component | em table linked cell', { + integration: true +}); + +test('Basic rendering test', function(assert) { + this.render(hbs`{{em-table-linked-cell}}`); + + assert.equal(this.$().text().trim(), 'Not Available!'); +}); diff --git a/tez-ui/src/main/webapp/tests/integration/components/em-table-pagination-ui-test.js b/tez-ui/src/main/webapp/tests/integration/components/em-table-pagination-ui-test.js new file mode 100644 index 0000000000..0333d0cf11 --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/components/em-table-pagination-ui-test.js @@ -0,0 +1,204 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; + +import DataProcessor from '../../../utils/data-processor'; +import TableDefinition from '../../../utils/table-definition'; + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +moduleForComponent('em-table-pagination-ui', 'Integration | Component | em table pagination ui', { + integration: true +}); + +test('Basic rendering test', function(assert) { + var customRowCount = 25, + definition = TableDefinition.create({ + rowCount: customRowCount + }), + processor; + + Ember.run(function () { + processor = DataProcessor.create({ + tableDefinition: definition, + rows: Ember.A([Ember.Object.create()]) + }); + }); + + this.set('definition', definition); + this.set('processor', processor); + this.render(hbs`{{em-table-pagination-ui tableDefinition=definition dataProcessor=processor}}`); + + var paginationItems = this.$('li'); + assert.equal(paginationItems.length, 1); + assert.equal($(paginationItems[0]).text().trim(), "1"); + + var rowSelection = this.$('select')[0]; + assert.ok(rowSelection); + assert.equal($(rowSelection).val(), customRowCount); +}); + +test('No data test', function(assert) { + var customRowCount = 2, + definition = TableDefinition.create({ + rowCount: customRowCount + }), + processor; + + Ember.run(function () { + processor = DataProcessor.create({ + tableDefinition: definition, + rows: Ember.A() + }); + }); + + this.set('definition', definition); + this.set('processor', processor); + this.render(hbs`{{em-table-pagination-ui tableDefinition=definition dataProcessor=processor}}`); + + var paginationItems = this.$('li'); + assert.equal(paginationItems.length, 0); +}); + +test('Multiple page test; without first & last', function(assert) { + var customRowCount = 2, + definition = TableDefinition.create({ + rowCount: customRowCount + }), + processor; + + Ember.run(function () { + processor = DataProcessor.create({ + tableDefinition: definition, + rows: Ember.A([Ember.Object.create(), Ember.Object.create(), Ember.Object.create()]) + }); + }); + + this.set('definition', definition); + this.set('processor', processor); + this.render(hbs`{{em-table-pagination-ui tableDefinition=definition dataProcessor=processor}}`); + + var paginationItems = this.$('li'); + assert.equal(paginationItems.length, 2); + assert.equal($(paginationItems[0]).text().trim(), "1"); + assert.equal($(paginationItems[1]).text().trim(), "2"); +}); + +test('Display last test', function(assert) { + var customRowCount = 5, + definition = TableDefinition.create({ + rowCount: customRowCount + }), + processor, + rows = []; + + for(var i = 0; i < 100; i++) { + rows.push(Ember.Object.create()); + } + + Ember.run(function () { + processor = DataProcessor.create({ + tableDefinition: definition, + rows: Ember.A(rows) + }); + }); + + this.set('definition', definition); + this.set('processor', processor); + this.render(hbs`{{em-table-pagination-ui tableDefinition=definition dataProcessor=processor}}`); + + var paginationItems = this.$('li'); + assert.equal(paginationItems.length, 6); + assert.equal($(paginationItems[0]).text().trim(), "1"); + assert.equal($(paginationItems[1]).text().trim(), "2"); + assert.equal($(paginationItems[2]).text().trim(), "3"); + assert.equal($(paginationItems[3]).text().trim(), "4"); + assert.equal($(paginationItems[4]).text().trim(), "5"); + assert.equal($(paginationItems[5]).text().trim(), "Last - 20"); +}); + +test('Display first test', function(assert) { + var customRowCount = 5, + definition = TableDefinition.create({ + pageNum: 20, + rowCount: customRowCount + }), + processor, + rows = []; + + for(var i = 0; i < 100; i++) { + rows.push(Ember.Object.create()); + } + + Ember.run(function () { + processor = DataProcessor.create({ + tableDefinition: definition, + rows: Ember.A(rows) + }); + }); + + this.set('definition', definition); + this.set('processor', processor); + this.render(hbs`{{em-table-pagination-ui tableDefinition=definition dataProcessor=processor}}`); + + var paginationItems = this.$('li'); + assert.equal(paginationItems.length, 6); + assert.equal($(paginationItems[0]).text().trim(), "First"); + assert.equal($(paginationItems[1]).text().trim(), "16"); + assert.equal($(paginationItems[2]).text().trim(), "17"); + assert.equal($(paginationItems[3]).text().trim(), "18"); + assert.equal($(paginationItems[4]).text().trim(), "19"); + assert.equal($(paginationItems[5]).text().trim(), "20"); +}); + +test('Display first & last test', function(assert) { + var customRowCount = 5, + definition = TableDefinition.create({ + pageNum: 10, + rowCount: customRowCount + }), + processor, + rows = []; + + for(var i = 0; i < 100; i++) { + rows.push(Ember.Object.create()); + } + + Ember.run(function () { + processor = DataProcessor.create({ + tableDefinition: definition, + rows: Ember.A(rows) + }); + }); + + this.set('definition', definition); + this.set('processor', processor); + this.render(hbs`{{em-table-pagination-ui tableDefinition=definition dataProcessor=processor}}`); + + var paginationItems = this.$('li'); + assert.equal(paginationItems.length, 7); + assert.equal($(paginationItems[0]).text().trim(), "First"); + assert.equal($(paginationItems[1]).text().trim(), "8"); + assert.equal($(paginationItems[2]).text().trim(), "9"); + assert.equal($(paginationItems[3]).text().trim(), "10"); + assert.equal($(paginationItems[4]).text().trim(), "11"); + assert.equal($(paginationItems[5]).text().trim(), "12"); + assert.equal($(paginationItems[6]).text().trim(), "Last - 20"); +}); diff --git a/tez-ui/src/main/webapp/tests/integration/components/em-table-progress-cell-test.js b/tez-ui/src/main/webapp/tests/integration/components/em-table-progress-cell-test.js new file mode 100644 index 0000000000..b7eced31d6 --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/components/em-table-progress-cell-test.js @@ -0,0 +1,43 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +moduleForComponent('em-table-progress-cell', 'Integration | Component | em table progress cell', { + integration: true +}); + +test('Basic creation test', function(assert) { + + // Set any properties with this.set('myProperty', 'value'); + // Handle any actions with this.on('myAction', function(val) { ... });" + EOL + EOL + + + this.render(hbs`{{em-table-progress-cell content=0.5}}`); + + assert.equal(this.$().text().trim(), '50%'); + + // Template block usage:" + EOL + + this.render(hbs` + {{#em-table-progress-cell content=0.5}} + template block text + {{/em-table-progress-cell}} + `); + + assert.equal(this.$().text().trim(), '50%'); +}); diff --git a/tez-ui/src/main/webapp/tests/integration/components/em-table-search-ui-test.js b/tez-ui/src/main/webapp/tests/integration/components/em-table-search-ui-test.js new file mode 100644 index 0000000000..0cd2bbca15 --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/components/em-table-search-ui-test.js @@ -0,0 +1,30 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +moduleForComponent('em-table-search-ui', 'Integration | Component | em table search ui', { + integration: true +}); + +test('Basic rendering test', function(assert) { + this.render(hbs`{{em-table-search-ui}}`); + + assert.equal(this.$().text().trim(), 'Search'); +}); diff --git a/tez-ui/src/main/webapp/tests/integration/components/em-table-test.js b/tez-ui/src/main/webapp/tests/integration/components/em-table-test.js new file mode 100644 index 0000000000..96baf79fa9 --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/components/em-table-test.js @@ -0,0 +1,48 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +import TableDefinition from '../../../utils/table-definition'; +import ColumnDefinition from '../../../utils/column-definition'; + +moduleForComponent('em-table', 'Integration | Component | em table', { + integration: true +}); + +test('Basic rendering test', function(assert) { + this.render(hbs`{{em-table}}`); + + assert.equal(this.$('.table-message').text().trim(), 'No columns available!'); +}); + +test('Records missing test', function(assert) { + var definition = TableDefinition.create({ + recordType: "vertex" + }); + + this.set("columns", [ColumnDefinition.fillerColumn]); + + this.render(hbs`{{em-table columns=columns}}`); + assert.equal(this.$('.table-message').text().trim(), 'No records available!'); + + this.set("definition", definition); + this.render(hbs`{{em-table columns=columns definition=definition}}`); + assert.equal(this.$('.table-message').text().trim(), 'No vertices available!'); +}); diff --git a/tez-ui/src/main/webapp/tests/integration/em-table-status-cell-test.js b/tez-ui/src/main/webapp/tests/integration/em-table-status-cell-test.js new file mode 100644 index 0000000000..31483395e9 --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/em-table-status-cell-test.js @@ -0,0 +1,40 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +moduleForComponent('em-table-status-cell', 'Integration | Component | em table status cell', { + integration: true +}); + +test('Basic creation test', function(assert) { + + this.render(hbs`{{em-table-status-cell}}`); + + assert.equal(this.$().text().trim(), 'Not Available!'); + + // Template block usage:" + EOL + + this.render(hbs` + {{#em-table-status-cell}} + template block text + {{/em-table-status-cell}} + `); + + assert.equal(this.$().text().trim(), 'Not Available!'); +}); diff --git a/tez-ui/src/main/webapp/tests/unit/utils/column-definition-test.js b/tez-ui/src/main/webapp/tests/unit/utils/column-definition-test.js new file mode 100644 index 0000000000..5ee9a49023 --- /dev/null +++ b/tez-ui/src/main/webapp/tests/unit/utils/column-definition-test.js @@ -0,0 +1,104 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import ColumnDefinition from '../../../utils/column-definition'; +import { module, test } from 'qunit'; + +module('Unit | Utility | column definition'); + +test('Class creation test', function(assert) { + assert.ok(ColumnDefinition); + + assert.ok(ColumnDefinition.make); + assert.ok(ColumnDefinition.makeFromModel); +}); + +test('make - Instance creation test', function(assert) { + + var definition = ColumnDefinition.make({ + id: "testId" + }); + var definitions = ColumnDefinition.make([{ + id: "testId 1" + },{ + id: "testId 2" + }]); + + // Single + assert.ok(definition); + + // Multiple + assert.ok(definitions); + assert.ok(Array.isArray(definitions)); + assert.equal(definitions.length, 2); +}); + +test('make - Instance creation failure test', function(assert) { + assert.throws(function () { + ColumnDefinition.make({}); + }); +}); + +test('makeFromModel test', function(assert) { + var attributes = Ember.Map.create(), + DummyModel = Ember.Object.create({ + attributes: attributes + }), + getCellContent = function () {}, + columns; + + attributes.set("attr1", "path1"); + attributes.set("attr2", "path2"); + attributes.set("attr3", "path3"); + + columns = ColumnDefinition.makeFromModel(DummyModel, { + getCellContent: getCellContent + }); + + assert.equal(columns.length, 3); + assert.equal(columns[0].id, "attr1"); + assert.equal(columns[0].headerTitle, "Attr1"); + assert.equal(columns[0].contentPath, "attr1"); + assert.equal(columns[0].getCellContent, getCellContent); +}); + +test('Instance test', function(assert) { + var definition = ColumnDefinition.make({ + id: "testId", + contentPath: "a.b" + }); + var data = Ember.Object.create({ + a: { + b: 42 + } + }); + + assert.ok(definition.getCellContent); + assert.ok(definition.getSearchValue); + assert.ok(definition.getSortValue); + + assert.equal(definition.id, "testId"); + assert.equal(definition.headerTitle, "Not Available!"); + assert.equal(definition.minWidth, "150px"); + assert.equal(definition.contentPath, "a.b"); + + assert.equal(definition.getCellContent(data), 42); + assert.equal(definition.getSearchValue(data), 42); + assert.equal(definition.getSortValue(data), 42); +}); diff --git a/tez-ui/src/main/webapp/tests/unit/utils/data-processor-test.js b/tez-ui/src/main/webapp/tests/unit/utils/data-processor-test.js new file mode 100644 index 0000000000..58f52dd013 --- /dev/null +++ b/tez-ui/src/main/webapp/tests/unit/utils/data-processor-test.js @@ -0,0 +1,137 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; + +import DataProcessor from '../../../utils/data-processor'; +import ColumnDefinition from '../../../utils/column-definition'; +import { module, test } from 'qunit'; + +module('Unit | Utility | data processor'); + +test('Class creation test', function(assert) { + assert.ok(DataProcessor); +}); + +test('Instance default test', function(assert) { + var processor; + + Ember.run(function () { + processor = DataProcessor.create({ + tableDefinition: Ember.Object.create(), + startSearch: function () { + // Test Search + }, + startSort: function () { + // Test Sort + } + }); + }); + + assert.ok(processor); + assert.equal(processor.get('isSorting'), false); + assert.equal(processor.get('isSearching'), false); + + assert.ok(processor._searchObserver); + assert.ok(processor._sortObserver); + assert.ok(processor.startSearch); + assert.ok(processor.startSort); + assert.ok(processor.compareFunction); + assert.ok(processor.totalPages); + assert.ok(processor.processedRows); +}); + +test('compareFunction test', function(assert) { + var processor; + + Ember.run(function () { + processor = DataProcessor.create({ + tableDefinition: Ember.Object.create(), + startSearch: function () {}, + startSort: function () {} + }); + }); + + assert.equal(processor.compareFunction(1, 1), 0); + assert.equal(processor.compareFunction(1, 2), -1); + assert.equal(processor.compareFunction(2, 1), 1); + + assert.equal(processor.compareFunction("a", "a"), 0); + assert.equal(processor.compareFunction("a", "b"), -1); + assert.equal(processor.compareFunction("b", "a"), 1); + + assert.equal(processor.compareFunction(null, null), -1); + assert.equal(processor.compareFunction(1, null), 1); + assert.equal(processor.compareFunction(null, 2), -1); + assert.equal(processor.compareFunction("a", null), 1); + assert.equal(processor.compareFunction(null, "b"), -1); + + assert.equal(processor.compareFunction(undefined, undefined), -1); + assert.equal(processor.compareFunction(1, undefined), 1); + assert.equal(processor.compareFunction(undefined, 2), -1); + assert.equal(processor.compareFunction("a", undefined), 1); + assert.equal(processor.compareFunction(undefined, "b"), -1); +}); + +test('startSearch test', function(assert) { + var processor, + runLater = Ember.run.later; + + assert.expect(3); + + Ember.run.later = function (callback) { + callback(); + assert.equal(processor.get("_searchedRows.length"), 2); + assert.equal(processor.get("_searchedRows.0.foo"), "Foo1"); + assert.equal(processor.get("_searchedRows.1.foo"), "Foo12"); + + Ember.run.later = runLater; // Reset + }; + + Ember.run(function () { + processor = DataProcessor.create({ + tableDefinition: Ember.Object.create({ + searchText: "foo1", + columns: [ColumnDefinition.create({ + id: "foo", + contentPath: 'foo' + }), ColumnDefinition.create({ + id: "bar", + contentPath: 'bar' + })] + }), + startSort: function () { + // Test Sort + }, + _sortedRows: [Ember.Object.create({ + foo: "Foo1", + bar: "Bar1" + }), Ember.Object.create({ + foo: "Foo12", + bar: "Bar2" + }), Ember.Object.create({ + foo: "Foo3", + bar: "Bar3" + }), Ember.Object.create({ + foo: "Foo4", + bar: "Bar4" + })], + }); + }); + +}); diff --git a/tez-ui/src/main/webapp/tests/unit/utils/facet-types-test.js b/tez-ui/src/main/webapp/tests/unit/utils/facet-types-test.js new file mode 100644 index 0000000000..f3af95249f --- /dev/null +++ b/tez-ui/src/main/webapp/tests/unit/utils/facet-types-test.js @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import facetTypes from '../../../utils/facet-types'; +import { module, test } from 'qunit'; + +module('Unit | Utility | facet types'); + +test('Basic creation test', function(assert) { + assert.ok(facetTypes); + + assert.ok(facetTypes.VALUES); +}); diff --git a/tez-ui/src/main/webapp/tests/unit/utils/sql-test.js b/tez-ui/src/main/webapp/tests/unit/utils/sql-test.js new file mode 100644 index 0000000000..7aed218801 --- /dev/null +++ b/tez-ui/src/main/webapp/tests/unit/utils/sql-test.js @@ -0,0 +1,90 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import SQL from '../../../utils/sql'; +import ColumnDefinition from '../../../utils/column-definition'; +import { module, test } from 'qunit'; + +module('Unit | Utility | sql'); + +test('Class creation test', function(assert) { + var sql = SQL.create(); + + assert.ok(sql.constructQuery); + assert.ok(sql.validateClause); + assert.ok(sql.normaliseClause); + assert.ok(sql.search); +}); + +test('constructQuery test', function(assert) { + var sql = SQL.create(); + + assert.equal(sql.constructQuery("x = y"), "SELECT * FROM ? WHERE x = y"); +}); + +test('validateClause test', function(assert) { + var sql = SQL.create(); + + assert.ok(sql.validateClause("x = y")); + assert.ok(sql.validateClause("x = y AND a = b")); + assert.ok(sql.validateClause("(x = y OR y = z) AND a = b")); + assert.ok(sql.validateClause("x BETWEEN 1 AND 2")); + + assert.notOk(sql.validateClause("foo")); + assert.notOk(sql.validateClause("foo bar")); + assert.notOk(sql.validateClause("^[a-z0-9_-]{3,16}$")); + assert.notOk(sql.validateClause("^[a-z0-9_-]{6,18}$")); + assert.notOk(sql.validateClause("^[a-z0-9-]+$")); +}); + +test('normaliseClause test', function(assert) { + var sql = SQL.create(), + column = ColumnDefinition.create({ + headerTitle: "Column Header", + id: "columnID", + contentPath: "col" + }); + + assert.equal(sql.normaliseClause('"Column Header" = value', [column]), "columnID = value"); + assert.equal(sql.normaliseClause('"Another Column Header" = value', [column]), '"Another Column Header" = value'); +}); + +test('search test', function(assert) { + var sql = SQL.create(), + data = [{ + colA: "x1", + colB: "y1" + }, { + colA: "x2", + colB: "y2" + }, { + colA: "x1", + colB: "y3" + }], + columns = [ColumnDefinition.create({ + headerTitle: "Column A", + id: "colA", + contentPath: "colA" + })]; + + var result = sql.search('"Column A" = "x1"', data, columns); + + assert.equal(result.length, 2); + assert.equal(result[0].colB, "y1"); + assert.equal(result[1].colB, "y3"); +}); diff --git a/tez-ui/src/main/webapp/tests/unit/utils/table-definition-test.js b/tez-ui/src/main/webapp/tests/unit/utils/table-definition-test.js new file mode 100644 index 0000000000..234994b192 --- /dev/null +++ b/tez-ui/src/main/webapp/tests/unit/utils/table-definition-test.js @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import TableDefinition from '../../../utils/table-definition'; +import { module, test } from 'qunit'; + +module('Unit | Utility | table definition'); + +test('Class creation test', function(assert) { + assert.ok(TableDefinition); +}); + +test('Default instance test', function(assert) { + var definition = TableDefinition.create(); + + assert.ok(definition); + + assert.equal(definition.pageNum, 1); + assert.equal(definition.rowCount, 10); + assert.equal(definition.minRowsForFooter, 25); +}); + +test('Page-num reset test', function(assert) { + var definition = TableDefinition.create(); + + assert.equal(definition.pageNum, 1); + + definition.set("pageNum", 5); + assert.equal(definition.pageNum, 5); + + definition.set("searchText", "x"); + assert.equal(definition.pageNum, 1); + + definition.set("pageNum", 5); + definition.set("rowCount", 5); + assert.equal(definition.pageNum, 1); +}); diff --git a/tez-ui/src/main/webapp/yarn.lock b/tez-ui/src/main/webapp/yarn.lock index 00250e82cf..660ac80d87 100644 --- a/tez-ui/src/main/webapp/yarn.lock +++ b/tez-ui/src/main/webapp/yarn.lock @@ -1391,16 +1391,6 @@ ee-first@1.1.1: version "1.1.1" resolved "https://registry.yarnpkg.com/ee-first/-/ee-first-1.1.1.tgz#590c61156b0ae2f4f0255732a158b266bc56b21d" -em-table@0.11.3: - version "0.11.3" - resolved "https://registry.yarnpkg.com/em-table/-/em-table-0.11.3.tgz#20e605cc3814214e644199399a2383cee8d23eeb" - dependencies: - ember-cli-htmlbars "^1.0.1" - ember-cli-less "^1.4.0" - source-map "^0.5.6" - optionalDependencies: - phantomjs-prebuilt "2.1.13" - em-tgraph@0.0.14: version "0.0.14" resolved "https://registry.yarnpkg.com/em-tgraph/-/em-tgraph-0.0.14.tgz#4d48b911760f85dec41904e4056ec52542391cc1" From c875b8216fa6c236230a04c8cdbdc25c38e7ef61 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Thu, 26 Aug 2021 21:46:23 +0200 Subject: [PATCH 332/512] TEZ-4332: Change ASF logo on Tez webpage according to recent guidelines (#148) --- docs/src/site/site.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/src/site/site.xml b/docs/src/site/site.xml index c8e6479a8f..5964cbc877 100644 --- a/docs/src/site/site.xml +++ b/docs/src/site/site.xml @@ -47,8 +47,8 @@ Apache Software Foundation - http://www.apache.org/images/asf_logo.gif - http://www.apache.org/ + https://apache.org/foundation/press/kit/asf_logo_wide.png + https://www.apache.org/ From 58fca8bb77d63c1ca6e6eb400eb60588159d6ae0 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Thu, 14 Oct 2021 16:12:57 +0200 Subject: [PATCH 333/512] TEZ-4180: Show convenient input -> output vertex names in output/sort messages (#154) --- .../apache/tez/runtime/api/OutputContext.java | 14 +++-- .../apache/tez/mapreduce/output/MROutput.java | 4 +- .../api/impl/TezOutputContextImpl.java | 6 ++ .../common/sort/impl/ExternalSorter.java | 4 +- .../common/sort/impl/PipelinedSorter.java | 59 ++++++++++--------- .../common/sort/impl/dflt/DefaultSorter.java | 29 ++++----- .../writers/UnorderedPartitionedKVWriter.java | 42 +++++++------ .../output/OrderedPartitionedKVOutput.java | 4 +- .../library/output/UnorderedKVOutput.java | 4 +- .../output/UnorderedPartitionedKVOutput.java | 2 +- .../library/output/OutputTestHelpers.java | 1 + .../output/TestOnFileSortedOutput.java | 1 + 12 files changed, 97 insertions(+), 73 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/OutputContext.java b/tez-api/src/main/java/org/apache/tez/runtime/api/OutputContext.java index 33fe772f85..f0de897fda 100644 --- a/tez-api/src/main/java/org/apache/tez/runtime/api/OutputContext.java +++ b/tez-api/src/main/java/org/apache/tez/runtime/api/OutputContext.java @@ -33,21 +33,27 @@ public interface OutputContext extends TaskContext { * Output's data * @return Name of the Destination Vertex */ - public String getDestinationVertexName(); - + String getDestinationVertexName(); + + /** + * Returns a convenient, human-readable string describing the input and output vertices. + * @return the convenient string + */ + String getInputOutputVertexNames(); + /** * Get the index of the output in the set of all outputs for the task. The * index will be consistent and valid only among the tasks of this vertex. * @return index */ - public int getOutputIndex(); + int getOutputIndex(); /** * Get an {@link OutputStatisticsReporter} for this {@link Output} that can * be used to report statistics like data size * @return {@link OutputStatisticsReporter} */ - public OutputStatisticsReporter getStatisticsReporter(); + OutputStatisticsReporter getStatisticsReporter(); /** * Notify the context that at this point no more events should be sent. diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java index 19ece5a0f6..9aeae25bd9 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java @@ -458,7 +458,7 @@ protected List initializeBase() throws IOException, InterruptedException initCommitter(jobConf, useNewApi); } - LOG.info(getContext().getDestinationVertexName() + ": " + LOG.info(getContext().getInputOutputVertexNames() + ": " + "outputFormat=" + outputFormatClassName + ", using newmapreduce API=" + useNewApi); return null; @@ -576,7 +576,7 @@ public void handleEvents(List outputEvents) { @Override public synchronized List close() throws IOException { flush(); - LOG.info(getContext().getDestinationVertexName() + " closed"); + LOG.info(getContext().getInputOutputVertexNames() + " closed"); long outputRecords = getContext().getCounters() .findCounter(TaskCounter.OUTPUT_RECORDS).getValue(); getContext().getStatisticsReporter().reportItemsProcessed(outputRecords); diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java index 20ec0622c7..a17bc8900d 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java @@ -154,6 +154,12 @@ public String getDestinationVertexName() { return destinationVertexName; } + + @Override + public String getInputOutputVertexNames() { + return String.format("%s -> %s", getTaskVertexName(), getDestinationVertexName()); + } + @Override public void fatalError(Throwable exception, String message) { super.signalFatalError(exception, message, sourceInfo); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java index 3ff74f72bb..758c069799 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java @@ -183,7 +183,7 @@ public ExternalSorter(OutputContext outputContext, Configuration conf, int numOu rfs = ((LocalFileSystem)FileSystem.getLocal(this.conf)).getRaw(); if (LOG.isDebugEnabled()) { - LOG.debug(outputContext.getDestinationVertexName() + ": Initial Mem bytes : " + + LOG.debug(outputContext.getInputOutputVertexNames() + ": Initial Mem bytes : " + initialMemoryAvailable + ", in MB=" + ((initialMemoryAvailable >> 20))); } int assignedMb = (int) (initialMemoryAvailable >> 20); @@ -201,7 +201,7 @@ public ExternalSorter(OutputContext outputContext, Configuration conf, int numOu this.serializationContext = new SerializationContext(this.conf); keySerializer = serializationContext.getKeySerializer(); valSerializer = serializationContext.getValueSerializer(); - LOG.info(outputContext.getDestinationVertexName() + " using: " + LOG.info(outputContext.getInputOutputVertexNames() + " using: " + "memoryMb=" + assignedMb + ", keySerializerClass=" + serializationContext.getKeyClass() + ", valueSerializerClass=" + valSerializer diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java index b70d6c4360..08786c9b2c 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java @@ -161,7 +161,7 @@ public PipelinedSorter(OutputContext outputContext, Configuration conf, int numO } StringBuilder initialSetupLogLine = new StringBuilder("Setting up PipelinedSorter for ") - .append(outputContext.getDestinationVertexName()).append(": "); + .append(outputContext.getInputOutputVertexNames()).append(": "); partitionBits = bitcount(partitions)+1; boolean confPipelinedShuffle = this.conf.getBoolean(TezRuntimeConfiguration @@ -235,10 +235,9 @@ public PipelinedSorter(OutputContext outputContext, Configuration conf, int numO TezRuntimeConfiguration.TEZ_RUNTIME_PIPELINED_SORTER_SORT_THREADS_DEFAULT); sortmaster = Executors.newFixedThreadPool(sortThreads, new ThreadFactoryBuilder().setDaemon(true) - .setNameFormat("Sorter {" + TezUtilsInternal - .cleanVertexName(outputContext.getDestinationVertexName()) + "} #%d") - .build()); - + .setNameFormat("Sorter {" + TezUtilsInternal.cleanVertexName(outputContext.getTaskVertexName()) + " -> " + + TezUtilsInternal.cleanVertexName(outputContext.getDestinationVertexName()) + "} #%d") + .build()); valSerializer.open(span.out); keySerializer.open(span.out); @@ -336,7 +335,8 @@ public void sort() throws IOException { boolean ret = spill(true); stopWatch.stop(); if (LOG.isDebugEnabled()) { - LOG.debug(outputContext.getDestinationVertexName() + ": Time taken for spill " + (stopWatch.now(TimeUnit.MILLISECONDS)) + " ms"); + LOG.debug(outputContext.getInputOutputVertexNames() + ": Time taken for spill " + + (stopWatch.now(TimeUnit.MILLISECONDS)) + " ms"); } if (pipelinedShuffle && ret) { sendPipelinedShuffleEvents(); @@ -380,7 +380,7 @@ private void sendPipelinedShuffleEvents() throws IOException{ partitions, sendEmptyPartitionDetails, pathComponent, partitionStats, reportDetailedPartitionStats(), auxiliaryService, deflater); outputContext.sendEvents(events); - LOG.info(outputContext.getDestinationVertexName() + + LOG.info(outputContext.getInputOutputVertexNames() + ": Added spill event for spill (final update=false), spillId=" + (numSpills - 1)); } @@ -496,7 +496,7 @@ private void spillSingleRecord(final Object key, final Object value, ensureSpillFilePermissions(filename, rfs); try { - LOG.info(outputContext.getDestinationVertexName() + ": Spilling to " + filename.toString() + + LOG.info(outputContext.getInputOutputVertexNames() + ": Spilling to " + filename.toString() + ", indexFilename=" + indexFilename); for (int i = 0; i < partitions; ++i) { if (isThreadInterrupted()) { @@ -568,8 +568,9 @@ public boolean spill(boolean ignoreEmptySpills) throws IOException { } } catch (InterruptedException e) { Thread.currentThread().interrupt(); - LOG.info(outputContext.getDestinationVertexName() + ": Interrupted while waiting for mergers to complete"); - throw new IOInterruptedException(outputContext.getDestinationVertexName() + ": Interrupted while waiting for mergers to complete", e); + LOG.info(outputContext.getInputOutputVertexNames() + ": Interrupted while waiting for mergers to complete"); + throw new IOInterruptedException( + outputContext.getInputOutputVertexNames() + ": Interrupted while waiting for mergers to complete", e); } // create spill file @@ -581,7 +582,7 @@ public boolean spill(boolean ignoreEmptySpills) throws IOException { spillFilePaths.put(numSpills, filename); out = rfs.create(filename, true, 4096); ensureSpillFilePermissions(filename, rfs); - LOG.info(outputContext.getDestinationVertexName() + ": Spilling to " + filename.toString()); + LOG.info(outputContext.getInputOutputVertexNames() + ": Spilling to " + filename.toString()); for (int i = 0; i < partitions; ++i) { if (isThreadInterrupted()) { return false; @@ -652,8 +653,9 @@ private boolean isThreadInterrupted() throws IOException { cleanup(); } sortmaster.shutdownNow(); - LOG.info(outputContext.getDestinationVertexName() + ": Thread interrupted, cleaned up stale data, sorter threads shutdown=" + sortmaster - .isShutdown() + ", terminated=" + sortmaster.isTerminated()); + LOG.info(outputContext.getInputOutputVertexNames() + + ": Thread interrupted, cleaned up stale data, sorter threads shutdown=" + sortmaster.isShutdown() + + ", terminated=" + sortmaster.isTerminated()); return true; } return false; @@ -674,7 +676,7 @@ public void flush() throws IOException { } try { - LOG.info(outputContext.getDestinationVertexName() + ": Starting flush of map output"); + LOG.info(outputContext.getInputOutputVertexNames() + ": Starting flush of map output"); span.end(); merger.add(span.sort(sorter)); // force a spill in flush() @@ -698,7 +700,7 @@ public void flush() throws IOException { * NPE leading to distraction when debugging. */ if (LOG.isDebugEnabled()) { - LOG.debug(outputContext.getDestinationVertexName() + LOG.debug(outputContext.getInputOutputVertexNames() + ": Index list is empty... returning"); } return; @@ -717,7 +719,8 @@ public void flush() throws IOException { outputContext, i, indexCacheList.get(i), partitions, sendEmptyPartitionDetails, pathComponent, partitionStats, reportDetailedPartitionStats(), auxiliaryService, deflater); - LOG.info(outputContext.getDestinationVertexName() + ": Adding spill event for spill (final update=" + isLastEvent + "), spillId=" + i); + LOG.info(outputContext.getInputOutputVertexNames() + ": Adding spill event for spill (final update=" + + isLastEvent + "), spillId=" + i); } return; } @@ -736,7 +739,7 @@ public void flush() throws IOException { sameVolRename(filename, finalOutputFile); sameVolRename(indexFilename, finalIndexFile); if (LOG.isDebugEnabled()) { - LOG.debug(outputContext.getDestinationVertexName() + ": numSpills=" + numSpills + + LOG.debug(outputContext.getInputOutputVertexNames() + ": numSpills=" + numSpills + ", finalOutputFile=" + finalOutputFile + ", " + "finalIndexFile=" + finalIndexFile + ", filename=" + filename + ", indexFilename=" + indexFilename); @@ -759,7 +762,7 @@ public void flush() throws IOException { mapOutputFile.getOutputIndexFileForWrite(0); //TODO if (LOG.isDebugEnabled()) { - LOG.debug(outputContext.getDestinationVertexName() + ": " + + LOG.debug(outputContext.getInputOutputVertexNames() + ": " + "numSpills: " + numSpills + ", finalOutputFile:" + finalOutputFile + ", finalIndexFile:" + finalIndexFile); } @@ -944,7 +947,7 @@ public SortSpan(ByteBuffer source, int maxItems, int perItem, RawComparator comp } ByteBuffer reserved = source.duplicate(); reserved.mark(); - LOG.info(outputContext.getDestinationVertexName() + ": " + "reserved.remaining()=" + + LOG.info(outputContext.getInputOutputVertexNames() + ": " + "reserved.remaining()=" + reserved.remaining() + ", reserved.metasize=" + metasize); reserved.position(metasize); kvbuffer = reserved.slice(); @@ -966,8 +969,8 @@ public SpanIterator sort(IndexedSorter sorter) { if(length() > 1) { sorter.sort(this, 0, length(), progressable); } - LOG.info(outputContext.getDestinationVertexName() + ": " + "done sorting span=" + index + ", length=" + length() + ", " - + "time=" + (System.currentTimeMillis() - start)); + LOG.info(outputContext.getInputOutputVertexNames() + ": " + "done sorting span=" + index + ", length=" + length() + + ", " + "time=" + (System.currentTimeMillis() - start)); return new SpanIterator((SortSpan)this); } @@ -1042,8 +1045,9 @@ public SortSpan next() { } newSpan = new SortSpan(remaining, items, perItem, newComparator); newSpan.index = index+1; - LOG.info(String.format(outputContext.getDestinationVertexName() + ": " + "New Span%d.length = %d, perItem = %d", newSpan.index, newSpan - .length(), perItem) + ", counter:" + mapOutputRecordCounter.getValue()); + LOG.info( + String.format(outputContext.getInputOutputVertexNames() + ": " + "New Span%d.length = %d, perItem = %d", + newSpan.index, newSpan.length(), perItem) + ", counter:" + mapOutputRecordCounter.getValue()); return newSpan; } return null; @@ -1064,13 +1068,14 @@ public ByteBuffer end() { return null; } int perItem = kvbuffer.position()/items; - LOG.info(outputContext.getDestinationVertexName() + ": " + String.format("Span%d.length = %d, perItem = %d", index, length(), perItem)); + LOG.info(outputContext.getInputOutputVertexNames() + ": " + + String.format("Span%d.length = %d, perItem = %d", index, length(), perItem)); if(remaining.remaining() < METASIZE+perItem) { //Check if we can get the next Buffer from the main buffer list ByteBuffer space = allocateSpace(); if (space != null) { - LOG.info(outputContext.getDestinationVertexName() + ": " + "Getting memory from next block in the list, recordsWritten=" + - mapOutputRecordCounter.getValue()); + LOG.info(outputContext.getInputOutputVertexNames() + ": " + + "Getting memory from next block in the list, recordsWritten=" + mapOutputRecordCounter.getValue()); reinit = true; return space; } @@ -1403,7 +1408,7 @@ public final boolean ready() throws IOException, InterruptedException { total += sp.span.length(); eq += sp.span.getEq(); } - LOG.info(outputContext.getDestinationVertexName() + ": " + "Heap = " + sb.toString()); + LOG.info(outputContext.getInputOutputVertexNames() + ": " + "Heap = " + sb.toString()); return true; } catch(ExecutionException e) { LOG.error("Heap size={}, total={}, eq={}, partition={}, gallop={}, totalItr={}," diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java index dd6c083109..7c678749b2 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java @@ -153,7 +153,7 @@ public DefaultSorter(OutputContext outputContext, Configuration conf, int numOut .TEZ_RUNTIME_PIPELINED_SHUFFLE_ENABLED_DEFAULT); if (confPipelinedShuffle) { - LOG.warn(outputContext.getDestinationVertexName() + ": " + + LOG.warn(outputContext.getInputOutputVertexNames() + ": " + TezRuntimeConfiguration.TEZ_RUNTIME_PIPELINED_SHUFFLE_ENABLED + " does not work " + "with DefaultSorter. It is supported only with PipelinedSorter."); } @@ -371,7 +371,8 @@ synchronized void collect(Object key, Object value, final int partition kvindex = (int)(((long)kvindex - NMETA + kvmeta.capacity()) % kvmeta.capacity()); totalKeys++; } catch (MapBufferTooSmallException e) { - LOG.info(outputContext.getDestinationVertexName() + ": Record too large for in-memory buffer: " + e.getMessage()); + LOG.info( + outputContext.getInputOutputVertexNames() + ": Record too large for in-memory buffer: " + e.getMessage()); spillSingleRecord(key, value, partition); mapOutputRecordCounter.increment(1); return; @@ -390,7 +391,7 @@ private void setEquator(int pos) { // Cast one of the operands to long to avoid integer overflow kvindex = (int) (((long) aligned - METASIZE + kvbuffer.length) % kvbuffer.length) / 4; if (LOG.isInfoEnabled()) { - LOG.info(outputContext.getDestinationVertexName() + ": " + "(EQUATOR) " + pos + " kvi " + kvindex + + LOG.info(outputContext.getInputOutputVertexNames() + ": " + "(EQUATOR) " + pos + " kvi " + kvindex + "(" + (kvindex * 4) + ")"); } } @@ -408,7 +409,7 @@ private void resetSpill() { // Cast one of the operands to long to avoid integer overflow kvstart = kvend = (int) (((long) aligned - METASIZE + kvbuffer.length) % kvbuffer.length) / 4; if (LOG.isInfoEnabled()) { - LOG.info(outputContext.getDestinationVertexName() + ": " + "(RESET) equator " + e + " kv " + kvstart + "(" + + LOG.info(outputContext.getInputOutputVertexNames() + ": " + "(RESET) equator " + e + " kv " + kvstart + "(" + (kvstart * 4) + ")" + " kvi " + kvindex + "(" + (kvindex * 4) + ")"); } } @@ -664,7 +665,7 @@ void interruptSpillThread() throws IOException { spillThread.interrupt(); spillThread.join(); } catch (InterruptedException e) { - LOG.info(outputContext.getDestinationVertexName() + ": " + "Spill thread interrupted"); + LOG.info(outputContext.getInputOutputVertexNames() + ": " + "Spill thread interrupted"); //Reset status Thread.currentThread().interrupt(); throw new IOInterruptedException("Spill failed", e); @@ -673,7 +674,7 @@ void interruptSpillThread() throws IOException { @Override public void flush() throws IOException { - LOG.info(outputContext.getDestinationVertexName() + ": " + "Starting flush of map output"); + LOG.info(outputContext.getInputOutputVertexNames() + ": " + "Starting flush of map output"); outputContext.notifyProgress(); if (Thread.currentThread().isInterrupted()) { /** @@ -710,7 +711,7 @@ public void flush() throws IOException { bufend = bufmark; if (LOG.isInfoEnabled()) { LOG.info( - outputContext.getDestinationVertexName() + ": " + "Sorting & Spilling map output. " + outputContext.getInputOutputVertexNames() + ": " + "Sorting & Spilling map output. " + "bufstart = " + bufstart + ", bufend = " + bufmark + ", bufvoid = " + bufvoid + "; " + "kvstart=" + kvstart + "(" + (kvstart * 4) + ")" + ", kvend = " + kvend + "(" + (kvend * 4) + ")" @@ -781,7 +782,7 @@ public void run() { spillLock.unlock(); sortAndSpill(sameKeyCount, totalKeysCount); } catch (Throwable t) { - LOG.warn(outputContext.getDestinationVertexName() + ": " + "Got an exception in sortAndSpill", t); + LOG.warn(outputContext.getInputOutputVertexNames() + ": " + "Got an exception in sortAndSpill", t); sortSpillException = t; } finally { spillLock.lock(); @@ -794,7 +795,7 @@ public void run() { } } } catch (InterruptedException e) { - LOG.info(outputContext.getDestinationVertexName() + ": " + "Spill thread interrupted"); + LOG.info(outputContext.getInputOutputVertexNames() + ": " + "Spill thread interrupted"); Thread.currentThread().interrupt(); } finally { spillLock.unlock(); @@ -830,7 +831,7 @@ private void startSpill() { bufend = bufmark; spillInProgress = true; if (LOG.isInfoEnabled()) { - LOG.info(outputContext.getDestinationVertexName() + ": Spilling map output." + LOG.info(outputContext.getInputOutputVertexNames() + ": Spilling map output." + "bufstart=" + bufstart + ", bufend = " + bufmark + ", bufvoid = " + bufvoid +"; kvstart=" + kvstart + "(" + (kvstart * 4) + ")" +", kvend = " + kvend + "(" + (kvend * 4) + ")" @@ -936,7 +937,7 @@ protected void spill(int mstart, int mend, long sameKeyCount, long totalKeysCoun TezRawKeyValueIterator kvIter = new MRResultIterator(spstart, spindex); if (LOG.isDebugEnabled()) { - LOG.debug(outputContext.getDestinationVertexName() + ": " + "Running combine processor"); + LOG.debug(outputContext.getInputOutputVertexNames() + ": " + "Running combine processor"); } runCombineProcessor(kvIter, writer); } @@ -975,7 +976,7 @@ protected void spill(int mstart, int mend, long sameKeyCount, long totalKeysCoun totalIndexCacheMemory += spillRec.size() * MAP_OUTPUT_INDEX_RECORD_LENGTH; } - LOG.info(outputContext.getDestinationVertexName() + ": " + "Finished spill " + numSpills + LOG.info(outputContext.getInputOutputVertexNames() + ": " + "Finished spill " + numSpills + " at " + filename.toString()); ++numSpills; if (!isFinalMergeEnabled()) { @@ -1172,7 +1173,7 @@ private void maybeSendEventForSpill(List events, boolean isLastEvent, outputContext, index, spillRecord, partitions, sendEmptyPartitionDetails, pathComponent, partitionStats, reportDetailedPartitionStats(), auxiliaryService, deflater); - LOG.info(outputContext.getDestinationVertexName() + ": " + + LOG.info(outputContext.getInputOutputVertexNames() + ": " + "Adding spill event for spill (final update=" + isLastEvent + "), spillId=" + index); if (sendEvent) { @@ -1339,7 +1340,7 @@ private void mergeParts() throws IOException, InterruptedException { segmentList.add(s); } if (LOG.isDebugEnabled()) { - LOG.debug(outputContext.getDestinationVertexName() + ": " + LOG.debug(outputContext.getInputOutputVertexNames() + ": " + "TaskIdentifier=" + taskIdentifier + " Partition=" + parts + "Spill =" + i + "(" + indexRecord.getStartOffset() + "," + indexRecord.getRawLength() + ", " + diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java index 5ff2944766..faf75866b9 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java @@ -104,7 +104,7 @@ public class UnorderedPartitionedKVWriter extends BaseUnorderedPartitionedKVWrit // Maybe setup a separate statistics class which can be shared between the // buffer and the main path instead of having multiple arrays. - private final String destNameTrimmed; + private final String sourceDestNameTrimmed; private final long availableMemory; @VisibleForTesting final WrappedBuffer[] buffers; @@ -206,7 +206,8 @@ public UnorderedPartitionedKVWriter(OutputContext outputContext, Configuration c Preconditions.checkArgument(availableMemoryBytes >= 0, "availableMemory should be >= 0 bytes"); - this.destNameTrimmed = TezUtilsInternal.cleanVertexName(outputContext.getDestinationVertexName()); + this.sourceDestNameTrimmed = TezUtilsInternal.cleanVertexName(outputContext.getTaskVertexName()) + " -> " + + TezUtilsInternal.cleanVertexName(outputContext.getDestinationVertexName()); //Not checking for TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT as it might not add much value in // this case. Add it later if needed. boolean pipelinedShuffleConf = this.conf.getBoolean(TezRuntimeConfiguration @@ -257,7 +258,7 @@ public UnorderedPartitionedKVWriter(OutputContext outputContext, Configuration c buffers[0] = new WrappedBuffer(numOutputs, sizePerBuffer); numInitializedBuffers = 1; if (LOG.isDebugEnabled()) { - LOG.debug(destNameTrimmed + ": " + "Initializing Buffer #" + + LOG.debug(sourceDestNameTrimmed + ": " + "Initializing Buffer #" + numInitializedBuffers + " with size=" + sizePerBuffer); } currentBuffer = buffers[0]; @@ -313,7 +314,7 @@ public UnorderedPartitionedKVWriter(OutputContext outputContext, Configuration c skipBuffers = false; writer = null; } - LOG.info(destNameTrimmed + ": " + LOG.info(sourceDestNameTrimmed + ": " + "numBuffers=" + numBuffers + ", sizePerBuffer=" + sizePerBuffer + ", skipBuffers=" + skipBuffers @@ -493,7 +494,7 @@ private void setupNextBuffer() throws IOException { // Update overall stats final int filledBufferCount = filledBuffers.size(); if (LOG.isDebugEnabled() || (filledBufferCount % 10) == 0) { - LOG.info(destNameTrimmed + ": " + "Moving to next buffer. Total filled buffers: " + filledBufferCount); + LOG.info(sourceDestNameTrimmed + ": " + "Moving to next buffer. Total filled buffers: " + filledBufferCount); } updateGlobalStats(currentBuffer); @@ -531,7 +532,7 @@ private boolean scheduleSpill(boolean block) throws IOException { final int filledBufferCount = filledBuffers.size(); if (LOG.isDebugEnabled() || (filledBufferCount % 10) == 0) { - LOG.info(destNameTrimmed + ": triggering spill. filledBuffers.size=" + filledBufferCount); + LOG.info(sourceDestNameTrimmed + ": triggering spill. filledBuffers.size=" + filledBufferCount); } pendingSpillCount.incrementAndGet(); int spillNumber = numSpills.getAndIncrement(); @@ -673,10 +674,10 @@ protected SpillResult callInternal() throws IOException { spillResult = new SpillResult(compressedLength, this.filledBuffers); handleSpillIndex(spillPathDetails, spillRecord); - LOG.info(destNameTrimmed + ": " + "Finished spill " + spillIndex); + LOG.info(sourceDestNameTrimmed + ": " + "Finished spill " + spillIndex); if (LOG.isDebugEnabled()) { - LOG.debug(destNameTrimmed + ": " + "Spill=" + spillIndex + ", indexPath=" + LOG.debug(sourceDestNameTrimmed + ": " + "Spill=" + spillIndex + ", indexPath=" + spillPathDetails.indexFilePath + ", outputPath=" + spillPathDetails.outputFilePath); } return spillResult; @@ -754,7 +755,8 @@ public List close() throws IOException, InterruptedException { isShutdown.set(true); spillLock.lock(); try { - LOG.info(destNameTrimmed + ": " + "Waiting for all spills to complete : Pending : " + pendingSpillCount.get()); + LOG.info( + sourceDestNameTrimmed + ": " + "Waiting for all spills to complete : Pending : " + pendingSpillCount.get()); while (pendingSpillCount.get() != 0 && spillException == null) { spillInProgress.await(); } @@ -762,7 +764,7 @@ public List close() throws IOException, InterruptedException { spillLock.unlock(); } if (spillException != null) { - LOG.error(destNameTrimmed + ": " + "Error during spill, throwing"); + LOG.error(sourceDestNameTrimmed + ": " + "Error during spill, throwing"); // Assuming close will be called on the same thread as the write cleanup(); currentBuffer.cleanup(); @@ -773,7 +775,7 @@ public List close() throws IOException, InterruptedException { throw new IOException(spillException); } } else { - LOG.info(destNameTrimmed + ": " + "All spills complete"); + LOG.info(sourceDestNameTrimmed + ": " + "All spills complete"); // Assuming close will be called on the same thread as the write cleanup(); @@ -1082,7 +1084,8 @@ private void mergeAll() throws IOException { for (int i = 0; i < numPartitions; i++) { long segmentStart = out.getPos(); if (numRecordsPerPartition[i] == 0) { - LOG.info(destNameTrimmed + ": " + "Skipping partition: " + i + " in final merge since it has no records"); + LOG.info( + sourceDestNameTrimmed + ": " + "Skipping partition: " + i + " in final merge since it has no records"); continue; } writer = new Writer(keySerialization, valSerialization, out, keyClass, valClass, codec, null, null); @@ -1136,7 +1139,7 @@ private void mergeAll() throws IOException { } finalSpillRecord.writeToFile(finalIndexPath, conf, localFs); fileOutputBytesCounter.increment(indexFileSizeEstimate); - LOG.info(destNameTrimmed + ": " + "Finished final spill after merging : " + numSpills.get() + " spills"); + LOG.info(sourceDestNameTrimmed + ": " + "Finished final spill after merging : " + numSpills.get() + " spills"); } private void deleteIntermediateSpills() { @@ -1208,9 +1211,10 @@ private void writeLargeRecord(final Object key, final Object value, final int pa mayBeSendEventsForSpill(emptyPartitions, sizePerPartition, spillIndex, false); - LOG.info(destNameTrimmed + ": " + "Finished writing large record of size " + outSize + " to spill file " + spillIndex); + LOG.info(sourceDestNameTrimmed + ": " + "Finished writing large record of size " + outSize + " to spill file " + + spillIndex); if (LOG.isDebugEnabled()) { - LOG.debug(destNameTrimmed + ": " + "LargeRecord Spill=" + spillIndex + ", indexPath=" + LOG.debug(sourceDestNameTrimmed + ": " + "LargeRecord Spill=" + spillIndex + ", indexPath=" + spillPathDetails.indexFilePath + ", outputPath=" + spillPathDetails.outputFilePath); } @@ -1346,7 +1350,7 @@ private void mayBeSendEventsForSpill( try { events = generateEventForSpill(emptyPartitions, sizePerPartition, spillNumber, isFinalUpdate); - LOG.info(destNameTrimmed + ": " + "Adding spill event for spill" + LOG.info(sourceDestNameTrimmed + ": " + "Adding spill event for spill" + " (final update=" + isFinalUpdate + "), spillId=" + spillNumber); if (pipelinedShuffle) { //Send out an event for consuming. @@ -1355,7 +1359,7 @@ private void mayBeSendEventsForSpill( this.finalEvents.addAll(events); } } catch (IOException e) { - LOG.error(destNameTrimmed + ": " + "Error in sending pipelined events", e); + LOG.error(sourceDestNameTrimmed + ": " + "Error in sending pipelined events", e); outputContext.reportFailure(TaskFailureType.NON_FATAL, e, "Error in sending events."); } @@ -1414,7 +1418,7 @@ public void onSuccess(SpillResult result) { availableBuffers.add(buffer); } } catch (Throwable e) { - LOG.error(destNameTrimmed + ": Failure while attempting to reset buffer after spill", e); + LOG.error(sourceDestNameTrimmed + ": Failure while attempting to reset buffer after spill", e); outputContext.reportFailure(TaskFailureType.NON_FATAL, e, "Failure while attempting to reset buffer after spill"); } @@ -1444,7 +1448,7 @@ public void onSuccess(SpillResult result) { public void onFailure(Throwable t) { // spillException setup to throw an exception back to the user. Requires synchronization. // Consider removing it in favor of having Tez kill the task - LOG.error(destNameTrimmed + ": " + "Failure while spilling to disk", t); + LOG.error(sourceDestNameTrimmed + ": " + "Failure while spilling to disk", t); spillException = t; outputContext.reportFailure(TaskFailureType.NON_FATAL, t, "Failure while spilling to disk"); spillLock.lock(); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java index 676fe17a5f..44cb9d6aae 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java @@ -135,7 +135,7 @@ public synchronized void start() throws Exception { if (pipelinedShuffle) { if (finalMergeEnabled) { - LOG.info(getContext().getDestinationVertexName() + " disabling final merge as " + LOG.info(getContext().getInputOutputVertexNames() + " disabling final merge as " + TezRuntimeConfiguration.TEZ_RUNTIME_PIPELINED_SHUFFLE_ENABLED + " is enabled."); finalMergeEnabled = false; conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT, false); @@ -194,7 +194,7 @@ public synchronized List close() throws IOException { returnEvents.addAll(generateEvents()); sorter = null; } else { - LOG.warn(getContext().getDestinationVertexName() + + LOG.warn(getContext().getInputOutputVertexNames() + ": Attempting to close output {} of type {} before it was started. Generating empty events", getContext().getDestinationVertexName(), this.getClass().getSimpleName()); returnEvents = generateEmptyEvents(); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java index e7a4429d95..bcacc5238e 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java @@ -106,7 +106,7 @@ public synchronized void start() throws Exception { this.kvWriter = new UnorderedPartitionedKVWriter(getContext(), conf, 1, memoryUpdateCallbackHandler.getMemoryAssigned()); isStarted.set(true); - LOG.info(getContext().getDestinationVertexName() + " started. MemoryAssigned=" + LOG.info(getContext().getInputOutputVertexNames() + " started. MemoryAssigned=" + memoryUpdateCallbackHandler.getMemoryAssigned()); } } @@ -130,7 +130,7 @@ public synchronized List close() throws Exception { returnEvents = kvWriter.close(); kvWriter = null; } else { - LOG.warn(getContext().getDestinationVertexName() + + LOG.warn(getContext().getInputOutputVertexNames() + ": Attempting to close output {} of type {} before it was started. Generating empty events", getContext().getDestinationVertexName(), this.getClass().getSimpleName()); returnEvents = new LinkedList(); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java index 439b732db5..9bc7ea40cd 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java @@ -108,7 +108,7 @@ public synchronized List close() throws Exception { returnEvents = kvWriter.close(); kvWriter = null; } else { - LOG.warn(getContext().getDestinationVertexName() + + LOG.warn(getContext().getInputOutputVertexNames() + ": Attempting to close output {} of type {} before it was started. Generating empty events", getContext().getDestinationVertexName(), this.getClass().getSimpleName()); returnEvents = new LinkedList(); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java index b81c2bd036..a7c7ca28cd 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java @@ -69,6 +69,7 @@ static OutputContext createOutputContext(Configuration conf, Configuration userP }).when(ctx).requestInitialMemory(anyLong(), any(MemoryUpdateCallback.class)); doReturn(conf).when(ctx).getContainerConfiguration(); doReturn(TezUtils.createUserPayloadFromConf(userPayloadConf)).when(ctx).getUserPayload(); + doReturn("taskVertex").when(ctx).getTaskVertexName(); doReturn("destinationVertex").when(ctx).getDestinationVertexName(); doReturn("UUID").when(ctx).getUniqueIdentifier(); doReturn(new String[] { workingDir.toString() }).when(ctx).getWorkDirs(); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileSortedOutput.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileSortedOutput.java index 2c9c3b2ace..7999d45fcd 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileSortedOutput.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileSortedOutput.java @@ -406,6 +406,7 @@ private OutputContext createTezOutputContext() throws IOException { doReturn(payLoad).when(context).getUserPayload(); doReturn(5 * 1024 * 1024l).when(context).getTotalMemoryAvailableToTask(); doReturn(UniqueID).when(context).getUniqueIdentifier(); + doReturn("v0").when(context).getTaskVertexName(); doReturn("v1").when(context).getDestinationVertexName(); doReturn(ByteBuffer.wrap(serviceProviderMetaData.getData())).when(context) .getServiceProviderMetaData From 3326978dfbe799f92f262d9a2b16c99a8ef61836 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Mon, 25 Oct 2021 13:19:48 +0200 Subject: [PATCH 334/512] TEZ-4342: TestSecureShuffle is broken - No subject alternative names present (#158) (Laszlo Bodor reviewed by Rajesh Balamohan) --- .../apache/tez/test/TestSecureShuffle.java | 50 +++++++++++++++++-- 1 file changed, 46 insertions(+), 4 deletions(-) diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java b/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java index aed240997e..2b2221230f 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java @@ -25,14 +25,19 @@ import java.io.File; import java.io.IOException; import java.io.OutputStreamWriter; +import java.math.BigInteger; import java.net.InetAddress; import java.security.KeyPair; +import java.security.SecureRandom; import java.security.cert.X509Certificate; import java.util.ArrayList; import java.util.Collection; +import java.util.Date; import java.util.HashMap; import java.util.Map; +import javax.security.auth.x500.X500Principal; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -46,6 +51,10 @@ import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.mapreduce.examples.TestOrderedWordCount; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; +import org.bouncycastle.asn1.x509.GeneralName; +import org.bouncycastle.asn1.x509.GeneralNames; +import org.bouncycastle.asn1.x509.X509Extensions; +import org.bouncycastle.x509.X509V3CertificateGenerator; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; @@ -222,8 +231,9 @@ private static void setupKeyStores() throws Exception { * (as discussed in https://github.com/AsyncHttpClient/async-http-client/issues/928), that's why * it cannot be set for an async http connection. So instead of hacking an ALLOW_ALL verifier * somehow (which cannot be propagated to netty), a valid certificate with the actual hostname - * should be generated in setupSSLConfig, so the only change is the usage of - * "InetAddress.getLocalHost().getHostName()". + * should be generated in setupSSLConfig. So, one change is the usage of + * InetAddress.getLocalHost().getHostName(), the other is using local generateCertificate, + * which fixes another issue. */ public static void setupSSLConfig(String keystoresDir, String sslConfDir, Configuration config, boolean useClientCert, boolean trustStore, String excludeCiphers) throws Exception { @@ -242,7 +252,7 @@ public static void setupSSLConfig(String keystoresDir, String sslConfDir, Config if (useClientCert) { KeyPair cKP = KeyStoreTestUtil.generateKeyPair("RSA"); X509Certificate cCert = - KeyStoreTestUtil.generateCertificate("CN=localhost, O=client", cKP, 30, "SHA1withRSA"); + generateCertificate("CN=localhost, O=client", cKP, 30, "SHA1withRSA"); KeyStoreTestUtil.createKeyStore(clientKS, clientPassword, "client", cKP.getPrivate(), cCert); certs.put("client", cCert); } @@ -250,7 +260,7 @@ public static void setupSSLConfig(String keystoresDir, String sslConfDir, Config String localhostName = InetAddress.getLocalHost().getHostName(); KeyPair sKP = KeyStoreTestUtil.generateKeyPair("RSA"); X509Certificate sCert = - KeyStoreTestUtil.generateCertificate("CN="+localhostName+", O=server", sKP, 30, "SHA1withRSA"); + generateCertificate("CN="+localhostName+", O=server", sKP, 30, "SHA1withRSA"); KeyStoreTestUtil.createKeyStore(serverKS, serverPassword, "server", sKP.getPrivate(), sCert); certs.put("server", sCert); @@ -274,4 +284,36 @@ public static void setupSSLConfig(String keystoresDir, String sslConfDir, Config config.set(SSLFactory.SSL_SERVER_CONF_KEY, sslServerConfFile.getName()); config.setBoolean(SSLFactory.SSL_REQUIRE_CLIENT_CERT_KEY, useClientCert); } + + /** + * This is a copied version of hadoop's KeyStoreTestUtil.generateCertificate, which takes care of setting + * IP address as a SSL Subject Alternative Name (SAN). Without this, SSL shuffle failed with async http client. + * Introduced by TEZ-4342. + */ + public static X509Certificate generateCertificate(String dn, KeyPair pair, int days, String algorithm) + throws Exception { + + Date from = new Date(); + Date to = new Date(from.getTime() + days * 86400000L); + BigInteger sn = new BigInteger(64, new SecureRandom()); + KeyPair keyPair = pair; + X509V3CertificateGenerator certGen = new X509V3CertificateGenerator(); + + String hostAddress = InetAddress.getLocalHost().getHostAddress(); + certGen.addExtension(X509Extensions.SubjectAlternativeName, false, + new GeneralNames(new GeneralName(GeneralName.iPAddress, hostAddress))); + + X500Principal dnName = new X500Principal(dn); + + certGen.setSerialNumber(sn); + certGen.setIssuerDN(dnName); + certGen.setNotBefore(from); + certGen.setNotAfter(to); + certGen.setSubjectDN(dnName); + certGen.setPublicKey(keyPair.getPublic()); + certGen.setSignatureAlgorithm(algorithm); + + X509Certificate cert = certGen.generate(pair.getPrivate()); + return cert; + } } From 6863a2d9923ed1633d44c708631e454303503d46 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Tue, 2 Nov 2021 18:26:47 +0100 Subject: [PATCH 335/512] TEZ-4336: ShuffleScheduler should try to report the original exception (when shuffle becomes unhealthy) (#155) (Laszlo Bodor reviewed by Rajesh Balamohan) --- .../common/shuffle/InputAttemptFetchFailure.java | 10 ++++++++++ .../orderedgrouped/FetcherOrderedGrouped.java | 4 ++-- .../shuffle/orderedgrouped/ShuffleScheduler.java | 12 +++++++----- 3 files changed, 19 insertions(+), 7 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/InputAttemptFetchFailure.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/InputAttemptFetchFailure.java index d94db35c2f..4ce1699cf5 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/InputAttemptFetchFailure.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/InputAttemptFetchFailure.java @@ -33,6 +33,7 @@ public class InputAttemptFetchFailure { private final InputAttemptIdentifier inputAttemptIdentifier; private final boolean isLocalFetch; private final boolean isDiskErrorAtSource; + private Throwable cause = null; public InputAttemptFetchFailure(InputAttemptIdentifier inputAttemptIdentifier) { this(inputAttemptIdentifier, false, false); @@ -112,4 +113,13 @@ public String toString() { return String.format("%s, isLocalFetch: %s, isDiskErrorAtSource: %s", inputAttemptIdentifier.toString(), isLocalFetch, isDiskErrorAtSource); } + + public InputAttemptFetchFailure withCause(Throwable throwable) { + this.cause = throwable; + return this; + } + + public Throwable getCause() { + return cause; + } } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java index c9bd092f05..a4328af44a 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java @@ -378,7 +378,7 @@ boolean setupConnection(MapHost host, Collection attempt for (InputAttemptIdentifier left : remaining.values()) { // Need to be handling temporary glitches .. // Report read error to the AM to trigger source failure heuristics - scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(left), host, connectSucceeded, + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(left).withCause(ie), host, connectSucceeded, !connectSucceeded); } return false; @@ -738,7 +738,7 @@ protected void setupLocalDiskFetch(MapHost host) throws InterruptedException { if (!stopped) { hasFailures = true; ioErrs.increment(1); - scheduler.copyFailed(InputAttemptFetchFailure.fromLocalFetchFailure(srcAttemptId), + scheduler.copyFailed(InputAttemptFetchFailure.fromLocalFetchFailure(srcAttemptId).withCause(e), host, true, false); LOG.warn("Failed to read local disk output of " + srcAttemptId + " from " + host.getHostIdentifier(), e); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java index 540d44f409..dd27d45aed 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java @@ -178,6 +178,7 @@ enum ShuffleErrors { private final Referee referee; @VisibleForTesting final Map failureCounts = new HashMap(); + final Set uniqueHosts = Sets.newHashSet(); private final Map hostFailures = new HashMap(); private final InputContext inputContext; @@ -792,7 +793,7 @@ public synchronized void copyFailed(InputAttemptFetchFailure fetchFailure, MapHo } //Restart consumer in case shuffle is not healthy - if (!isShuffleHealthy(fetchFailure.getInputAttemptIdentifier())) { + if (!isShuffleHealthy(fetchFailure)) { return; } @@ -1006,8 +1007,8 @@ private boolean isFetcherHealthy(String logContext) { return fetcherHealthy; } - boolean isShuffleHealthy(InputAttemptIdentifier srcAttempt) { - + boolean isShuffleHealthy(InputAttemptFetchFailure fetchFailure) { + InputAttemptIdentifier srcAttempt = fetchFailure.getInputAttemptIdentifier(); if (isAbortLimitExceeedFor(srcAttempt)) { return false; } @@ -1049,14 +1050,15 @@ boolean isShuffleHealthy(InputAttemptIdentifier srcAttempt) { + ", pendingInputs=" + (numInputs - doneMaps) + ", fetcherHealthy=" + fetcherHealthy + ", reducerProgressedEnough=" + reducerProgressedEnough - + ", reducerStalled=" + reducerStalled) + + ", reducerStalled=" + reducerStalled + + ", hostFailures=" + hostFailures) + "]"; LOG.error(errorMsg); if (LOG.isDebugEnabled()) { LOG.debug("Host failures=" + hostFailures.keySet()); } // Shuffle knows how to deal with failures post shutdown via the onFailure hook - exceptionReporter.reportException(new IOException(errorMsg)); + exceptionReporter.reportException(new IOException(errorMsg, fetchFailure.getCause())); return false; } return true; From f39a51e588fe7f731356cf3aa68755a5d98ff12a Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Tue, 2 Nov 2021 18:32:45 +0100 Subject: [PATCH 336/512] TEZ-4339: Expose real-time memory consumption of AM and task containers via DagClient (#157) (Laszlo Bodor reviewed by Rajesh Balamohan) --- .../apache/tez/dag/api/DagTypeConverters.java | 4 ++++ .../apache/tez/dag/api/client/DAGStatus.java | 20 +++++++++++++------ .../tez/dag/api/client/StatusGetOpts.java | 3 ++- tez-api/src/main/proto/DAGApiRecords.proto | 3 +++ .../tez/dag/api/client/DAGStatusBuilder.java | 7 +++++++ .../tez/dag/app/TaskCommunicatorManager.java | 8 ++++++++ .../app/TaskCommunicatorManagerInterface.java | 1 + .../tez/dag/app/TezTaskCommunicatorImpl.java | 6 ++++++ .../apache/tez/dag/app/dag/impl/DAGImpl.java | 8 ++++++++ .../serviceplugins/api/TaskCommunicator.java | 9 +++++++++ .../apache/tez/dag/app/MockDAGAppMaster.java | 4 ++-- .../runtime/api/impl/TezHeartbeatRequest.java | 11 +++++++++- .../apache/tez/runtime/task/TaskReporter.java | 18 ++++++++++++++++- .../tez/mapreduce/TestMRRJobsDAGApi.java | 8 +++++++- 14 files changed, 98 insertions(+), 12 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java index 5a2cb64cc3..c563f1fb47 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java @@ -628,6 +628,8 @@ public static DAGProtos.StatusGetOptsProto convertStatusGetOptsToProto( switch (statusGetOpts) { case GET_COUNTERS: return DAGProtos.StatusGetOptsProto.GET_COUNTERS; + case GET_MEMORY_USAGE: + return DAGProtos.StatusGetOptsProto.GET_MEMORY_USAGE; } throw new TezUncheckedException("Could not convert StatusGetOpts to" + " proto"); } @@ -636,6 +638,8 @@ public static StatusGetOpts convertStatusGetOptsFromProto(DAGProtos.StatusGetOpt switch (proto) { case GET_COUNTERS: return StatusGetOpts.GET_COUNTERS; + case GET_MEMORY_USAGE: + return StatusGetOpts.GET_MEMORY_USAGE; } throw new TezUncheckedException("Could not convert to StatusGetOpts from" + " proto"); } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java index cbf641e00f..1f8db62b80 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java @@ -152,6 +152,14 @@ public TezCounters getDAGCounters() { return dagCounters; } + public long getMemoryUsedByAM() { + return proxy.getMemoryUsedByAM(); + } + + public long getMemoryUsedByTasks() { + return proxy.getMemoryUsedByTasks(); + } + @InterfaceAudience.Private DagStatusSource getSource() { return this.source; @@ -201,12 +209,12 @@ public int hashCode() { @Override public String toString() { StringBuilder sb = new StringBuilder(); - sb.append("status=" + getState() - + ", progress=" + getDAGProgress() - + ", diagnostics=" - + StringUtils.join(getDiagnostics(), LINE_SEPARATOR) - + ", counters=" - + (getDAGCounters() == null ? "null" : getDAGCounters().toString())); + sb.append("status=" + getState()); + sb.append(", progress=" + getDAGProgress()); + sb.append(", diagnostics=" + StringUtils.join(getDiagnostics(), LINE_SEPARATOR)); + sb.append(", memoryUsedByAM=").append(proxy.getMemoryUsedByAM()); + sb.append(", memoryUsedByTasks=").append(proxy.getMemoryUsedByTasks()); + sb.append(", counters=" + (getDAGCounters() == null ? "null" : getDAGCounters().toString())); return sb.toString(); } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/StatusGetOpts.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/StatusGetOpts.java index 1a9df7afa1..3518d33c2c 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/StatusGetOpts.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/StatusGetOpts.java @@ -29,5 +29,6 @@ @Evolving public enum StatusGetOpts { /** Retrieve Counters with Status */ - GET_COUNTERS + GET_COUNTERS, + GET_MEMORY_USAGE } diff --git a/tez-api/src/main/proto/DAGApiRecords.proto b/tez-api/src/main/proto/DAGApiRecords.proto index 4c8c7f63a9..15f681db96 100644 --- a/tez-api/src/main/proto/DAGApiRecords.proto +++ b/tez-api/src/main/proto/DAGApiRecords.proto @@ -275,6 +275,8 @@ message DAGStatusProto { optional ProgressProto DAGProgress = 3; repeated StringProgressPairProto vertexProgress = 4; optional TezCountersProto dagCounters = 5; + optional int64 memoryUsedByAM = 6; + optional int64 memoryUsedByTasks = 7; } message PlanLocalResourcesProto { @@ -299,6 +301,7 @@ message TezCountersProto { enum StatusGetOptsProto { GET_COUNTERS = 0; + GET_MEMORY_USAGE = 1; } message VertexLocationHintProto { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGStatusBuilder.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGStatusBuilder.java index 0002d8b604..931c6d05ca 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGStatusBuilder.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGStatusBuilder.java @@ -61,6 +61,13 @@ public void addVertexProgress(String name, ProgressBuilder progress) { getBuilder().addVertexProgress(builder.build()); } + //TODO: let this be a map of values in protobuf 3.x + public void setMemoryUsage(long memoryUsedByAM, long memoryUsedByTasks) { + Builder builder = getBuilder(); + builder.setMemoryUsedByAM(memoryUsedByAM); + builder.setMemoryUsedByTasks(memoryUsedByTasks); + } + public DAGStatusProto getProto() { return getBuilder().build(); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java index 3a99456ed7..ac2f7605ae 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java @@ -678,4 +678,12 @@ public String getCompletedLogsUrl(int taskCommId, TezTaskAttemptID attemptID, No return null; } + @Override + public long getTotalUsedMemory() { + long totalUsedMemory = 0; + for (int i = 0; i < taskCommunicators.length; i++) { + totalUsedMemory += taskCommunicators[i].getTaskCommunicator().getTotalUsedMemory(); + } + return totalUsedMemory; + } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManagerInterface.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManagerInterface.java index 254e74c734..150977a94e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManagerInterface.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManagerInterface.java @@ -54,4 +54,5 @@ public interface TaskCommunicatorManagerInterface { String getCompletedLogsUrl(int taskCommId, TezTaskAttemptID attemptID, NodeId containerNodeId); + long getTotalUsedMemory(); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java index 6d69d36014..48aee3103b 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java @@ -104,6 +104,7 @@ public static final class ContainerInfo { Credentials credentials = null; boolean credentialsChanged = false; boolean taskPulled = false; + long usedMemory = 0; void reset() { taskSpec = null; @@ -382,6 +383,7 @@ public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request) throws IOExce response.setLastRequestId(requestId); containerInfo.lastRequestId = requestId; containerInfo.lastResponse = response; + containerInfo.usedMemory = request.getUsedMemory(); return response; } @@ -466,4 +468,8 @@ protected ContainerInfo getContainerInfo(ContainerId containerId) { protected ContainerId getContainerForAttempt(TezTaskAttemptID taskAttemptId) { return attemptToContainerMap.get(taskAttemptId); } + + public long getTotalUsedMemory() { + return registeredContainers.values().stream().mapToLong(c -> c.usedMemory).sum(); + } } \ No newline at end of file diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java index 026ca29b7a..07715cdfe7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java @@ -19,6 +19,8 @@ package org.apache.tez.dag.app.dag.impl; import java.io.IOException; +import java.lang.management.ManagementFactory; +import java.lang.management.MemoryMXBean; import java.net.URL; import java.security.PrivilegedExceptionAction; import java.util.ArrayList; @@ -244,6 +246,8 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG, private static final CommitCompletedTransition COMMIT_COMPLETED_TRANSITION = new CommitCompletedTransition(); + private final MemoryMXBean memoryMXBean = ManagementFactory.getMemoryMXBean(); + protected static final StateMachineFactory stateMachineFactory @@ -940,6 +944,10 @@ public DAGStatusBuilder getDAGStatus(Set statusOptions) { if (statusOptions.contains(StatusGetOpts.GET_COUNTERS)) { status.setDAGCounters(getAllCounters()); } + if (statusOptions.contains(StatusGetOpts.GET_MEMORY_USAGE)) { + status.setMemoryUsage(memoryMXBean.getHeapMemoryUsage().getUsed(), + taskCommunicatorManagerInterface.getTotalUsedMemory()); + } return status; } finally { readLock.unlock(); diff --git a/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicator.java b/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicator.java index fceddf2522..be6ad68d55 100644 --- a/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicator.java +++ b/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicator.java @@ -237,4 +237,13 @@ public String getCompletedLogsUrl(TezTaskAttemptID attemptID, NodeId containerNo return null; } + /** + * Return the amount of memory used by the containers. Each container is supposed to refresh + * its current state via heartbeat requests, and the TaskCommunicator implementation is supposed + * to aggregate this properly. + * @return memory in MB + */ + public long getTotalUsedMemory() { + return 0; + } } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java index 9bceaece93..b3ddaa0f86 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java @@ -431,7 +431,7 @@ public Void call() throws Exception { EventProducerConsumerType.SYSTEM, cData.vName, "", cData.taId), MockDAGAppMaster.this.getContext().getClock().getTime())); TezHeartbeatRequest request = new TezHeartbeatRequest(cData.numUpdates, events, - cData.nextPreRoutedFromEventId, cData.cIdStr, cData.taId, cData.nextFromEventId, 50000); + cData.nextPreRoutedFromEventId, cData.cIdStr, cData.taId, cData.nextFromEventId, 50000, 0); doHeartbeat(request, cData); } else if (version != null && cData.taId.getId() <= version.intValue()) { preemptContainer(cData); @@ -443,7 +443,7 @@ public Void call() throws Exception { EventProducerConsumerType.SYSTEM, cData.vName, "", cData.taId), MockDAGAppMaster.this.getContext().getClock().getTime())); TezHeartbeatRequest request = new TezHeartbeatRequest(++cData.numUpdates, events, - cData.nextPreRoutedFromEventId, cData.cIdStr, cData.taId, cData.nextFromEventId, 10000); + cData.nextPreRoutedFromEventId, cData.cIdStr, cData.taId, cData.nextFromEventId, 10000, 0); doHeartbeat(request, cData); cData.clear(); } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatRequest.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatRequest.java index 7ed89f813b..fd5bc17521 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatRequest.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatRequest.java @@ -39,13 +39,14 @@ public class TezHeartbeatRequest implements Writable { private int preRoutedStartIndex; private int maxEvents; private long requestId; + private long usedMemory; public TezHeartbeatRequest() { } public TezHeartbeatRequest(long requestId, List events, int preRoutedStartIndex, String containerIdentifier, - TezTaskAttemptID taskAttemptID, int startIndex, int maxEvents) { + TezTaskAttemptID taskAttemptID, int startIndex, int maxEvents, long usedMemory) { this.containerIdentifier = containerIdentifier; this.requestId = requestId; this.events = Collections.unmodifiableList(events); @@ -53,6 +54,7 @@ public TezHeartbeatRequest(long requestId, List events, this.preRoutedStartIndex = preRoutedStartIndex; this.maxEvents = maxEvents; this.currentTaskAttemptID = taskAttemptID; + this.usedMemory = usedMemory; } public String getContainerIdentifier() { @@ -83,6 +85,10 @@ public TezTaskAttemptID getCurrentTaskAttemptID() { return currentTaskAttemptID; } + public long getUsedMemory() { + return usedMemory; + } + @Override public void write(DataOutput out) throws IOException { if (events != null) { @@ -105,6 +111,7 @@ public void write(DataOutput out) throws IOException { out.writeInt(maxEvents); out.writeLong(requestId); Text.writeString(out, containerIdentifier); + out.writeLong(usedMemory); } @Override @@ -128,6 +135,7 @@ public void readFields(DataInput in) throws IOException { maxEvents = in.readInt(); requestId = in.readLong(); containerIdentifier = Text.readString(in); + usedMemory = in.readLong(); } @Override @@ -140,6 +148,7 @@ public String toString() { + ", maxEventsToGet=" + maxEvents + ", taskAttemptId=" + currentTaskAttemptID + ", eventCount=" + (events != null ? events.size() : 0) + + ", usedMemory=" + usedMemory + " }"; } } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskReporter.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskReporter.java index 978942d4e7..eeb24343be 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskReporter.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskReporter.java @@ -19,6 +19,8 @@ package org.apache.tez.runtime.task; import java.io.IOException; +import java.lang.management.ManagementFactory; +import java.lang.management.MemoryMXBean; import java.util.ArrayList; import java.util.Collection; import java.util.List; @@ -136,6 +138,7 @@ static class HeartbeatCallable implements Callable { private static final int LOG_COUNTER_START_INTERVAL = 5000; // 5 seconds private static final float LOG_COUNTER_BACKOFF = 1.3f; + private static final int HEAP_MEMORY_USAGE_UPDATE_INTERVAL = 5000; // 5 seconds private final RuntimeTask task; private final EventMetaData updateEventMetadata; @@ -157,6 +160,10 @@ static class HeartbeatCallable implements Callable { private final ReentrantLock lock = new ReentrantLock(); private final Condition condition = lock.newCondition(); + private final MemoryMXBean memoryMXBean = ManagementFactory.getMemoryMXBean(); + private long usedMemory = 0; + private long heapMemoryUsageUpdatedTime = System.currentTimeMillis() - HEAP_MEMORY_USAGE_UPDATE_INTERVAL; + /* * Keeps track of regular timed heartbeats. Is primarily used as a timing mechanism to send / * log counters. @@ -263,7 +270,7 @@ private synchronized ResponseWrapper heartbeat(Collection eventsArg) t int fromPreRoutedEventId = task.getNextPreRoutedEventId(); int maxEvents = Math.min(maxEventsToGet, task.getMaxEventsToHandle()); TezHeartbeatRequest request = new TezHeartbeatRequest(requestId, events, fromPreRoutedEventId, - containerIdStr, task.getTaskAttemptID(), fromEventId, maxEvents); + containerIdStr, task.getTaskAttemptID(), fromEventId, maxEvents, getUsedMemory()); LOG.debug("Sending heartbeat to AM, request={}", request); maybeLogCounters(); @@ -305,6 +312,15 @@ private synchronized ResponseWrapper heartbeat(Collection eventsArg) t return new ResponseWrapper(false, numEventsReceived); } + private long getUsedMemory() { + long now = System.currentTimeMillis(); + if (now - heapMemoryUsageUpdatedTime > HEAP_MEMORY_USAGE_UPDATE_INTERVAL) { + usedMemory = memoryMXBean.getHeapMemoryUsage().getUsed(); + heapMemoryUsageUpdatedTime = now; + } + return usedMemory; + } + public void markComplete() { // Notify to clear pending events, if any. lock.lock(); diff --git a/tez-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java b/tez-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java index 96b7bbf655..95d5bcf30b 100644 --- a/tez-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java +++ b/tez-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; import java.io.File; import java.io.IOException; @@ -211,13 +212,18 @@ public void testSleepJob() throws TezException, IOException, InterruptedExceptio + dagStatus.getState()); Thread.sleep(500l); dagStatus = dagClient.getDAGStatus(null); + assertTrue("Memory used by AM is supposed to be 0 if not requested", dagStatus.getMemoryUsedByAM() == 0); + assertTrue("Memory used by tasks is supposed to be 0 if not requested", dagStatus.getMemoryUsedByTasks() == 0); } - dagStatus = dagClient.getDAGStatus(Sets.newHashSet(StatusGetOpts.GET_COUNTERS)); + dagStatus = dagClient.getDAGStatus(Sets.newHashSet(StatusGetOpts.GET_COUNTERS, StatusGetOpts.GET_MEMORY_USAGE)); assertEquals(DAGStatus.State.SUCCEEDED, dagStatus.getState()); assertNotNull(dagStatus.getDAGCounters()); assertNotNull(dagStatus.getDAGCounters().getGroup(FileSystemCounter.class.getName())); assertNotNull(dagStatus.getDAGCounters().findCounter(TaskCounter.GC_TIME_MILLIS)); + assertTrue("Memory used by AM is supposed to be >0", dagStatus.getMemoryUsedByAM() > 0); + assertTrue("Memory used by tasks is supposed to be >0", dagStatus.getMemoryUsedByTasks() > 0); + ExampleDriver.printDAGStatus(dagClient, new String[] { "SleepVertex" }, true, true); tezSession.stop(); } From 211b59bf4a9692491de0c8306c94407d6a6536f4 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Fri, 19 Nov 2021 11:09:47 +0100 Subject: [PATCH 337/512] TEZ-4338: Tez should consider node information to realize OUTPUT_LOST as early as possible - upstream(mapper) problems (#152) (Laszlo Bodor reviewed by Rajesh Balamohan) --- .../apache/tez/dag/api/TezConfiguration.java | 18 +++ .../api/events/InputReadErrorEvent.java | 29 +++-- .../api/ContainerLauncherContext.java | 2 +- tez-api/src/main/proto/Events.proto | 1 + .../org/apache/tez/dag/app/dag/Vertex.java | 5 + .../event/TaskAttemptEventOutputFailed.java | 4 +- .../org/apache/tez/dag/app/dag/impl/Edge.java | 15 ++- .../tez/dag/app/dag/impl/TaskAttemptImpl.java | 101 +++++++++++----- .../tez/dag/app/dag/impl/VertexImpl.java | 27 +++++ .../tez/dag/app/rm/node/AMNodeImpl.java | 6 + .../tez/dag/app/rm/node/AMNodeTracker.java | 13 +- .../dag/app/rm/node/PerSourceNodeTracker.java | 4 + .../tez/dag/app/dag/impl/TestTaskAttempt.java | 77 +++++++++++- .../apache/tez/runtime/api/impl/TezEvent.java | 4 +- .../library/api/TezRuntimeConfiguration.java | 27 ++++- .../library/common/shuffle/Fetcher.java | 33 ++++-- .../shuffle/FetcherErrorTestingConfig.java | 111 ++++++++++++++++++ .../shuffle/FetcherWithInjectableErrors.java | 87 ++++++++++++++ .../common/shuffle/impl/ShuffleManager.java | 14 ++- .../orderedgrouped/FetcherOrderedGrouped.java | 13 +- ...herOrderedGroupedWithInjectableErrors.java | 68 +++++++++++ .../orderedgrouped/ShuffleScheduler.java | 30 +++-- 22 files changed, 608 insertions(+), 81 deletions(-) create mode 100644 tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherErrorTestingConfig.java create mode 100644 tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherWithInjectableErrors.java create mode 100644 tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGroupedWithInjectableErrors.java diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 17a826e7e7..5063b04eac 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -300,6 +300,24 @@ public TezConfiguration(boolean loadDefaults) { TEZ_AM_PREFIX + "max.allowed.time-sec.for-read-error"; public static final int TEZ_AM_MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC_DEFAULT = 300; + /** + * Double value. Assuming that a certain number of downstream hosts reported fetch failure for a + * given upstream host, this config drives the max allowed ratio of (downstream hosts) / (all hosts). + * The total number of used hosts are tracked by AMNodeTracker, which divides the distinct number of + * downstream hosts blaming source(upstream) tasks in a given vertex. If the fraction is beyond this + * limit, the upstream task attempt is marked as failed (so blamed for the fetch failure). + * E.g. if this set to 0.2, in case of 3 different hosts reporting fetch failure + * for the same upstream host in a cluster which currently utilizes 10 nodes, the upstream task + * is immediately blamed for the fetch failure. + * + * Expert level setting. + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty(type="integer") + public static final String TEZ_AM_MAX_ALLOWED_DOWNSTREAM_HOST_FAILURES_FRACTION = + TEZ_AM_PREFIX + "max.allowed.downstream.host.failures.fraction"; + public static final double TEZ_AM_MAX_ALLOWED_DOWNSTREAM_HOST_FAILURES_FRACTION_DEFAULT = 0.2; + /** * Boolean value. Determines when the final outputs to data sinks are committed. Commit is an * output specific operation and typically involves making the output visible for consumption. diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java index 8ef50ebac4..1d0c44d6c1 100644 --- a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java +++ b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java @@ -62,8 +62,13 @@ public final class InputReadErrorEvent extends Event { */ private final boolean isDiskErrorAtSource; + /** + * The localhostName of the destination task attempt. + */ + private final String destinationLocalhostName; + private InputReadErrorEvent(final String diagnostics, final int index, final int version, - final int numFailures, boolean isLocalFetch, boolean isDiskErrorAtSource) { + final int numFailures, boolean isLocalFetch, boolean isDiskErrorAtSource, String destinationLocalhostName) { super(); this.diagnostics = diagnostics; this.index = index; @@ -71,24 +76,30 @@ private InputReadErrorEvent(final String diagnostics, final int index, final int this.numFailures = numFailures; this.isLocalFetch = isLocalFetch; this.isDiskErrorAtSource = isDiskErrorAtSource; + this.destinationLocalhostName = destinationLocalhostName; } public static InputReadErrorEvent create(String diagnostics, int index, int version, boolean isLocalFetch, boolean isDiskErrorAtSource) { - return create(diagnostics, index, version, 1, isLocalFetch, isDiskErrorAtSource); + return create(diagnostics, index, version, 1, isLocalFetch, isDiskErrorAtSource, null); } public static InputReadErrorEvent create(String diagnostics, int index, int version) { - return create(diagnostics, index, version, 1, false, false); + return create(diagnostics, index, version, 1, false, false, null); + } + + public static InputReadErrorEvent create(String diagnostics, int index, int version, boolean isLocalFetch, + boolean isDiskErrorAtSource, String destinationLocalhostName) { + return create(diagnostics, index, version, 1, isLocalFetch, isDiskErrorAtSource, destinationLocalhostName); } /** * Create an InputReadErrorEvent. */ - public static InputReadErrorEvent create(final String diagnostics, final int index, - final int version, final int numFailures, boolean isLocalFetch, boolean isDiskErrorAtSource) { - return new InputReadErrorEvent(diagnostics, index, version, numFailures, isLocalFetch, - isDiskErrorAtSource); + public static InputReadErrorEvent create(final String diagnostics, final int index, final int version, + final int numFailures, boolean isLocalFetch, boolean isDiskErrorAtSource, String destinationLocalhostName) { + return new InputReadErrorEvent(diagnostics, index, version, numFailures, isLocalFetch, isDiskErrorAtSource, + destinationLocalhostName); } public String getDiagnostics() { @@ -118,6 +129,10 @@ public boolean isDiskErrorAtSource() { return isDiskErrorAtSource; } + public String getDestinationLocalhostName(){ + return destinationLocalhostName; + } + @Override public int hashCode() { return Objects.hash(index, version); diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java index ed1d58f78a..16d54f05b7 100644 --- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java +++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java @@ -80,7 +80,7 @@ void containerCompleted(ContainerId containerId, int exitStatus, String diagnost * Get the number of nodes being handled by the specified source * * @param sourceName the relevant source name - * @return the initial payload + * @return the number of nodes */ int getNumNodes(String sourceName); diff --git a/tez-api/src/main/proto/Events.proto b/tez-api/src/main/proto/Events.proto index e041c33f60..9949b0bc8c 100644 --- a/tez-api/src/main/proto/Events.proto +++ b/tez-api/src/main/proto/Events.proto @@ -41,6 +41,7 @@ message InputReadErrorEventProto { optional int32 version = 3; optional bool is_local_fetch = 4; optional bool is_disk_error_at_source = 5; + optional string destination_localhost_name = 6; } message InputFailedEventProto { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java index ba3079d4e1..ff83e19a85 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java @@ -230,8 +230,13 @@ interface VertexConfig { * @return tez.am.max.allowed.time-sec.for-read-error. */ int getMaxAllowedTimeForTaskReadErrorSec(); + /** + * @return tez.am.max.allowed.downstream.host.failures.fraction. + */ + double getMaxAllowedDownstreamHostFailuresFraction(); } void incrementRejectedTaskAttemptCount(); int getRejectedTaskAttemptCount(); + Map> getDownstreamBlamingHosts(); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventOutputFailed.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventOutputFailed.java index 6bc110a1a0..fbdd2305c7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventOutputFailed.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventOutputFailed.java @@ -28,9 +28,9 @@ public class TaskAttemptEventOutputFailed extends TaskAttemptEvent private TezEvent inputFailedEvent; private int consumerTaskNumber; - public TaskAttemptEventOutputFailed(TezTaskAttemptID attemptId, + public TaskAttemptEventOutputFailed(TezTaskAttemptID sourceTaskAttemptId, TezEvent tezEvent, int numConsumers) { - super(attemptId, TaskAttemptEventType.TA_OUTPUT_FAILED); + super(sourceTaskAttemptId, TaskAttemptEventType.TA_OUTPUT_FAILED); this.inputFailedEvent = tezEvent; this.consumerTaskNumber = numConsumers; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java index 848b49199d..99b56fbf07 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java @@ -374,7 +374,7 @@ public void sendTezEventToSourceTasks(TezEvent tezEvent) throws AMUserCodeExcept if (!bufferEvents.get()) { switch (tezEvent.getEventType()) { case INPUT_READ_ERROR_EVENT: - InputReadErrorEvent event = (InputReadErrorEvent) tezEvent.getEvent(); + InputReadErrorEvent inputReadErrorEvent = (InputReadErrorEvent) tezEvent.getEvent(); TezTaskAttemptID destAttemptId = tezEvent.getSourceInfo() .getTaskAttemptID(); int destTaskIndex = destAttemptId.getTaskID().getId(); @@ -383,10 +383,10 @@ public void sendTezEventToSourceTasks(TezEvent tezEvent) throws AMUserCodeExcept try { if (onDemandRouting) { srcTaskIndex = ((EdgeManagerPluginOnDemand) edgeManager).routeInputErrorEventToSource( - destTaskIndex, event.getIndex()); + destTaskIndex, inputReadErrorEvent.getIndex()); } else { - srcTaskIndex = edgeManager.routeInputErrorEventToSource(event, - destTaskIndex, event.getIndex()); + srcTaskIndex = edgeManager.routeInputErrorEventToSource(inputReadErrorEvent, + destTaskIndex, inputReadErrorEvent.getIndex()); } Preconditions.checkArgument(srcTaskIndex >= 0, "SourceTaskIndex should not be negative," @@ -414,11 +414,10 @@ public void sendTezEventToSourceTasks(TezEvent tezEvent) throws AMUserCodeExcept " edgeManager=" + edgeManager.getClass().getName()); } TezTaskID srcTaskId = srcTask.getTaskId(); - int taskAttemptIndex = event.getVersion(); + int srcTaskAttemptIndex = inputReadErrorEvent.getVersion(); TezTaskAttemptID srcTaskAttemptId = TezTaskAttemptID.getInstance(srcTaskId, - taskAttemptIndex); - sendEvent(new TaskAttemptEventOutputFailed(srcTaskAttemptId, - tezEvent, numConsumers)); + srcTaskAttemptIndex); + sendEvent(new TaskAttemptEventOutputFailed(srcTaskAttemptId, tezEvent, numConsumers)); break; default: throw new TezUncheckedException("Unhandled tez event type: " diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java index 593ea6cabf..e6b9e82000 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java @@ -194,7 +194,7 @@ public static DataEventDependencyInfo fromProto(DataEventDependencyInfoProto pro private Container container; private long allocationTime; private ContainerId containerId; - private NodeId containerNodeId; + protected NodeId containerNodeId; private String nodeHttpAddress; private String nodeRackName; @@ -1793,62 +1793,95 @@ protected static class OutputReportedFailedTransition implements MultipleArcTransition { @Override - public TaskAttemptStateInternal transition(TaskAttemptImpl attempt, + public TaskAttemptStateInternal transition(TaskAttemptImpl sourceAttempt, TaskAttemptEvent event) { TaskAttemptEventOutputFailed outputFailedEvent = (TaskAttemptEventOutputFailed) event; - TezEvent tezEvent = outputFailedEvent.getInputFailedEvent(); - TezTaskAttemptID failedDestTaId = tezEvent.getSourceInfo().getTaskAttemptID(); - InputReadErrorEvent readErrorEvent = (InputReadErrorEvent)tezEvent.getEvent(); + TezEvent inputFailedEvent = outputFailedEvent.getInputFailedEvent(); + TezTaskAttemptID failedDestTaId = inputFailedEvent.getSourceInfo().getTaskAttemptID(); + + InputReadErrorEvent readErrorEvent = (InputReadErrorEvent)inputFailedEvent.getEvent(); int failedInputIndexOnDestTa = readErrorEvent.getIndex(); - if (readErrorEvent.getVersion() != attempt.getID().getId()) { - throw new TezUncheckedException(attempt.getID() + + if (readErrorEvent.getVersion() != sourceAttempt.getID().getId()) { + throw new TezUncheckedException(sourceAttempt.getID() + " incorrectly blamed for read error from " + failedDestTaId + " at inputIndex " + failedInputIndexOnDestTa + " version" + readErrorEvent.getVersion()); } - LOG.info(attempt.getID() - + " blamed for read error from " + failedDestTaId - + " at inputIndex " + failedInputIndexOnDestTa); - long time = attempt.clock.getTime(); - Long firstErrReportTime = attempt.uniquefailedOutputReports.get(failedDestTaId); + // source host: where the data input is supposed to come from + String sHost = sourceAttempt.getNodeId().getHost(); + // destination: where the data is tried to be fetched to + String dHost = readErrorEvent.getDestinationLocalhostName(); + + LOG.info("{} (on {}) blamed for read error from {} (on {}) at inputIndex {}", sourceAttempt.getID(), + sHost, failedDestTaId, dHost, failedInputIndexOnDestTa); + + boolean tooManyDownstreamHostsBlamedTheSameUpstreamHost = false; + Map> downstreamBlamingHosts = sourceAttempt.getVertex().getDownstreamBlamingHosts(); + if (!downstreamBlamingHosts.containsKey(sHost)) { + LOG.info("Host {} is blamed for fetch failure from {} for the first time", sHost, dHost); + downstreamBlamingHosts.put(sHost, new HashSet()); + } + + downstreamBlamingHosts.get(sHost).add(dHost); + int currentNumberOfFailingDownstreamHosts = downstreamBlamingHosts.get(sHost).size(); + int numNodes = getNumNodes(sourceAttempt); + float hostFailureFraction = numNodes > 0 ? ((float) currentNumberOfFailingDownstreamHosts) / numNodes : 0; + double maxAllowedHostFailureFraction = sourceAttempt.getVertex().getVertexConfig() + .getMaxAllowedDownstreamHostFailuresFraction(); + + if (hostFailureFraction > maxAllowedHostFailureFraction) { + LOG.info("Host will be marked fail: {} because of host failure fraction {} is beyond the limit {}", sHost, + hostFailureFraction, maxAllowedHostFailureFraction); + tooManyDownstreamHostsBlamedTheSameUpstreamHost = true; + } + long time = sourceAttempt.clock.getTime(); + + Long firstErrReportTime = sourceAttempt.uniquefailedOutputReports.get(failedDestTaId); if (firstErrReportTime == null) { - attempt.uniquefailedOutputReports.put(failedDestTaId, time); + sourceAttempt.uniquefailedOutputReports.put(failedDestTaId, time); firstErrReportTime = time; } - int maxAllowedOutputFailures = attempt.getVertex().getVertexConfig() + int maxAllowedOutputFailures = sourceAttempt.getVertex().getVertexConfig() .getMaxAllowedOutputFailures(); - int maxAllowedTimeForTaskReadErrorSec = attempt.getVertex() + int maxAllowedTimeForTaskReadErrorSec = sourceAttempt.getVertex() .getVertexConfig().getMaxAllowedTimeForTaskReadErrorSec(); - double maxAllowedOutputFailuresFraction = attempt.getVertex() + double maxAllowedOutputFailuresFraction = sourceAttempt.getVertex() .getVertexConfig().getMaxAllowedOutputFailuresFraction(); int readErrorTimespanSec = (int)((time - firstErrReportTime)/1000); boolean crossTimeDeadline = readErrorTimespanSec >= maxAllowedTimeForTaskReadErrorSec; - int runningTasks = attempt.appContext.getCurrentDAG().getVertex( + int runningTasks = sourceAttempt.appContext.getCurrentDAG().getVertex( failedDestTaId.getTaskID().getVertexID()).getRunningTasks(); - float failureFraction = runningTasks > 0 ? ((float) attempt.uniquefailedOutputReports.size()) / runningTasks : 0; + float failureFraction = + runningTasks > 0 ? ((float) sourceAttempt.uniquefailedOutputReports.size()) / runningTasks : 0; boolean withinFailureFractionLimits = (failureFraction <= maxAllowedOutputFailuresFraction); boolean withinOutputFailureLimits = - (attempt.uniquefailedOutputReports.size() < maxAllowedOutputFailures); + (sourceAttempt.uniquefailedOutputReports.size() < maxAllowedOutputFailures); // If needed we can launch a background task without failing this task // to generate a copy of the output just in case. // If needed we can consider only running consumer tasks if (!crossTimeDeadline && withinFailureFractionLimits && withinOutputFailureLimits - && !(readErrorEvent.isLocalFetch() || readErrorEvent.isDiskErrorAtSource())) { - return attempt.getInternalState(); + && !(readErrorEvent.isLocalFetch() || readErrorEvent.isDiskErrorAtSource()) + && !tooManyDownstreamHostsBlamedTheSameUpstreamHost) { + return sourceAttempt.getInternalState(); } - String message = attempt.getID() + " being failed for too many output errors. " + String message = sourceAttempt.getID() + " being failed for too many output errors. " + "failureFraction=" + failureFraction + ", MAX_ALLOWED_OUTPUT_FAILURES_FRACTION=" + maxAllowedOutputFailuresFraction - + ", uniquefailedOutputReports=" + attempt.uniquefailedOutputReports.size() + + ", uniquefailedOutputReports=" + sourceAttempt.uniquefailedOutputReports.size() + ", MAX_ALLOWED_OUTPUT_FAILURES=" + maxAllowedOutputFailures + + ", hostFailureFraction=" + hostFailureFraction + + " (" + currentNumberOfFailingDownstreamHosts + " / " + numNodes + ")" + + ", MAX_ALLOWED_DOWNSTREAM_HOST_FAILURES_FRACTION=" + + maxAllowedHostFailureFraction + ", MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC=" + maxAllowedTimeForTaskReadErrorSec + ", readErrorTimespan=" + readErrorTimespanSec @@ -1856,22 +1889,34 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl attempt, + ", isDiskErrorAtSource=" + readErrorEvent.isDiskErrorAtSource(); LOG.info(message); - attempt.addDiagnosticInfo(message); + sourceAttempt.addDiagnosticInfo(message); // send input failed event - attempt.sendInputFailedToConsumers(); + sourceAttempt.sendInputFailedToConsumers(); // Not checking for leafVertex since a READ_ERROR should only be reported for intermediate tasks. - if (attempt.getInternalState() == TaskAttemptStateInternal.SUCCEEDED) { + if (sourceAttempt.getInternalState() == TaskAttemptStateInternal.SUCCEEDED) { (new TerminatedAfterSuccessHelper(FAILED_HELPER)).transition( - attempt, event); + sourceAttempt, event); return TaskAttemptStateInternal.FAILED; } else { (new TerminatedWhileRunningTransition(FAILED_HELPER)).transition( - attempt, event); + sourceAttempt, event); return TaskAttemptStateInternal.FAIL_IN_PROGRESS; } // TODO at some point. Nodes may be interested in FetchFailure info. // Can be used to blacklist nodes. } + + private int getNumNodes(TaskAttemptImpl sourceAttempt) { + Vertex vertex = sourceAttempt.getVertex(); + String taskSchedulerName = vertex.getServicePluginInfo().getTaskSchedulerName(); + int sourceIndex = vertex.getAppContext().getTaskScheduerIdentifier(taskSchedulerName); + int numActiveNodes = vertex.getAppContext().getNodeTracker().getNumActiveNodes(sourceIndex); + if (LOG.isDebugEnabled()) { + int numAllNodes = vertex.getAppContext().getNodeTracker().getNumNodes(sourceIndex); + LOG.debug("Getting nodes, active/all: {}/{}", numActiveNodes, numAllNodes); + } + return numActiveNodes; + } } @VisibleForTesting diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index 912339e15e..f22f6de250 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -264,6 +264,13 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl final ServicePluginInfo servicePluginInfo; + /* + * For every upstream host (as map keys) contains every unique downstream hostnames that reported INPUT_READ_ERROR. + * This map helps to decide if there is a problem with the host that produced the map outputs. There is an assumption + * that if multiple downstream hosts report input errors for the same upstream host, then it's likely that the output + * has to be blamed and needs to rerun. + */ + private final Map> downstreamBlamingHosts = Maps.newHashMap(); private final float maxFailuresPercent; private boolean logSuccessDiagnostics = false; @@ -4833,6 +4840,10 @@ static class VertexConfigImpl implements VertexConfig { * See tez.am.max.allowed.time-sec.for-read-error. */ private final int maxAllowedTimeForTaskReadErrorSec; + /** + * See tez.am.max.allowed.downstream.host.failures.fraction. + */ + private final double maxAllowedDownstreamHostFailuresFraction; public VertexConfigImpl(Configuration conf) { this.maxFailedTaskAttempts = conf.getInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS, @@ -4857,6 +4868,10 @@ public VertexConfigImpl(Configuration conf) { this.maxAllowedTimeForTaskReadErrorSec = conf.getInt( TezConfiguration.TEZ_AM_MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC, TezConfiguration.TEZ_AM_MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC_DEFAULT); + + this.maxAllowedDownstreamHostFailuresFraction = conf.getDouble( + TezConfiguration.TEZ_AM_MAX_ALLOWED_DOWNSTREAM_HOST_FAILURES_FRACTION, + TezConfiguration.TEZ_AM_MAX_ALLOWED_DOWNSTREAM_HOST_FAILURES_FRACTION_DEFAULT); } @Override @@ -4899,8 +4914,20 @@ public boolean getTaskRescheduleRelaxedLocality() { @Override public int getMaxAllowedTimeForTaskReadErrorSec() { return maxAllowedTimeForTaskReadErrorSec; } + + /** + * @return maxAllowedDownstreamHostsReportingFetchFailure. + */ + @Override public double getMaxAllowedDownstreamHostFailuresFraction() { + return maxAllowedDownstreamHostFailuresFraction; + } } @Override public AbstractService getSpeculator() { return speculator; } + + @Override + public Map> getDownstreamBlamingHosts(){ + return downstreamBlamingHosts; + } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java index df19534bfc..26796d2af4 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java @@ -509,4 +509,10 @@ public void dagComplete(DAG dag) { this.writeLock.unlock(); } } + + public String toString() { + return String.format( + "{AMNodeImpl: nodeId: %s, state: %s, containers: %d, completed containers: %d, healthy: %s, blackListed: %s}", + nodeId, getState(), getContainers().size(), completedContainers.size(), !isUnhealthy(), isBlacklisted()); + } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java index 1536170fac..8c81cb52c9 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java @@ -134,6 +134,17 @@ public int getNumNodes(int schedulerId) { return perSourceNodeTrackers.get(schedulerId).getNumNodes(); } + /** + * Retrieve the number of nodes in ACTIVE state. This number is suitable for deciding + * how many nodes can be potentially used for running containers at the moment. + * + * @param schedulerId the schedulerId for which the node count is required + * @return the number of nodes from the scheduler being in ACTIVE state + */ + public int getNumActiveNodes(int schedulerId) { + return perSourceNodeTrackers.get(schedulerId).getNumActiveNodes(); + } + @Private @VisibleForTesting public boolean isBlacklistingIgnored(int schedulerId) { @@ -158,6 +169,4 @@ private PerSourceNodeTracker getAndCreateIfNeededPerSourceTracker(int schedulerI } return nodeTracker; } - - } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/PerSourceNodeTracker.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/PerSourceNodeTracker.java index 74c6176e4c..9906644fe1 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/PerSourceNodeTracker.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/PerSourceNodeTracker.java @@ -84,6 +84,10 @@ public int getNumNodes() { return nodeMap.size(); } + public int getNumActiveNodes() { + return (int) nodeMap.values().stream().filter(node -> node.getState() == AMNodeState.ACTIVE).count(); + } + public void handle(AMNodeEvent rEvent) { // No synchronization required until there's multiple dispatchers. NodeId nodeId = rEvent.getNodeId(); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java index 6862bec2ee..7a2a05fb26 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java @@ -108,6 +108,7 @@ import org.apache.tez.dag.app.rm.AMSchedulerEventTALaunchRequest; import org.apache.tez.dag.app.rm.container.AMContainerMap; import org.apache.tez.dag.app.rm.container.ContainerContextMatcher; +import org.apache.tez.dag.app.rm.node.AMNodeTracker; import org.apache.tez.dag.history.DAGHistoryEvent; import org.apache.tez.dag.history.HistoryEventHandler; import org.apache.tez.dag.history.events.TaskAttemptFinishedEvent; @@ -131,6 +132,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.collect.Maps; + @SuppressWarnings({ "unchecked", "rawtypes" }) public class TestTaskAttempt { @@ -174,9 +177,16 @@ public void setupTest() { private void createMockVertex(Configuration conf) { mockVertex = mock(Vertex.class); + when(mockVertex.getDownstreamBlamingHosts()).thenReturn(Maps.newHashMap()); when(mockVertex.getServicePluginInfo()).thenReturn(servicePluginInfo); - when(mockVertex.getVertexConfig()).thenReturn( - new VertexImpl.VertexConfigImpl(conf)); + when(mockVertex.getVertexConfig()).thenReturn(new VertexImpl.VertexConfigImpl(conf)); + AppContext appContext = mock(AppContext.class); + when(appContext.getTaskScheduerIdentifier(Mockito.anyString())).thenReturn(0); + when(mockVertex.getAppContext()).thenReturn(appContext); + AMNodeTracker nodeTracker = mock(AMNodeTracker.class); + when(nodeTracker.getNumNodes(Mockito.anyInt())).thenReturn(10); + when(nodeTracker.getNumActiveNodes(Mockito.anyInt())).thenReturn(8); + when(appContext.getNodeTracker()).thenReturn(nodeTracker); } @Test(timeout = 5000) @@ -2173,11 +2183,11 @@ private void testMapTaskFailingForFetchFailureType(boolean isLocalFetch, TezTaskID.getInstance(TezVertexID.getInstance(TezDAGID.getInstance("1", 1, 1), 1), 1); TaskAttemptImpl sourceAttempt = new MockTaskAttemptImpl(taskID, 1, eventHandler, null, new Configuration(), SystemClock.getInstance(), mock(TaskHeartbeatHandler.class), appCtx, - false, null, null, false); + false, null, null, false).setNodeId(NodeId.newInstance("somehost", 0)); // the original read error event, sent by reducer task InputReadErrorEvent inputReadErrorEvent = - InputReadErrorEvent.create("", 0, 1, 1, isLocalFetch, isDiskErrorAtSource); + InputReadErrorEvent.create("", 0, 1, 1, isLocalFetch, isDiskErrorAtSource, null); TezTaskAttemptID destTaskAttemptId = mock(TezTaskAttemptID.class); when(destTaskAttemptId.getTaskID()).thenReturn(mock(TezTaskID.class)); when(destTaskAttemptId.getTaskID().getVertexID()).thenReturn(mock(TezVertexID.class)); @@ -2201,6 +2211,56 @@ private void testMapTaskFailingForFetchFailureType(boolean isLocalFetch, Assert.assertEquals(expectedState, resultState); } + @Test + public void testMapTaskIsBlamedByDownstreamAttemptsFromDifferentHosts() { + EventHandler eventHandler = mock(EventHandler.class); + TezTaskID taskID = TezTaskID.getInstance(TezVertexID.getInstance(TezDAGID.getInstance("1", 1, 1), 1), 1); + TaskAttemptImpl sourceAttempt = new MockTaskAttemptImpl(taskID, 1, eventHandler, null, new Configuration(), + SystemClock.getInstance(), mock(TaskHeartbeatHandler.class), appCtx, false, null, null, false) + .setNodeId(NodeId.newInstance("somehost", 0)); + + // input read error events from 2 different hosts + InputReadErrorEvent inputReadErrorEvent1 = + InputReadErrorEvent.create("", 0, 1, 1, false, false, "downstream_host_1"); + InputReadErrorEvent inputReadErrorEvent2 = + InputReadErrorEvent.create("", 1, 1, 1, false, false, "downstream_host_2"); + + TezTaskAttemptID destTaskAttemptId = mock(TezTaskAttemptID.class); + when(destTaskAttemptId.getTaskID()).thenReturn(mock(TezTaskID.class)); + when(destTaskAttemptId.getTaskID().getVertexID()).thenReturn(mock(TezVertexID.class)); + when(appCtx.getCurrentDAG()).thenReturn(mock(DAG.class)); + when(appCtx.getCurrentDAG().getVertex(Mockito.any(TezVertexID.class))).thenReturn(mock(Vertex.class)); + when(appCtx.getCurrentDAG().getVertex(Mockito.any(TezVertexID.class)).getRunningTasks()).thenReturn(100); + + EventMetaData mockMeta = mock(EventMetaData.class); + when(mockMeta.getTaskAttemptID()).thenReturn(destTaskAttemptId); + + // mapper task succeeded earlier + sourceAttempt.handle(new TaskAttemptEvent(sourceAttempt.getID(), TaskAttemptEventType.TA_DONE)); + Assert.assertEquals(TaskAttemptStateInternal.SUCCEEDED, sourceAttempt.getInternalState()); + + // the event is propagated to map task's event handler + TezEvent tezEvent = new TezEvent(inputReadErrorEvent1, mockMeta); + TaskAttemptEventOutputFailed outputFailedEvent = + new TaskAttemptEventOutputFailed(sourceAttempt.getID(), tezEvent, 11); + TaskAttemptStateInternal resultState = + new TaskAttemptImpl.OutputReportedFailedTransition().transition(sourceAttempt, outputFailedEvent); + // SUCCEEDED, as we haven't reached the host limit fraction + // active nodes: 8, failed hosts: 1, fraction 0.125 (< 0.2) + Assert.assertEquals(TaskAttemptStateInternal.SUCCEEDED, resultState); + + // the second event is propagated to map task's event handler + TezEvent tezEvent2 = new TezEvent(inputReadErrorEvent2, mockMeta); + TaskAttemptEventOutputFailed outputFailedEvent2 = + new TaskAttemptEventOutputFailed(sourceAttempt.getID(), tezEvent2, 11); + TaskAttemptStateInternal resultState2 = + new TaskAttemptImpl.OutputReportedFailedTransition().transition(sourceAttempt, outputFailedEvent2); + + // now it's marked as FAILED + // active nodes: 8, failed hosts: 2, fraction 0.25 (> 0.2) + Assert.assertEquals(TaskAttemptStateInternal.FAILED, resultState2); + } + private Event verifyEventType(List events, Class eventClass, int expectedOccurences) { int count = 0; @@ -2247,9 +2307,14 @@ public MockTaskAttemptImpl(TezTaskID taskId, int attemptNumber, isRescheduled, resource, containerContext, leafVertex, mockTask, locationHint, null, null); } - + boolean inputFailedReported = false; - + + public MockTaskAttemptImpl setNodeId(NodeId nodeId) { + this.containerNodeId = nodeId; + return this; + } + @Override protected Vertex getVertex() { return mockVertex; diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java index ebea9a4f3f..f96a437a49 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java @@ -193,6 +193,7 @@ private void serializeEvent(DataOutput out) throws IOException { .setVersion(ideEvt.getVersion()) .setIsLocalFetch(ideEvt.isLocalFetch()) .setIsDiskErrorAtSource(ideEvt.isDiskErrorAtSource()) + .setDestinationLocalhostName(ideEvt.getDestinationLocalhostName()) .build(); break; case TASK_ATTEMPT_FAILED_EVENT: @@ -298,7 +299,8 @@ private void deserializeEvent(DataInput in) throws IOException { case INPUT_READ_ERROR_EVENT: InputReadErrorEventProto ideProto = InputReadErrorEventProto.parseFrom(input); event = InputReadErrorEvent.create(ideProto.getDiagnostics(), ideProto.getIndex(), - ideProto.getVersion(), ideProto.getIsLocalFetch(), ideProto.getIsDiskErrorAtSource()); + ideProto.getVersion(), ideProto.getIsLocalFetch(), ideProto.getIsDiskErrorAtSource(), + ideProto.getDestinationLocalhostName()); break; case TASK_ATTEMPT_FAILED_EVENT: TaskAttemptFailedEventProto tfProto = diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java index 64e6a71d7e..1c747af2bb 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java @@ -414,6 +414,29 @@ public class TezRuntimeConfiguration { public static final float TEZ_RUNTIME_SHUFFLE_FETCH_BUFFER_PERCENT_DEFAULT = 0.90f; + /** + * Enables fetch failures by a configuration. Should be used for testing only. + */ + @ConfigurationProperty(type = "boolean") + public static final String TEZ_RUNTIME_SHUFFLE_FETCH_ENABLE_TESTING_ERRORS = + TEZ_RUNTIME_PREFIX + "shuffle.fetch.testing.errors.enable"; + public static final boolean TEZ_RUNTIME_SHUFFLE_FETCH_ENABLE_TESTING_ERRORS_DEFAULT = false; + + /** + * Configures the injectable fetch failures, in a form of: + * maphost#mapvertex#probability#comma,separated,features + * Possible values are (fetch fails...): + * "*#*#50": from all map hosts with 50% likelihood + * "_first_#*#80": for the first ever seen map host with 80% likelihood (user doesn't want to use hostnames) + * "host1#*#100": from host1 with 100% likelihood (simulates single node failure) + * "host1#Map_1#100": from host1 for Map 1 source tasks with 100% likelihood + * "host1#Map_1#100#fail_only_first": as above but only for input attempts with index 0 + */ + @ConfigurationProperty(type = "string") + public static final String TEZ_RUNTIME_SHUFFLE_FETCH_TESTING_ERRORS_CONFIG = + TEZ_RUNTIME_PREFIX + "shuffle.fetch.testing.errors.config"; + public static final String TEZ_RUNTIME_SHUFFLE_FETCH_TESTING_ERRORS_CONFIG_DEFAULT = "*#50"; + @ConfigurationProperty(type = "float") public static final String TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT = TEZ_RUNTIME_PREFIX + "shuffle.memory.limit.percent"; @@ -543,7 +566,7 @@ public class TezRuntimeConfiguration { /** - * Share data fetched between tasks running on the same host if applicable + * Share data fetched between tasks running on the same host if applicable. */ @ConfigurationProperty(type = "boolean") public static final String TEZ_RUNTIME_OPTIMIZE_SHARED_FETCH = TEZ_RUNTIME_PREFIX @@ -626,6 +649,8 @@ public class TezRuntimeConfiguration { tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_MAX_ALLOWED_FAILED_FETCH_ATTEMPT_FRACTION); tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_MIN_REQUIRED_PROGRESS_FRACTION); tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_FAILED_CHECK_SINCE_LAST_COMPLETION); + tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_FETCH_TESTING_ERRORS_CONFIG); + tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_FETCH_ENABLE_TESTING_ERRORS); tezRuntimeKeys.add(TEZ_RUNTIME_REPORT_PARTITION_STATS); tezRuntimeKeys.add(TEZ_RUNTIME_INPUT_POST_MERGE_BUFFER_PERCENT); tezRuntimeKeys.add(TEZ_RUNTIME_GROUP_COMPARATOR_CLASS); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java index f295c06b8e..6039df3a2f 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java @@ -58,6 +58,7 @@ import org.apache.tez.common.CallableWithNdc; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.dag.api.TezUncheckedException; +import org.apache.tez.runtime.api.ObjectRegistry; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; import org.apache.tez.runtime.library.common.Constants; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; @@ -144,7 +145,7 @@ public String toString() { private final AtomicBoolean isShutDown = new AtomicBoolean(false); - private final int fetcherIdentifier; + protected final int fetcherIdentifier; // Parameters to track work. private List srcAttempts; @@ -192,7 +193,7 @@ public String getHost() { private final boolean isDebugEnabled = LOG.isDebugEnabled(); - private Fetcher(FetcherCallback fetcherCallback, HttpConnectionParams params, + protected Fetcher(FetcherCallback fetcherCallback, HttpConnectionParams params, FetchedInputAllocator inputManager, ApplicationId appId, int dagIdentifier, JobTokenSecretManager jobTokenSecretManager, String srcNameTrimmed, Configuration conf, RawLocalFileSystem localFs, @@ -527,8 +528,7 @@ private HostFetchResult setupConnection(Collection attem } try { - input = httpConnection.getInputStream(); - httpConnection.validate(); + setupConnectionInternal(host, attempts); //validateConnectionResponse(msgToEncode, encHash); } catch (IOException e) { // ioErrs.increment(1); @@ -556,6 +556,13 @@ private HostFetchResult setupConnection(Collection attem return null; } + + protected void setupConnectionInternal(String host, Collection attempts) + throws IOException, InterruptedException { + input = httpConnection.getInputStream(); + httpConnection.validate(); + } + @VisibleForTesting protected HostFetchResult doHttpFetch(CachingCallBack callback) { @@ -1141,11 +1148,19 @@ public FetcherBuilder(FetcherCallback fetcherCallback, Configuration conf, RawLocalFileSystem localFs, LocalDirAllocator localDirAllocator, Path lockPath, boolean localDiskFetchEnabled, boolean sharedFetchEnabled, - String localHostname, int shufflePort, boolean asyncHttp, boolean verifyDiskChecksum, boolean compositeFetch) { - this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId, dagIdentifier, - jobTokenSecretMgr, srcNameTrimmed, conf, localFs, localDirAllocator, - lockPath, localDiskFetchEnabled, sharedFetchEnabled, localHostname, shufflePort, asyncHttp, - verifyDiskChecksum, compositeFetch); + String localHostname, int shufflePort, boolean asyncHttp, boolean verifyDiskChecksum, boolean compositeFetch, + boolean enableFetcherTestingErrors, ObjectRegistry objectRegistry) { + if (enableFetcherTestingErrors) { + this.fetcher = new FetcherWithInjectableErrors(fetcherCallback, params, inputManager, appId, dagIdentifier, + jobTokenSecretMgr, srcNameTrimmed, conf, localFs, localDirAllocator, + lockPath, localDiskFetchEnabled, sharedFetchEnabled, localHostname, shufflePort, asyncHttp, + verifyDiskChecksum, compositeFetch, objectRegistry); + } else { + this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId, dagIdentifier, + jobTokenSecretMgr, srcNameTrimmed, conf, localFs, localDirAllocator, + lockPath, localDiskFetchEnabled, sharedFetchEnabled, localHostname, shufflePort, asyncHttp, + verifyDiskChecksum, compositeFetch); + } } public FetcherBuilder setHttpConnectionParameters(HttpConnectionParams httpParams) { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherErrorTestingConfig.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherErrorTestingConfig.java new file mode 100644 index 0000000000..ce15a87fb9 --- /dev/null +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherErrorTestingConfig.java @@ -0,0 +1,111 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.runtime.library.common.shuffle; + +import java.util.Arrays; +import java.util.List; +import java.util.Random; + +import org.apache.hadoop.conf.Configuration; +import org.apache.tez.common.TezUtilsInternal; +import org.apache.tez.runtime.api.ObjectRegistry; +import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; +import org.apache.tez.runtime.library.common.InputAttemptIdentifier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class FetcherErrorTestingConfig { + private static final Logger LOG = LoggerFactory.getLogger(FetcherErrorTestingConfig.class); + private static final String KEY_CACHED_HOSTNAME = "FetcherErrorTestingConfig.host"; + + private String hostToFail = "*"; + private String srcNameTrimmedToFail = "*"; + private int probabilityPercent = 50; + private Random random = new Random(); + /** + * Whether to fail only in case of input attempts with index 0, + * this prevents continuous failure, and helps simulating a real-life node failure. + */ + private boolean failForFirstAttemptOnly = false; + private ObjectRegistry objectRegistry; + + public FetcherErrorTestingConfig(Configuration conf, ObjectRegistry objectRegistry) { + String errorConfig = conf.get(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_TESTING_ERRORS_CONFIG, + TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_TESTING_ERRORS_CONFIG_DEFAULT); + String[] configParts = errorConfig.split("#"); + + // e.g. host_1 + if (configParts.length > 0) { + hostToFail = configParts[0]; + } + + // e.g. Map 1 or Map_1, both will work + if (configParts.length > 1) { + srcNameTrimmedToFail = TezUtilsInternal.cleanVertexName(configParts[1]); + } + + // e.g. 50 + if (configParts.length > 2) { + probabilityPercent = Integer.parseInt(configParts[2]); + } + + // e.g. fail_only_first + if (configParts.length > 3) { + List features = Arrays.asList(configParts[3].split(",")); + if (features.contains("fail_only_first")) { + failForFirstAttemptOnly = true; + } + } + + this.objectRegistry = objectRegistry; + if (hostToFail.equals("_first_")) { + String host = (String) objectRegistry.get(KEY_CACHED_HOSTNAME); + if (host != null) { + LOG.info("Get already stored hostname for fetcher test failures: " + host); + hostToFail = host; + } + } + } + + public boolean shouldFail(String host, String srcNameTrimmed, InputAttemptIdentifier inputAttemptIdentifier) { + if (matchHost(host) && matchSourceVertex(srcNameTrimmed)) { + return (!failForFirstAttemptOnly || failForFirstAttemptOnly && inputAttemptIdentifier.getAttemptNumber() == 0) + && random.nextInt(100) < probabilityPercent; + } + return false; + } + + private boolean matchHost(String host) { + if (hostToFail.equals("_first_")) { + objectRegistry.cacheForVertex(KEY_CACHED_HOSTNAME, host); + hostToFail = host; + } + return "*".equals(hostToFail) || host.equalsIgnoreCase(hostToFail); + } + + private boolean matchSourceVertex(String srcNameTrimmed) { + return "*".equals(srcNameTrimmedToFail) || srcNameTrimmed.equalsIgnoreCase(srcNameTrimmedToFail); + } + + @Override + public String toString() { + return String.format( + "[FetcherErrorTestingConfig: host: %s, source vertex: %s, probability: %d%%, failForFirstAttemptOnly: %s]", + hostToFail, srcNameTrimmedToFail, probabilityPercent, failForFirstAttemptOnly); + } +} diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherWithInjectableErrors.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherWithInjectableErrors.java new file mode 100644 index 0000000000..951adf9f5d --- /dev/null +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherWithInjectableErrors.java @@ -0,0 +1,87 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.runtime.library.common.shuffle; + +import java.io.IOException; +import java.util.Collection; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RawLocalFileSystem; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.tez.common.security.JobTokenSecretManager; +import org.apache.tez.http.HttpConnectionParams; +import org.apache.tez.runtime.api.ObjectRegistry; +import org.apache.tez.runtime.library.common.InputAttemptIdentifier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class FetcherWithInjectableErrors extends Fetcher { + private static final Logger LOG = LoggerFactory.getLogger(FetcherWithInjectableErrors.class); + + private FetcherErrorTestingConfig fetcherErrorTestingConfig; + private String srcNameTrimmed; + + protected FetcherWithInjectableErrors(FetcherCallback fetcherCallback, HttpConnectionParams params, + FetchedInputAllocator inputManager, ApplicationId appId, int dagIdentifier, + JobTokenSecretManager jobTokenSecretManager, String srcNameTrimmed, Configuration conf, + RawLocalFileSystem localFs, LocalDirAllocator localDirAllocator, Path lockPath, boolean localDiskFetchEnabled, + boolean sharedFetchEnabled, String localHostname, int shufflePort, boolean asyncHttp, boolean verifyDiskChecksum, + boolean compositeFetch, ObjectRegistry objectRegistry) { + super(fetcherCallback, params, inputManager, appId, dagIdentifier, jobTokenSecretManager, srcNameTrimmed, conf, + localFs, localDirAllocator, lockPath, localDiskFetchEnabled, sharedFetchEnabled, localHostname, shufflePort, + asyncHttp, verifyDiskChecksum, compositeFetch); + this.fetcherErrorTestingConfig = new FetcherErrorTestingConfig(conf, objectRegistry); + this.srcNameTrimmed = srcNameTrimmed; + LOG.info("Initialized FetcherWithInjectableErrors with config: {}", fetcherErrorTestingConfig); + } + + @Override + protected void setupConnectionInternal(String host, Collection attempts) + throws IOException, InterruptedException { + LOG.info("Checking if fetcher should fail for host: {} ...", host); + for (InputAttemptIdentifier inputAttemptIdentifier : attempts) { + if (fetcherErrorTestingConfig.shouldFail(host, srcNameTrimmed, inputAttemptIdentifier)) { + throw new IOException(String.format( + "FetcherWithInjectableErrors tester made failure for host: %s, input attempt: %s", host, + inputAttemptIdentifier.getAttemptNumber())); + } + } + super.setupConnectionInternal(host, attempts); + } + + @Override + public int hashCode() { + return fetcherIdentifier; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + FetcherWithInjectableErrors other = (FetcherWithInjectableErrors) obj; + if (fetcherIdentifier != other.fetcherIdentifier) { + return false; + } + return true; + } +} diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java index 56195a8641..fa539c8374 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java @@ -174,7 +174,8 @@ public class ShuffleManager implements FetcherCallback { private final boolean sharedFetchEnabled; private final boolean verifyDiskChecksum; private final boolean compositeFetch; - + private final boolean enableFetcherTestingErrors; + private final int ifileBufferSize; private final boolean ifileReadAhead; private final int ifileReadAheadLength; @@ -259,6 +260,10 @@ public ShuffleManager(InputContext inputContext, Configuration conf, int numInpu this.lastEventReceived = inputContext.getCounters().findCounter(TaskCounter.LAST_EVENT_RECEIVED); this.compositeFetch = ShuffleUtils.isTezShuffleHandler(conf); + this.enableFetcherTestingErrors = + conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_ENABLE_TESTING_ERRORS, + TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_ENABLE_TESTING_ERRORS_DEFAULT); + this.srcNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName()); completedInputSet = new BitSet(numInputs); @@ -395,7 +400,7 @@ protected Void callInternal() throws Exception { for (InputReadErrorEvent key : failedEvents.keySet()) { failedEventsToSend.add(InputReadErrorEvent.create(key.getDiagnostics(), key.getIndex(), key.getVersion(), failedEvents.get(key), key.isLocalFetch(), - key.isDiskErrorAtSource())); + key.isDiskErrorAtSource(), localhostName)); } inputContext.sendEvents(failedEventsToSend); failedEvents.clear(); @@ -543,7 +548,8 @@ Fetcher constructFetcherForHost(InputHost inputHost, Configuration conf) { httpConnectionParams, inputManager, inputContext.getApplicationId(), inputContext.getDagIdentifier(), jobTokenSecretMgr, srcNameTrimmed, conf, localFs, localDirAllocator, lockDisk, localDiskFetchEnabled, sharedFetchEnabled, - localhostName, shufflePort, asyncHttp, verifyDiskChecksum, compositeFetch); + localhostName, shufflePort, asyncHttp, verifyDiskChecksum, compositeFetch, enableFetcherTestingErrors, + inputContext.getObjectRegistry()); if (codec != null) { fetcherBuilder.setCompressionParameters(codec); @@ -960,7 +966,7 @@ public void fetchFailed(String host, srcAttemptIdentifier.getInputIdentifier(), srcAttemptIdentifier.getAttemptNumber(), inputAttemptFetchFailure.isLocalFetch(), - inputAttemptFetchFailure.isDiskErrorAtSource()); + inputAttemptFetchFailure.isDiskErrorAtSource(), localhostName); if (maxTimeToWaitForReportMillis > 0) { try { reportLock.lock(); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java index a4328af44a..2725debb56 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java @@ -81,7 +81,7 @@ class FetcherOrderedGrouped extends CallableWithNdc { private final int localShufflePort; private final String applicationId; private final int dagId; - private final MapHost mapHost; + protected final MapHost mapHost; private final int minPartition; private final int maxPartition; @@ -350,8 +350,7 @@ boolean setupConnection(MapHost host, Collection attempt LOG.debug("Detected fetcher has been shutdown after connection establishment. Returning"); return false; } - input = httpConnection.getInputStream(); - httpConnection.validate(); + setupConnectionInternal(host, attempts); return true; } catch (IOException | InterruptedException ie) { if (ie instanceof InterruptedException) { @@ -385,6 +384,12 @@ boolean setupConnection(MapHost host, Collection attempt } } + protected void setupConnectionInternal(MapHost host, Collection attempts) + throws IOException, InterruptedException { + input = httpConnection.getInputStream(); + httpConnection.validate(); + } + @VisibleForTesting protected void putBackRemainingMapOutputs(MapHost host) { // Cycle through remaining MapOutputs @@ -426,7 +431,7 @@ public String toString() { } protected InputAttemptFetchFailure[] copyMapOutput(MapHost host, DataInputStream input, - InputAttemptIdentifier inputAttemptIdentifier) throws FetcherReadTimeoutException { + InputAttemptIdentifier inputAttemptIdentifier) throws FetcherReadTimeoutException, IOException { MapOutput mapOutput = null; InputAttemptIdentifier srcAttemptId = null; long decompressedLength = 0; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGroupedWithInjectableErrors.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGroupedWithInjectableErrors.java new file mode 100644 index 0000000000..94172d12b9 --- /dev/null +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGroupedWithInjectableErrors.java @@ -0,0 +1,68 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.runtime.library.common.shuffle.orderedgrouped; + +import java.io.IOException; +import java.util.Collection; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.RawLocalFileSystem; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.tez.common.counters.TezCounter; +import org.apache.tez.common.security.JobTokenSecretManager; +import org.apache.tez.http.HttpConnectionParams; +import org.apache.tez.runtime.api.ObjectRegistry; +import org.apache.tez.runtime.library.common.InputAttemptIdentifier; +import org.apache.tez.runtime.library.common.shuffle.FetcherErrorTestingConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class FetcherOrderedGroupedWithInjectableErrors extends FetcherOrderedGrouped { + private static final Logger LOG = LoggerFactory.getLogger(FetcherOrderedGroupedWithInjectableErrors.class); + + private FetcherErrorTestingConfig fetcherErrorTestingConfig; + private String srcNameTrimmed; + + public FetcherOrderedGroupedWithInjectableErrors(HttpConnectionParams httpConnectionParams, + ShuffleScheduler scheduler, FetchedInputAllocatorOrderedGrouped allocator, ExceptionReporter exceptionReporter, + JobTokenSecretManager jobTokenSecretMgr, boolean ifileReadAhead, int ifileReadAheadLength, CompressionCodec codec, + Configuration conf, RawLocalFileSystem localFs, boolean localDiskFetchEnabled, String localHostname, + int shufflePort, String srcNameTrimmed, MapHost mapHost, TezCounter ioErrsCounter, + TezCounter wrongLengthErrsCounter, TezCounter badIdErrsCounter, TezCounter wrongMapErrsCounter, + TezCounter connectionErrsCounter, TezCounter wrongReduceErrsCounter, String applicationId, int dagId, + boolean asyncHttp, boolean sslShuffle, boolean verifyDiskChecksum, boolean compositeFetch, + ObjectRegistry objectRegistry) { + super(httpConnectionParams, scheduler, allocator, exceptionReporter, jobTokenSecretMgr, ifileReadAhead, + ifileReadAheadLength, codec, conf, localFs, localDiskFetchEnabled, localHostname, shufflePort, srcNameTrimmed, + mapHost, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, connectionErrsCounter, + wrongReduceErrsCounter, applicationId, dagId, asyncHttp, sslShuffle, verifyDiskChecksum, compositeFetch); + this.fetcherErrorTestingConfig = new FetcherErrorTestingConfig(conf, objectRegistry); + this.srcNameTrimmed = srcNameTrimmed; + LOG.info("Initialized FetcherOrderedGroupedWithInjectableErrors with config: {}", fetcherErrorTestingConfig); + } + + @Override + protected void setupConnectionInternal(MapHost host, Collection attempts) + throws IOException, InterruptedException { + LOG.info("Checking if fetcher should fail for host: {} ...", mapHost.getHost()); + for (InputAttemptIdentifier inputAttemptIdentifier : attempts) { + if (fetcherErrorTestingConfig.shouldFail(mapHost.getHost(), srcNameTrimmed, inputAttemptIdentifier)) { + throw new IOException(String.format( + "FetcherOrderedGroupedWithInjectableErrors tester made failure for host: %s, input attempt: %s", + mapHost.getHost(), inputAttemptIdentifier.getAttemptNumber())); + } + } + super.setupConnectionInternal(host, attempts); + } +} diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java index dd27d45aed..cd4b14ca1b 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java @@ -248,7 +248,7 @@ enum ShuffleErrors { private final boolean checkFailedFetchSinceLastCompletion; private final boolean verifyDiskChecksum; private final boolean compositeFetch; - + private final boolean enableFetcherTestingErrors; private volatile Thread shuffleSchedulerThread = null; private final int maxPenaltyTime; @@ -427,6 +427,10 @@ public ShuffleScheduler(InputContext inputContext, this.maxPenaltyTime = conf.getInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_HOST_PENALTY_TIME_LIMIT_MS, TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_HOST_PENALTY_TIME_LIMIT_MS_DEFAULT); + this.enableFetcherTestingErrors = + conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_ENABLE_TESTING_ERRORS, + TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_ENABLE_TESTING_ERRORS_DEFAULT); + pipelinedShuffleInfoEventsMap = Maps.newConcurrentMap(); LOG.info("ShuffleScheduler running for sourceVertex: " + inputContext.getSourceVertexName() + " with configuration: " @@ -443,6 +447,7 @@ public ShuffleScheduler(InputContext inputContext, + ", minReqProgressFraction=" + minReqProgressFraction + ", checkFailedFetchSinceLastCompletion=" + checkFailedFetchSinceLastCompletion + ", asyncHttp=" + asyncHttp + + ", enableFetcherTestingErrors=" + enableFetcherTestingErrors ); } @@ -885,7 +890,7 @@ private void informAM(InputAttemptFetchFailure fetchFailure) { srcAttempt.getInputIdentifier(), srcAttempt.getAttemptNumber()) + " to jobtracker.", srcAttempt.getInputIdentifier(), srcAttempt.getAttemptNumber(), - fetchFailure.isLocalFetch(), fetchFailure.isDiskErrorAtSource())); + fetchFailure.isLocalFetch(), fetchFailure.isDiskErrorAtSource(), localHostname)); inputContext.sendEvents(failedEvents); } @@ -1466,12 +1471,21 @@ private synchronized void waitAndNotifyProgress() throws InterruptedException { @VisibleForTesting FetcherOrderedGrouped constructFetcherForHost(MapHost mapHost) { - return new FetcherOrderedGrouped(httpConnectionParams, ShuffleScheduler.this, allocator, - exceptionReporter, jobTokenSecretManager, ifileReadAhead, ifileReadAheadLength, - codec, conf, localFs, localDiskFetchEnabled, localHostname, shufflePort, srcNameTrimmed, mapHost, - ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, - connectionErrsCounter, wrongReduceErrsCounter, applicationId, dagId, asyncHttp, sslShuffle, - verifyDiskChecksum, compositeFetch); + if (enableFetcherTestingErrors) { + return new FetcherOrderedGroupedWithInjectableErrors(httpConnectionParams, ShuffleScheduler.this, allocator, + exceptionReporter, jobTokenSecretManager, ifileReadAhead, ifileReadAheadLength, + codec, conf, localFs, localDiskFetchEnabled, localHostname, shufflePort, srcNameTrimmed, mapHost, + ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, + connectionErrsCounter, wrongReduceErrsCounter, applicationId, dagId, asyncHttp, sslShuffle, + verifyDiskChecksum, compositeFetch, inputContext.getObjectRegistry()); + } else { + return new FetcherOrderedGrouped(httpConnectionParams, ShuffleScheduler.this, allocator, + exceptionReporter, jobTokenSecretManager, ifileReadAhead, ifileReadAheadLength, + codec, conf, localFs, localDiskFetchEnabled, localHostname, shufflePort, srcNameTrimmed, mapHost, + ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, + connectionErrsCounter, wrongReduceErrsCounter, applicationId, dagId, asyncHttp, sslShuffle, + verifyDiskChecksum, compositeFetch); + } } private class FetchFutureCallback implements FutureCallback { From 8a75317bb9f954108aac44fc8ca30bfde0c0e462 Mon Sep 17 00:00:00 2001 From: dmmkr Date: Thu, 25 Nov 2021 17:52:26 +0530 Subject: [PATCH 338/512] TEZ-4353: Update commons-io to 2.8.0 (#165) (D M Murali Krishna Reddy reviewed by Laszlo Bodor) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 3785352bf5..27b3e399fe 100644 --- a/pom.xml +++ b/pom.xml @@ -253,7 +253,7 @@ commons-io commons-io - 2.4 + 2.8.0 commons-lang From bf616b22012b5e0f60242c80c697d58fc2eed005 Mon Sep 17 00:00:00 2001 From: jteagles Date: Sat, 25 Dec 2021 16:30:24 -0600 Subject: [PATCH 339/512] TEZ-4294. Tez UI counters missing in settings view (#144) (Jonathan Eagles reviewed by Laszlo Bodor) --- .../apache/tez/common/counters/DAGCounter.java | 1 + .../apache/tez/common/counters/TaskCounter.java | 1 + .../src/main/webapp/config/default-app-conf.js | 16 ++++++++++++++++ 3 files changed, 18 insertions(+) diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java index 0a32d388bc..1ef56778c3 100644 --- a/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java +++ b/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java @@ -22,6 +22,7 @@ import org.apache.hadoop.classification.InterfaceStability; // Per-job counters +// Keep in sync with tez-ui/src/main/webapp/config/default-app-conf.js @InterfaceAudience.Public @InterfaceStability.Evolving public enum DAGCounter { diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java index 2ee82a3965..a65388ffd0 100644 --- a/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java +++ b/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java @@ -21,6 +21,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; // Counters used by Task classes +// Keep in sync with tez-ui/src/main/webapp/config/default-app-conf.js @Private public enum TaskCounter { // TODO Eventually, rename counters to be non-MR specific and map them to MR equivalent. diff --git a/tez-ui/src/main/webapp/config/default-app-conf.js b/tez-ui/src/main/webapp/config/default-app-conf.js index 388ca4ce0f..a49a3e0134 100644 --- a/tez-ui/src/main/webapp/config/default-app-conf.js +++ b/tez-ui/src/main/webapp/config/default-app-conf.js @@ -194,6 +194,10 @@ module.exports = { // Tez App configurations counterName: "INPUT_RECORDS_PROCESSED", counterGroupName: "org.apache.tez.common.counters.TaskCounter", }, + { + counterName: "INPUT_SPLIT_LENGTH_BYTES", + counterGroupName: "org.apache.tez.common.counters.TaskCounter", + }, { counterName: "OUTPUT_RECORDS", counterGroupName: "org.apache.tez.common.counters.TaskCounter", @@ -226,6 +230,10 @@ module.exports = { // Tez App configurations counterName: "ADDITIONAL_SPILL_COUNT", counterGroupName: "org.apache.tez.common.counters.TaskCounter", }, + { + counterName: "SHUFFLE_CHUNK_COUNT", + counterGroupName: "org.apache.tez.common.counters.TaskCounter", + }, { counterName: "SHUFFLE_BYTES", counterGroupName: "org.apache.tez.common.counters.TaskCounter", @@ -321,6 +329,14 @@ module.exports = { // Tez App configurations counterName :"NUM_FAILED_UBERTASKS", counterGroupName :"org.apache.tez.common.counters.DAGCounter", }, + { + counterName :"AM_CPU_MILLISECONDS", + counterGroupName :"org.apache.tez.common.counters.DAGCounter", + }, + { + counterName :"AM_GC_TIME_MILLIS", + counterGroupName :"org.apache.tez.common.counters.DAGCounter", + }, { counterName: "REDUCE_OUTPUT_RECORDS", From 8f846dcfe131aca69f8d3cb5cc1b47c6413e3b01 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Sun, 26 Dec 2021 18:10:36 +0100 Subject: [PATCH 340/512] TEZ-4303: Exclude compile-time httpclient dependency from Tez (#119) (Laszlo Bodor reviewed by Jonathan Eagles) --- pom.xml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/pom.xml b/pom.xml index 27b3e399fe..5a2c4fa45c 100644 --- a/pom.xml +++ b/pom.xml @@ -341,6 +341,10 @@ io.netty netty + + org.apache.httpcomponents + httpclient + @@ -352,6 +356,10 @@ io.netty netty + + org.apache.httpcomponents + httpclient + From c9b8e90db5af2b9648fccaa1501ad6dda82f31e0 Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Sun, 26 Dec 2021 12:35:43 -0500 Subject: [PATCH 341/512] TEZ-4270: HeartbeatHandlerBase Does Not Need Stop Condition (#93) (David Mollitor reviewed by Laszlo Bodor) --- .../java/org/apache/tez/dag/app/HeartbeatHandlerBase.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/HeartbeatHandlerBase.java b/tez-dag/src/main/java/org/apache/tez/dag/app/HeartbeatHandlerBase.java index 5c786b2232..b5ded81500 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/HeartbeatHandlerBase.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/HeartbeatHandlerBase.java @@ -42,7 +42,6 @@ public abstract class HeartbeatHandlerBase extends AbstractService { protected final AppContext appContext; private ConcurrentMap runningMap; - private volatile boolean stopped; public HeartbeatHandlerBase(AppContext appContext, int expectedConcurrency, String name) { super(name); @@ -70,7 +69,6 @@ public void serviceStart() { @Override public void serviceStop() { - stopped = true; if (timeOutCheckerThread != null) { timeOutCheckerThread.interrupt(); } @@ -140,7 +138,7 @@ private class PingChecker implements Runnable { @Override public void run() { - while (!stopped && !Thread.currentThread().isInterrupted()) { + while (!Thread.currentThread().isInterrupted()) { Iterator> iterator = runningMap.entrySet().iterator(); @@ -158,7 +156,7 @@ public void run() { try { Thread.sleep(timeOutCheckInterval); } catch (InterruptedException e) { - break; + Thread.currentThread().interrupt(); } } } From a602a07dc2c18f2a59eddbcdade9d116d06174ec Mon Sep 17 00:00:00 2001 From: Ahmed Hussein <50450311+amahussein@users.noreply.github.com> Date: Mon, 3 Jan 2022 01:39:40 -0600 Subject: [PATCH 342/512] TEZ-4349. DAGClient gets stuck with invalid cached DAGStatus (#161) (Ahmed Hussein reviewed by Laszlo Bodor, Jonathan Eagles) --- .../org/apache/tez/common/CachedEntity.java | 104 +++++++++++++ .../apache/tez/dag/api/TezConfiguration.java | 12 ++ .../tez/dag/api/client/DAGClientImpl.java | 66 +++++--- .../tez/dag/api/client/rpc/TestDAGClient.java | 144 +++++++++++++++++- 4 files changed, 303 insertions(+), 23 deletions(-) create mode 100644 tez-api/src/main/java/org/apache/tez/common/CachedEntity.java diff --git a/tez-api/src/main/java/org/apache/tez/common/CachedEntity.java b/tez-api/src/main/java/org/apache/tez/common/CachedEntity.java new file mode 100644 index 0000000000..47ccd77e48 --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/common/CachedEntity.java @@ -0,0 +1,104 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.common; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.hadoop.yarn.util.Clock; +import org.apache.hadoop.yarn.util.MonotonicClock; + +/** + * A thread safe implementation used as a container for cacheable entries with Expiration times. + * It supports custom {@link Clock} to control the elapsed time calculation. + * @param the data object type. + */ +public class CachedEntity { + private final AtomicReference entryDataRef; + private final Clock cacheClock; + private final long expiryDurationMS; + private volatile long entryTimeStamp; + + public CachedEntity(TimeUnit expiryTimeUnit, long expiryLength, Clock clock) { + entryDataRef = new AtomicReference<>(null); + cacheClock = clock; + expiryDurationMS = TimeUnit.MILLISECONDS.convert(expiryLength, expiryTimeUnit); + entryTimeStamp = 0; + } + + public CachedEntity(TimeUnit expiryTimeUnit, long expiryLength) { + this(expiryTimeUnit, expiryLength, new MonotonicClock()); + } + + /** + * + * @return true if expiration timestamp is 0, or the elapsed time since last update is + * greater than {@link #expiryDurationMS} + */ + public boolean isExpired() { + return (entryTimeStamp == 0) + || ((cacheClock.getTime() - entryTimeStamp) > expiryDurationMS); + } + + /** + * If the entry has expired, it reset the cache reference through {@link #clearExpiredEntry()}. + * @return cached data if the timestamp is valid. Null, if the timestamp has expired. + */ + public T getValue() { + if (isExpired()) { // quick check for expiration + if (clearExpiredEntry()) { // remove reference to the expired entry + return null; + } + } + return entryDataRef.get(); + } + + /** + * Safely sets the cached data. + * @param newEntry + */ + public void setValue(T newEntry) { + T currentEntry = entryDataRef.get(); + while (!entryDataRef.compareAndSet(currentEntry, newEntry)) { + currentEntry = entryDataRef.get(); + } + entryTimeStamp = cacheClock.getTime(); + } + + /** + * Enforces the expiration of the cached entry. + */ + public void enforceExpiration() { + entryTimeStamp = 0; + } + + /** + * Safely deletes the reference to the data if it was not null. + * @return true if the reference is set to Null. False indicates that another thread + * updated the cache. + */ + private boolean clearExpiredEntry() { + T currentEntry = entryDataRef.get(); + if (currentEntry == null) { + return true; + } + // the current value is not null: try to reset it. + // if the CAS is successful, then we won't override a recent update to the cache. + return (entryDataRef.compareAndSet(currentEntry, null)); + } +} diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 5063b04eac..ee991ccf53 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -1993,6 +1993,18 @@ static Set getPropertySet() { TEZ_PREFIX + "test.minicluster.app.wait.on.shutdown.secs"; public static final long TEZ_TEST_MINI_CLUSTER_APP_WAIT_ON_SHUTDOWN_SECS_DEFAULT = 30; + /** + * Long value + * Status Cache timeout window in minutes for the DAGClient. + */ + @Private + @ConfigurationScope(Scope.CLIENT) + @ConfigurationProperty(type="long") + public static final String TEZ_CLIENT_DAG_STATUS_CACHE_TIMEOUT_SECS = TEZ_PREFIX + + "client.dag.status.cache.timeout-secs"; + // Default timeout is 60 seconds. + public static final long TEZ_CLIENT_DAG_STATUS_CACHE_TIMEOUT_SECS_DEFAULT = 60; + /** * Long value * Time to wait (in milliseconds) for yarn app's diagnotics is available diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java index e58863f614..3c0de00819 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java @@ -27,9 +27,11 @@ import java.util.HashMap; import java.util.Map; import java.util.Set; +import java.util.concurrent.TimeUnit; import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.tez.common.CachedEntity; import org.apache.tez.common.Preconditions; import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException; @@ -58,13 +60,15 @@ public class DAGClientImpl extends DAGClient { private final String dagId; private final TezConfiguration conf; private final FrameworkClient frameworkClient; - + /** + * Container to cache the last {@link DAGStatus}. + */ + private final CachedEntity cachedDAGStatusRef; @VisibleForTesting protected DAGClientInternal realClient; - private boolean dagCompleted = false; + private volatile boolean dagCompleted = false; @VisibleForTesting protected boolean isATSEnabled = false; - private DAGStatus cachedDagStatus = null; Map cachedVertexStatus = new HashMap(); private static final long SLEEP_FOR_COMPLETION = 500; @@ -110,6 +114,28 @@ public DAGClientImpl(ApplicationId appId, String dagId, TezConfiguration conf, this.diagnoticsWaitTimeout = conf.getLong( TezConfiguration.TEZ_CLIENT_DIAGNOSTICS_WAIT_TIMEOUT_MS, TezConfiguration.TEZ_CLIENT_DIAGNOSTICS_WAIT_TIMEOUT_MS_DEFAULT); + cachedDAGStatusRef = initCacheDAGRefFromConf(conf); + } + + /** + * Constructs a new {@link CachedEntity} for {@link DAGStatus}. + * @param tezConf TEZ configuration parameters. + * @return a caching entry to hold the {@link DAGStatus}. + */ + protected CachedEntity initCacheDAGRefFromConf(TezConfiguration tezConf) { + long clientDAGStatusCacheTimeOut = tezConf.getLong( + TezConfiguration.TEZ_CLIENT_DAG_STATUS_CACHE_TIMEOUT_SECS, + TezConfiguration.TEZ_CLIENT_DAG_STATUS_CACHE_TIMEOUT_SECS_DEFAULT); + if (clientDAGStatusCacheTimeOut <= 0) { + LOG.error("DAG Status cache timeout interval should be positive. Enforcing default value."); + clientDAGStatusCacheTimeOut = + TezConfiguration.TEZ_CLIENT_DAG_STATUS_CACHE_TIMEOUT_SECS_DEFAULT; + } + return new CachedEntity<>(TimeUnit.SECONDS, clientDAGStatusCacheTimeOut); + } + + protected CachedEntity getCachedDAGStatusRef() { + return cachedDAGStatusRef; } @Override @@ -133,13 +159,11 @@ public DAGStatus getDAGStatus(@Nullable Set statusOptions, } long startTime = System.currentTimeMillis(); - boolean refreshStatus; - DAGStatus dagStatus; - if(cachedDagStatus != null) { - dagStatus = cachedDagStatus; - refreshStatus = true; - } else { - // For the first lookup only. After this cachedDagStatus should be populated. + + DAGStatus dagStatus = cachedDAGStatusRef.getValue(); + boolean refreshStatus = true; + if (dagStatus == null) { + // the first lookup only or when the cachedDAG has expired dagStatus = getDAGStatus(statusOptions); refreshStatus = false; } @@ -221,13 +245,14 @@ protected DAGStatus getDAGStatusInternal(@Nullable Set statusOpti final DAGStatus dagStatus = getDAGStatusViaAM(statusOptions, timeout); if (!dagCompleted) { - if (dagStatus != null) { - cachedDagStatus = dagStatus; + if (dagStatus != null) { // update the cached DAGStatus + cachedDAGStatusRef.setValue(dagStatus); return dagStatus; } - if (cachedDagStatus != null) { + DAGStatus cachedDAG = cachedDAGStatusRef.getValue(); + if (cachedDAG != null) { // could not get from AM (not reachable/ was killed). return cached status. - return cachedDagStatus; + return cachedDAG; } } @@ -253,8 +278,11 @@ protected DAGStatus getDAGStatusInternal(@Nullable Set statusOpti // dag completed and Timeline service is either not enabled or does not have completion status // return cached status if completion info is present. - if (dagCompleted && cachedDagStatus != null && cachedDagStatus.isCompleted()) { - return cachedDagStatus; + if (dagCompleted) { + DAGStatus cachedDag = cachedDAGStatusRef.getValue(); + if (cachedDag != null && cachedDag.isCompleted()) { + return cachedDag; + } } // everything else fails rely on RM. @@ -377,9 +405,11 @@ private DAGStatus getDAGStatusViaAM(@Nullable Set statusOptions, LOG.info("DAG is no longer running - application not found by YARN", e); dagCompleted = true; } catch (TezException e) { - // can be either due to a n/w issue of due to AM completed. + // can be either due to a n/w issue or due to AM completed. + LOG.info("Cannot retrieve DAG Status due to TezException: {}", e.getMessage()); } catch (IOException e) { - // can be either due to a n/w issue of due to AM completed. + // can be either due to a n/w issue or due to AM completed. + LOG.info("Cannot retrieve DAG Status due to IOException: {}", e.getMessage()); } if (dagStatus == null && !dagCompleted) { diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java index 50c9a6061c..c1ebaa8db1 100644 --- a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java +++ b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java @@ -34,6 +34,7 @@ import java.lang.reflect.Field; import java.util.EnumSet; import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.ssl.KeyStoreTestUtil; @@ -41,12 +42,14 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.tez.client.FrameworkClient; +import org.apache.tez.common.CachedEntity; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.api.client.DAGClient; import org.apache.tez.dag.api.client.DAGClientImpl; import org.apache.tez.dag.api.client.DAGClientTimelineImpl; import org.apache.tez.dag.api.client.DAGStatus; +import org.apache.tez.dag.api.client.DAGStatus.State; import org.apache.tez.dag.api.client.DagStatusSource; import org.apache.tez.dag.api.client.StatusGetOpts; import org.apache.tez.dag.api.client.VertexStatus; @@ -374,7 +377,7 @@ public void testGetDagStatusWithTimeout() throws Exception { // Fetch from AM. RUNNING dagClient.resetCounters(); - dagClientRpc.resetCountesr(); + dagClientRpc.resetCounters(); rmDagStatus = new DAGStatus(constructDagStatusProto(DAGStatusStateProto.DAG_RUNNING), DagStatusSource.RM); dagClient.setRmDagStatus(rmDagStatus); @@ -394,7 +397,7 @@ public void testGetDagStatusWithTimeout() throws Exception { // Fetch from AM. Success. dagClient.resetCounters(); - dagClientRpc.resetCountesr(); + dagClientRpc.resetCounters(); rmDagStatus = new DAGStatus(constructDagStatusProto(DAGStatusStateProto.DAG_RUNNING), DagStatusSource.RM); dagClient.setRmDagStatus(rmDagStatus); @@ -438,20 +441,21 @@ private static void testAtsEnabled(ApplicationId appId, String dagIdStr, boolean } private static class DAGClientRPCImplForTest extends DAGClientRPCImpl { - + private AtomicReference faultAMInjectedRef; int numGetStatusViaAmInvocations = 0; public DAGClientRPCImplForTest(ApplicationId appId, String dagId, TezConfiguration conf, @Nullable FrameworkClient frameworkClient) throws IOException { super(appId, dagId, conf, frameworkClient, UserGroupInformation.getCurrentUser()); + faultAMInjectedRef = new AtomicReference<>(null); } void setAMProxy(DAGClientAMProtocolBlockingPB proxy) { this.proxy = proxy; } - void resetCountesr() { + void resetCounters() { numGetStatusViaAmInvocations = 0; } @@ -468,15 +472,22 @@ boolean createAMProxyIfNeeded() throws IOException, TezException { DAGStatus getDAGStatusViaAM(Set statusOptions, long timeout) throws IOException, TezException { numGetStatusViaAmInvocations++; + if (faultAMInjectedRef.get() != null) { + throw faultAMInjectedRef.get(); + } return super.getDAGStatusViaAM(statusOptions, timeout); } + + void injectAMFault(IOException exception) { + faultAMInjectedRef.set(exception); + } } private static class DAGClientImplForTest extends DAGClientImpl { private DAGStatus rmDagStatus; int numGetStatusViaRmInvocations = 0; - + private volatile boolean faultInjected; public DAGClientImplForTest(ApplicationId appId, String dagId, TezConfiguration conf, @Nullable FrameworkClient frameworkClient) throws IOException { super(appId, dagId, conf, frameworkClient, UserGroupInformation.getCurrentUser()); @@ -497,6 +508,9 @@ void resetCounters() { @Override protected DAGStatus getDAGStatusViaRM() throws TezException, IOException { numGetStatusViaRmInvocations++; + if (faultInjected) { + throw new IOException("Fault Injected for RM"); + } return rmDagStatus; } @@ -504,6 +518,18 @@ public boolean getIsATSEnabled() { return isATSEnabled; } + void injectFault() { + faultInjected = true; + } + + DAGStatus getCachedDAGStatus() { + CachedEntity cacheRef = getCachedDAGStatusRef(); + return cacheRef.getValue(); + } + + void enforceExpirationCachedDAGStatus() { + getCachedDAGStatusRef().enforceExpiration(); + } } private DAGProtos.DAGStatusProto.Builder constructDagStatusProto(DAGStatusStateProto stateProto) { @@ -578,4 +604,112 @@ public void testTimelineClientCleanup() throws Exception { } Assert.assertFalse("Reloader is still alive", reloaderStillAlive); } + + @Test(timeout = 50000) + public void testGetDagStatusWithCachedStatusExpiration() throws Exception { + long startTime; + long endTime; + long diff; + + TezConfiguration tezConf = new TezConfiguration(); + tezConf.setLong(TezConfiguration.TEZ_DAG_STATUS_POLLINTERVAL_MS, 800L); + tezConf.setLong(TezConfiguration.TEZ_CLIENT_DAG_STATUS_CACHE_TIMEOUT_SECS, 100000L); + try (DAGClientImplForTest dagClientImpl = + new DAGClientImplForTest(mockAppId, dagIdStr, tezConf, null)) { + DAGClientRPCImplForTest dagClientRpc = + new DAGClientRPCImplForTest(mockAppId, dagIdStr, tezConf, null); + dagClientImpl.setRealClient(dagClientRpc); + + DAGStatus dagStatus; + DAGStatus rmDagStatus; + + // Fetch from AM. RUNNING + rmDagStatus = + new DAGStatus(constructDagStatusProto(DAGStatusStateProto.DAG_RUNNING), + DagStatusSource.RM); + dagClientImpl.setRmDagStatus(rmDagStatus); + dagClientRpc.setAMProxy(createMockProxy(DAGStatusStateProto.DAG_RUNNING, -1)); + + startTime = System.currentTimeMillis(); + dagStatus = dagClientImpl.getDAGStatus(EnumSet.noneOf(StatusGetOpts.class), 2000L); + endTime = System.currentTimeMillis(); + diff = endTime - startTime; + assertTrue(diff > 1500L && diff < 2500L); + // Directly from AM + assertEquals(0, dagClientImpl.numGetStatusViaRmInvocations); + // Directly from AM - one refresh. One with timeout. + assertEquals(2, dagClientRpc.numGetStatusViaAmInvocations); + assertEquals(DAGStatus.State.RUNNING, dagStatus.getState()); + + // Fetch from AM. Success. + dagClientImpl.resetCounters(); + dagClientRpc.resetCounters(); + rmDagStatus = + new DAGStatus(constructDagStatusProto(DAGStatusStateProto.DAG_RUNNING), + DagStatusSource.RM); + dagClientImpl.setRmDagStatus(rmDagStatus); + dagClientRpc.setAMProxy(createMockProxy(DAGStatusStateProto.DAG_SUCCEEDED, 1000L)); + + startTime = System.currentTimeMillis(); + dagStatus = dagClientImpl.getDAGStatus(EnumSet.noneOf(StatusGetOpts.class), 2000L); + endTime = System.currentTimeMillis(); + diff = endTime - startTime; + assertTrue("diff is " + diff, diff > 500L && diff < 1500L); + // Directly from AM + assertEquals(0, dagClientImpl.numGetStatusViaRmInvocations); + // Directly from AM - previous request cached, so single invocation only. + assertEquals(1, dagClientRpc.numGetStatusViaAmInvocations); + assertEquals(DAGStatus.State.SUCCEEDED, dagStatus.getState()); + + // verify that the cachedDAGStatus is correct + DAGStatus cachedDagStatus = dagClientImpl.getCachedDAGStatus(); + Assert.assertNotNull(cachedDagStatus); + Assert.assertSame(dagStatus, cachedDagStatus); + + // When AM proxy throws an exception, the cachedDAGStatus should be returned + dagClientImpl.resetCounters(); + dagClientRpc.resetCounters(); + dagClientRpc.injectAMFault(new IOException("injected Fault")); + dagStatus = dagClientImpl.getDAGStatus(EnumSet.noneOf(StatusGetOpts.class)); + // get the Status from the cache + assertEquals(0, dagClientImpl.numGetStatusViaRmInvocations); + // Directly from AM - previous request cached, so single invocation only. + assertEquals(1, dagClientRpc.numGetStatusViaAmInvocations); + assertEquals(DAGStatus.State.SUCCEEDED, dagStatus.getState()); + Assert.assertSame(dagStatus, cachedDagStatus); + + // test that RM is invoked when the cacheExpires and the AM fails. + dagClientRpc.setAMProxy(createMockProxy(DAGStatusStateProto.DAG_SUCCEEDED, 1000L)); + dagClientRpc.injectAMFault(new IOException("injected AM Fault")); + dagClientImpl.resetCounters(); + dagClientRpc.resetCounters(); + dagClientImpl.enforceExpirationCachedDAGStatus(); + dagStatus = dagClientImpl.getDAGStatus(EnumSet.noneOf(StatusGetOpts.class)); + // get the Status from the cache + assertEquals(1, dagClientImpl.numGetStatusViaRmInvocations); + assertEquals(1, dagClientRpc.numGetStatusViaAmInvocations); + assertEquals(State.RUNNING, dagStatus.getState()); + Assert.assertNotSame(dagStatus, cachedDagStatus); + + // verify that the cachedDAGStatus is null because AM threw exception before setting the + // cache. + cachedDagStatus = dagClientImpl.getCachedDAGStatus(); + Assert.assertNull(cachedDagStatus); + Assert.assertNotNull(dagStatus); + + // inject fault in RM too. getDAGStatus should return null; + dagClientImpl.resetCounters(); + dagClientRpc.resetCounters(); + dagClientRpc.setAMProxy(createMockProxy(DAGStatusStateProto.DAG_SUCCEEDED, 1000L)); + dagClientImpl.injectFault(); + try { + dagClientImpl.getDAGStatus(EnumSet.noneOf(StatusGetOpts.class)); + Assert.fail("The RM should throw IOException"); + } catch (IOException ioException) { + Assert.assertEquals(ioException.getMessage(), "Fault Injected for RM"); + assertEquals(1, dagClientImpl.numGetStatusViaRmInvocations); + assertEquals(1, dagClientRpc.numGetStatusViaAmInvocations); + } + } + } } From 56a7bf46cda95d9de2a3ef0c0e1ec73a68a7e95d Mon Sep 17 00:00:00 2001 From: dmmkr Date: Mon, 3 Jan 2022 13:12:00 +0530 Subject: [PATCH 343/512] TEZ-4354: Update netty to 4.1.61.Final (#164) (D M Murali Krishna Reddy reviewed by Laszlo Bodor) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 5a2c4fa45c..3f219eaa43 100644 --- a/pom.xml +++ b/pom.xml @@ -59,7 +59,7 @@ ${user.home}/clover.license 27.0-jre 3.2.2 - 4.0.52.Final + 4.1.61.Final 0.13.0 1.19 1.7.30 From 7e09dfa5081797b721f219472bd511b1633b55a7 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Mon, 3 Jan 2022 10:57:11 +0100 Subject: [PATCH 344/512] TEZ-4364: TestFaultTolerance timeout on master (#171) (Laszlo Bodor reviewed by Rajesh Balamohan) --- .../runtime/api/events/InputReadErrorEvent.java | 15 ++++++++++----- .../test/java/org/apache/tez/test/TestInput.java | 7 ++++--- 2 files changed, 14 insertions(+), 8 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java index 1d0c44d6c1..6c9f1fbe6b 100644 --- a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java +++ b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java @@ -21,6 +21,8 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.tez.runtime.api.Event; +import com.google.common.annotations.VisibleForTesting; + import java.util.Objects; /** @@ -79,11 +81,14 @@ private InputReadErrorEvent(final String diagnostics, final int index, final int this.destinationLocalhostName = destinationLocalhostName; } - public static InputReadErrorEvent create(String diagnostics, int index, int version, - boolean isLocalFetch, boolean isDiskErrorAtSource) { - return create(diagnostics, index, version, 1, isLocalFetch, isDiskErrorAtSource, null); - } - + /** + * Creates an InputReadErrorEvent from less parameters, supposed to be used from tests only. + * @param diagnostics + * @param index + * @param version + * @return InputReadErrorEvent instance + */ + @VisibleForTesting public static InputReadErrorEvent create(String diagnostics, int index, int version) { return create(diagnostics, index, version, 1, false, false, null); } diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestInput.java b/tez-tests/src/test/java/org/apache/tez/test/TestInput.java index 811ca3cc17..56bfe49dc8 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestInput.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestInput.java @@ -175,7 +175,7 @@ public int doRead() { for (int i=0; i Date: Wed, 5 Jan 2022 21:40:06 +0100 Subject: [PATCH 345/512] TEZ-4351: ShuffleHandler port should respect value in config (#163) (Laszlo Bodor reviewed by Jonathan Eagles) --- .../tez/auxservices/ShuffleHandler.java | 7 ++-- .../tez/auxservices/TestShuffleHandler.java | 33 +++++++++++++++++++ 2 files changed, 38 insertions(+), 2 deletions(-) diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java index fb28a0f4b1..a54a7cfe01 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java @@ -464,6 +464,10 @@ static Token deserializeServiceData(ByteBuffer secret) throw return jt; } + public int getPort() { + return port; + } + @Override public void initializeApplication(ApplicationInitializationContext context) { @@ -537,7 +541,7 @@ public Thread newThread(Runnable r) { return new Thread(r, WORKER_THREAD_NAME_PREFIX + workerThreadCounter.incrementAndGet()); } }); - + port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT); super.serviceInit(new YarnConfiguration(conf)); } @@ -556,7 +560,6 @@ protected void serviceStart() throws Exception { conf.getInt(SHUFFLE_LISTEN_QUEUE_SIZE, DEFAULT_SHUFFLE_LISTEN_QUEUE_SIZE)) .childOption(ChannelOption.SO_KEEPALIVE, true); initPipeline(bootstrap, conf); - port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT); Channel ch = bootstrap.bind().sync().channel(); accepted.add(ch); diff --git a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java index 21addd393e..6a2e1cc68e 100644 --- a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java @@ -42,6 +42,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Random; import java.util.concurrent.atomic.AtomicBoolean; import java.util.zip.Checksum; @@ -1432,4 +1433,36 @@ public FullHttpRequest createHttpRequest() { } return new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, uri); } + + @Test + public void testConfigPortStatic() throws Exception { + Random rand = new Random(); + int port = rand.nextInt(10) + 50000; + Configuration conf = new Configuration(); + // provide a port for ShuffleHandler + conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, port); + MockShuffleHandler2 shuffleHandler = new MockShuffleHandler2(); + shuffleHandler.serviceInit(conf); + try { + shuffleHandler.serviceStart(); + Assert.assertEquals(port, shuffleHandler.getPort()); + } finally { + shuffleHandler.stop(); + } + } + + @Test + public void testConfigPortDynamic() throws Exception { + Configuration conf = new Configuration(); + // 0 as config, should be dynamically chosen by netty + conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); + MockShuffleHandler2 shuffleHandler = new MockShuffleHandler2(); + shuffleHandler.serviceInit(conf); + try { + shuffleHandler.serviceStart(); + Assert.assertTrue("ShuffleHandler should use a random chosen port", shuffleHandler.getPort() > 0); + } finally { + shuffleHandler.stop(); + } + } } From 4953f2983b315bbb008de434017d6ff41121260d Mon Sep 17 00:00:00 2001 From: jteagles Date: Thu, 6 Jan 2022 14:09:29 -0600 Subject: [PATCH 346/512] TEZ-4167. Speed up TestPipelinedSorter (#174) (Jonathan Eagles reviewed by Laszlo Bodor) --- .../common/sort/impl/TestPipelinedSorter.java | 45 ++++++++++++------- 1 file changed, 29 insertions(+), 16 deletions(-) diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java index eed423b494..bb0f22fabd 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java @@ -60,6 +60,7 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.Map; +import java.util.Random; import java.util.TreeMap; import java.util.UUID; @@ -83,7 +84,7 @@ public class TestPipelinedSorter { private long initialAvailableMem; //TODO: Need to make it nested structure so that multiple partition cases can be validated - private static TreeMap sortedDataMap = Maps.newTreeMap(); + private static TreeMap sortedDataMap = Maps.newTreeMap(); static { conf = getConf(); @@ -541,10 +542,10 @@ private void writeData2(ExternalSorter sorter, int counter = 0; for (int numkey : numKeys) { int curKeyLen = keyLen[counter]; + char[] buffer = new char[curKeyLen]; for (int i = 0; i < numkey; i++) { - Text key = new Text(RandomStringUtils.randomAlphanumeric(curKeyLen)); - Text value = new Text(RandomStringUtils.randomAlphanumeric(curKeyLen)); - sorter.write(key, value); + Text random = new Text(randomAlphanumeric(buffer)); + sorter.write(random, random); } counter++; } @@ -876,29 +877,41 @@ private void writeData(ExternalSorter sorter, int numKeys, int keyLen) throws IO private void writeSimilarKeys(ExternalSorter sorter, int numKeys, int keyLen, boolean autoClose) throws IOException { sortedDataMap.clear(); - String keyStr = RandomStringUtils.randomAlphanumeric(keyLen); + char[] buffer = new char[keyLen]; + String keyStr = randomAlphanumeric(buffer); for (int i = 0; i < numKeys; i++) { if (i % 4 == 0) { - keyStr = RandomStringUtils.randomAlphanumeric(keyLen); + keyStr = randomAlphanumeric(buffer); } Text key = new Text(keyStr); Text value = new Text(RandomStringUtils.randomAlphanumeric(keyLen)); sorter.write(key, value); - sortedDataMap.put(key.toString(), value.toString()); //for verifying data later + sortedDataMap.put(key, value); //for verifying data later } if (autoClose) { closeSorter(sorter); } } + static private final Random RANDOM = new Random(); + int start = ' '; + int end = 'z' + 1; + int gap = end - start; + private String randomAlphanumeric(char[] buffer) { + for (int i = 0; i < buffer.length; ++i) { + buffer[i] = (char)(RANDOM.nextInt(gap) + start); + } + return new String(buffer); + } private void writeData(ExternalSorter sorter, int numKeys, int keyLen, boolean autoClose) throws IOException { + char[] buffer = new char[keyLen]; sortedDataMap.clear(); for (int i = 0; i < numKeys; i++) { - Text key = new Text(RandomStringUtils.randomAlphanumeric(keyLen)); - Text value = new Text(RandomStringUtils.randomAlphanumeric(keyLen)); - sorter.write(key, value); - sortedDataMap.put(key.toString(), value.toString()); //for verifying data later + String randomStr = randomAlphanumeric(buffer); + Text random = new Text(randomStr); + sorter.write(random, random); + sortedDataMap.put(random, random); //for verifying data later } if (autoClose) { closeSorter(sorter); @@ -926,15 +939,15 @@ private void verifyData(IFile.Reader reader) int numRecordsRead = 0; - for (Map.Entry entry : sortedDataMap.entrySet()) { - String key = entry.getKey(); - String val = entry.getValue(); + for (Map.Entry entry : sortedDataMap.entrySet()) { + Text key = entry.getKey(); + Text val = entry.getValue(); if (reader.nextRawKey(keyIn)) { reader.nextRawValue(valIn); readKey = keyDeserializer.deserialize(readKey); readValue = valDeserializer.deserialize(readValue); - Assert.assertTrue(key.equalsIgnoreCase(readKey.toString())); - Assert.assertTrue(val.equalsIgnoreCase(readValue.toString())); + Assert.assertTrue(key.equals(readKey)); + Assert.assertTrue(val.equals(readValue)); numRecordsRead++; } } From a6a936dad34397226adcb672f25184169ecbcb71 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Thu, 6 Jan 2022 21:30:58 +0100 Subject: [PATCH 347/512] TEZ-4311: Bump hadoop dependency version to 3.3.x (#132) (Laszlo Bodor reviewed by Jonathan Eagles) --- pom.xml | 11 +++- .../tez/dag/api/client/rpc/TestDAGClient.java | 6 ++- .../ats/acls/TestATSHistoryWithACLs.java | 54 +++++++++++++++++-- tez-runtime-library/pom.xml | 4 ++ .../apache/tez/test/TestSecureShuffle.java | 5 +- 5 files changed, 71 insertions(+), 9 deletions(-) diff --git a/pom.xml b/pom.xml index 3f219eaa43..5cf77deedf 100644 --- a/pom.xml +++ b/pom.xml @@ -58,7 +58,7 @@ ${user.home}/clover.license 27.0-jre - 3.2.2 + 3.3.1 4.1.61.Final 0.13.0 1.19 @@ -244,6 +244,12 @@ org.asynchttpclient async-http-client 2.12.1 + + + io.netty + * + + org.slf4j @@ -337,9 +343,10 @@ commons-el commons-el + io.netty - netty + * org.apache.httpcomponents diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java index c1ebaa8db1..f0cc20b062 100644 --- a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java +++ b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java @@ -587,7 +587,11 @@ public void testTimelineClientCleanup() throws Exception { threadGroup.enumerate(threads); Thread reloaderThread = null; for (Thread thread : threads) { - if ((thread.getName() != null) && (thread.getName().contains("Truststore reloader thread"))) { + /* Since HADOOP-16524, the reloader thread's name is changed, let's handle the backward compatibility + * with a simple OR, as this is just a unit test, it's not worth involving a hadoop version check. + */ + if ((thread.getName() != null) && (thread.getName().contains("Truststore reloader thread")) + || (thread.getName().contains("SSL Certificates Store Monitor"))) { reloaderThread = thread; } } diff --git a/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java b/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java index 4b5702ef7e..f79a78b9fa 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java +++ b/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java @@ -25,8 +25,10 @@ import static org.mockito.Mockito.*; import java.io.IOException; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.List; import java.util.Random; import javax.ws.rs.core.MediaType; @@ -42,6 +44,7 @@ import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain; import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.tez.client.TezClient; import org.apache.tez.common.ReflectionUtils; @@ -63,6 +66,9 @@ import org.apache.tez.runtime.library.processor.SleepProcessor; import org.apache.tez.runtime.library.processor.SleepProcessor.SleepProcessorConfig; import org.apache.tez.tests.MiniTezClusterWithTimeline; +import org.codehaus.jettison.json.JSONArray; +import org.codehaus.jettison.json.JSONException; +import org.codehaus.jettison.json.JSONObject; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -149,9 +155,48 @@ private K getTimelineData(String url, Class clazz) { assertEquals(200, response.getStatus()); assertTrue(MediaType.APPLICATION_JSON_TYPE.isCompatible(response.getType())); - K entity = response.getEntity(clazz); - assertNotNull(entity); - return entity; + JSONObject entity = response.getEntity(JSONObject.class); + K converted = null; + try { + converted = convertJSONObjectToTimelineObject(entity, clazz); + } catch (JSONException e) { + throw new RuntimeException(e); + } + assertNotNull(converted); + return converted; + } + + private K convertJSONObjectToTimelineObject(JSONObject jsonObj, Class clazz) throws JSONException { + LOG.info("convertJSONObjectToEntity got object: " + jsonObj); + if (clazz == TimelineDomain.class) { + TimelineDomain domain = new TimelineDomain(); + domain.setId(jsonObj.getString("id")); + domain.setOwner(jsonObj.getString("owner")); + domain.setReaders(jsonObj.getString("readers")); + domain.setWriters(jsonObj.getString("writers")); + return (K) domain; + } else if (clazz == TimelineEntity.class) { + TimelineEntity entity = new TimelineEntity(); + entity.setEntityId(jsonObj.getString("entity")); + entity.setEntityType(jsonObj.getString("entitytype")); + entity.setDomainId(jsonObj.getString("domain")); + entity.setEvents(getEventsFromJSON(jsonObj)); + return (K) entity; + } else { + throw new RuntimeException( + "convertJSONObjectToTimelineObject doesn't support conversion from JSONObject to " + clazz); + } + } + + private List getEventsFromJSON(JSONObject jsonObj) throws JSONException { + List events = new ArrayList<>(); + JSONArray arrEvents = jsonObj.getJSONArray("events"); + for (int i = 0; i < arrEvents.length(); i++) { + TimelineEvent event = new TimelineEvent(); + event.setEventType(((JSONObject) arrEvents.get(i)).getString("eventtype")); + events.add(event); + } + return events; } private TimelineDomain getDomain(String domainId) { @@ -459,7 +504,8 @@ public void testDagLoggingEnabled() throws Exception { .get(ClientResponse.class); assertEquals(200, response.getStatus()); assertTrue(MediaType.APPLICATION_JSON_TYPE.isCompatible(response.getType())); - TimelineEntity entity = response.getEntity(TimelineEntity.class); + JSONObject entityJson = response.getEntity(JSONObject.class); + TimelineEntity entity = convertJSONObjectToTimelineObject(entityJson, TimelineEntity.class); assertEquals(entity.getEntityType(), "TEZ_DAG_ID"); assertEquals(entity.getEvents().get(0).getEventType(), HistoryEventType.DAG_SUBMITTED.toString()); } diff --git a/tez-runtime-library/pom.xml b/tez-runtime-library/pom.xml index 2caff344ec..e43f90f415 100644 --- a/tez-runtime-library/pom.xml +++ b/tez-runtime-library/pom.xml @@ -33,6 +33,10 @@ org.asynchttpclient async-http-client + + io.netty + netty-all + org.apache.tez tez-api diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java b/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java index 2b2221230f..8e298506b8 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java @@ -114,6 +114,7 @@ public static Collection getParameters() { public static void setupDFSCluster() throws Exception { conf = new Configuration(); conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_EDITS_NOEDITLOGCHANNELFLUSH, false); + conf.setBoolean("fs.hdfs.impl.disable.cache", true); EditLogFileOutputStream.setShouldSkipFsyncForTesting(true); conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, TEST_ROOT_DIR); miniDFSCluster = @@ -301,8 +302,8 @@ public static X509Certificate generateCertificate(String dn, KeyPair pair, int d String hostAddress = InetAddress.getLocalHost().getHostAddress(); certGen.addExtension(X509Extensions.SubjectAlternativeName, false, - new GeneralNames(new GeneralName(GeneralName.iPAddress, hostAddress))); - + new GeneralNames(new GeneralName[] { new GeneralName(GeneralName.iPAddress, hostAddress), + new GeneralName(GeneralName.dNSName, "localhost") })); X500Principal dnName = new X500Principal(dn); certGen.setSerialNumber(sn); From 41cbc17aab3a2cc5fb177fc6a15dec867e795b72 Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Wed, 12 Jan 2022 08:11:58 -0500 Subject: [PATCH 348/512] TEZ-4276: Clean Up Use of Preconditions (#96) (David Mollitor reviewed by Laszlo Bodor) --- .../tez/dag/api/NamedEntityDescriptor.java | 5 +- .../java/org/apache/tez/dag/api/Vertex.java | 5 +- .../dag/api/VertexManagerPluginContext.java | 5 +- .../api/events/VertexManagerEvent.java | 17 ++++--- .../tez/common/TezContainerLogAppender.java | 13 ++--- .../org/apache/tez/dag/records/TezDAGID.java | 10 ++-- .../tez/dag/records/TezTaskAttemptID.java | 6 +-- .../org/apache/tez/dag/records/TezTaskID.java | 6 +-- .../apache/tez/dag/records/TezVertexID.java | 5 +- .../dag/app/TaskCommunicatorContextImpl.java | 8 ++-- .../tez/dag/app/dag/impl/VertexManager.java | 1 - .../tez/dag/app/rm/node/ExtendedNodeId.java | 4 +- .../split/TezGroupedSplitsInputFormat.java | 7 ++- .../mapred/split/TezMapredSplitsGrouper.java | 4 +- .../split/TezGroupedSplitsInputFormat.java | 7 ++- .../split/TezMapReduceSplitsGrouper.java | 4 +- .../mapreduce/grouper/TezSplitGrouper.java | 3 +- .../apache/tez/mapreduce/input/MRInput.java | 3 +- .../tez/mapreduce/input/MultiMRInput.java | 3 +- .../org/apache/tez/history/ATSImportTool.java | 4 +- .../tez/history/parser/ATSFileParser.java | 47 +++++++++---------- .../parser/datamodel/TaskAttemptInfo.java | 4 +- .../history/parser/datamodel/TaskInfo.java | 4 +- .../history/parser/datamodel/VertexInfo.java | 4 +- .../tez/runtime/api/impl/TaskStatistics.java | 4 +- .../java/org/apache/tez/http/SSLFactory.java | 9 ++-- .../runtime/library/common/ConfigUtils.java | 37 +++++++-------- .../library/common/shuffle/ShuffleUtils.java | 3 +- .../conf/OrderedGroupedKVInputConfig.java | 14 ++---- .../OrderedPartitionedKVOutputConfig.java | 18 +++---- .../library/conf/UnorderedKVInputConfig.java | 11 ++--- .../library/conf/UnorderedKVOutputConfig.java | 11 ++--- .../UnorderedPartitionedKVOutputConfig.java | 11 ++--- .../org/apache/tez/analyzer/CSVResult.java | 3 +- .../tez/analyzer/plugins/SkewAnalyzer.java | 5 +- 35 files changed, 149 insertions(+), 156 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java index 452deaa77b..86ae26c876 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java @@ -17,8 +17,8 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.Objects; -import org.apache.tez.common.Preconditions; import org.apache.hadoop.classification.InterfaceAudience; @SuppressWarnings("unchecked") @@ -28,8 +28,7 @@ public class NamedEntityDescriptor> extends E @InterfaceAudience.Private public NamedEntityDescriptor(String entityName, String className) { super(className); - Preconditions.checkArgument(entityName != null, "EntityName must be specified"); - this.entityName = entityName; + this.entityName = Objects.requireNonNull(entityName, "EntityName must be specified"); } public String getEntityName() { diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java b/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java index ccfa92a9f6..8ce3fb67e6 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java @@ -23,6 +23,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import org.apache.commons.lang.StringUtils; import org.apache.hadoop.classification.InterfaceAudience; @@ -276,10 +277,10 @@ public Map getTaskLocalFiles() { * set environment for all vertices via Tezconfiguration#TEZ_TASK_LAUNCH_ENV * @param environment * @return this Vertex + * NullPointerException if {@code environment} is {@code null} */ public Vertex setTaskEnvironment(Map environment) { - Preconditions.checkArgument(environment != null); - this.taskEnvironment.putAll(environment); + this.taskEnvironment.putAll(Objects.requireNonNull(environment)); return this; } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/VertexManagerPluginContext.java b/tez-api/src/main/java/org/apache/tez/dag/api/VertexManagerPluginContext.java index ba43ecd9c1..0862f49034 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/VertexManagerPluginContext.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/VertexManagerPluginContext.java @@ -21,6 +21,7 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import javax.annotation.Nullable; @@ -29,7 +30,6 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.tez.dag.api.event.VertexState; -import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.InputSpecUpdate; import org.apache.tez.runtime.api.VertexStatistics; import org.apache.tez.runtime.api.events.CustomProcessorEvent; @@ -72,8 +72,7 @@ public class TaskWithLocationHint { Integer taskIndex; TaskLocationHint locationHint; public TaskWithLocationHint(Integer taskIndex, @Nullable TaskLocationHint locationHint) { - Preconditions.checkState(taskIndex != null); - this.taskIndex = taskIndex; + this.taskIndex = Objects.requireNonNull(taskIndex); this.locationHint = locationHint; } diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/events/VertexManagerEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/VertexManagerEvent.java index 6e5738918c..7a450c6cdf 100644 --- a/tez-api/src/main/java/org/apache/tez/runtime/api/events/VertexManagerEvent.java +++ b/tez-api/src/main/java/org/apache/tez/runtime/api/events/VertexManagerEvent.java @@ -19,6 +19,7 @@ package org.apache.tez.runtime.api.events; import java.nio.ByteBuffer; +import java.util.Objects; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; @@ -27,8 +28,6 @@ import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.TaskAttemptIdentifier; -import org.apache.tez.common.Preconditions; - /** * Event used to send information from a Task to the VertexManager for a vertex. * This may be used to send statistics like samples etc to the VertexManager for @@ -50,11 +49,17 @@ public class VertexManagerEvent extends Event { */ private final ByteBuffer userPayload; + /** + * Constructor. + * + * @param vertexName + * @param userPayload + * @throws NullPointerException if {@code vertexName} or {@code userPayload} + * is {@code null} + */ private VertexManagerEvent(String vertexName, ByteBuffer userPayload) { - Preconditions.checkArgument(vertexName != null); - Preconditions.checkArgument(userPayload != null); - this.targetVertexName = vertexName; - this.userPayload = userPayload; + this.targetVertexName = Objects.requireNonNull(vertexName); + this.userPayload = Objects.requireNonNull(userPayload); } /** diff --git a/tez-common/src/main/java/org/apache/tez/common/TezContainerLogAppender.java b/tez-common/src/main/java/org/apache/tez/common/TezContainerLogAppender.java index 2cfacfb08e..1649a0b318 100644 --- a/tez-common/src/main/java/org/apache/tez/common/TezContainerLogAppender.java +++ b/tez-common/src/main/java/org/apache/tez/common/TezContainerLogAppender.java @@ -19,6 +19,7 @@ package org.apache.tez.common; import java.io.File; +import java.util.Objects; import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.log4j.FileAppender; @@ -48,14 +49,14 @@ public void activateOptions() { * The file will be created within the container's log directory. * * @param fileName + * @throws NullPointerException if {@code fileName} is {@code null} + * @throws IllegalArgumentException if {@code fileName} is an absolute path */ public void setLogFileName(String fileName) { - if (fileName == null || fileName.contains(File.pathSeparator)) { - throw new RuntimeException( - "Invalid filename specified: " - + fileName - + " . FileName should not have a path component and should not be empty."); - } + Objects.requireNonNull(fileName); + Preconditions.checkArgument(!fileName.contains(File.pathSeparator), + "Invalid filename specified: " + fileName + + " . FileName should not have a path component and should not be empty."); this.logFileName = fileName; } diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezDAGID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezDAGID.java index 68184fc8c0..c46aa6088b 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezDAGID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezDAGID.java @@ -21,10 +21,10 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.Objects; import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.tez.common.Preconditions; import org.apache.tez.util.FastNumberFormat; import com.google.common.collect.Interner; @@ -49,12 +49,13 @@ public class TezDAGID extends TezID { * Get a DAGID object from given {@link ApplicationId}. * @param applicationId Application that this dag belongs to * @param id the dag number + * @throws NullPointerException if {@code obj} is {@code applicationId} */ public static TezDAGID getInstance(ApplicationId applicationId, int id) { // The newly created TezDAGIds are primarily for their hashCode method, and // will be short-lived. // Alternately the cache can be keyed by the hash of the incoming paramters. - Preconditions.checkArgument(applicationId != null, "ApplicationID cannot be null"); + Objects.requireNonNull(applicationId, "ApplicationID cannot be null"); return tezDAGIDCache.intern(new TezDAGID(applicationId, id)); } @@ -63,15 +64,16 @@ public static TezDAGID getInstance(ApplicationId applicationId, int id) { * @param yarnRMIdentifier YARN RM identifier * @param appId application number * @param id the dag number + * @throws NullPointerException if {@code yarnRMIdentifier} is {@code null} */ public static TezDAGID getInstance(String yarnRMIdentifier, int appId, int id) { // The newly created TezDAGIds are primarily for their hashCode method, and // will be short-lived. // Alternately the cache can be keyed by the hash of the incoming paramters. - Preconditions.checkArgument(yarnRMIdentifier != null, "yarnRMIdentifier cannot be null"); + Objects.requireNonNull(yarnRMIdentifier, "yarnRMIdentifier cannot be null"); return tezDAGIDCache.intern(new TezDAGID(yarnRMIdentifier, appId, id)); } - + // Public for Writable serialization. Verify if this is actually required. public TezDAGID() { } diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java index 9ce1b10aa0..7c92be9449 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java @@ -21,6 +21,7 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.Objects; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -59,16 +60,15 @@ public TezTaskAttemptID() { * Constructs a TaskAttemptID object from given {@link TezTaskID}. * @param taskID TaskID that this task belongs to * @param id the task attempt number + * @throws NullPointerException if {@code taskID} is {@code null} */ public static TezTaskAttemptID getInstance(TezTaskID taskID, int id) { + Objects.requireNonNull(taskID); return tezTaskAttemptIDCache.intern(new TezTaskAttemptID(taskID, id)); } private TezTaskAttemptID(TezTaskID taskId, int id) { super(id); - if(taskId == null) { - throw new IllegalArgumentException("taskId cannot be null"); - } this.taskId = taskId; } diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java index 993df7c87a..a1a556c9ba 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java @@ -21,11 +21,11 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.Objects; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.tez.common.Preconditions; import org.apache.tez.util.FastNumberFormat; import com.google.common.collect.Interner; @@ -61,15 +61,15 @@ public FastNumberFormat initialValue() { * Constructs a TezTaskID object from given {@link TezVertexID}. * @param vertexID the vertexID object for this TezTaskID * @param id the tip number + * @throws NullPointerException if {@code vertexID} is {@code null} */ public static TezTaskID getInstance(TezVertexID vertexID, int id) { - Preconditions.checkArgument(vertexID != null, "vertexID cannot be null"); + Objects.requireNonNull(vertexID, "vertexID cannot be null"); return tezTaskIDCache.intern(new TezTaskID(vertexID, id)); } private TezTaskID(TezVertexID vertexID, int id) { super(id); - Preconditions.checkArgument(vertexID != null, "vertexID cannot be null"); this.vertexId = vertexID; this.serializingHash = getHashCode(true); } diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java index 5ecfca6a49..e9ddd77d38 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java @@ -21,11 +21,11 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.Objects; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.tez.common.Preconditions; import org.apache.tez.util.FastNumberFormat; import com.google.common.collect.Interner; @@ -67,9 +67,10 @@ public TezVertexID() { * Constructs a TezVertexID object from given {@link TezDAGID}. * @param dagId TezDAGID object for this TezVertexID * @param id the tip number + * @throws NullPointerException if {@code dagId} is {@code null} */ public static TezVertexID getInstance(TezDAGID dagId, int id) { - Preconditions.checkArgument(dagId != null, "DagID cannot be null"); + Objects.requireNonNull(dagId, "DagID cannot be null"); return tezVertexIDCache.intern(new TezVertexID(dagId, id)); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java index faa6fe1967..4f9e8723aa 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java @@ -191,7 +191,7 @@ public String apply(Vertex input) { @Override public int getVertexTotalTaskCount(String vertexName) { - Preconditions.checkArgument(vertexName != null, "VertexName must be specified"); + Objects.requireNonNull(vertexName, "VertexName must be specified"); DAG dag = getDag(); Vertex vertex = dag.getVertex(vertexName); return vertex.getTotalTasks(); @@ -199,7 +199,7 @@ public int getVertexTotalTaskCount(String vertexName) { @Override public int getVertexCompletedTaskCount(String vertexName) { - Preconditions.checkArgument(vertexName != null, "VertexName must be specified"); + Objects.requireNonNull(vertexName, "VertexName must be specified"); DAG dag = getDag(); Vertex vertex = dag.getVertex(vertexName); return vertex.getCompletedTasks(); @@ -207,7 +207,7 @@ public int getVertexCompletedTaskCount(String vertexName) { @Override public int getVertexRunningTaskCount(String vertexName) { - Preconditions.checkArgument(vertexName != null, "VertexName must be specified"); + Objects.requireNonNull(vertexName, "VertexName must be specified"); DAG dag = getDag(); Vertex vertex = dag.getVertex(vertexName); return vertex.getRunningTasks(); @@ -215,7 +215,7 @@ public int getVertexRunningTaskCount(String vertexName) { @Override public long getFirstAttemptStartTime(String vertexName, int taskIndex) { - Preconditions.checkArgument(vertexName != null, "VertexName must be specified"); + Objects.requireNonNull(vertexName, "VertexName must be specified"); Preconditions.checkArgument(taskIndex >=0, "TaskIndex must be > 0"); DAG dag = getDag(); Vertex vertex = dag.getVertex(vertexName); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java index 292742530a..57c8c72be1 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java @@ -425,7 +425,6 @@ public VertexManager(VertexManagerPluginDescriptor pluginDesc, UserGroupInformat this.rootInputInitEventQueue = new LinkedBlockingQueue(); pluginContext = new VertexManagerPluginContextImpl(); - Preconditions.checkArgument(pluginDesc != null); payload = pluginDesc.getUserPayload(); pluginFailed = new AtomicBoolean(false); plugin = ReflectionUtils.createClazzInstance(pluginDesc.getClassName(), diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/ExtendedNodeId.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/ExtendedNodeId.java index ea58d8624c..07b2dd4252 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/ExtendedNodeId.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/ExtendedNodeId.java @@ -21,7 +21,6 @@ import java.util.Objects; import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.tez.common.Preconditions; /** * ExtendedNodeId extends NodeId with unique identifier in addition to hostname and port. @@ -33,8 +32,7 @@ public class ExtendedNodeId extends NodeId { private final String uniqueIdentifier; public ExtendedNodeId(NodeId nodeId, String uniqueIdentifier) { - Preconditions.checkArgument(nodeId != null); - this.nodeId = nodeId; + this.nodeId = Objects.requireNonNull(nodeId); this.uniqueIdentifier = uniqueIdentifier == null ? "" : uniqueIdentifier.trim(); } diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java index 61e1f6c431..61ba560300 100644 --- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java +++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java @@ -19,6 +19,7 @@ package org.apache.hadoop.mapred.split; import java.io.IOException; +import java.util.Objects; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,14 +67,12 @@ public void setInputFormat(InputFormat wrappedInputFormat) { } public void setSplitSizeEstimator(SplitSizeEstimator estimator) { - Preconditions.checkArgument(estimator != null); - this.estimator = estimator; + this.estimator = Objects.requireNonNull(estimator); LOG.debug("Split size estimator : {}", estimator); } public void setSplitLocationProvider(SplitLocationProvider locationProvider) { - Preconditions.checkArgument(locationProvider != null); - this.locationProvider = locationProvider; + this.locationProvider = Objects.requireNonNull(locationProvider); LOG.debug("Split size location provider: {}", locationProvider); } diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezMapredSplitsGrouper.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezMapredSplitsGrouper.java index ccaccd627f..38a213ec9a 100644 --- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezMapredSplitsGrouper.java +++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezMapredSplitsGrouper.java @@ -21,9 +21,9 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; +import java.util.Objects; import com.google.common.base.Function; -import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import org.apache.tez.mapreduce.grouper.GroupedSplitContainer; import org.apache.tez.mapreduce.grouper.MapredSplitContainer; @@ -66,7 +66,7 @@ public InputSplit[] getGroupedSplits(Configuration conf, public InputSplit[] getGroupedSplits(Configuration conf, InputSplit[] originalSplits, int desiredNumSplits, String wrappedInputFormatName, SplitSizeEstimator estimator, SplitLocationProvider locationProvider) throws IOException { - Preconditions.checkArgument(originalSplits != null, "Splits must be specified"); + Objects.requireNonNull(originalSplits, "Splits must be specified"); List originalSplitContainers = Lists.transform(Arrays.asList(originalSplits), new Function() { diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezGroupedSplitsInputFormat.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezGroupedSplitsInputFormat.java index 863f9aa792..b85e97e467 100644 --- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezGroupedSplitsInputFormat.java +++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezGroupedSplitsInputFormat.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Objects; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -74,14 +75,12 @@ public void setDesiredNumberOfSplits(int num) { } public void setSplitSizeEstimator(SplitSizeEstimator estimator) { - Preconditions.checkArgument(estimator != null); - this.estimator = estimator; + this.estimator = Objects.requireNonNull(estimator); LOG.debug("Split size estimator : {}", estimator); } public void setSplitLocationProvider(SplitLocationProvider locationProvider) { - Preconditions.checkArgument(locationProvider != null); - this.locationProvider = locationProvider; + this.locationProvider = Objects.requireNonNull(locationProvider); LOG.debug("Split location provider : {}", locationProvider); } diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezMapReduceSplitsGrouper.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezMapReduceSplitsGrouper.java index 629f01e8e5..4f638e1400 100644 --- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezMapReduceSplitsGrouper.java +++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezMapReduceSplitsGrouper.java @@ -20,11 +20,11 @@ import java.io.IOException; import java.util.List; +import java.util.Objects; import javax.annotation.Nullable; import com.google.common.base.Function; -import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import org.apache.tez.mapreduce.grouper.GroupedSplitContainer; import org.apache.tez.mapreduce.grouper.MapReduceSplitContainer; @@ -154,7 +154,7 @@ public List getGroupedSplits(Configuration conf, SplitSizeEstimator estimator, SplitLocationProvider locationProvider) throws IOException, InterruptedException { - Preconditions.checkArgument(originalSplits != null, "Splits must be specified"); + Objects.requireNonNull(originalSplits, "Splits must be specified"); List originalSplitContainers = Lists.transform(originalSplits, new Function() { @Override diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java index 6f3d9df033..a1d6b6c806 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java @@ -22,6 +22,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.TreeMap; @@ -166,7 +167,7 @@ public List getGroupedSplits(Configuration conf, SplitLocationProviderWrapper locationProvider) throws IOException, InterruptedException { LOG.info("Grouping splits in Tez"); - Preconditions.checkArgument(originalSplits != null, "Splits must be specified"); + Objects.requireNonNull(originalSplits, "Splits must be specified"); int configNumSplits = conf.getInt(TEZ_GROUPING_SPLIT_COUNT, 0); if (configNumSplits > 0) { diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java index 891249b63e..964c0b8301 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java @@ -23,6 +23,7 @@ import java.net.URI; import java.util.Collection; import java.util.List; +import java.util.Objects; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; @@ -672,7 +673,7 @@ private void initFromEventInternal(InputDataInformationEvent initEvent) throws I if (LOG.isDebugEnabled()) { LOG.debug(getContext().getSourceVertexName() + " initializing RecordReader from event"); } - Preconditions.checkState(initEvent != null, "InitEvent must be specified"); + Objects.requireNonNull(initEvent, "InitEvent must be specified"); MRSplitProto splitProto = MRSplitProto.parseFrom(ByteString.copyFrom(initEvent.getUserPayload())); Object splitObj = null; long splitLength = -1; diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java index ee776a59f2..3bc8629565 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java @@ -25,6 +25,7 @@ import java.util.Collections; import java.util.LinkedList; import java.util.List; +import java.util.Objects; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; @@ -164,7 +165,7 @@ public void handleEvents(List inputEvents) throws Exception { } private MRReader initFromEvent(InputDataInformationEvent event) throws IOException { - Preconditions.checkState(event != null, "Event must be specified"); + Objects.requireNonNull(event, "Event must be specified"); if (LOG.isDebugEnabled()) { LOG.debug(getContext().getSourceVertexName() + " initializing Reader: " + eventCount.get()); } diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/ATSImportTool.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/ATSImportTool.java index f05e946acc..c909f7aa0d 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/ATSImportTool.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/ATSImportTool.java @@ -69,6 +69,7 @@ import java.net.URL; import java.net.URLEncoder; import java.util.Iterator; +import java.util.Objects; import java.util.zip.ZipEntry; import java.util.zip.ZipOutputStream; @@ -242,11 +243,12 @@ private void downloadData(ZipOutputStream zos) throws TezException, JSONExceptio * @throws IOException * @throws TezException * @throws JSONException + * @throws NullPointerException if {@code zos} is {@code null} */ private void downloadJSONArrayFromATS(String url, ZipOutputStream zos, String tag) throws IOException, TezException, JSONException { - Preconditions.checkArgument(zos != null, "ZipOutputStream can not be null"); + Objects.requireNonNull(zos, "ZipOutputStream can not be null"); String baseUrl = url; JSONArray entities; diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java index e64fb43f94..caeb406f21 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java @@ -42,6 +42,7 @@ import java.util.Enumeration; import java.util.Iterator; import java.util.List; +import java.util.Objects; import java.util.zip.ZipEntry; import java.util.zip.ZipFile; @@ -89,18 +90,17 @@ public DagInfo getDAGData(String dagId) throws TezException { * * @param verticesJson * @throws JSONException + * @throws NullPointerException if {@code verticesJson} is {@code null} */ private void processVertices(JSONArray verticesJson) throws JSONException { //Process vertex information - Preconditions.checkState(verticesJson != null, "Vertex json can not be null"); - if (verticesJson != null) { - LOG.debug("Started parsing vertex"); - for (int i = 0; i < verticesJson.length(); i++) { - VertexInfo vertexInfo = VertexInfo.create(verticesJson.getJSONObject(i)); - vertexList.add(vertexInfo); - } - LOG.debug("Finished parsing vertex"); + Objects.requireNonNull(verticesJson, "Vertex json cannot be null"); + LOG.debug("Started parsing vertex"); + for (int i = 0; i < verticesJson.length(); i++) { + VertexInfo vertexInfo = VertexInfo.create(verticesJson.getJSONObject(i)); + vertexList.add(vertexInfo); } + LOG.debug("Finished parsing vertex"); } /** @@ -108,18 +108,17 @@ private void processVertices(JSONArray verticesJson) throws JSONException { * * @param tasksJson * @throws JSONException + * @throws NullPointerException if {@code verticesJson} is {@code null} */ private void processTasks(JSONArray tasksJson) throws JSONException { //Process Task information - Preconditions.checkState(tasksJson != null, "Task json can not be null"); - if (tasksJson != null) { - LOG.debug("Started parsing task"); - for (int i = 0; i < tasksJson.length(); i++) { - TaskInfo taskInfo = TaskInfo.create(tasksJson.getJSONObject(i)); - taskList.add(taskInfo); - } - LOG.debug("Finished parsing task"); + Objects.requireNonNull(tasksJson, "Task json can not be null"); + LOG.debug("Started parsing task"); + for (int i = 0; i < tasksJson.length(); i++) { + TaskInfo taskInfo = TaskInfo.create(tasksJson.getJSONObject(i)); + taskList.add(taskInfo); } + LOG.debug("Finished parsing task"); } /** @@ -127,18 +126,18 @@ private void processTasks(JSONArray tasksJson) throws JSONException { * * @param taskAttemptsJson * @throws JSONException + * @throws NullPointerException if {@code taskAttemptsJson} is {@code null} */ private void processAttempts(JSONArray taskAttemptsJson) throws JSONException { //Process TaskAttempt information - Preconditions.checkState(taskAttemptsJson != null, "Attempts json can not be null"); - if (taskAttemptsJson != null) { - LOG.debug("Started parsing task attempts"); - for (int i = 0; i < taskAttemptsJson.length(); i++) { - TaskAttemptInfo attemptInfo = TaskAttemptInfo.create(taskAttemptsJson.getJSONObject(i)); - attemptList.add(attemptInfo); - } - LOG.debug("Finished parsing task attempts"); + Objects.requireNonNull(taskAttemptsJson, "Attempts json can not be null"); + LOG.debug("Started parsing task attempts"); + for (int i = 0; i < taskAttemptsJson.length(); i++) { + TaskAttemptInfo attemptInfo = + TaskAttemptInfo.create(taskAttemptsJson.getJSONObject(i)); + attemptList.add(attemptInfo); } + LOG.debug("Finished parsing task attempts"); } /** diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java index c8d400cc35..c45fcf5ff3 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java @@ -41,6 +41,7 @@ import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.Objects; import static org.apache.hadoop.classification.InterfaceStability.Evolving; import static org.apache.hadoop.classification.InterfaceAudience.Public; @@ -169,8 +170,7 @@ public int compare(TaskAttemptInfo o1, TaskAttemptInfo o2) { } void setTaskInfo(TaskInfo taskInfo) { - Preconditions.checkArgument(taskInfo != null, "Provide valid taskInfo"); - this.taskInfo = taskInfo; + this.taskInfo = Objects.requireNonNull(taskInfo, "Provide valid taskInfo"); taskInfo.addTaskAttemptInfo(this); } diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskInfo.java index 43886fa5c6..fd01093244 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskInfo.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskInfo.java @@ -41,6 +41,7 @@ import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.Objects; import static org.apache.hadoop.classification.InterfaceAudience.Public; import static org.apache.hadoop.classification.InterfaceStability.Evolving; @@ -153,8 +154,7 @@ void addTaskAttemptInfo(TaskAttemptInfo taskAttemptInfo) { } void setVertexInfo(VertexInfo vertexInfo) { - Preconditions.checkArgument(vertexInfo != null, "Provide valid vertexInfo"); - this.vertexInfo = vertexInfo; + this.vertexInfo = Objects.requireNonNull(vertexInfo, "Provide valid vertexInfo"); //link it to vertex vertexInfo.addTaskInfo(this); } diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java index c196c48cd4..106f2bc729 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java @@ -41,6 +41,7 @@ import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.Objects; import static org.apache.hadoop.classification.InterfaceAudience.Public; import static org.apache.hadoop.classification.InterfaceStability.Evolving; @@ -198,8 +199,7 @@ void addOutEdge(EdgeInfo edgeInfo) { } void setDagInfo(DagInfo dagInfo) { - Preconditions.checkArgument(dagInfo != null, "Provide valid dagInfo"); - this.dagInfo = dagInfo; + this.dagInfo = Objects.requireNonNull(dagInfo, "Provide valid dagInfo"); //link vertex to dagInfo dagInfo.addVertexInfo(this); updateEdgeInfo(); diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskStatistics.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskStatistics.java index d9825c9f43..961c28ef60 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskStatistics.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskStatistics.java @@ -21,11 +21,11 @@ import java.io.DataOutput; import java.io.IOException; import java.util.Map; +import java.util.Objects; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; -import org.apache.tez.common.Preconditions; import org.apache.tez.util.StringInterner; import com.google.common.collect.Maps; @@ -40,7 +40,7 @@ public void addIO(String edgeName) { } public void addIO(String edgeName, IOStatistics stats) { - Preconditions.checkArgument(stats != null, edgeName); + Objects.requireNonNull(stats, edgeName); ioStatistics.put(StringInterner.intern(edgeName), stats); } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java b/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java index 4147be8980..8c98764bfe 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java @@ -41,6 +41,7 @@ import java.io.IOException; import java.net.HttpURLConnection; import java.security.GeneralSecurityException; +import java.util.Objects; import static org.apache.hadoop.security.ssl.SSLFactory.KEYSTORES_FACTORY_CLASS_KEY; import static org.apache.hadoop.security.ssl.SSLFactory.SSL_CLIENT_CONF_KEY; @@ -81,13 +82,11 @@ public class SSLFactory implements ConnectionConfigurator { * @param mode SSLFactory mode, client or server. * @param conf Hadoop configuration from where the SSLFactory configuration * will be read. + * @throws NullPointerException if {@code mode} or {@code conf} is {@code null} */ public SSLFactory(Mode mode, Configuration conf) { - this.conf = conf; - if (mode == null) { - throw new IllegalArgumentException("mode cannot be NULL"); - } - this.mode = mode; + this.conf = Objects.requireNonNull(conf); + this.mode = Objects.requireNonNull(mode, "mode cannot be NULL"); requireClientCert = conf.getBoolean(SSL_REQUIRE_CLIENT_CERT_KEY, DEFAULT_SSL_REQUIRE_CLIENT_CERT); // Rest of ssl configs are pre-populated in incoming conf payload diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java index f83fdc99d7..74e74f2780 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java @@ -22,10 +22,9 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; -import org.apache.tez.common.Preconditions; - import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.RawComparator; @@ -134,8 +133,8 @@ public static boolean useNewApi(Configuration conf) { @InterfaceAudience.Private public static Map extractConfigurationMap(Map confMap, Set allowedKeys) { - Preconditions.checkArgument(confMap != null, "ConfMap cannot be null"); - Preconditions.checkArgument(allowedKeys != null, "Valid key set cannot be empty"); + Objects.requireNonNull(confMap, "ConfMap cannot be null"); + Objects.requireNonNull(allowedKeys, "Valid key set cannot be empty"); Map map = new HashMap(); for (Map.Entry entry : confMap.entrySet()) { if (allowedKeys.contains(entry.getKey())) { @@ -147,8 +146,8 @@ public static Map extractConfigurationMap(Map co @InterfaceAudience.Private public static void addConfigMapToConfiguration(Configuration conf, Map confMap) { - Preconditions.checkArgument(conf != null, "Configuration cannot be null"); - Preconditions.checkArgument(confMap != null, "Configuration map cannot be null"); + Objects.requireNonNull(conf, "Configuration cannot be null"); + Objects.requireNonNull(confMap, "Configuration map cannot be null"); for (Map.Entry entry : confMap.entrySet()) { conf.set(entry.getKey(), entry.getValue()); } @@ -158,9 +157,9 @@ public static void addConfigMapToConfiguration(Configuration conf, Map extractConfigurationMap(Map confMap, List> validKeySets, List allowedPrefixes) { - Preconditions.checkArgument(confMap != null, "ConfMap cannot be null"); - Preconditions.checkArgument(validKeySets != null, "Valid key set cannot be empty"); - Preconditions.checkArgument(allowedPrefixes != null, "Allowed prefixes cannot be null"); + Objects.requireNonNull(confMap, "ConfMap cannot be null"); + Objects.requireNonNull(validKeySets, "Valid key set cannot be empty"); + Objects.requireNonNull(allowedPrefixes, "Allowed prefixes cannot be null"); return extractConfigurationMapInternal(confMap.entrySet(), validKeySets, allowedPrefixes); } @@ -169,17 +168,17 @@ public static Map extractConfigurationMap(Map co public static Map extractConfigurationMap(Configuration conf, List> validKeySets, List allowedPrefixes) { - Preconditions.checkArgument(conf != null, "conf cannot be null"); - Preconditions.checkArgument(validKeySets != null, "Valid key set cannot be empty"); - Preconditions.checkArgument(allowedPrefixes != null, "Allowed prefixes cannot be null"); + Objects.requireNonNull(conf, "conf cannot be null"); + Objects.requireNonNull(validKeySets, "Valid key set cannot be empty"); + Objects.requireNonNull(allowedPrefixes, "Allowed prefixes cannot be null"); return extractConfigurationMapInternal(conf, validKeySets, allowedPrefixes); } @InterfaceAudience.Private public static boolean doesKeyQualify(String key, List> validKeySets, List allowedPrefixes) { - Preconditions.checkArgument(key != null, "key cannot be null"); - Preconditions.checkArgument(validKeySets != null, "Valid key set cannot be empty"); - Preconditions.checkArgument(allowedPrefixes != null, "Allowed prefixes cannot be null"); + Objects.requireNonNull(key, "key cannot be null"); + Objects.requireNonNull(validKeySets, "Valid key set cannot be empty"); + Objects.requireNonNull(allowedPrefixes, "Allowed prefixes cannot be null"); for (Set set : validKeySets) { if (set.contains(key)) { return true; @@ -195,8 +194,8 @@ public static boolean doesKeyQualify(String key, List> validKeySets, @InterfaceAudience.Private public static void mergeConfsWithExclusions(Configuration destConf, Map srcConf, Set excludedKeySet) { - Preconditions.checkState(destConf != null, "Destination conf cannot be null"); - Preconditions.checkState(srcConf != null, "Source conf cannot be null"); + Objects.requireNonNull(destConf, "Destination conf cannot be null"); + Objects.requireNonNull(srcConf, "Source conf cannot be null"); for (Map.Entry entry : srcConf.entrySet()) { if (!excludedKeySet.contains(entry.getKey())) { destConf.set(entry.getKey(), entry.getValue()); @@ -205,8 +204,8 @@ public static void mergeConfsWithExclusions(Configuration destConf, Map entry : srcConf) { // Explicit get to have parameter replacement work. String val = srcConf.get(entry.getKey()); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java index 6a61474a5a..ca8b84724e 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java @@ -29,6 +29,7 @@ import java.util.BitSet; import java.util.Collection; import java.util.List; +import java.util.Objects; import java.util.concurrent.atomic.AtomicLong; import java.util.zip.Deflater; @@ -407,7 +408,7 @@ public static void generateEventOnSpill(List eventList, boolean finalMerg int numPhysicalOutputs, boolean sendEmptyPartitionDetails, String pathComponent, @Nullable long[] partitionStats, boolean reportDetailedPartitionStats, String auxiliaryService, Deflater deflater) throws IOException { - Preconditions.checkArgument(eventList != null, "EventList can't be null"); + Objects.requireNonNull(eventList, "EventList can't be null"); context.notifyProgress(); if (finalMergeEnabled) { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedGroupedKVInputConfig.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedGroupedKVInputConfig.java index e7b96930a9..d9c1d21403 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedGroupedKVInputConfig.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedGroupedKVInputConfig.java @@ -27,7 +27,6 @@ import java.util.Objects; import com.google.common.annotations.VisibleForTesting; -import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import org.apache.hadoop.classification.InterfaceAudience; @@ -448,7 +447,7 @@ public Builder setAdditionalConfiguration(Map confMap) { @Override public Builder setFromConfiguration(Configuration conf) { // Maybe ensure this is the first call ? Otherwise this can end up overriding other parameters - Preconditions.checkArgument(conf != null, "Configuration cannot be null"); + Objects.requireNonNull(conf, "Configuration cannot be null"); Map map = ConfigUtils.extractConfigurationMap(conf, Lists.newArrayList(OrderedGroupedKVInput.getConfigurationKeySet(), TezRuntimeConfiguration.getRuntimeAdditionalConfigKeySet()), TezRuntimeConfiguration.getAllowedPrefixes()); @@ -460,7 +459,7 @@ public Builder setFromConfiguration(Configuration conf) { @Override public Builder setFromConfigurationUnfiltered(Configuration conf) { // Maybe ensure this is the first call ? Otherwise this can end up overriding other parameters - Preconditions.checkArgument(conf != null, "Configuration cannot be null"); + Objects.requireNonNull(conf, "Configuration cannot be null"); ConfigUtils.mergeConfs(this.conf, conf); return this; } @@ -494,10 +493,8 @@ public Builder setCompression(boolean enabled, @Nullable String compressionCodec */ public Builder setKeySerializationClass(String serializationClassName, String comparatorClassName, @Nullable Map serializerConf) { - Preconditions.checkArgument(serializationClassName != null, - "serializationClassName cannot be null"); - Preconditions.checkArgument(comparatorClassName != null, - "comparator cannot be null"); + Objects.requireNonNull(serializationClassName, "serializationClassName cannot be null"); + Objects.requireNonNull(comparatorClassName, "comparator cannot be null"); this.conf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, serializationClassName + "," + conf.get(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY)); setKeyComparatorClass(comparatorClassName, null); @@ -520,8 +517,7 @@ public Builder setKeySerializationClass(String serializationClassName, */ public Builder setValueSerializationClass(String serializationClassName, @Nullable Map serializerConf) { - Preconditions.checkArgument(serializationClassName != null, - "serializationClassName cannot be null"); + Objects.requireNonNull(serializationClassName, "serializationClassName cannot be null"); this.conf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, serializationClassName + "," + conf.get(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY)); if (serializerConf != null) { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedPartitionedKVOutputConfig.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedPartitionedKVOutputConfig.java index 7c92f38260..9671feb8b9 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedPartitionedKVOutputConfig.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedPartitionedKVOutputConfig.java @@ -27,7 +27,6 @@ import java.util.Objects; import com.google.common.annotations.VisibleForTesting; -import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import org.apache.hadoop.classification.InterfaceAudience; @@ -374,7 +373,7 @@ public Builder setAdditionalConfiguration(Map confMap) { @Override public Builder setFromConfiguration(Configuration conf) { // Maybe ensure this is the first call ? Otherwise this can end up overriding other parameters - Preconditions.checkArgument(conf != null, "Configuration cannot be null"); + Objects.requireNonNull(conf, "Configuration cannot be null"); Map map = ConfigUtils.extractConfigurationMap(conf, Lists.newArrayList(OrderedPartitionedKVOutput.getConfigurationKeySet(), TezRuntimeConfiguration.getRuntimeAdditionalConfigKeySet()), TezRuntimeConfiguration.getAllowedPrefixes()); @@ -386,7 +385,7 @@ public Builder setFromConfiguration(Configuration conf) { @Override public Builder setFromConfigurationUnfiltered(Configuration conf) { // Maybe ensure this is the first call ? Otherwise this can end up overriding other parameters - Preconditions.checkArgument(conf != null, "Configuration cannot be null"); + Objects.requireNonNull(conf, "Configuration cannot be null"); ConfigUtils.mergeConfs(this.conf, conf); return this; } @@ -412,6 +411,7 @@ public Builder setKeyComparatorClass(String comparatorClassName) { * java.util.Map} of key-value pairs. The keys should be limited to * the ones required by the comparator. * @return instance of the current builder + * @throws NullPointerException if {@code comparatorClassName} is {@code null} */ public Builder setKeyComparatorClass(String comparatorClassName, @Nullable Map comparatorConf) { @@ -452,13 +452,13 @@ public Builder setCompression(boolean enabled, @Nullable String compressionCodec * {@link java.util.Map} of key-value pairs. The keys should be limited * to the ones required by the comparator. * @return this object for further chained method calls + * @throws NullPointerException if {@code serializationClassName} or + * {@code comparatorClassName} is {@code null} */ public Builder setKeySerializationClass(String serializationClassName, String comparatorClassName, @Nullable Map serializerConf) { - Preconditions.checkArgument(serializationClassName != null, - "serializationClassName cannot be null"); - Preconditions.checkArgument(comparatorClassName != null, - "comparator cannot be null"); + Objects.requireNonNull(serializationClassName, "serializationClassName cannot be null"); + Objects.requireNonNull(comparatorClassName, "comparator cannot be null"); this.conf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, serializationClassName + "," + conf.get(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY)); setKeyComparatorClass(comparatorClassName, null); @@ -478,11 +478,11 @@ public Builder setKeySerializationClass(String serializationClassName, * {@link java.util.Map} of key-value pairs. The keys should be limited * to the ones required by the comparator. * @return this object for further chained method calls + * @throws NullPointerException if {@code serializationClassName} is {@code null} */ public Builder setValueSerializationClass(String serializationClassName, @Nullable Map serializerConf) { - Preconditions.checkArgument(serializationClassName != null, - "serializationClassName cannot be null"); + Objects.requireNonNull(serializationClassName, "serializationClassName cannot be null"); this.conf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, serializationClassName + "," + conf.get(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY)); if (serializerConf != null) { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVInputConfig.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVInputConfig.java index 6ac0dbb3a8..6f285e7995 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVInputConfig.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVInputConfig.java @@ -27,7 +27,6 @@ import java.util.Objects; import com.google.common.annotations.VisibleForTesting; -import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import org.apache.hadoop.classification.InterfaceAudience; @@ -295,7 +294,7 @@ public Builder setAdditionalConfiguration(Map confMap) { @Override public Builder setFromConfiguration(Configuration conf) { // Maybe ensure this is the first call ? Otherwise this can end up overriding other parameters - Preconditions.checkArgument(conf != null, "Configuration cannot be null"); + Objects.requireNonNull(conf, "Configuration cannot be null"); Map map = ConfigUtils.extractConfigurationMap(conf, Lists.newArrayList(UnorderedKVInput.getConfigurationKeySet(), TezRuntimeConfiguration.getRuntimeAdditionalConfigKeySet()), TezRuntimeConfiguration.getAllowedPrefixes()); @@ -307,7 +306,7 @@ public Builder setFromConfiguration(Configuration conf) { @Override public Builder setFromConfigurationUnfiltered(Configuration conf) { // Maybe ensure this is the first call ? Otherwise this can end up overriding other parameters - Preconditions.checkArgument(conf != null, "Configuration cannot be null"); + Objects.requireNonNull(conf, "Configuration cannot be null"); ConfigUtils.mergeConfs(this.conf, conf); return this; } @@ -339,8 +338,7 @@ public Builder setCompression(boolean enabled, @Nullable String compressionCodec */ public Builder setKeySerializationClass(String serializationClassName, @Nullable Map serializerConf) { - Preconditions.checkArgument(serializationClassName != null, - "serializationClassName cannot be null"); + Objects.requireNonNull(serializationClassName, "serializationClassName cannot be null"); this.conf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, serializationClassName + "," + conf.get(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY)); if (serializerConf != null) { @@ -362,8 +360,7 @@ public Builder setKeySerializationClass(String serializationClassName, */ public Builder setValueSerializationClass(String serializationClassName, @Nullable Map serializerConf) { - Preconditions.checkArgument(serializationClassName != null, - "serializationClassName cannot be null"); + Objects.requireNonNull(serializationClassName, "serializationClassName cannot be null"); this.conf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, serializationClassName + "," + conf.get(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY)); if (serializerConf != null) { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVOutputConfig.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVOutputConfig.java index 34513badf4..a6ac7f2b5a 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVOutputConfig.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVOutputConfig.java @@ -27,7 +27,6 @@ import java.util.Objects; import com.google.common.annotations.VisibleForTesting; -import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import org.apache.hadoop.classification.InterfaceAudience; @@ -222,7 +221,7 @@ public Builder setAdditionalConfiguration(Map confMap) { @Override public Builder setFromConfiguration(Configuration conf) { // Maybe ensure this is the first call ? Otherwise this can end up overriding other parameters - Preconditions.checkArgument(conf != null, "Configuration cannot be null"); + Objects.requireNonNull(conf, "Configuration cannot be null"); Map map = ConfigUtils.extractConfigurationMap(conf, Lists.newArrayList(UnorderedKVOutput.getConfigurationKeySet(), TezRuntimeConfiguration.getRuntimeAdditionalConfigKeySet()), TezRuntimeConfiguration.getAllowedPrefixes()); @@ -234,7 +233,7 @@ public Builder setFromConfiguration(Configuration conf) { @Override public Builder setFromConfigurationUnfiltered(Configuration conf) { // Maybe ensure this is the first call ? Otherwise this can end up overriding other parameters - Preconditions.checkArgument(conf != null, "Configuration cannot be null"); + Objects.requireNonNull(conf, "Configuration cannot be null"); ConfigUtils.mergeConfs(this.conf, conf); return this; } @@ -250,8 +249,7 @@ public Builder setFromConfigurationUnfiltered(Configuration conf) { */ public Builder setKeySerializationClass(String serializationClassName, @Nullable Map serializerConf) { - Preconditions.checkArgument(serializationClassName != null, - "serializationClassName cannot be null"); + Objects.requireNonNull(serializationClassName, "serializationClassName cannot be null"); this.conf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, serializationClassName + "," + conf.get(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY)); if (serializerConf != null) { @@ -273,8 +271,7 @@ public Builder setKeySerializationClass(String serializationClassName, */ public Builder setValueSerializationClass(String serializationClassName, @Nullable Map serializerConf) { - Preconditions.checkArgument(serializationClassName != null, - "serializationClassName cannot be null"); + Objects.requireNonNull(serializationClassName, "serializationClassName cannot be null"); this.conf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, serializationClassName + "," + conf.get(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY)); if (serializerConf != null) { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedPartitionedKVOutputConfig.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedPartitionedKVOutputConfig.java index c924e7db66..dbbfb9ea6d 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedPartitionedKVOutputConfig.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedPartitionedKVOutputConfig.java @@ -27,7 +27,6 @@ import java.util.Objects; import com.google.common.annotations.VisibleForTesting; -import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import org.apache.hadoop.classification.InterfaceAudience; @@ -266,7 +265,7 @@ public Builder setAdditionalConfiguration(Map confMap) { @Override public Builder setFromConfiguration(Configuration conf) { // Maybe ensure this is the first call ? Otherwise this can end up overriding other parameters - Preconditions.checkArgument(conf != null, "Configuration cannot be null"); + Objects.requireNonNull(conf, "Configuration cannot be null"); Map map = ConfigUtils.extractConfigurationMap(conf, Lists.newArrayList(UnorderedPartitionedKVOutput.getConfigurationKeySet(), TezRuntimeConfiguration.getRuntimeAdditionalConfigKeySet()), TezRuntimeConfiguration.getAllowedPrefixes()); @@ -278,7 +277,7 @@ public Builder setFromConfiguration(Configuration conf) { @Override public Builder setFromConfigurationUnfiltered(Configuration conf) { // Maybe ensure this is the first call ? Otherwise this can end up overriding other parameters - Preconditions.checkArgument(conf != null, "Configuration cannot be null"); + Objects.requireNonNull(conf, "Configuration cannot be null"); ConfigUtils.mergeConfs(this.conf, conf); return this; } @@ -309,8 +308,7 @@ public Builder setCompression(boolean enabled, @Nullable String compressionCodec */ public Builder setKeySerializationClass(String serializationClassName, @Nullable Map serializerConf) { - Preconditions.checkArgument(serializationClassName != null, - "serializationClassName cannot be null"); + Objects.requireNonNull(serializationClassName, "serializationClassName cannot be null"); this.conf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, serializationClassName + "," + conf.get(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY)); if (serializerConf != null) { @@ -331,8 +329,7 @@ public Builder setKeySerializationClass(String serializationClassName, */ public Builder setValueSerializationClass(String serializationClassName, @Nullable Map serializerConf) { - Preconditions.checkArgument(serializationClassName != null, - "serializationClassName cannot be null"); + Objects.requireNonNull(serializationClassName, "serializationClassName cannot be null"); this.conf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, serializationClassName + "," + conf.get(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY)); if (serializerConf != null) { diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/CSVResult.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/CSVResult.java index 0e167b2dcf..1144f7643c 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/CSVResult.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/CSVResult.java @@ -36,6 +36,7 @@ import java.util.Comparator; import java.util.Iterator; import java.util.List; +import java.util.Objects; /** * Simple placeholder for storing CSV results. @@ -57,7 +58,7 @@ public String[] getHeaders() { } public void addRecord(String[] record) { - Preconditions.checkArgument(record != null, "Record can't be null"); + Objects.requireNonNull(record, "Record cannot be null"); Preconditions.checkArgument(record.length == headers.length, "Record length" + record.length + " does not match headers length " + headers.length); recordsList.add(record); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SkewAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SkewAnalyzer.java index a7d14fae25..b0387d1973 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SkewAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SkewAnalyzer.java @@ -18,7 +18,6 @@ package org.apache.tez.analyzer.plugins; -import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import org.apache.hadoop.conf.Configuration; @@ -34,6 +33,7 @@ import java.util.List; import java.util.Map; +import java.util.Objects; /** @@ -101,8 +101,7 @@ public SkewAnalyzer(Configuration config) { @Override public void analyze(DagInfo dagInfo) throws TezException { - Preconditions.checkArgument(dagInfo != null, "DAG can't be null"); - analyzeReducers(dagInfo); + analyzeReducers(Objects.requireNonNull(dagInfo, "DAG can't be null")); } private void analyzeReducers(DagInfo dagInfo) { From 0b3f3b69f9eefc883b69439a37d93122904ee574 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Sun, 16 Jan 2022 09:42:08 +0100 Subject: [PATCH 349/512] TEZ-4376: Remove javax.security from the tez-auxservices shaded jar (#177) (Laszlo Bodor reviewed by Jonathan Eagles) --- tez-plugins/tez-aux-services/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/tez-plugins/tez-aux-services/pom.xml b/tez-plugins/tez-aux-services/pom.xml index 2a936d6fa5..76f431b3ac 100644 --- a/tez-plugins/tez-aux-services/pom.xml +++ b/tez-plugins/tez-aux-services/pom.xml @@ -246,6 +246,7 @@ org.apache.tez.shaded.$0 javax.crypto.* + javax.security.** From cc8249e932c8c5b117c901188dbd92bf6f8d5391 Mon Sep 17 00:00:00 2001 From: csjuhasz-c <66361392+csjuhasz-c@users.noreply.github.com> Date: Mon, 17 Jan 2022 14:15:27 +0100 Subject: [PATCH 350/512] TEZ-4340: Show convenient input -> output vertex names in input messages (#170) (Csaba Juhasz reviewed by Laszlo Bodor) --- .../apache/tez/runtime/api/InputContext.java | 6 ++ .../apache/tez/mapreduce/input/MRInput.java | 16 ++-- .../tez/mapreduce/input/MRInputLegacy.java | 4 +- .../tez/mapreduce/input/MultiMRInput.java | 10 +-- .../runtime/api/impl/TezInputContextImpl.java | 5 ++ .../library/common/shuffle/Fetcher.java | 44 ++++----- .../shuffle/FetcherWithInjectableErrors.java | 16 ++-- .../impl/ShuffleInputEventHandlerImpl.java | 2 +- .../common/shuffle/impl/ShuffleManager.java | 62 ++++++------- .../orderedgrouped/FetcherOrderedGrouped.java | 16 ++-- ...herOrderedGroupedWithInjectableErrors.java | 18 ++-- .../shuffle/orderedgrouped/MergeManager.java | 17 ++-- .../shuffle/orderedgrouped/Shuffle.java | 35 ++++---- ...huffleInputEventHandlerOrderedGrouped.java | 2 +- .../orderedgrouped/ShuffleScheduler.java | 18 ++-- .../library/input/OrderedGroupedKVInput.java | 4 +- .../library/input/UnorderedKVInput.java | 4 +- .../library/common/shuffle/TestFetcher.java | 25 ++++-- .../TestShuffleInputEventHandlerImpl.java | 1 + .../shuffle/impl/TestShuffleManager.java | 1 + .../shuffle/orderedgrouped/TestFetcher.java | 90 ++++++++----------- .../shuffle/orderedgrouped/TestShuffle.java | 1 + .../java/org/apache/tez/test/TestInput.java | 3 +- 23 files changed, 209 insertions(+), 191 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/InputContext.java b/tez-api/src/main/java/org/apache/tez/runtime/api/InputContext.java index 479a7dba1b..6eac2dfc95 100644 --- a/tez-api/src/main/java/org/apache/tez/runtime/api/InputContext.java +++ b/tez-api/src/main/java/org/apache/tez/runtime/api/InputContext.java @@ -32,6 +32,12 @@ public interface InputContext extends TaskContext { * @return Name of the Source Vertex */ public String getSourceVertexName(); + + /** + * Returns a convenient, human-readable string describing the input and output vertices. + * @return the convenient string + */ + String getInputOutputVertexNames(); /** * Get the index of the input in the set of all inputs for the task. The diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java index 964c0b8301..e64d273b44 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java @@ -461,7 +461,7 @@ public List initialize() throws IOException { getContext().inputIsReady(); this.splitInfoViaEvents = jobConf.getBoolean(MRJobConfig.MR_TEZ_SPLITS_VIA_EVENTS, MRJobConfig.MR_TEZ_SPLITS_VIA_EVENTS_DEFAULT); - LOG.info(getContext().getSourceVertexName() + " using newmapreduce API=" + useNewApi + + LOG.info(getContext().getInputOutputVertexNames() + " using newmapreduce API=" + useNewApi + ", split via event=" + splitInfoViaEvents + ", numPhysicalInputs=" + getNumPhysicalInputs()); initializeInternal(); @@ -526,7 +526,7 @@ inputRecordCounter, getContext().getApplicationId().getClusterTimestamp(), } finally { rrLock.unlock(); } - LOG.info("Initialized MRInput: " + getContext().getSourceVertexName()); + LOG.info("Initialized MRInput: " + getContext().getInputOutputVertexNames()); } /** @@ -634,7 +634,7 @@ void processSplitEvent(InputDataInformationEvent event) try { initFromEventInternal(event); if (LOG.isDebugEnabled()) { - LOG.debug(getContext().getSourceVertexName() + " notifying on RecordReader initialized"); + LOG.debug(getContext().getInputOutputVertexNames() + " notifying on RecordReader initialized"); } rrInited.signal(); } finally { @@ -647,7 +647,7 @@ void checkAndAwaitRecordReaderInitialization() throws IOException { rrLock.lock(); try { if (LOG.isDebugEnabled()) { - LOG.debug(getContext().getSourceVertexName() + " awaiting RecordReader initialization"); + LOG.debug(getContext().getInputOutputVertexNames() + " awaiting RecordReader initialization"); } rrInited.await(); } catch (Exception e) { @@ -671,7 +671,7 @@ void initFromEvent(InputDataInformationEvent initEvent) private void initFromEventInternal(InputDataInformationEvent initEvent) throws IOException { if (LOG.isDebugEnabled()) { - LOG.debug(getContext().getSourceVertexName() + " initializing RecordReader from event"); + LOG.debug(getContext().getInputOutputVertexNames() + " initializing RecordReader from event"); } Objects.requireNonNull(initEvent, "InitEvent must be specified"); MRSplitProto splitProto = MRSplitProto.parseFrom(ByteString.copyFrom(initEvent.getUserPayload())); @@ -686,7 +686,7 @@ private void initFromEventInternal(InputDataInformationEvent initEvent) throws I LOG.warn("Thread interrupted while getting split length: ", e); } if (LOG.isDebugEnabled()) { - LOG.debug(getContext().getSourceVertexName() + " split Details -> SplitClass: " + + LOG.debug(getContext().getInputOutputVertexNames() + " split Details -> SplitClass: " + split.getClass().getName() + ", NewSplit: " + split + ", length: " + splitLength); } @@ -696,7 +696,7 @@ private void initFromEventInternal(InputDataInformationEvent initEvent) throws I splitObj = split; splitLength = split.getLength(); if (LOG.isDebugEnabled()) { - LOG.debug(getContext().getSourceVertexName() + " split Details -> SplitClass: " + + LOG.debug(getContext().getInputOutputVertexNames() + " split Details -> SplitClass: " + split.getClass().getName() + ", OldSplit: " + split + ", length: " + splitLength); } } @@ -705,7 +705,7 @@ private void initFromEventInternal(InputDataInformationEvent initEvent) throws I .increment(splitLength); } mrReader.setSplit(splitObj); - LOG.info(getContext().getSourceVertexName() + " initialized RecordReader from event"); + LOG.info(getContext().getInputOutputVertexNames() + " initialized RecordReader from event"); } private static class MRInputHelpersInternal extends MRInputHelpers { diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInputLegacy.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInputLegacy.java index 70be7ee444..bbf145eb6c 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInputLegacy.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInputLegacy.java @@ -74,7 +74,7 @@ public MRInputLegacy(InputContext inputContext, int numPhysicalInputs) { @Private protected void initializeInternal() throws IOException { - LOG.info(getContext().getSourceVertexName() + " MRInputLegacy deferring initialization"); + LOG.info(getContext().getInputOutputVertexNames() + " MRInputLegacy deferring initialization"); } @Private @@ -136,7 +136,7 @@ void checkAndAwaitRecordReaderInitialization() throws IOException { if (splitInfoViaEvents && !inited) { if (initEvent == null) { if (LOG.isDebugEnabled()) { - LOG.debug(getContext().getSourceVertexName() + + LOG.debug(getContext().getInputOutputVertexNames() + " awaiting init event before initializing record reader"); } diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java index 3bc8629565..4a98052211 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java @@ -110,7 +110,7 @@ private MultiMRInputConfigBuilder(Configuration conf, Class inputFormat) { @Override public List initialize() throws IOException { super.initialize(); - LOG.info(getContext().getSourceVertexName() + " using newmapreduce API=" + useNewApi + + LOG.info(getContext().getInputOutputVertexNames() + " using newmapreduce API=" + useNewApi + ", numPhysicalInputs=" + getNumPhysicalInputs()); if (getNumPhysicalInputs() == 0) { getContext().inputIsReady(); @@ -167,7 +167,7 @@ public void handleEvents(List inputEvents) throws Exception { private MRReader initFromEvent(InputDataInformationEvent event) throws IOException { Objects.requireNonNull(event, "Event must be specified"); if (LOG.isDebugEnabled()) { - LOG.debug(getContext().getSourceVertexName() + " initializing Reader: " + eventCount.get()); + LOG.debug(getContext().getInputOutputVertexNames() + " initializing Reader: " + eventCount.get()); } MRSplitProto splitProto = MRSplitProto.parseFrom(ByteString.copyFrom(event.getUserPayload())); MRReader reader = null; @@ -186,7 +186,7 @@ private MRReader initFromEvent(InputDataInformationEvent event) throws IOExcepti .getApplicationId().getId(), getContext().getTaskIndex(), getContext() .getTaskAttemptNumber(), getContext()); if (LOG.isDebugEnabled()) { - LOG.debug(getContext().getSourceVertexName() + " split Details -> SplitClass: " + + LOG.debug(getContext().getInputOutputVertexNames() + " split Details -> SplitClass: " + split.getClass().getName() + ", NewSplit: " + split + ", length: " + splitLength); } } else { @@ -196,7 +196,7 @@ private MRReader initFromEvent(InputDataInformationEvent event) throws IOExcepti reader = new MRReaderMapred(localJobConf, split, getContext().getCounters(), inputRecordCounter, getContext()); if (LOG.isDebugEnabled()) { - LOG.debug(getContext().getSourceVertexName() + " split Details -> SplitClass: " + + LOG.debug(getContext().getInputOutputVertexNames() + " split Details -> SplitClass: " + split.getClass().getName() + ", OldSplit: " + split + ", length: " + splitLength); } } @@ -204,7 +204,7 @@ private MRReader initFromEvent(InputDataInformationEvent event) throws IOExcepti getContext().getCounters().findCounter(TaskCounter.INPUT_SPLIT_LENGTH_BYTES) .increment(splitLength); } - LOG.info(getContext().getSourceVertexName() + " initialized RecordReader from event"); + LOG.info(getContext().getInputOutputVertexNames() + " initialized RecordReader from event"); return reader; } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java index f28573a649..1c1c10bf86 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java @@ -152,6 +152,11 @@ public String getSourceVertexName() { return sourceVertexName; } + @Override + public String getInputOutputVertexNames() { + return String.format("%s -> %s", getSourceVertexName(), getTaskVertexName()); + } + @Override public void fatalError(Throwable exception, String message) { super.signalFatalError(exception, message, sourceInfo); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java index 6039df3a2f..eb34ec2993 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java @@ -58,7 +58,7 @@ import org.apache.tez.common.CallableWithNdc; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.dag.api.TezUncheckedException; -import org.apache.tez.runtime.api.ObjectRegistry; +import org.apache.tez.runtime.api.InputContext; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; import org.apache.tez.runtime.library.common.Constants; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; @@ -69,6 +69,7 @@ import org.apache.tez.runtime.library.common.shuffle.FetchedInput.Type; import org.apache.tez.runtime.library.common.shuffle.api.ShuffleHandlerError; import org.apache.tez.common.Preconditions; +import org.apache.tez.common.TezUtilsInternal; /** * Responsible for fetching inputs served by the ShuffleHandler for a single @@ -194,8 +195,8 @@ public String getHost() { private final boolean isDebugEnabled = LOG.isDebugEnabled(); protected Fetcher(FetcherCallback fetcherCallback, HttpConnectionParams params, - FetchedInputAllocator inputManager, ApplicationId appId, int dagIdentifier, - JobTokenSecretManager jobTokenSecretManager, String srcNameTrimmed, Configuration conf, + FetchedInputAllocator inputManager, InputContext inputContext, + JobTokenSecretManager jobTokenSecretManager, Configuration conf, RawLocalFileSystem localFs, LocalDirAllocator localDirAllocator, Path lockPath, @@ -208,8 +209,8 @@ protected Fetcher(FetcherCallback fetcherCallback, HttpConnectionParams params, this.fetcherCallback = fetcherCallback; this.inputManager = inputManager; this.jobTokenSecretMgr = jobTokenSecretManager; - this.appId = appId; - this.dagIdentifier = dagIdentifier; + this.appId = inputContext.getApplicationId(); + this.dagIdentifier = inputContext.getDagIdentifier(); this.pathToAttemptMap = new HashMap(); this.httpConnectionParams = params; this.conf = conf; @@ -218,7 +219,10 @@ protected Fetcher(FetcherCallback fetcherCallback, HttpConnectionParams params, this.sharedFetchEnabled = sharedFetchEnabled; this.fetcherIdentifier = fetcherIdGen.getAndIncrement(); - this.logIdentifier = " fetcher [" + srcNameTrimmed +"] " + fetcherIdentifier; + + String sourceDestNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName()) + " -> " + + TezUtilsInternal.cleanVertexName(inputContext.getTaskVertexName()); + this.logIdentifier = " fetcher [" + sourceDestNameTrimmed +"] " + fetcherIdentifier; this.localFs = localFs; this.localDirAllocator = localDirAllocator; @@ -1133,31 +1137,29 @@ public static class FetcherBuilder { private boolean workAssigned = false; public FetcherBuilder(FetcherCallback fetcherCallback, - HttpConnectionParams params, FetchedInputAllocator inputManager, - ApplicationId appId, int dagIdentifier, JobTokenSecretManager jobTokenSecretMgr, String srcNameTrimmed, - Configuration conf, boolean localDiskFetchEnabled, String localHostname, int shufflePort, - boolean asyncHttp, boolean verifyDiskChecksum, boolean compositeFetch) { - this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId, dagIdentifier, - jobTokenSecretMgr, srcNameTrimmed, conf, null, null, null, localDiskFetchEnabled, + HttpConnectionParams params, FetchedInputAllocator inputManager, InputContext inputContext, + JobTokenSecretManager jobTokenSecretMgr, Configuration conf, boolean localDiskFetchEnabled, + String localHostname, int shufflePort, boolean asyncHttp, boolean verifyDiskChecksum, boolean compositeFetch) { + this.fetcher = new Fetcher(fetcherCallback, params, inputManager, inputContext, + jobTokenSecretMgr, conf, null, null, null, localDiskFetchEnabled, false, localHostname, shufflePort, asyncHttp, verifyDiskChecksum, compositeFetch); } public FetcherBuilder(FetcherCallback fetcherCallback, - HttpConnectionParams params, FetchedInputAllocator inputManager, - ApplicationId appId, int dagIdentifier, JobTokenSecretManager jobTokenSecretMgr, String srcNameTrimmed, - Configuration conf, RawLocalFileSystem localFs, + HttpConnectionParams params, FetchedInputAllocator inputManager, InputContext inputContext, + JobTokenSecretManager jobTokenSecretMgr, Configuration conf, RawLocalFileSystem localFs, LocalDirAllocator localDirAllocator, Path lockPath, boolean localDiskFetchEnabled, boolean sharedFetchEnabled, String localHostname, int shufflePort, boolean asyncHttp, boolean verifyDiskChecksum, boolean compositeFetch, - boolean enableFetcherTestingErrors, ObjectRegistry objectRegistry) { + boolean enableFetcherTestingErrors) { if (enableFetcherTestingErrors) { - this.fetcher = new FetcherWithInjectableErrors(fetcherCallback, params, inputManager, appId, dagIdentifier, - jobTokenSecretMgr, srcNameTrimmed, conf, localFs, localDirAllocator, + this.fetcher = new FetcherWithInjectableErrors(fetcherCallback, params, inputManager, inputContext, + jobTokenSecretMgr, conf, localFs, localDirAllocator, lockPath, localDiskFetchEnabled, sharedFetchEnabled, localHostname, shufflePort, asyncHttp, - verifyDiskChecksum, compositeFetch, objectRegistry); + verifyDiskChecksum, compositeFetch); } else { - this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId, dagIdentifier, - jobTokenSecretMgr, srcNameTrimmed, conf, localFs, localDirAllocator, + this.fetcher = new Fetcher(fetcherCallback, params, inputManager, inputContext, + jobTokenSecretMgr, conf, localFs, localDirAllocator, lockPath, localDiskFetchEnabled, sharedFetchEnabled, localHostname, shufflePort, asyncHttp, verifyDiskChecksum, compositeFetch); } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherWithInjectableErrors.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherWithInjectableErrors.java index 951adf9f5d..cf53a57a9e 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherWithInjectableErrors.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherWithInjectableErrors.java @@ -20,10 +20,10 @@ import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RawLocalFileSystem; -import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.http.HttpConnectionParams; -import org.apache.tez.runtime.api.ObjectRegistry; +import org.apache.tez.runtime.api.InputContext; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,16 +35,16 @@ public class FetcherWithInjectableErrors extends Fetcher { private String srcNameTrimmed; protected FetcherWithInjectableErrors(FetcherCallback fetcherCallback, HttpConnectionParams params, - FetchedInputAllocator inputManager, ApplicationId appId, int dagIdentifier, - JobTokenSecretManager jobTokenSecretManager, String srcNameTrimmed, Configuration conf, + FetchedInputAllocator inputManager, InputContext inputContext, + JobTokenSecretManager jobTokenSecretManager, Configuration conf, RawLocalFileSystem localFs, LocalDirAllocator localDirAllocator, Path lockPath, boolean localDiskFetchEnabled, boolean sharedFetchEnabled, String localHostname, int shufflePort, boolean asyncHttp, boolean verifyDiskChecksum, - boolean compositeFetch, ObjectRegistry objectRegistry) { - super(fetcherCallback, params, inputManager, appId, dagIdentifier, jobTokenSecretManager, srcNameTrimmed, conf, + boolean compositeFetch) { + super(fetcherCallback, params, inputManager, inputContext, jobTokenSecretManager, conf, localFs, localDirAllocator, lockPath, localDiskFetchEnabled, sharedFetchEnabled, localHostname, shufflePort, asyncHttp, verifyDiskChecksum, compositeFetch); - this.fetcherErrorTestingConfig = new FetcherErrorTestingConfig(conf, objectRegistry); - this.srcNameTrimmed = srcNameTrimmed; + this.fetcherErrorTestingConfig = new FetcherErrorTestingConfig(conf, inputContext.getObjectRegistry()); + this.srcNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName()); LOG.info("Initialized FetcherWithInjectableErrors with config: {}", fetcherErrorTestingConfig); } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java index bcb7bb58ea..ca1259f30a 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java @@ -161,7 +161,7 @@ private void handleEvent(Event event) throws IOException { @Override public void logProgress(boolean updateOnClose) { - LOG.info(inputContext.getSourceVertexName() + ": " + LOG.info(inputContext.getInputOutputVertexNames() + ": " + "numDmeEventsSeen=" + numDmeEvents.get() + ", numDmeEventsSeenWithNoData=" + numDmeEventsNoData.get() + ", numObsoletionEventsSeen=" + numObsoletionEvents.get() diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java index fa539c8374..769ac68f7e 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java @@ -185,7 +185,7 @@ public class ShuffleManager implements FetcherCallback { */ private final int maxTimeToWaitForReportMillis; - private final String srcNameTrimmed; + private final String sourceDestNameTrimmed; private final int maxTaskOutputAtOnce; @@ -264,8 +264,9 @@ public ShuffleManager(InputContext inputContext, Configuration conf, int numInpu conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_ENABLE_TESTING_ERRORS, TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_ENABLE_TESTING_ERRORS_DEFAULT); - this.srcNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName()); - + this.sourceDestNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName()) + " -> " + + TezUtilsInternal.cleanVertexName(inputContext.getTaskVertexName()); + completedInputSet = new BitSet(numInputs); /** * In case of pipelined shuffle, it is possible to get multiple FetchedInput per attempt. @@ -288,15 +289,15 @@ public ShuffleManager(InputContext inputContext, Configuration conf, int numInpu if (conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCHER_USE_SHARED_POOL, TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCHER_USE_SHARED_POOL_DEFAULT)) { fetcherRawExecutor = inputContext.createTezFrameworkExecutorService(numFetchers, - "Fetcher_B {" + srcNameTrimmed + "} #%d"); + "Fetcher_B {" + sourceDestNameTrimmed + "} #%d"); } else { fetcherRawExecutor = Executors.newFixedThreadPool(numFetchers, new ThreadFactoryBuilder() - .setDaemon(true).setNameFormat("Fetcher_B {" + srcNameTrimmed + "} #%d").build()); + .setDaemon(true).setNameFormat("Fetcher_B {" + sourceDestNameTrimmed + "} #%d").build()); } this.fetcherExecutor = MoreExecutors.listeningDecorator(fetcherRawExecutor); ExecutorService schedulerRawExecutor = Executors.newFixedThreadPool(1, new ThreadFactoryBuilder() - .setDaemon(true).setNameFormat("ShuffleRunner {" + srcNameTrimmed + "}").build()); + .setDaemon(true).setNameFormat("ShuffleRunner {" + sourceDestNameTrimmed + "}").build()); this.schedulerExecutor = MoreExecutors.listeningDecorator(schedulerRawExecutor); this.schedulerCallable = new RunShuffleCallable(conf); @@ -336,7 +337,7 @@ public ShuffleManager(InputContext inputContext, Configuration conf, int numInpu shuffleInfoEventsMap = new ConcurrentHashMap(); - LOG.info(srcNameTrimmed + ": numInputs=" + numInputs + ", compressionCodec=" + LOG.info(sourceDestNameTrimmed + ": numInputs=" + numInputs + ", compressionCodec=" + (codec == null ? "NoCompressionCodec" : codec.getClass().getName()) + ", numFetchers=" + numFetchers + ", ifileBufferSize=" + ifileBufferSize + ", ifileReadAheadEnabled=" + ifileReadAhead + ", ifileReadAheadLength=" + ifileReadAheadLength +", " @@ -361,7 +362,7 @@ public void run() throws IOException { if (maxTimeToWaitForReportMillis > 0) { reporterExecutor = Executors.newSingleThreadExecutor( new ThreadFactoryBuilder().setDaemon(true) - .setNameFormat("ShuffleRunner {" + srcNameTrimmed + "}") + .setNameFormat("ShuffleRunner {" + sourceDestNameTrimmed + "}") .build()); Future reporterFuture = reporterExecutor.submit(new ReporterCallable()); } @@ -446,7 +447,7 @@ protected Void callInternal() throws Exception { break; } - LOG.debug("{}: NumCompletedInputs: {}", srcNameTrimmed, numCompletedInputs); + LOG.debug("{}: NumCompletedInputs: {}", sourceDestNameTrimmed, numCompletedInputs); if (numCompletedInputs.get() < numInputs && !isShutdown.get()) { lock.lock(); try { @@ -458,7 +459,8 @@ protected Void callInternal() throws Exception { inputHost = pendingHosts.take(); } catch (InterruptedException e) { if (isShutdown.get()) { - LOG.info(srcNameTrimmed + ": " + "Interrupted and hasBeenShutdown, Breaking out of ShuffleScheduler Loop"); + LOG.info(sourceDestNameTrimmed + ": " + + "Interrupted and hasBeenShutdown, Breaking out of ShuffleScheduler Loop"); Thread.currentThread().interrupt(); break; } else { @@ -466,14 +468,14 @@ protected Void callInternal() throws Exception { } } if (LOG.isDebugEnabled()) { - LOG.debug(srcNameTrimmed + ": " + "Processing pending host: " + + LOG.debug(sourceDestNameTrimmed + ": " + "Processing pending host: " + inputHost.toDetailedString()); } if (inputHost.getNumPendingPartitions() > 0 && !isShutdown.get()) { Fetcher fetcher = constructFetcherForHost(inputHost, conf); runningFetchers.add(fetcher); if (isShutdown.get()) { - LOG.info(srcNameTrimmed + ": " + "hasBeenShutdown," + + LOG.info(sourceDestNameTrimmed + ": " + "hasBeenShutdown," + "Breaking out of ShuffleScheduler Loop"); break; } @@ -485,7 +487,7 @@ protected Void callInternal() throws Exception { } } else { if (LOG.isDebugEnabled()) { - LOG.debug(srcNameTrimmed + ": " + "Skipping host: " + + LOG.debug(sourceDestNameTrimmed + ": " + "Skipping host: " + inputHost.getIdentifier() + " since it has no inputs to process"); } @@ -497,7 +499,8 @@ protected Void callInternal() throws Exception { } } shufflePhaseTime.setValue(System.currentTimeMillis() - startTime); - LOG.info(srcNameTrimmed + ": " + "Shutting down FetchScheduler, Was Interrupted: " + Thread.currentThread().isInterrupted()); + LOG.info(sourceDestNameTrimmed + ": " + "Shutting down FetchScheduler, Was Interrupted: " + + Thread.currentThread().isInterrupted()); if (!fetcherExecutor.isShutdown()) { fetcherExecutor.shutdownNow(); } @@ -540,16 +543,15 @@ Fetcher constructFetcherForHost(InputHost inputHost, Configuration conf) { if (sharedFetchEnabled) { // pick a single lock disk from the edge name's hashcode + host hashcode - final int h = Math.abs(Objects.hashCode(this.srcNameTrimmed, inputHost.getHost())); + final int h = Math.abs(Objects.hashCode(this.sourceDestNameTrimmed, inputHost.getHost())); lockDisk = new Path(this.localDisks[h % this.localDisks.length], "locks"); } FetcherBuilder fetcherBuilder = new FetcherBuilder(ShuffleManager.this, - httpConnectionParams, inputManager, inputContext.getApplicationId(), inputContext.getDagIdentifier(), - jobTokenSecretMgr, srcNameTrimmed, conf, localFs, localDirAllocator, + httpConnectionParams, inputManager, inputContext, + jobTokenSecretMgr, conf, localFs, localDirAllocator, lockDisk, localDiskFetchEnabled, sharedFetchEnabled, - localhostName, shufflePort, asyncHttp, verifyDiskChecksum, compositeFetch, enableFetcherTestingErrors, - inputContext.getObjectRegistry()); + localhostName, shufflePort, asyncHttp, verifyDiskChecksum, compositeFetch, enableFetcherTestingErrors); if (codec != null) { fetcherBuilder.setCompressionParameters(codec); @@ -632,7 +634,7 @@ public void addKnownInput(String hostName, int port, } } if (LOG.isDebugEnabled()) { - LOG.debug(srcNameTrimmed + ": " + "Adding input: " + + LOG.debug(sourceDestNameTrimmed + ": " + "Adding input: " + srcAttemptIdentifier + ", to host: " + host); } @@ -878,7 +880,7 @@ private void adjustCompletedInputs(FetchedInput fetchedInput) { if (fetchedInput instanceof NullFetchedInput) { completedInputs.add(fetchedInput); } - LOG.info("All inputs fetched for input vertex : " + inputContext.getSourceVertexName()); + LOG.info("All inputs fetched for input vertex : " + inputContext.getInputOutputVertexNames()); } } finally { lock.unlock(); @@ -950,7 +952,7 @@ public void fetchFailed(String host, LOG.info( "{}: Fetch failed for src: {} InputIdentifier: {}, connectFailed: {}, " + "local fetch: {}, remote fetch failure reported as local failure: {})", - srcNameTrimmed, srcAttemptIdentifier, srcAttemptIdentifier, connectFailed, + sourceDestNameTrimmed, srcAttemptIdentifier, srcAttemptIdentifier, connectFailed, inputAttemptFetchFailure.isLocalFetch(), inputAttemptFetchFailure.isDiskErrorAtSource()); failedShufflesCounter.increment(1); inputContext.notifyProgress(); @@ -988,11 +990,11 @@ public void shutdown() throws InterruptedException { if (Thread.currentThread().isInterrupted()) { //TODO: need to cleanup all FetchedInput (DiskFetchedInput, LocalDisFetchedInput), lockFile //As of now relying on job cleanup (when all directories would be cleared) - LOG.info(srcNameTrimmed + ": " + "Thread interrupted. Need to cleanup the local dirs"); + LOG.info(sourceDestNameTrimmed + ": " + "Thread interrupted. Need to cleanup the local dirs"); } if (!isShutdown.getAndSet(true)) { // Shut down any pending fetchers - LOG.info("Shutting down pending fetchers on source" + srcNameTrimmed + ": " + LOG.info("Shutting down pending fetchers on source" + sourceDestNameTrimmed + ": " + runningFetchers.size()); lock.lock(); try { @@ -1140,15 +1142,15 @@ private class SchedulerFutureCallback implements FutureCallback { @Override public void onSuccess(Void result) { - LOG.info(srcNameTrimmed + ": " + "Scheduler thread completed"); + LOG.info(sourceDestNameTrimmed + ": " + "Scheduler thread completed"); } @Override public void onFailure(Throwable t) { if (isShutdown.get()) { - LOG.debug("{}: Already shutdown. Ignoring error.", srcNameTrimmed, t); + LOG.debug("{}: Already shutdown. Ignoring error.", sourceDestNameTrimmed, t); } else { - LOG.error(srcNameTrimmed + ": " + "Scheduler failed with error: ", t); + LOG.error(sourceDestNameTrimmed + ": " + "Scheduler failed with error: ", t); inputContext.reportFailure(TaskFailureType.NON_FATAL, t, "Shuffle Scheduler Failed"); } } @@ -1177,7 +1179,7 @@ private void doBookKeepingForFetcherComplete() { public void onSuccess(FetchResult result) { fetcher.shutdown(); if (isShutdown.get()) { - LOG.debug("{}: Already shutdown. Ignoring event from fetcher", srcNameTrimmed); + LOG.debug("{}: Already shutdown. Ignoring event from fetcher", sourceDestNameTrimmed); } else { Iterable pendingInputs = result.getPendingInputs(); if (pendingInputs != null && pendingInputs.iterator().hasNext()) { @@ -1200,9 +1202,9 @@ public void onFailure(Throwable t) { // Unsuccessful - the fetcher may not have shutdown correctly. Try shutting it down. fetcher.shutdown(); if (isShutdown.get()) { - LOG.debug("{}: Already shutdown. Ignoring error from fetcher.", srcNameTrimmed, t); + LOG.debug("{}: Already shutdown. Ignoring error from fetcher.", sourceDestNameTrimmed, t); } else { - LOG.error(srcNameTrimmed + ": " + "Fetcher failed with error: ", t); + LOG.error(sourceDestNameTrimmed + ": " + "Fetcher failed with error: ", t); shuffleError = t; inputContext.reportFailure(TaskFailureType.NON_FATAL, t, "Fetch failed"); doBookKeepingForFetcherComplete(); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java index 2725debb56..5887dcb3ca 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java @@ -42,8 +42,10 @@ import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.tez.common.TezRuntimeFrameworkConfigs; +import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.common.counters.TezCounter; import org.apache.tez.common.security.JobTokenSecretManager; +import org.apache.tez.runtime.api.InputContext; import org.apache.tez.runtime.library.common.Constants; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.MapOutput.Type; @@ -121,7 +123,6 @@ public FetcherOrderedGrouped(HttpConnectionParams httpConnectionParams, boolean localDiskFetchEnabled, String localHostname, int shufflePort, - String srcNameTrimmed, MapHost mapHost, TezCounter ioErrsCounter, TezCounter wrongLengthErrsCounter, @@ -129,12 +130,11 @@ public FetcherOrderedGrouped(HttpConnectionParams httpConnectionParams, TezCounter wrongMapErrsCounter, TezCounter connectionErrsCounter, TezCounter wrongReduceErrsCounter, - String applicationId, - int dagId, boolean asyncHttp, boolean sslShuffle, boolean verifyDiskChecksum, - boolean compositeFetch) { + boolean compositeFetch, + InputContext inputContext) { this.scheduler = scheduler; this.allocator = allocator; this.exceptionReporter = exceptionReporter; @@ -149,8 +149,8 @@ public FetcherOrderedGrouped(HttpConnectionParams httpConnectionParams, this.badIdErrs = badIdErrsCounter; this.connectionErrs = connectionErrsCounter; this.wrongReduceErrs = wrongReduceErrsCounter; - this.applicationId = applicationId; - this.dagId = dagId; + this.applicationId = inputContext.getApplicationId().toString(); + this.dagId = inputContext.getDagIdentifier(); this.ifileReadAhead = ifileReadAhead; this.ifileReadAheadLength = ifileReadAheadLength; @@ -171,7 +171,9 @@ public FetcherOrderedGrouped(HttpConnectionParams httpConnectionParams, this.verifyDiskChecksum = verifyDiskChecksum; this.compositeFetch = compositeFetch; - this.logIdentifier = "fetcher [" + srcNameTrimmed + "] #" + id; + String sourceDestNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName()) + " -> " + + TezUtilsInternal.cleanVertexName(inputContext.getTaskVertexName()); + this.logIdentifier = "fetcher [" + sourceDestNameTrimmed + "] #" + id; } @VisibleForTesting diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGroupedWithInjectableErrors.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGroupedWithInjectableErrors.java index 94172d12b9..9c782f6585 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGroupedWithInjectableErrors.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGroupedWithInjectableErrors.java @@ -19,10 +19,11 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.common.counters.TezCounter; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.http.HttpConnectionParams; -import org.apache.tez.runtime.api.ObjectRegistry; +import org.apache.tez.runtime.api.InputContext; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; import org.apache.tez.runtime.library.common.shuffle.FetcherErrorTestingConfig; import org.slf4j.Logger; @@ -38,17 +39,16 @@ public FetcherOrderedGroupedWithInjectableErrors(HttpConnectionParams httpConnec ShuffleScheduler scheduler, FetchedInputAllocatorOrderedGrouped allocator, ExceptionReporter exceptionReporter, JobTokenSecretManager jobTokenSecretMgr, boolean ifileReadAhead, int ifileReadAheadLength, CompressionCodec codec, Configuration conf, RawLocalFileSystem localFs, boolean localDiskFetchEnabled, String localHostname, - int shufflePort, String srcNameTrimmed, MapHost mapHost, TezCounter ioErrsCounter, + int shufflePort, MapHost mapHost, TezCounter ioErrsCounter, TezCounter wrongLengthErrsCounter, TezCounter badIdErrsCounter, TezCounter wrongMapErrsCounter, - TezCounter connectionErrsCounter, TezCounter wrongReduceErrsCounter, String applicationId, int dagId, - boolean asyncHttp, boolean sslShuffle, boolean verifyDiskChecksum, boolean compositeFetch, - ObjectRegistry objectRegistry) { + TezCounter connectionErrsCounter, TezCounter wrongReduceErrsCounter, boolean asyncHttp, + boolean sslShuffle, boolean verifyDiskChecksum, boolean compositeFetch, InputContext inputContext) { super(httpConnectionParams, scheduler, allocator, exceptionReporter, jobTokenSecretMgr, ifileReadAhead, - ifileReadAheadLength, codec, conf, localFs, localDiskFetchEnabled, localHostname, shufflePort, srcNameTrimmed, + ifileReadAheadLength, codec, conf, localFs, localDiskFetchEnabled, localHostname, shufflePort, mapHost, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, connectionErrsCounter, - wrongReduceErrsCounter, applicationId, dagId, asyncHttp, sslShuffle, verifyDiskChecksum, compositeFetch); - this.fetcherErrorTestingConfig = new FetcherErrorTestingConfig(conf, objectRegistry); - this.srcNameTrimmed = srcNameTrimmed; + wrongReduceErrsCounter, asyncHttp, sslShuffle, verifyDiskChecksum, compositeFetch, inputContext); + this.fetcherErrorTestingConfig = new FetcherErrorTestingConfig(conf, inputContext.getObjectRegistry()); + this.srcNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName()); LOG.info("Initialized FetcherOrderedGroupedWithInjectableErrors with config: {}", fetcherErrorTestingConfig); } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java index 46360e1287..8d3f37dbcd 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java @@ -250,7 +250,7 @@ public MergeManager(Configuration conf, if (LOG.isDebugEnabled()) { LOG.debug( - inputContext.getSourceVertexName() + ": " + "InitialRequest: ShuffleMem=" + memLimit + + inputContext.getInputOutputVertexNames() + ": " + "InitialRequest: ShuffleMem=" + memLimit + ", postMergeMem=" + maxRedBuffer + ", RuntimeTotalAvailable=" + this.initialMemoryAvailable + ". Updated to: ShuffleMem=" @@ -285,7 +285,7 @@ public MergeManager(Configuration conf, conf.getFloat( TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_MERGE_PERCENT, TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_MERGE_PERCENT_DEFAULT)); - LOG.info(inputContext.getSourceVertexName() + ": MergerManager: memoryLimit=" + memoryLimit + ", " + + LOG.info(inputContext.getInputOutputVertexNames() + ": MergerManager: memoryLimit=" + memoryLimit + ", " + "maxSingleShuffleLimit=" + maxSingleShuffleLimit + ", " + "mergeThreshold=" + mergeThreshold + ", " + "ioSortFactor=" + ioSortFactor + ", " + @@ -549,8 +549,9 @@ private void trackAndLogCloseInMemoryFile(MapOutput mapOutput) { private void startMemToDiskMerge() { synchronized (inMemoryMerger) { if (!inMemoryMerger.isInProgress()) { - LOG.info(inputContext.getSourceVertexName() + ": " + "Starting inMemoryMerger's merge since commitMemory=" + - commitMemory + " > mergeThreshold=" + mergeThreshold + + LOG.info(inputContext.getInputOutputVertexNames() + ": " + + "Starting inMemoryMerger's merge since commitMemory=" + commitMemory + + " > mergeThreshold=" + mergeThreshold + ". Current usedMemory=" + usedMemory); inMemoryMapOutputs.addAll(inMemoryMergedMapOutputs); inMemoryMergedMapOutputs.clear(); @@ -788,8 +789,8 @@ public void merge(List inputs) throws IOException, InterruptedExcepti Writer writer = new InMemoryWriter(mergedMapOutputs.getMemory()); - LOG.info(inputContext.getSourceVertexName() + ": " + "Initiating Memory-to-Memory merge with " + noInMemorySegments + - " segments of total-size: " + mergeOutputSize); + LOG.info(inputContext.getInputOutputVertexNames() + ": " + "Initiating Memory-to-Memory merge with " + + noInMemorySegments + " segments of total-size: " + mergeOutputSize); if (Thread.currentThread().isInterrupted()) { return; // early exit @@ -808,7 +809,7 @@ public void merge(List inputs) throws IOException, InterruptedExcepti TezMerger.writeFile(rIter, writer, progressable, TezRuntimeConfiguration.TEZ_RUNTIME_RECORDS_BEFORE_PROGRESS_DEFAULT); writer.close(); - LOG.info(inputContext.getSourceVertexName() + + LOG.info(inputContext.getInputOutputVertexNames() + " Memory-to-Memory merge of the " + noInMemorySegments + " files in-memory complete with mergeOutputSize=" + mergeOutputSize); @@ -1051,7 +1052,7 @@ public void merge(List inputs) throws IOException, InterruptedExcepti final long outputLen = localFS.getFileStatus(outputPath).getLen(); closeOnDiskFile(new FileChunk(outputPath, 0, outputLen)); - LOG.info(inputContext.getSourceVertexName() + + LOG.info(inputContext.getInputOutputVertexNames() + " Finished merging " + inputs.size() + " map output files on disk of total-size " + approxOutputSize + "." + diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java index db5ef734de..2b99739a58 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java @@ -93,7 +93,7 @@ public class Shuffle implements ExceptionReporter { private volatile ListenableFuture runShuffleFuture; private final ListeningExecutorService executor; - private final String srcNameTrimmed; + private final String sourceDestNameTrimmed; private AtomicBoolean isShutDown = new AtomicBoolean(false); private AtomicBoolean fetchersClosed = new AtomicBoolean(false); @@ -109,7 +109,8 @@ public Shuffle(InputContext inputContext, Configuration conf, int numInputs, this.inputContext = inputContext; this.conf = conf; - this.srcNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName()); + this.sourceDestNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName()) + " -> " + + TezUtilsInternal.cleanVertexName(inputContext.getTaskVertexName()); this.codec = CodecUtils.getCodec(conf); @@ -138,7 +139,7 @@ public Shuffle(InputContext inputContext, Configuration conf, int numInputs, TezCounter mergedMapOutputsCounter = inputContext.getCounters().findCounter(TaskCounter.MERGED_MAP_OUTPUTS); - LOG.info(srcNameTrimmed + ": " + "Shuffle assigned with " + numInputs + " inputs" + ", codec: " + LOG.info(sourceDestNameTrimmed + ": " + "Shuffle assigned with " + numInputs + " inputs" + ", codec: " + (codec == null ? "None" : codec.getClass().getName()) + ", ifileReadAhead: " + ifileReadAhead); @@ -169,7 +170,7 @@ public Shuffle(InputContext inputContext, Configuration conf, int numInputs, codec, ifileReadAhead, ifileReadAheadLength, - srcNameTrimmed); + sourceDestNameTrimmed); this.mergePhaseTime = inputContext.getCounters().findCounter(TaskCounter.MERGE_PHASE_TIME); this.shufflePhaseTime = inputContext.getCounters().findCounter(TaskCounter.SHUFFLE_PHASE_TIME); @@ -182,7 +183,7 @@ public Shuffle(InputContext inputContext, Configuration conf, int numInputs, ShuffleUtils.isTezShuffleHandler(conf)); ExecutorService rawExecutor = Executors.newFixedThreadPool(1, new ThreadFactoryBuilder() - .setDaemon(true).setNameFormat("ShuffleAndMergeRunner {" + srcNameTrimmed + "}").build()); + .setDaemon(true).setNameFormat("ShuffleAndMergeRunner {" + sourceDestNameTrimmed + "}").build()); executor = MoreExecutors.listeningDecorator(rawExecutor); @@ -193,7 +194,7 @@ public void handleEvents(List events) throws IOException { if (!isShutDown.get()) { eventHandler.handleEvents(events); } else { - LOG.info(srcNameTrimmed + ": " + "Ignoring events since already shutdown. EventCount: " + events.size()); + LOG.info(sourceDestNameTrimmed + ": " + "Ignoring events since already shutdown. EventCount: " + events.size()); } } @@ -267,7 +268,7 @@ public void run() throws IOException { public void shutdown() { if (!isShutDown.getAndSet(true)) { // Interrupt so that the scheduler / merger sees this interrupt. - LOG.info("Shutting down Shuffle for source: " + srcNameTrimmed); + LOG.info("Shutting down Shuffle for source: " + sourceDestNameTrimmed); runShuffleFuture.cancel(true); cleanupIgnoreErrors(); } @@ -323,7 +324,7 @@ protected TezRawKeyValueIterator callInternal() throws IOException, InterruptedE } inputContext.inputIsReady(); - LOG.info("merge complete for input vertex : " + srcNameTrimmed); + LOG.info("merge complete for input vertex : " + sourceDestNameTrimmed); return kvIter; } } @@ -333,7 +334,8 @@ private void cleanupShuffleSchedulerIgnoreErrors() { cleanupShuffleScheduler(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - LOG.info(srcNameTrimmed + ": " + "Interrupted while attempting to close the scheduler during cleanup. Ignoring"); + LOG.info(sourceDestNameTrimmed + ": " + + "Interrupted while attempting to close the scheduler during cleanup. Ignoring"); } } @@ -351,13 +353,14 @@ private void cleanupMerger(boolean ignoreErrors) throws Throwable { if (ignoreErrors) { //Reset the status Thread.currentThread().interrupt(); - LOG.info(srcNameTrimmed + ": " + "Interrupted while attempting to close the merger during cleanup. Ignoring"); + LOG.info(sourceDestNameTrimmed + ": " + + "Interrupted while attempting to close the merger during cleanup. Ignoring"); } else { throw e; } } catch (Throwable e) { if (ignoreErrors) { - LOG.info(srcNameTrimmed + ": " + "Exception while trying to shutdown merger, Ignoring", e); + LOG.info(sourceDestNameTrimmed + ": " + "Exception while trying to shutdown merger, Ignoring", e); } else { throw e; } @@ -379,7 +382,7 @@ private void cleanupIgnoreErrors() { } cleanupMerger(true); } catch (Throwable t) { - LOG.info(srcNameTrimmed + ": " + "Error in cleaning up.., ", t); + LOG.info(sourceDestNameTrimmed + ": " + "Error in cleaning up.., ", t); } } @@ -388,7 +391,7 @@ private void cleanupIgnoreErrors() { public synchronized void reportException(Throwable t) { // RunShuffleCallable onFailure deals with ignoring errors on shutdown. if (throwable.get() == null) { - LOG.info(srcNameTrimmed + ": " + "Setting throwable in reportException with message [" + t.getMessage() + + LOG.info(sourceDestNameTrimmed + ": " + "Setting throwable in reportException with message [" + t.getMessage() + "] from thread [" + Thread.currentThread().getName()); throwable.set(t); throwingThreadName = Thread.currentThread().getName(); @@ -423,15 +426,15 @@ public static long getInitialMemoryRequirement(Configuration conf, long maxAvail private class ShuffleRunnerFutureCallback implements FutureCallback { @Override public void onSuccess(TezRawKeyValueIterator result) { - LOG.info(srcNameTrimmed + ": " + "Shuffle Runner thread complete"); + LOG.info(sourceDestNameTrimmed + ": " + "Shuffle Runner thread complete"); } @Override public void onFailure(Throwable t) { if (isShutDown.get()) { - LOG.info(srcNameTrimmed + ": " + "Already shutdown. Ignoring error"); + LOG.info(sourceDestNameTrimmed + ": " + "Already shutdown. Ignoring error"); } else { - LOG.error(srcNameTrimmed + ": " + "ShuffleRunner failed with error", t); + LOG.error(sourceDestNameTrimmed + ": " + "ShuffleRunner failed with error", t); // In case of an abort / Interrupt - the runtime makes sure that this is ignored. inputContext.reportFailure(TaskFailureType.NON_FATAL, t, "Shuffle Runner Failed"); cleanupIgnoreErrors(); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java index c97cfdf3ce..9984c5af10 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java @@ -77,7 +77,7 @@ public void handleEvents(List events) throws IOException { @Override public void logProgress(boolean updateOnClose) { - LOG.info(inputContext.getSourceVertexName() + ": " + LOG.info(inputContext.getInputOutputVertexNames() + ": " + "numDmeEventsSeen=" + numDmeEvents.get() + ", numDmeEventsSeenWithNoData=" + numDmeEventsNoData.get() + ", numObsoletionEventsSeen=" + numObsoletionEvents.get() diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java index cd4b14ca1b..470b04cc5f 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java @@ -222,8 +222,6 @@ enum ShuffleErrors { private final boolean localDiskFetchEnabled; private final String localHostname; private final int shufflePort; - private final String applicationId; - private final int dagId; private final boolean asyncHttp; private final boolean sslShuffle; @@ -338,8 +336,6 @@ public ShuffleScheduler(InputContext inputContext, TezRuntimeConfiguration .TEZ_RUNTIME_SHUFFLE_FAILED_CHECK_SINCE_LAST_COMPLETION_DEFAULT); - this.applicationId = inputContext.getApplicationId().toString(); - this.dagId = inputContext.getDagIdentifier(); this.localHostname = inputContext.getExecutionContext().getHostName(); String auxiliaryService = conf.get(TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID, TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID_DEFAULT); @@ -673,7 +669,7 @@ public synchronized void copySucceeded(InputAttemptIdentifier srcAttemptIdentifi if (remainingMaps.get() == 0) { notifyAll(); // Notify the getHost() method. - LOG.info("All inputs fetched for input vertex : " + inputContext.getSourceVertexName()); + LOG.info("All inputs fetched for input vertex : " + inputContext.getInputOutputVertexNames()); } // update the status @@ -1474,17 +1470,17 @@ FetcherOrderedGrouped constructFetcherForHost(MapHost mapHost) { if (enableFetcherTestingErrors) { return new FetcherOrderedGroupedWithInjectableErrors(httpConnectionParams, ShuffleScheduler.this, allocator, exceptionReporter, jobTokenSecretManager, ifileReadAhead, ifileReadAheadLength, - codec, conf, localFs, localDiskFetchEnabled, localHostname, shufflePort, srcNameTrimmed, mapHost, + codec, conf, localFs, localDiskFetchEnabled, localHostname, shufflePort, mapHost, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, - connectionErrsCounter, wrongReduceErrsCounter, applicationId, dagId, asyncHttp, sslShuffle, - verifyDiskChecksum, compositeFetch, inputContext.getObjectRegistry()); + connectionErrsCounter, wrongReduceErrsCounter, asyncHttp, sslShuffle, + verifyDiskChecksum, compositeFetch, inputContext); } else { return new FetcherOrderedGrouped(httpConnectionParams, ShuffleScheduler.this, allocator, exceptionReporter, jobTokenSecretManager, ifileReadAhead, ifileReadAheadLength, - codec, conf, localFs, localDiskFetchEnabled, localHostname, shufflePort, srcNameTrimmed, mapHost, + codec, conf, localFs, localDiskFetchEnabled, localHostname, shufflePort, mapHost, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, - connectionErrsCounter, wrongReduceErrsCounter, applicationId, dagId, asyncHttp, sslShuffle, - verifyDiskChecksum, compositeFetch); + connectionErrsCounter, wrongReduceErrsCounter, asyncHttp, sslShuffle, + verifyDiskChecksum, compositeFetch, inputContext); } } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java index 313c13d188..1463cfabbe 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java @@ -104,7 +104,7 @@ public synchronized List initialize() throws IOException { isStarted.set(true); getContext().inputIsReady(); LOG.info("input fetch not required since there are 0 physical inputs for input vertex: " - + getContext().getSourceVertexName()); + + getContext().getInputOutputVertexNames()); return Collections.emptyList(); } @@ -305,7 +305,7 @@ protected synchronized void createValuesIterator() RawComparator rawComparator = ConfigUtils.getIntermediateInputKeyComparator(conf); Class keyClass = ConfigUtils.getIntermediateInputKeyClass(conf); Class valClass = ConfigUtils.getIntermediateInputValueClass(conf); - LOG.info(getContext().getSourceVertexName() + ": " + "creating ValuesIterator with " + LOG.info(getContext().getInputOutputVertexNames() + ": " + "creating ValuesIterator with " + "comparator=" + rawComparator.getClass().getName() + ", keyClass=" + keyClass.getName() + ", valClass=" + valClass.getName()); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java index c67c405b43..38d5295094 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java @@ -92,7 +92,7 @@ public synchronized List initialize() throws Exception { isStarted.set(true); getContext().inputIsReady(); LOG.info("input fetch not required since there are 0 physical inputs for input vertex: " - + getContext().getSourceVertexName()); + + getContext().getInputOutputVertexNames()); return Collections.emptyList(); } else { long initalMemReq = getInitialMemoryReq(); @@ -148,7 +148,7 @@ public synchronized void start() throws IOException { pendingEvents.drainTo(pending); if (pending.size() > 0) { if (LOG.isDebugEnabled()) { - LOG.debug(getContext().getSourceVertexName() + ": " + "NoAutoStart delay in processing first event: " + LOG.debug(getContext().getInputOutputVertexNames() + ": " + "NoAutoStart delay in processing first event: " + (System.currentTimeMillis() - firstEventReceivedTime)); } inputEventHandler.handleEvents(pending); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java index cea3272818..338f640f91 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java @@ -81,7 +81,7 @@ public void testLocalFetchModeSetting() throws Exception { final boolean DISABLE_LOCAL_FETCH = false; Fetcher.FetcherBuilder builder = new Fetcher.FetcherBuilder(fetcherCallback, null, null, - ApplicationId.newInstance(0, 1), 1, null, "fetcherTest", conf, ENABLE_LOCAL_FETCH, HOST, + createMockInputContext(), null, conf, ENABLE_LOCAL_FETCH, HOST, PORT, false, true, false); builder.assignWork(HOST, PORT, 0, 1, Arrays.asList(srcAttempts)); Fetcher fetcher = spy(builder.build()); @@ -100,7 +100,7 @@ public void testLocalFetchModeSetting() throws Exception { // when enabled and hostname does not match use http fetch. builder = new Fetcher.FetcherBuilder(fetcherCallback, null, null, - ApplicationId.newInstance(0, 1), -1, null, "fetcherTest", conf, ENABLE_LOCAL_FETCH, HOST, + createMockInputContext(), null, conf, ENABLE_LOCAL_FETCH, HOST, PORT, false, true, false); builder.assignWork(HOST + "_OTHER", PORT, 0, 1, Arrays.asList(srcAttempts)); fetcher = spy(builder.build()); @@ -116,7 +116,7 @@ public void testLocalFetchModeSetting() throws Exception { // when enabled and port does not match use http fetch. builder = new Fetcher.FetcherBuilder(fetcherCallback, null, null, - ApplicationId.newInstance(0, 1), -1, null, "fetcherTest", conf, ENABLE_LOCAL_FETCH, HOST, + createMockInputContext(), null, conf, ENABLE_LOCAL_FETCH, HOST, PORT, false, true, false); builder.assignWork(HOST, PORT + 1, 0, 1, Arrays.asList(srcAttempts)); fetcher = spy(builder.build()); @@ -133,7 +133,7 @@ public void testLocalFetchModeSetting() throws Exception { // When disabled use http fetch conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, false); builder = new Fetcher.FetcherBuilder(fetcherCallback, null, null, - ApplicationId.newInstance(0, 1), 1, null, "fetcherTest", conf, DISABLE_LOCAL_FETCH, HOST, + createMockInputContext(), null, conf, DISABLE_LOCAL_FETCH, HOST, PORT, false, true, false); builder.assignWork(HOST, PORT, 0, 1, Arrays.asList(srcAttempts)); fetcher = spy(builder.build()); @@ -167,7 +167,7 @@ public void testSetupLocalDiskFetch() throws Exception { int partition = 42; FetcherCallback callback = mock(FetcherCallback.class); Fetcher.FetcherBuilder builder = new Fetcher.FetcherBuilder(callback, null, null, - ApplicationId.newInstance(0, 1), 1, null, "fetcherTest", conf, true, HOST, PORT, + createMockInputContext(), null, conf, true, HOST, PORT, false, true, true); ArrayList inputAttemptIdentifiers = new ArrayList<>(); for(CompositeInputAttemptIdentifier compositeInputAttemptIdentifier : srcAttempts) { @@ -306,7 +306,7 @@ public void testInputAttemptIdentifierMap() { int partition = 42; FetcherCallback callback = mock(FetcherCallback.class); Fetcher.FetcherBuilder builder = new Fetcher.FetcherBuilder(callback, null, null, - ApplicationId.newInstance(0, 1), 1, null, "fetcherTest", conf, true, HOST, PORT, + createMockInputContext(), null, conf, true, HOST, PORT, false, true, false); builder.assignWork(HOST, PORT, partition, 1, Arrays.asList(srcAttempts)); Fetcher fetcher = spy(builder.build()); @@ -330,7 +330,7 @@ public void testShuffleHandlerDiskErrorUnordered() doReturn("vertex").when(inputContext).getSourceVertexName(); Fetcher.FetcherBuilder builder = new Fetcher.FetcherBuilder(mock(ShuffleManager.class), null, - null, ApplicationId.newInstance(0, 1), 1, null, "fetcherTest", conf, true, HOST, PORT, + null, createMockInputContext(), null, conf, true, HOST, PORT, false, true, false); builder.assignWork(HOST, PORT, 0, 1, Arrays.asList(new InputAttemptIdentifier(0, 0))); @@ -345,4 +345,15 @@ public void testShuffleHandlerDiskErrorUnordered() Assert.assertTrue(failures[0].isDiskErrorAtSource()); Assert.assertFalse(failures[0].isLocalFetch()); } + + private InputContext createMockInputContext() { + InputContext inputContext = mock(InputContext.class); + + doReturn(ApplicationId.newInstance(0, 1)).when(inputContext).getApplicationId(); + doReturn(1).when(inputContext).getDagIdentifier(); + doReturn("sourceVertex").when(inputContext).getSourceVertexName(); + doReturn("taskVertex").when(inputContext).getTaskVertexName(); + + return inputContext; + } } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java index 683422b1a7..8a774dc1a5 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java @@ -189,6 +189,7 @@ private InputContext createInputContext() throws IOException { InputContext inputContext = mock(InputContext.class); doReturn(new TezCounters()).when(inputContext).getCounters(); doReturn("sourceVertex").when(inputContext).getSourceVertexName(); + doReturn("taskVertex").when(inputContext).getTaskVertexName(); doReturn(shuffleMetaData).when(inputContext) .getServiceProviderMetaData(conf.get(TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID, TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID_DEFAULT)); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java index 041fd03854..fda2c896d1 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java @@ -169,6 +169,7 @@ private InputContext createInputContext() throws IOException { InputContext inputContext = mock(InputContext.class); doReturn(new TezCounters()).when(inputContext).getCounters(); doReturn("sourceVertex").when(inputContext).getSourceVertexName(); + doReturn("taskVertex").when(inputContext).getTaskVertexName(); doReturn(shuffleMetaData).when(inputContext) .getServiceProviderMetaData(conf.get(TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID, TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID_DEFAULT)); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java index 028fbce96a..f283780c12 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java @@ -90,7 +90,7 @@ public class TestFetcher { public static final String HOST = "localhost"; public static final int PORT = 65; public static final int DAG_ID = 1; - public static final String APP_ID = "application_1234_1"; + public static final ApplicationId APP_ID = ApplicationId.newInstance(0, 1); private TezCounters tezCounters = new TezCounters(); private TezCounter ioErrsCounter = tezCounters.findCounter(ShuffleScheduler.SHUFFLE_ERR_GRP_NAME, @@ -121,10 +121,6 @@ public void testInputsReturnedOnConnectionException() throws Exception { Shuffle shuffle = mock(Shuffle.class); - InputContext inputContext = mock(InputContext.class); - doReturn(new TezCounters()).when(inputContext).getCounters(); - doReturn("src vertex").when(inputContext).getSourceVertexName(); - MapHost mapHost = new MapHost(HOST, PORT, 0, 1); InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(0, 0, "attempt"); mapHost.addKnownMap(inputAttemptIdentifier); @@ -133,9 +129,9 @@ public void testInputsReturnedOnConnectionException() throws Exception { FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, null, conf, - getRawFs(conf), false, HOST, PORT, "src vertex", mapHost, ioErrsCounter, + getRawFs(conf), false, HOST, PORT, mapHost, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, connectionErrsCounter, - wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); + wrongReduceErrsCounter, false, false, true, false, createMockInputContext()); fetcher.call(); verify(scheduler).getMapsForHost(mapHost); @@ -151,18 +147,14 @@ public void testLocalFetchModeSetting1() throws Exception { MergeManager merger = mock(MergeManager.class); Shuffle shuffle = mock(Shuffle.class); - InputContext inputContext = mock(InputContext.class); - doReturn(new TezCounters()).when(inputContext).getCounters(); - doReturn("src vertex").when(inputContext).getSourceVertexName(); - final boolean ENABLE_LOCAL_FETCH = true; final boolean DISABLE_LOCAL_FETCH = false; MapHost mapHost = new MapHost(HOST, PORT, 0, 1); FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, null, conf, - getRawFs(conf), ENABLE_LOCAL_FETCH, HOST, PORT, "src vertex", mapHost, ioErrsCounter, + getRawFs(conf), ENABLE_LOCAL_FETCH, HOST, PORT, mapHost, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, connectionErrsCounter, - wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); + wrongReduceErrsCounter, false, false, true, false, createMockInputContext()); // when local mode is enabled and host and port matches use local fetch FetcherOrderedGrouped spyFetcher = spy(fetcher); @@ -177,9 +169,9 @@ public void testLocalFetchModeSetting1() throws Exception { mapHost = new MapHost(HOST + "_OTHER", PORT, 0, 1); fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, null, conf, - getRawFs(conf), ENABLE_LOCAL_FETCH, HOST, PORT, "src vertex", mapHost, ioErrsCounter, + getRawFs(conf), ENABLE_LOCAL_FETCH, HOST, PORT, mapHost, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, connectionErrsCounter, - wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); + wrongReduceErrsCounter, false, false, true, false, createMockInputContext()); spyFetcher = spy(fetcher); doNothing().when(spyFetcher).setupLocalDiskFetch(mapHost); @@ -192,9 +184,9 @@ public void testLocalFetchModeSetting1() throws Exception { mapHost = new MapHost(HOST, PORT + 1, 0, 1); fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, null, conf, - getRawFs(conf), ENABLE_LOCAL_FETCH, HOST, PORT, "src vertex", mapHost, ioErrsCounter, + getRawFs(conf), ENABLE_LOCAL_FETCH, HOST, PORT, mapHost, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, connectionErrsCounter, - wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); + wrongReduceErrsCounter, false, false, true, false, createMockInputContext()); spyFetcher = spy(fetcher); doNothing().when(spyFetcher).setupLocalDiskFetch(mapHost); @@ -206,9 +198,9 @@ public void testLocalFetchModeSetting1() throws Exception { //if local fetch is not enabled mapHost = new MapHost(HOST, PORT, 0, 1); fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, null, - conf, getRawFs(conf), DISABLE_LOCAL_FETCH, HOST, PORT, "src vertex", mapHost, ioErrsCounter, + conf, getRawFs(conf), DISABLE_LOCAL_FETCH, HOST, PORT, mapHost, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, connectionErrsCounter, - wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); + wrongReduceErrsCounter, false, false, true, false, createMockInputContext()); spyFetcher = spy(fetcher); doNothing().when(spyFetcher).setupLocalDiskFetch(mapHost); @@ -224,15 +216,12 @@ public void testSetupLocalDiskFetch() throws Exception { ShuffleScheduler scheduler = mock(ShuffleScheduler.class); MergeManager merger = mock(MergeManager.class); Shuffle shuffle = mock(Shuffle.class); - InputContext inputContext = mock(InputContext.class); - when(inputContext.getCounters()).thenReturn(new TezCounters()); - when(inputContext.getSourceVertexName()).thenReturn(""); MapHost host = new MapHost(HOST, PORT, 1, 1); FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, - null, false, 0, null, conf, getRawFs(conf), true, HOST, PORT, "src vertex", host, + null, false, 0, null, conf, getRawFs(conf), true, HOST, PORT, host, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, - connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); + connectionErrsCounter, wrongReduceErrsCounter, false, false, true, false, createMockInputContext()); FetcherOrderedGrouped spyFetcher = spy(fetcher); @@ -338,15 +327,12 @@ public void testSetupLocalDiskFetchEmptyPartitions() throws Exception { ShuffleScheduler scheduler = mock(ShuffleScheduler.class); MergeManager merger = mock(MergeManager.class); Shuffle shuffle = mock(Shuffle.class); - InputContext inputContext = mock(InputContext.class); - when(inputContext.getCounters()).thenReturn(new TezCounters()); - when(inputContext.getSourceVertexName()).thenReturn(""); MapHost host = new MapHost(HOST, PORT, 1, 1); FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, - null, false, 0, null, conf, getRawFs(conf), true, HOST, PORT, "src vertex", host, + null, false, 0, null, conf, getRawFs(conf), true, HOST, PORT, host, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, - connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); + connectionErrsCounter, wrongReduceErrsCounter, false, false, true, false, createMockInputContext()); FetcherOrderedGrouped spyFetcher = spy(fetcher); final List srcAttempts = Arrays.asList( @@ -413,15 +399,12 @@ public void testSetupLocalDiskFetchAutoReduce() throws Exception { ShuffleScheduler scheduler = mock(ShuffleScheduler.class); MergeManager merger = mock(MergeManager.class); Shuffle shuffle = mock(Shuffle.class); - InputContext inputContext = mock(InputContext.class); - when(inputContext.getCounters()).thenReturn(new TezCounters()); - when(inputContext.getSourceVertexName()).thenReturn(""); MapHost host = new MapHost(HOST, PORT, 1, 2); FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, - null, false, 0, null, conf, getRawFs(conf), true, HOST, PORT, "src vertex", host, + null, false, 0, null, conf, getRawFs(conf), true, HOST, PORT, host, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, - connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); + connectionErrsCounter, wrongReduceErrsCounter, false, false, true, false, createMockInputContext()); FetcherOrderedGrouped spyFetcher = spy(fetcher); @@ -593,17 +576,12 @@ public void testWithRetry() throws Exception { MergeManager merger = mock(MergeManager.class); Shuffle shuffle = mock(Shuffle.class); - InputContext inputContext = mock(InputContext.class); - when(inputContext.getCounters()).thenReturn(new TezCounters()); - when(inputContext.getSourceVertexName()).thenReturn(""); - when(inputContext.getApplicationId()).thenReturn(ApplicationId.newInstance(0, 1)); - HttpConnectionParams httpConnectionParams = ShuffleUtils.getHttpConnectionParams(conf); final MapHost host = new MapHost(HOST, PORT, 1, 1); FetcherOrderedGrouped mockFetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, - null, false, 0, null, conf, getRawFs(conf), false, HOST, PORT, "src vertex", host, + null, false, 0, null, conf, getRawFs(conf), false, HOST, PORT, host, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, - connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); + connectionErrsCounter, wrongReduceErrsCounter, false, false, true, false, createMockInputContext()); final FetcherOrderedGrouped fetcher = spy(mockFetcher); @@ -676,11 +654,6 @@ public void testAsyncWithException() throws Exception { MergeManager merger = mock(MergeManager.class); Shuffle shuffle = mock(Shuffle.class); - TezCounters counters = new TezCounters(); - InputContext inputContext = mock(InputContext.class); - when(inputContext.getCounters()).thenReturn(counters); - when(inputContext.getSourceVertexName()).thenReturn(""); - JobTokenSecretManager jobMgr = mock(JobTokenSecretManager.class); doReturn(new byte[10]).when(jobMgr).computeHash(any(byte[].class)); @@ -688,9 +661,9 @@ public void testAsyncWithException() throws Exception { final MapHost host = new MapHost(HOST, PORT, 1, 1); FetcherOrderedGrouped mockFetcher = new FetcherOrderedGrouped(httpConnectionParams, scheduler, merger, shuffle, jobMgr, false, - 0, null, conf, getRawFs(conf), false, HOST, PORT, "src vertex", host, ioErrsCounter, + 0, null, conf, getRawFs(conf), false, HOST, PORT, host, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, connectionErrsCounter, - wrongReduceErrsCounter, APP_ID, DAG_ID, true, false, true, false); + wrongReduceErrsCounter, true, false, true, false, createMockInputContext()); final FetcherOrderedGrouped fetcher = spy(mockFetcher); fetcher.remaining = new LinkedHashMap(); final List srcAttempts = Arrays.asList( @@ -753,9 +726,9 @@ public void testInputAttemptIdentifierMap() { Shuffle shuffle = mock(Shuffle.class); MapHost mapHost = new MapHost(HOST, PORT, 0, 1); FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, - null, false, 0, null, conf, getRawFs(conf), false, HOST, PORT, "src vertex", mapHost, + null, false, 0, null, conf, getRawFs(conf), false, HOST, PORT, mapHost, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, - connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); + connectionErrsCounter, wrongReduceErrsCounter, false, false, true, false, createMockInputContext()); fetcher.populateRemainingMap(new LinkedList(Arrays.asList(srcAttempts))); Assert.assertEquals(expectedSrcAttempts.length, fetcher.remaining.size()); Iterator> iterator = fetcher.remaining.entrySet().iterator(); @@ -773,9 +746,9 @@ public void testShuffleHandlerDiskErrorOrdered() InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(0, 0, "attempt"); FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, null, null, null, null, false, - 0, null, new TezConfiguration(), null, false, HOST, PORT, "src vertex", mapHost, + 0, null, new TezConfiguration(), null, false, HOST, PORT, mapHost, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, - connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); + connectionErrsCounter, wrongReduceErrsCounter, false, false, true, false, createMockInputContext()); fetcher.remaining = new HashMap(); ShuffleHeader header = @@ -800,4 +773,17 @@ private RawLocalFileSystem getRawFs(Configuration conf) { throw new RuntimeException(e); } } + + + private InputContext createMockInputContext() { + InputContext inputContext = mock(InputContext.class); + + doReturn(APP_ID).when(inputContext).getApplicationId(); + doReturn(DAG_ID).when(inputContext).getDagIdentifier(); + doReturn(new TezCounters()).when(inputContext).getCounters(); + doReturn("src vertex").when(inputContext).getSourceVertexName(); + doReturn("task_Vertex").when(inputContext).getTaskVertexName(); + + return inputContext; + } } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffle.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffle.java index a28b1fa1a5..0a9c37e9ea 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffle.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffle.java @@ -144,6 +144,7 @@ private InputContext createTezInputContext() throws IOException { InputContext inputContext = mock(InputContext.class); doReturn(applicationId).when(inputContext).getApplicationId(); doReturn("sourceVertex").when(inputContext).getSourceVertexName(); + doReturn("taskVertex").when(inputContext).getTaskVertexName(); when(inputContext.getCounters()).thenReturn(new TezCounters()); ExecutionContext executionContext = new ExecutionContextImpl("localhost"); doReturn(executionContext).when(inputContext).getExecutionContext(); diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestInput.java b/tez-tests/src/test/java/org/apache/tez/test/TestInput.java index 56bfe49dc8..36bb983d1d 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestInput.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestInput.java @@ -352,7 +352,8 @@ public void handleEvents(List inputEvents) throws Exception { if (event instanceof DataMovementEvent) { DataMovementEvent dmEvent = (DataMovementEvent) event; numCompletedInputs++; - LOG.info(getContext().getSourceVertexName() + " Received DataMovement event sourceId : " + dmEvent.getSourceIndex() + + LOG.info(getContext().getInputOutputVertexNames() + + " Received DataMovement event sourceId : " + dmEvent.getSourceIndex() + " targetId: " + dmEvent.getTargetIndex() + " version: " + dmEvent.getVersion() + " numInputs: " + getNumPhysicalInputs() + From 0210dab45042b20d40437aa416740df9d2482309 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Fri, 21 Jan 2022 18:07:49 +0100 Subject: [PATCH 351/512] TEZ-4310: Avoid duplicated precommit comments (#176) --- Jenkinsfile | 4 ---- 1 file changed, 4 deletions(-) diff --git a/Jenkinsfile b/Jenkinsfile index aadfaa953e..4ce236598d 100644 --- a/Jenkinsfile +++ b/Jenkinsfile @@ -108,10 +108,6 @@ pipeline { YETUS_ARGS+=(--github-user="${GITHUB_USER}") YETUS_ARGS+=(--github-token="${GITHUB_TOKEN}") - # enable writing back to ASF JIRA - YETUS_ARGS+=(--jira-password="${JIRA_PASSWORD}") - YETUS_ARGS+=(--jira-user="${JIRA_USER}") - # auto-kill any surefire stragglers during unit test runs YETUS_ARGS+=("--reapermode=kill") From 02d574fd6a4681eca05cd1d6aba19304d6ada3d1 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Fri, 21 Jan 2022 18:08:50 +0100 Subject: [PATCH 352/512] TEZ-4369: Upgrade netty to 4.1.72 due to CVE-2021-37136, CVE-2021-37137 (#175) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 5cf77deedf..aff586bad9 100644 --- a/pom.xml +++ b/pom.xml @@ -59,7 +59,7 @@ ${user.home}/clover.license 27.0-jre 3.3.1 - 4.1.61.Final + 4.1.72.Final 0.13.0 1.19 1.7.30 From 267ca11b497d080042feca5b98cf0fb7ca9b6e15 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Fri, 21 Jan 2022 18:09:49 +0100 Subject: [PATCH 353/512] TEZ-4366: Disk utilization limit fix should be applied to all tez mini clusters (#173) * TEZ-4366: Disk utilization limit fix should be applied to all tez mini clusters --- .../org/apache/tez/common/TezTestUtils.java | 38 +++++++++++++++++++ .../tez/tests/MiniTezClusterWithTimeline.java | 7 ++++ .../org/apache/tez/test/MiniTezCluster.java | 6 +-- 3 files changed, 47 insertions(+), 4 deletions(-) create mode 100644 tez-common/src/main/java/org/apache/tez/common/TezTestUtils.java diff --git a/tez-common/src/main/java/org/apache/tez/common/TezTestUtils.java b/tez-common/src/main/java/org/apache/tez/common/TezTestUtils.java new file mode 100644 index 0000000000..4b9d5c3218 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/common/TezTestUtils.java @@ -0,0 +1,38 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.common; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.conf.YarnConfiguration; + +public class TezTestUtils { + /** + * Ensures a reasonably high limit for yarn disk utilization. This is very important for tests, + * as devs keep bumping into silent test hangs where yarn simply considers their machines as unhealthy, + * as the default limit is 90%, even if a machine with 90% full disk is still able to function. + * @param conf + */ + public static void ensureHighDiskUtilizationLimit(Configuration conf) { + if (conf.getFloat(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, + YarnConfiguration.DEFAULT_NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE) == + YarnConfiguration.DEFAULT_NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE) { + conf.setFloat(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, 99.0f); + } + } +} diff --git a/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/tests/MiniTezClusterWithTimeline.java b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/tests/MiniTezClusterWithTimeline.java index d13ebdbee2..f8a35a29c3 100644 --- a/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/tests/MiniTezClusterWithTimeline.java +++ b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/tests/MiniTezClusterWithTimeline.java @@ -49,6 +49,7 @@ import org.apache.hadoop.yarn.server.MiniYARNCluster; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor; +import org.apache.tez.common.TezTestUtils; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.TezUncheckedException; import org.apache.tez.dag.app.DAGAppMaster; @@ -92,6 +93,12 @@ public MiniTezClusterWithTimeline(String testName, int noOfNMs, super(testName, 1, noOfNMs, numLocalDirs, numLogDirs, enableAHS); } + @Override + public void init(Configuration conf) { + TezTestUtils.ensureHighDiskUtilizationLimit(conf); + super.init(conf); + } + @Override public void serviceInit(Configuration conf) throws Exception { conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.YARN_TEZ_FRAMEWORK_NAME); diff --git a/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java b/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java index 17c688590c..9af1e604b2 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java +++ b/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java @@ -50,6 +50,7 @@ import org.apache.hadoop.yarn.server.MiniYARNCluster; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor; +import org.apache.tez.common.TezTestUtils; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.TezUncheckedException; import org.apache.tez.dag.app.DAGAppMaster; @@ -90,10 +91,7 @@ public MiniTezCluster(String testName, int noOfNMs, @Override public void init(Configuration conf) { - if (conf.getFloat(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, - YarnConfiguration.DEFAULT_NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE) == YarnConfiguration.DEFAULT_NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE) { - conf.setFloat(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, 99.0f); - } + TezTestUtils.ensureHighDiskUtilizationLimit(conf); super.init(conf); } From 1176386c9519d90b69371e954f8b60b20dabc2f7 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Fri, 21 Jan 2022 21:40:28 +0100 Subject: [PATCH 354/512] TEZ-4350: Remove synchronized from DAGAppMaster.serviceInit (#162) --- tez-dag/findbugs-exclude.xml | 18 +--- .../org/apache/tez/dag/app/DAGAppMaster.java | 94 +++++++++---------- 2 files changed, 47 insertions(+), 65 deletions(-) diff --git a/tez-dag/findbugs-exclude.xml b/tez-dag/findbugs-exclude.xml index 50422ff0e0..e8755832f4 100644 --- a/tez-dag/findbugs-exclude.xml +++ b/tez-dag/findbugs-exclude.xml @@ -154,24 +154,8 @@ - - - - - - - - - - - - - - - - - + diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index abc10bd86c..f15c73c04a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -416,7 +416,7 @@ private long getAMGCTime() { } @Override - public synchronized void serviceInit(final Configuration conf) throws Exception { + protected void serviceInit(final Configuration conf) throws Exception { this.amConf = conf; initResourceCalculatorPlugins(); @@ -1923,7 +1923,7 @@ private DAGRecoveryData recoverDAG() throws IOException, TezException { } @Override - public synchronized void serviceStart() throws Exception { + public void serviceStart() throws Exception { //start all the components startServices(); super.serviceStart(); @@ -2115,57 +2115,55 @@ public void serviceStop() throws Exception { if (isSession) { sessionStopped.set(true); } - synchronized (this) { - if (this.dagSubmissionTimer != null) { - this.dagSubmissionTimer.cancel(); - } - if (this.clientAMHeartBeatTimeoutService != null) { - this.clientAMHeartBeatTimeoutService.shutdownNow(); - } - // release all the held containers before stop services TEZ-2687 - initiateStop(); - stopServices(); - - // Given pre-emption, we should delete tez scratch dir only if unregister is - // successful - boolean deleteTezScratchData = this.amConf.getBoolean( - TezConfiguration.TEZ_AM_STAGING_SCRATCH_DATA_AUTO_DELETE, - TezConfiguration.TEZ_AM_STAGING_SCRATCH_DATA_AUTO_DELETE_DEFAULT); - LOG.debug("Checking whether tez scratch data dir should be deleted, deleteTezScratchData={}", - deleteTezScratchData); - if (deleteTezScratchData && this.taskSchedulerManager != null - && this.taskSchedulerManager.hasUnregistered()) { - // Delete tez scratch data dir - if (this.tezSystemStagingDir != null) { - try { - this.appMasterUgi.doAs(new PrivilegedExceptionAction() { - @Override - public Void run() throws Exception { - FileSystem fs = tezSystemStagingDir.getFileSystem(amConf); - boolean deletedStagingDir = fs.delete(tezSystemStagingDir, true); - if (!deletedStagingDir) { - LOG.warn("Failed to delete tez scratch data dir, path=" - + tezSystemStagingDir); - } else { - LOG.info("Completed deletion of tez scratch data dir, path=" - + tezSystemStagingDir); - } - return null; + if (this.dagSubmissionTimer != null) { + this.dagSubmissionTimer.cancel(); + } + if (this.clientAMHeartBeatTimeoutService != null) { + this.clientAMHeartBeatTimeoutService.shutdownNow(); + } + // release all the held containers before stop services TEZ-2687 + initiateStop(); + stopServices(); + + // Given pre-emption, we should delete tez scratch dir only if unregister is + // successful + boolean deleteTezScratchData = this.amConf.getBoolean( + TezConfiguration.TEZ_AM_STAGING_SCRATCH_DATA_AUTO_DELETE, + TezConfiguration.TEZ_AM_STAGING_SCRATCH_DATA_AUTO_DELETE_DEFAULT); + LOG.debug("Checking whether tez scratch data dir should be deleted, deleteTezScratchData={}", + deleteTezScratchData); + if (deleteTezScratchData && this.taskSchedulerManager != null + && this.taskSchedulerManager.hasUnregistered()) { + // Delete tez scratch data dir + if (this.tezSystemStagingDir != null) { + try { + this.appMasterUgi.doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + FileSystem fs = tezSystemStagingDir.getFileSystem(amConf); + boolean deletedStagingDir = fs.delete(tezSystemStagingDir, true); + if (!deletedStagingDir) { + LOG.warn("Failed to delete tez scratch data dir, path=" + + tezSystemStagingDir); + } else { + LOG.info("Completed deletion of tez scratch data dir, path=" + + tezSystemStagingDir); } - }); - } catch (IOException e) { - // Best effort to delete tez scratch data dir - LOG.warn("Failed to delete tez scratch data dir", e); - } + return null; + } + }); + } catch (IOException e) { + // Best effort to delete tez scratch data dir + LOG.warn("Failed to delete tez scratch data dir", e); } } + } - if (execService != null) { - execService.shutdownNow(); - } - - super.serviceStop(); + if (execService != null) { + execService.shutdownNow(); } + + super.serviceStop(); } private class DagEventDispatcher implements EventHandler { From 4e3eb9574a9c59ea534a70021531337b2de13ed3 Mon Sep 17 00:00:00 2001 From: Syed Shameerur Rahman Date: Thu, 27 Jan 2022 19:47:53 +0530 Subject: [PATCH 355/512] TEZ-4129: Delete intermediate attempt data for failed attempts for Shuffle Handler (#72) (Syed Shameerur Rahman reviewed by Laszlo Bodor) --- .../apache/tez/dag/api/TezConfiguration.java | 12 ++- .../tez/common/DagContainerLauncher.java | 5 ++ .../org/apache/tez/dag/app/DAGAppMaster.java | 5 ++ .../apache/tez/dag/app/dag/impl/TaskImpl.java | 9 +- .../launcher/ContainerLauncherManager.java | 8 ++ .../launcher/ContainerLauncherWrapper.java | 9 ++ .../tez/dag/app/launcher/DeletionTracker.java | 6 ++ .../dag/app/launcher/DeletionTrackerImpl.java | 21 +++++ .../app/launcher/LocalContainerLauncher.java | 23 +++++- .../TaskAttemptFailedDeleteRunnable.java | 82 +++++++++++++++++++ .../launcher/TezContainerLauncherImpl.java | 25 ++++-- .../TestContainerLauncherWrapper.java | 2 +- .../tez/auxservices/ShuffleHandler.java | 54 +++++++++++- .../tez/auxservices/TestShuffleHandler.java | 82 +++++++++++++++++++ .../library/common/TezRuntimeUtils.java | 19 +++++ 15 files changed, 348 insertions(+), 14 deletions(-) create mode 100644 tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index ee991ccf53..16d1dfcb70 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -884,7 +884,17 @@ public TezConfiguration(boolean loadDefaults) { public static final boolean TEZ_AM_DAG_CLEANUP_ON_COMPLETION_DEFAULT = false; /** - * Int value. Upper limit on the number of threads used to delete DAG directories on nodes. + * Boolean value. Instructs AM to delete intermediate attempt data for failed task attempts. + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty(type="boolean") + public static final String TEZ_AM_TASK_ATTEMPT_CLEANUP_ON_FAILURE = TEZ_AM_PREFIX + + "task.attempt.cleanup.on.failure"; + public static final boolean TEZ_AM_TASK_ATTEMPT_CLEANUP_ON_FAILURE_DEFAULT = false; + + /** + * Int value. Upper limit on the number of threads used to delete DAG directories and failed task attempts + * directories on nodes. */ @ConfigurationScope(Scope.AM) @ConfigurationProperty(type="integer") diff --git a/tez-common/src/main/java/org/apache/tez/common/DagContainerLauncher.java b/tez-common/src/main/java/org/apache/tez/common/DagContainerLauncher.java index e3bd385a1a..6bda0a8da3 100644 --- a/tez-common/src/main/java/org/apache/tez/common/DagContainerLauncher.java +++ b/tez-common/src/main/java/org/apache/tez/common/DagContainerLauncher.java @@ -20,8 +20,10 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.dag.records.TezDAGID; +import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.serviceplugins.api.ContainerLauncher; import org.apache.tez.serviceplugins.api.ContainerLauncherContext; @@ -40,4 +42,7 @@ public DagContainerLauncher(ContainerLauncherContext containerLauncherContext) { } public abstract void dagComplete(TezDAGID dag, JobTokenSecretManager jobTokenSecretManager); + + public abstract void taskAttemptFailed(TezTaskAttemptID taskAttemptID, + JobTokenSecretManager jobTokenSecretManager, NodeId nodeId); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index f15c73c04a..3be9aacec5 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -67,6 +67,7 @@ import org.apache.commons.cli.GnuParser; import org.apache.commons.cli.Options; import org.apache.commons.lang.exception.ExceptionUtils; +import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.tez.Utils; import org.apache.tez.client.CallerContext; import org.apache.tez.client.TezClientUtils; @@ -178,6 +179,7 @@ import org.apache.tez.dag.history.events.DAGSubmittedEvent; import org.apache.tez.dag.history.utils.DAGUtils; import org.apache.tez.dag.records.TezDAGID; +import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.dag.utils.RelocalizationUtils; import org.apache.tez.dag.utils.Simple2LevelVersionComparator; @@ -2738,4 +2740,7 @@ String buildPluginComponentLog(List namedEntityDescriptor return sb.toString(); } + public void taskAttemptFailed(TezTaskAttemptID attemptID, NodeId nodeId) { + getContainerLauncherManager().taskAttemptFailed(attemptID, jobTokenSecretManager, nodeId); + } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java index cb8545f8ca..b840798c26 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java @@ -39,13 +39,13 @@ import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.hadoop.classification.InterfaceAudience.Private; -import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.tez.dag.app.dag.event.TaskEventTAFailed; import org.apache.tez.runtime.api.TaskFailureType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.state.InvalidStateTransitonException; @@ -1263,6 +1263,13 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { } // The attempt would have informed the scheduler about it's failure + // Delete the intermediate shuffle data for failed task attempt + TaskAttempt taskAttempt = task.getAttempt(castEvent.getTaskAttemptID()); + if (taskAttempt.getAssignedContainer() != null) { + NodeId nodeId = taskAttempt.getAssignedContainer().getNodeId(); + task.appContext.getAppMaster().taskAttemptFailed(taskAttempt.getID(), nodeId); + } + task.taskAttemptStatus.put(castEvent.getTaskAttemptID().getId(), true); if (task.failedAttempts < task.maxFailedAttempts && castEvent.getTaskFailureType() == TaskFailureType.NON_FATAL) { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherManager.java index d55787e25a..b0e0f0cf0d 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherManager.java @@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.event.Event; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.tez.Utils; @@ -35,6 +36,7 @@ import org.apache.tez.dag.app.dag.event.DAGAppMasterEventType; import org.apache.tez.dag.app.dag.event.DAGAppMasterEventUserServiceFatalError; import org.apache.tez.dag.records.TezDAGID; +import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.serviceplugins.api.ContainerLaunchRequest; import org.apache.tez.serviceplugins.api.ContainerLauncher; import org.apache.tez.serviceplugins.api.ContainerLauncherContext; @@ -200,6 +202,12 @@ public void dagComplete(TezDAGID dag, JobTokenSecretManager secretManager) { } } + public void taskAttemptFailed(TezTaskAttemptID taskAttemptID, JobTokenSecretManager secretManager, NodeId nodeId) { + for (int i = 0; i < containerLaunchers.length; i++) { + containerLaunchers[i].taskAttemptFailed(taskAttemptID, secretManager, nodeId); + } + } + public void dagSubmitted() { // Nothing to do right now. Indicates that a new DAG has been submitted and // the context has updated information. diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherWrapper.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherWrapper.java index 8ecac14856..5d262bdab4 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherWrapper.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherWrapper.java @@ -15,8 +15,10 @@ package org.apache.tez.dag.app.launcher; import org.apache.tez.common.DagContainerLauncher; +import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.dag.records.TezDAGID; +import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.serviceplugins.api.ContainerLaunchRequest; import org.apache.tez.serviceplugins.api.ContainerLauncher; import org.apache.tez.serviceplugins.api.ContainerStopRequest; @@ -46,4 +48,11 @@ public void dagComplete(TezDAGID dag, JobTokenSecretManager jobTokenSecretManage ((DagContainerLauncher)real).dagComplete(dag, jobTokenSecretManager); } } + + public void taskAttemptFailed(TezTaskAttemptID taskAttemptID, JobTokenSecretManager jobTokenSecretManager, + NodeId nodeId) { + if (real instanceof DagContainerLauncher) { + ((DagContainerLauncher) real).taskAttemptFailed(taskAttemptID, jobTokenSecretManager, nodeId); + } + } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTracker.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTracker.java index 27ece70513..87b7366bfc 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTracker.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTracker.java @@ -22,6 +22,7 @@ import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.dag.records.TezDAGID; +import org.apache.tez.dag.records.TezTaskAttemptID; public abstract class DeletionTracker { @@ -35,6 +36,11 @@ public void dagComplete(TezDAGID dag, JobTokenSecretManager jobTokenSecretManage //do nothing } + public void taskAttemptFailed(TezTaskAttemptID taskAttemptID, JobTokenSecretManager jobTokenSecretManager, + NodeId nodeId) { + //do nothing + } + public void addNodeShufflePort(NodeId nodeId, int port) { //do nothing } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTrackerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTrackerImpl.java index 06dae2d2b0..e4204bfc71 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTrackerImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTrackerImpl.java @@ -34,6 +34,7 @@ import org.apache.tez.dag.records.TezDAGID; import org.apache.hadoop.conf.Configuration; import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -70,6 +71,26 @@ public void dagComplete(TezDAGID dag, JobTokenSecretManager jobTokenSecretManage } } + @Override + public void taskAttemptFailed(TezTaskAttemptID taskAttemptID, JobTokenSecretManager jobTokenSecretManager, + NodeId nodeId) { + super.taskAttemptFailed(taskAttemptID, jobTokenSecretManager, nodeId); + if (nodeIdShufflePortMap == null || nodeIdShufflePortMap.get(nodeId) == null) { + LOG.warn("Unable to find the shuffle port for shuffle data deletion of failed task attempt."); + return; + } + int shufflePort = nodeIdShufflePortMap.get(nodeId); + if (shufflePort != TezRuntimeUtils.INVALID_PORT) { + TaskAttemptFailedRunnable taskAttemptFailedRunnable = new TaskAttemptFailedRunnable(nodeId, shufflePort, + taskAttemptID, TezRuntimeUtils.getHttpConnectionParams(conf), jobTokenSecretManager); + try { + dagCleanupService.submit(taskAttemptFailedRunnable); + } catch (RejectedExecutionException rejectedException) { + LOG.info("Ignoring failed task attempt deletion request for " + taskAttemptFailedRunnable); + } + } + } + @Override public void addNodeShufflePort(NodeId nodeId, int port) { if (port != TezRuntimeUtils.INVALID_PORT) { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java index ae308098a7..ebc8f95566 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java @@ -43,11 +43,13 @@ import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.tez.common.DagContainerLauncher; import org.apache.tez.common.ReflectionUtils; import org.apache.tez.common.TezUtils; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.common.security.JobTokenSecretManager; +import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.serviceplugins.api.ContainerLaunchRequest; @@ -93,6 +95,8 @@ public class LocalContainerLauncher extends DagContainerLauncher { private final boolean isLocalMode; int shufflePort = TezRuntimeUtils.INVALID_PORT; private DeletionTracker deletionTracker; + private boolean dagDelete; + private boolean failedTaskAttemptDelete; private final ConcurrentHashMap> runningContainers = @@ -155,10 +159,14 @@ public LocalContainerLauncher(ContainerLauncherContext containerLauncherContext, new ThreadFactoryBuilder().setDaemon(true).setNameFormat("LocalTaskExecutionThread #%d") .build()); this.taskExecutorService = MoreExecutors.listeningDecorator(rawExecutor); - boolean cleanupDagDataOnComplete = ShuffleUtils.isTezShuffleHandler(conf) - && conf.getBoolean(TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION, + dagDelete = ShuffleUtils.isTezShuffleHandler(conf) && + conf.getBoolean(TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION, TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION_DEFAULT); - if (cleanupDagDataOnComplete) { + failedTaskAttemptDelete = ShuffleUtils.isTezShuffleHandler(conf) && + conf.getBoolean(TezConfiguration.TEZ_AM_TASK_ATTEMPT_CLEANUP_ON_FAILURE, + TezConfiguration.TEZ_AM_TASK_ATTEMPT_CLEANUP_ON_FAILURE_DEFAULT); + + if (dagDelete || failedTaskAttemptDelete) { String deletionTrackerClassName = conf.get(TezConfiguration.TEZ_AM_DELETION_TRACKER_CLASS, TezConfiguration.TEZ_AM_DELETION_TRACKER_CLASS_DEFAULT); deletionTracker = ReflectionUtils.createClazzInstance( @@ -441,9 +449,16 @@ public void stopContainer(ContainerStopRequest stopRequest) { @Override public void dagComplete(TezDAGID dag, JobTokenSecretManager jobTokenSecretManager) { - if (deletionTracker != null) { + if (dagDelete && deletionTracker != null) { deletionTracker.dagComplete(dag, jobTokenSecretManager); } } + @Override + public void taskAttemptFailed(TezTaskAttemptID taskAttemptID, JobTokenSecretManager jobTokenSecretManager, + NodeId nodeId) { + if (failedTaskAttemptDelete && deletionTracker != null) { + deletionTracker.taskAttemptFailed(taskAttemptID, jobTokenSecretManager, nodeId); + } + } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java new file mode 100644 index 0000000000..d2587b5190 --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java @@ -0,0 +1,82 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.app.launcher; + +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.tez.common.security.JobTokenSecretManager; +import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.http.BaseHttpConnection; +import org.apache.tez.http.HttpConnectionParams; +import org.apache.tez.runtime.library.common.TezRuntimeUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.URL; + +class TaskAttemptFailedRunnable implements Runnable { + private static final Logger LOG = LoggerFactory.getLogger(TaskAttemptFailedRunnable.class); + private final NodeId nodeId; + private final TezTaskAttemptID taskAttemptID; + private final JobTokenSecretManager jobTokenSecretManager; + private final int shufflePort; + private final HttpConnectionParams httpConnectionParams; + + TaskAttemptFailedRunnable(NodeId nodeId, int shufflePort, TezTaskAttemptID taskAttemptID, + HttpConnectionParams httpConnectionParams, + JobTokenSecretManager jobTokenSecretMgr) { + this.nodeId = nodeId; + this.shufflePort = shufflePort; + this.taskAttemptID = taskAttemptID; + this.httpConnectionParams = httpConnectionParams; + this.jobTokenSecretManager = jobTokenSecretMgr; + } + + @Override + public void run() { + BaseHttpConnection httpConnection = null; + try { + URL baseURL = TezRuntimeUtils.constructBaseURIForShuffleHandlerTaskAttemptFailed( + nodeId.getHost(), shufflePort, taskAttemptID.getTaskID().getVertexID().getDAGId(). + getApplicationId().toString(), taskAttemptID.getTaskID().getVertexID().getDAGId().getId(), + taskAttemptID.toString(), false); + httpConnection = TezRuntimeUtils.getHttpConnection(true, baseURL, httpConnectionParams, + "FailedTaskAttemptDelete", jobTokenSecretManager); + httpConnection.connect(); + httpConnection.getInputStream(); + } catch (Exception e) { + LOG.warn("Could not setup HTTP Connection to the node " + nodeId.getHost() + + " for failed task attempt delete. ", e); + } finally { + try { + if (httpConnection != null) { + httpConnection.cleanup(true); + } + } catch (IOException ioe) { + LOG.warn("Encountered IOException for " + nodeId.getHost() + " during close. ", ioe); + } + } + } + + @Override + public String toString() { + return "TaskAttemptFailedRunnable nodeId=" + nodeId + ", shufflePort=" + shufflePort + ", taskAttemptId=" + + taskAttemptID.toString(); + } +} diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezContainerLauncherImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezContainerLauncherImpl.java index f5be74683e..88ed4f7b89 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezContainerLauncherImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezContainerLauncherImpl.java @@ -34,6 +34,7 @@ import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.hadoop.io.DataInputByteBuffer; +import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.tez.common.DagContainerLauncher; import org.apache.tez.common.ReflectionUtils; import org.apache.tez.common.TezUtils; @@ -41,6 +42,7 @@ import org.apache.tez.dag.api.TezConstants; import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.records.TezDAGID; +import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.serviceplugins.api.ContainerLaunchRequest; @@ -90,6 +92,8 @@ public class TezContainerLauncherImpl extends DagContainerLauncher { private ContainerManagementProtocolProxy cmProxy; private AtomicBoolean serviceStopped = new AtomicBoolean(false); private DeletionTracker deletionTracker = null; + private boolean dagDelete; + private boolean failedTaskAttemptDelete; private Container getContainer(ContainerOp event) { ContainerId id = event.getBaseOperation().getContainerId(); @@ -332,10 +336,14 @@ public void run() { }; eventHandlingThread.setName("ContainerLauncher Event Handler"); eventHandlingThread.start(); - boolean cleanupDagDataOnComplete = ShuffleUtils.isTezShuffleHandler(conf) - && conf.getBoolean(TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION, - TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION_DEFAULT); - if (cleanupDagDataOnComplete) { + dagDelete = ShuffleUtils.isTezShuffleHandler(conf) && + conf.getBoolean(TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION, + TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION_DEFAULT); + failedTaskAttemptDelete = ShuffleUtils.isTezShuffleHandler(conf) && + conf.getBoolean(TezConfiguration.TEZ_AM_TASK_ATTEMPT_CLEANUP_ON_FAILURE, + TezConfiguration.TEZ_AM_TASK_ATTEMPT_CLEANUP_ON_FAILURE_DEFAULT); + + if (dagDelete || failedTaskAttemptDelete) { String deletionTrackerClassName = conf.get(TezConfiguration.TEZ_AM_DELETION_TRACKER_CLASS, TezConfiguration.TEZ_AM_DELETION_TRACKER_CLASS_DEFAULT); deletionTracker = ReflectionUtils.createClazzInstance( @@ -441,9 +449,16 @@ public void stopContainer(ContainerStopRequest stopRequest) { @Override public void dagComplete(TezDAGID dag, JobTokenSecretManager jobTokenSecretManager) { - if (deletionTracker != null) { + if (dagDelete && deletionTracker != null) { deletionTracker.dagComplete(dag, jobTokenSecretManager); } } + @Override + public void taskAttemptFailed(TezTaskAttemptID taskAttemptID, JobTokenSecretManager jobTokenSecretManager, + NodeId nodeId) { + if (failedTaskAttemptDelete && deletionTracker != null) { + deletionTracker.taskAttemptFailed(taskAttemptID, jobTokenSecretManager, nodeId); + } + } } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherWrapper.java b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherWrapper.java index 8778f32d58..c4f4eff0cf 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherWrapper.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherWrapper.java @@ -24,7 +24,7 @@ public class TestContainerLauncherWrapper { @Test(timeout = 5000) public void testDelegation() throws Exception { PluginWrapperTestHelpers.testDelegation(ContainerLauncherWrapper.class, ContainerLauncher.class, - Sets.newHashSet("getContainerLauncher", "dagComplete")); + Sets.newHashSet("getContainerLauncher", "dagComplete", "taskAttemptFailed")); } } diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java index a54a7cfe01..9730be1869 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java @@ -57,6 +57,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileContext; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.DataInputByteBuffer; @@ -1007,6 +1009,7 @@ private void handleRequest(ChannelHandlerContext ctx, FullHttpRequest request) final Map> q = new QueryStringDecoder(request.getUri()).parameters(); final List keepAliveList = q.get("keepAlive"); final List dagCompletedQ = q.get("dagAction"); + final List taskAttemptFailedQ = q.get("taskAttemptAction"); boolean keepAliveParam = false; if (keepAliveList != null && keepAliveList.size() == 1) { keepAliveParam = Boolean.parseBoolean(keepAliveList.get(0)); @@ -1028,6 +1031,9 @@ private void handleRequest(ChannelHandlerContext ctx, FullHttpRequest request) if (deleteDagDirectories(ctx.channel(), dagCompletedQ, jobQ, dagIdQ)) { return; } + if (deleteTaskAttemptDirectories(ctx.channel(), taskAttemptFailedQ, jobQ, dagIdQ, mapIds)) { + return; + } if (mapIds == null || reduceRange == null || jobQ == null || dagIdQ == null) { sendError(ctx, "Required param job, dag, map and reduce", BAD_REQUEST); return; @@ -1115,14 +1121,24 @@ private void handleRequest(ChannelHandlerContext ctx, FullHttpRequest request) ch.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT); } + private boolean isNullOrEmpty(List entries) { + return entries == null || entries.isEmpty(); + } + + private boolean notEmptyAndContains(List entries, String key) { + if (entries == null || entries.isEmpty()) { + return false; + } + return entries.get(0).contains(key); + } + private boolean deleteDagDirectories(Channel channel, List dagCompletedQ, List jobQ, List dagIdQ) { if (jobQ == null || jobQ.isEmpty()) { return false; } - if (dagCompletedQ != null && !dagCompletedQ.isEmpty() && dagCompletedQ.get(0).contains("delete") - && dagIdQ != null && !dagIdQ.isEmpty()) { + if (notEmptyAndContains(dagCompletedQ,"delete") && !isNullOrEmpty(dagIdQ)) { String base = getDagLocation(jobQ.get(0), dagIdQ.get(0), userRsrc.get(jobQ.get(0))); try { FileContext lfc = FileContext.getLocalFSFileContext(); @@ -1139,6 +1155,40 @@ private boolean deleteDagDirectories(Channel channel, return false; } + private boolean deleteTaskAttemptDirectories(Channel channel, List taskAttemptFailedQ, + List jobQ, List dagIdQ, List taskAttemptIdQ) { + if (jobQ == null || jobQ.isEmpty()) { + return false; + } + if (notEmptyAndContains(taskAttemptFailedQ,"delete") && !isNullOrEmpty(taskAttemptIdQ)) { + for (String taskAttemptId : taskAttemptIdQ) { + String baseStr = getBaseLocation(jobQ.get(0), dagIdQ.get(0), userRsrc.get(jobQ.get(0))); + try { + FileSystem fs = FileSystem.getLocal(conf).getRaw(); + for (Path basePath : lDirAlloc.getAllLocalPathsToRead(baseStr, conf)) { + for (FileStatus fileStatus : fs.listStatus(basePath)) { + Path taskAttemptPath = fileStatus.getPath(); + if (taskAttemptPath.getName().startsWith(taskAttemptId)) { + if (fs.delete(taskAttemptPath, true)) { + LOG.info("Deleted directory : " + taskAttemptPath); + // remove entry from IndexCache + indexCache.removeMap(taskAttemptPath.getName()); + break; + } + } + } + } + } catch (IOException e) { + LOG.warn("Encountered exception during failed task attempt delete " + e); + } + } + channel.writeAndFlush(new DefaultHttpResponse(HTTP_1_1, OK)) + .addListener(ChannelFutureListener.CLOSE); + return true; + } + return false; + } + /** * Calls sendMapOutput for the mapId pointed by ReduceContext.mapsToSend * and increments it. This method is first called by channelRead() diff --git a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java index 6a2e1cc68e..45dd0adb06 100644 --- a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java @@ -1312,6 +1312,88 @@ protected void sendError(ChannelHandlerContext ctx, String message, } } + @Test(timeout = 5000) + public void testFailedTaskAttemptDelete() throws Exception { + final ArrayList failures = new ArrayList(1); + Configuration conf = new Configuration(); + conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, 3); + conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); + conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, + "simple"); + UserGroupInformation.setConfiguration(conf); + File absLogDir = new File("target", TestShuffleHandler.class. + getSimpleName() + "LocDir").getAbsoluteFile(); + conf.set(YarnConfiguration.NM_LOCAL_DIRS, absLogDir.getAbsolutePath()); + ApplicationId appId = ApplicationId.newInstance(12345, 1); + String appAttemptId = "attempt_12345_1_m_1_0"; + String user = "randomUser"; + List fileMap = new ArrayList(); + String taskAttemptDirStr = + StringUtils.join(Path.SEPARATOR, + new String[] {absLogDir.getAbsolutePath(), + ShuffleHandler.USERCACHE, user, + ShuffleHandler.APPCACHE, appId.toString(), "dag_1/output/", appAttemptId}); + File taskAttemptDir = new File(taskAttemptDirStr); + Assert.assertFalse("Task Attempt Directory should not exist", taskAttemptDir.exists()); + createShuffleHandlerFiles(absLogDir, user, appId.toString(), appAttemptId, + conf, fileMap); + ShuffleHandler shuffleHandler = new ShuffleHandler() { + @Override + protected Shuffle getShuffle(Configuration conf) { + // replace the shuffle handler with one stubbed for testing + return new Shuffle(conf) { + @Override + protected void sendError(ChannelHandlerContext ctx, String message, + HttpResponseStatus status) { + if (failures.size() == 0) { + failures.add(new Error(message)); + ctx.channel().close(); + } + } + }; + } + }; + shuffleHandler.init(conf); + try { + shuffleHandler.start(); + DataOutputBuffer outputBuffer = new DataOutputBuffer(); + outputBuffer.reset(); + Token jt = + new Token("identifier".getBytes(), + "password".getBytes(), new Text(user), new Text("shuffleService")); + jt.write(outputBuffer); + shuffleHandler + .initializeApplication(new ApplicationInitializationContext(user, + appId, ByteBuffer.wrap(outputBuffer.getData(), 0, + outputBuffer.getLength()))); + URL url = + new URL( + "http://127.0.0.1:" + + shuffleHandler.getConfig().get( + ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY) + + "/mapOutput?taskAttemptAction=delete&job=job_12345_0001&dag=1&map=" + appAttemptId); + HttpURLConnection conn = (HttpURLConnection) url.openConnection(); + conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME, + ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); + conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION, + ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); + Assert.assertTrue("Task Attempt Directory does not exist!", taskAttemptDir.exists()); + conn.connect(); + try { + DataInputStream is = new DataInputStream(conn.getInputStream()); + is.close(); + Assert.assertFalse("Task Attempt file was not deleted!", taskAttemptDir.exists()); + } catch (EOFException e) { + // ignore + } + Assert.assertEquals("sendError called due to shuffle error", + 0, failures.size()); + } finally { + shuffleHandler.stop(); + FileUtil.fullyDelete(absLogDir); + } + } + @Test(timeout = 4000) public void testSendMapCount() throws Exception { final List listenerList = diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java index 9d9b8c16c0..48b23bc694 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java @@ -187,6 +187,25 @@ public static URL constructBaseURIForShuffleHandlerDagComplete( return new URL(sb.toString()); } + public static URL constructBaseURIForShuffleHandlerTaskAttemptFailed( + String host, int port, String appId, int dagIdentifier, String taskAttemptIdentifier, boolean sslShuffle) + throws MalformedURLException { + String httpProtocol = (sslShuffle) ? "https://" : "http://"; + StringBuilder sb = new StringBuilder(httpProtocol); + sb.append(host); + sb.append(":"); + sb.append(port); + sb.append("/"); + sb.append("mapOutput?taskAttemptAction=delete"); + sb.append("&job="); + sb.append(appId.replace("application", "job")); + sb.append("&dag="); + sb.append(String.valueOf(dagIdentifier)); + sb.append("&map="); + sb.append(String.valueOf(taskAttemptIdentifier)); + return new URL(sb.toString()); + } + public static HttpConnectionParams getHttpConnectionParams(Configuration conf) { int connectionTimeout = conf.getInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_CONNECT_TIMEOUT, From 6d7ef203c69f00c47831cd3cb9760a94242eee64 Mon Sep 17 00:00:00 2001 From: Mark Bathori <57758037+mbathori-cloudera@users.noreply.github.com> Date: Sun, 30 Jan 2022 11:48:56 +0100 Subject: [PATCH 356/512] TEZ-4381: Speed up TestSecureShuffle (#180) (Mark Bathori reviewed by Laszlo Bodor) --- .../src/test/java/org/apache/tez/test/TestSecureShuffle.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java b/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java index 8e298506b8..7857c1ca2f 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java @@ -146,6 +146,8 @@ public void setupTezCluster() throws Exception { conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT, 3 * 1000); //set to low value so that it can detect failures quickly conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES_LIMIT, 2); + //reduce the maximum number of failed attempts per task + conf.setInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS, 1); conf.setLong(TezConfiguration.TEZ_AM_SLEEP_TIME_BEFORE_EXIT_MILLIS, 500); conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_USE_ASYNC_HTTP, asyncHttp); From 7b66d3da714ee09b434b8d8d438ba30a0026af3c Mon Sep 17 00:00:00 2001 From: ghanko <54805928+ghanko@users.noreply.github.com> Date: Thu, 10 Feb 2022 11:05:47 +0100 Subject: [PATCH 357/512] TEZ-4227 Introduce convenient methods in TezID subclasses (#166) (Gergely Hanko reviewed by Laszlo Bodor) --- .../apache/tez/dag/records/DAGIDAware.java | 29 ++ .../tez/dag/records/TaskAttemptIDAware.java | 28 ++ .../apache/tez/dag/records/TaskIDAware.java | 28 ++ .../tez/dag/records/TezTaskAttemptID.java | 3 +- .../org/apache/tez/dag/records/TezTaskID.java | 3 +- .../apache/tez/dag/records/TezVertexID.java | 6 +- .../apache/tez/dag/records/VertexIDAware.java | 28 ++ .../tez/dag/records/VertexIdentifierImpl.java | 2 +- .../apache/tez/dag/records/TestTezIds.java | 2 +- .../org/apache/tez/dag/app/DAGAppMaster.java | 17 +- .../apache/tez/dag/app/RecoveryParser.java | 12 +- .../tez/dag/app/TaskCommunicatorManager.java | 6 +- .../apache/tez/dag/app/dag/DAGScheduler.java | 8 +- .../app/dag/RootInputInitializerManager.java | 6 +- .../java/org/apache/tez/dag/app/dag/Task.java | 5 +- .../apache/tez/dag/app/dag/TaskAttempt.java | 11 +- .../tez/dag/app/dag/event/DAGEvent.java | 6 +- .../dag/event/DAGEventSchedulerUpdate.java | 13 +- .../dag/event/DAGEventVertexCompleted.java | 4 +- .../dag/event/DAGEventVertexReRunning.java | 2 +- ...peculatorEventTaskAttemptStatusUpdate.java | 2 +- .../dag/app/dag/event/TaskAttemptEvent.java | 4 +- .../tez/dag/app/dag/event/TaskEvent.java | 4 +- .../tez/dag/app/dag/event/VertexEvent.java | 6 +- .../VertexEventTaskAttemptCompleted.java | 2 +- .../apache/tez/dag/app/dag/impl/DAGImpl.java | 4 +- .../dag/impl/DAGSchedulerNaturalOrder.java | 6 +- .../DAGSchedulerNaturalOrderControlled.java | 15 +- .../org/apache/tez/dag/app/dag/impl/Edge.java | 2 +- .../tez/dag/app/dag/impl/TaskAttemptImpl.java | 62 ++-- .../apache/tez/dag/app/dag/impl/TaskImpl.java | 88 ++--- .../TezRootInputInitializerContextImpl.java | 2 +- .../tez/dag/app/dag/impl/VertexImpl.java | 26 +- .../tez/dag/app/dag/impl/VertexManager.java | 6 +- .../speculation/legacy/LegacySpeculator.java | 8 +- .../TaskAttemptFailedDeleteRunnable.java | 4 +- .../dag/app/rm/AMSchedulerEventTAEnded.java | 2 +- .../tez/dag/app/rm/TaskSchedulerManager.java | 28 +- .../AMContainerEventLaunchRequest.java | 2 +- .../tez/dag/app/web/AMWebController.java | 6 +- .../tez/dag/history/DAGHistoryEvent.java | 6 +- .../tez/dag/history/HistoryEventHandler.java | 6 +- .../dag/history/events/DAGFinishedEvent.java | 6 +- .../history/events/DAGInitializedEvent.java | 6 +- .../dag/history/events/DAGStartedEvent.java | 6 +- .../dag/history/events/DAGSubmittedEvent.java | 6 +- .../events/TaskAttemptFinishedEvent.java | 4 +- .../events/TaskAttemptStartedEvent.java | 4 +- .../dag/history/events/TaskFinishedEvent.java | 4 +- .../dag/history/events/TaskStartedEvent.java | 4 +- .../events/VertexCommitStartedEvent.java | 2 +- .../events/VertexConfigurationDoneEvent.java | 4 +- .../history/events/VertexFinishedEvent.java | 6 +- .../events/VertexInitializedEvent.java | 6 +- .../history/events/VertexStartedEvent.java | 6 +- .../impl/HistoryEventJsonConversion.java | 16 +- .../dag/history/recovery/RecoveryService.java | 12 +- .../tez/dag/app/TestMockDAGAppMaster.java | 12 +- .../apache/tez/dag/app/TestSpeculation.java | 10 +- .../tez/dag/app/dag/impl/TestCommit.java | 136 ++++---- .../tez/dag/app/dag/impl/TestDAGImpl.java | 40 +-- .../tez/dag/app/dag/impl/TestDAGRecovery.java | 7 +- .../dag/app/dag/impl/TestDAGScheduler.java | 41 ++- ...estDAGSchedulerNaturalOrderControlled.java | 6 +- .../apache/tez/dag/app/dag/impl/TestEdge.java | 6 +- .../tez/dag/app/dag/impl/TestTaskAttempt.java | 73 +++-- .../tez/dag/app/dag/impl/TestTaskImpl.java | 300 +++++++++--------- .../tez/dag/app/dag/impl/TestVertexImpl.java | 44 +-- .../tez/dag/app/rm/TestContainerReuse.java | 12 +- .../dag/app/rm/TestTaskSchedulerManager.java | 6 +- .../tez/dag/app/web/TestAMWebController.java | 14 +- .../TestHistoryEventsProtoConversion.java | 20 +- .../tez/service/impl/ContainerRunnerImpl.java | 4 +- .../tez/mapreduce/hadoop/IDConverter.java | 2 +- .../mapreduce/TaskAttemptContextImpl.java | 6 +- .../tez/mapreduce/processor/MRTask.java | 4 +- .../history/parser/SimpleHistoryParser.java | 12 +- .../history/parser/datamodel/BaseParser.java | 5 +- .../proto/HistoryEventProtoConverter.java | 16 +- .../proto/ProtoHistoryLoggingService.java | 12 +- .../logging/ats/TimelineCachePluginImpl.java | 6 +- .../ats/acls/TestATSHistoryWithACLs.java | 4 +- .../ats/ATSV15HistoryLoggingService.java | 12 +- .../logging/ats/ATSHistoryLoggingService.java | 8 +- .../ats/HistoryEventTimelineConversion.java | 81 +++-- .../tez/runtime/api/impl/EventMetaData.java | 4 +- .../apache/tez/runtime/api/impl/TaskSpec.java | 7 +- .../runtime/api/impl/TezTaskContextImpl.java | 7 +- .../org/apache/tez/runtime/task/TezChild.java | 6 +- .../RecoveryServiceWithEventHandlingHook.java | 16 +- .../org/apache/tez/test/TestAMRecovery.java | 4 +- 91 files changed, 834 insertions(+), 711 deletions(-) create mode 100644 tez-common/src/main/java/org/apache/tez/dag/records/DAGIDAware.java create mode 100644 tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptIDAware.java create mode 100644 tez-common/src/main/java/org/apache/tez/dag/records/TaskIDAware.java create mode 100644 tez-common/src/main/java/org/apache/tez/dag/records/VertexIDAware.java diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/DAGIDAware.java b/tez-common/src/main/java/org/apache/tez/dag/records/DAGIDAware.java new file mode 100644 index 0000000000..1234a30053 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/dag/records/DAGIDAware.java @@ -0,0 +1,29 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.records; + +import org.apache.hadoop.yarn.api.records.ApplicationId; + +public interface DAGIDAware { + TezDAGID getDAGID(); + + default ApplicationId getApplicationId() { + return getDAGID().getApplicationId(); + } +} diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptIDAware.java b/tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptIDAware.java new file mode 100644 index 0000000000..924fd07109 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptIDAware.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.records; + +public interface TaskAttemptIDAware extends TaskIDAware { + TezTaskAttemptID getTaskAttemptID(); + + @Override + default TezTaskID getTaskID() { + return getTaskAttemptID().getTaskID(); + } +} diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TaskIDAware.java b/tez-common/src/main/java/org/apache/tez/dag/records/TaskIDAware.java new file mode 100644 index 0000000000..0bee45dfa1 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TaskIDAware.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.records; + +public interface TaskIDAware extends VertexIDAware { + TezTaskID getTaskID(); + + @Override + default TezVertexID getVertexID() { + return getTaskID().getVertexID(); + } +} diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java index 7c92be9449..fe2b84449f 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java @@ -46,7 +46,7 @@ */ @InterfaceAudience.Public @InterfaceStability.Stable -public class TezTaskAttemptID extends TezID { +public class TezTaskAttemptID extends TezID implements TaskIDAware { public static final String ATTEMPT = "attempt"; private TezTaskID taskId; @@ -73,6 +73,7 @@ private TezTaskAttemptID(TezTaskID taskId, int id) { } /** Returns the {@link TezTaskID} object that this task attempt belongs to */ + @Override public TezTaskID getTaskID() { return taskId; } diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java index a1a556c9ba..08310f3dfc 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java @@ -41,7 +41,7 @@ */ @InterfaceAudience.Public @InterfaceStability.Stable -public class TezTaskID extends TezID { +public class TezTaskID extends TezID implements VertexIDAware { public static final String TASK = "task"; private final int serializingHash; @@ -79,6 +79,7 @@ public int getSerializingHash() { } /** Returns the {@link TezVertexID} object that this task belongs to */ + @Override public TezVertexID getVertexID() { return vertexId; } diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java index e9ddd77d38..f7becc250f 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java @@ -44,7 +44,7 @@ */ @InterfaceAudience.Public @InterfaceStability.Stable -public class TezVertexID extends TezID { +public class TezVertexID extends TezID implements DAGIDAware { public static final String VERTEX = "vertex"; static final ThreadLocal tezVertexIdFormat = new ThreadLocal() { @@ -80,7 +80,8 @@ private TezVertexID(TezDAGID dagId, int id) { } /** Returns the {@link TezDAGID} object that this tip belongs to */ - public TezDAGID getDAGId() { + @Override + public TezDAGID getDAGID() { return dagId; } @@ -159,5 +160,4 @@ public static TezVertexID fromString(String vertexIdStr) { } return null; } - } diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/VertexIDAware.java b/tez-common/src/main/java/org/apache/tez/dag/records/VertexIDAware.java new file mode 100644 index 0000000000..01bbe859b6 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/dag/records/VertexIDAware.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.records; + +public interface VertexIDAware extends DAGIDAware { + TezVertexID getVertexID(); + + @Override + default TezDAGID getDAGID() { + return getVertexID().getDAGID(); + } +} diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/VertexIdentifierImpl.java b/tez-common/src/main/java/org/apache/tez/dag/records/VertexIdentifierImpl.java index 4480f742fc..83b503203c 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/VertexIdentifierImpl.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/VertexIdentifierImpl.java @@ -30,7 +30,7 @@ public class VertexIdentifierImpl implements VertexIdentifier { public VertexIdentifierImpl(String dagName, String vertexName, TezVertexID vertexId) { this.vertexId = vertexId; this.vertexName = vertexName; - this.dagIdentifier = new DagIdentifierImpl(dagName, vertexId.getDAGId()); + this.dagIdentifier = new DagIdentifierImpl(dagName, vertexId.getDAGID()); } @Override diff --git a/tez-common/src/test/java/org/apache/tez/dag/records/TestTezIds.java b/tez-common/src/test/java/org/apache/tez/dag/records/TestTezIds.java index 5e1552d345..10b62b2bd1 100644 --- a/tez-common/src/test/java/org/apache/tez/dag/records/TestTezIds.java +++ b/tez-common/src/test/java/org/apache/tez/dag/records/TestTezIds.java @@ -38,7 +38,7 @@ private void verifyDagInfo(String[] splits, TezDAGID dagId) { } private void verifyVertexInfo(String[] splits, TezVertexID vId) { - verifyDagInfo(splits, vId.getDAGId()); + verifyDagInfo(splits, vId.getDAGID()); Assert.assertEquals(vId.getId(), Integer.valueOf(splits[4]).intValue()); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index 3be9aacec5..972fadf854 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -50,7 +50,6 @@ import java.util.TimerTask; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -2173,7 +2172,7 @@ private class DagEventDispatcher implements EventHandler { @Override public void handle(DAGEvent event) { DAG dag = context.getCurrentDAG(); - int eventDagIndex = event.getDAGId().getId(); + int eventDagIndex = event.getDAGID().getId(); if (dag == null || eventDagIndex != dag.getID().getId()) { return; // event not relevant any more } @@ -2187,12 +2186,12 @@ private class TaskEventDispatcher implements EventHandler { public void handle(TaskEvent event) { DAG dag = context.getCurrentDAG(); int eventDagIndex = - event.getTaskID().getVertexID().getDAGId().getId(); + event.getDAGID().getId(); if (dag == null || eventDagIndex != dag.getID().getId()) { return; // event not relevant any more } Task task = - dag.getVertex(event.getTaskID().getVertexID()). + dag.getVertex(event.getVertexID()). getTask(event.getTaskID()); ((EventHandler)task).handle(event); } @@ -2217,13 +2216,13 @@ private class TaskAttemptEventDispatcher public void handle(TaskAttemptEvent event) { DAG dag = context.getCurrentDAG(); int eventDagIndex = - event.getTaskAttemptID().getTaskID().getVertexID().getDAGId().getId(); + event.getDAGID().getId(); if (dag == null || eventDagIndex != dag.getID().getId()) { return; // event not relevant any more } Task task = - dag.getVertex(event.getTaskAttemptID().getTaskID().getVertexID()). - getTask(event.getTaskAttemptID().getTaskID()); + dag.getVertex(event.getVertexID()). + getTask(event.getTaskID()); TaskAttempt attempt = task.getAttempt(event.getTaskAttemptID()); ((EventHandler) attempt).handle(event); } @@ -2236,13 +2235,13 @@ private class VertexEventDispatcher public void handle(VertexEvent event) { DAG dag = context.getCurrentDAG(); int eventDagIndex = - event.getVertexId().getDAGId().getId(); + event.getDAGID().getId(); if (dag == null || eventDagIndex != dag.getID().getId()) { return; // event not relevant any more } Vertex vertex = - dag.getVertex(event.getVertexId()); + dag.getVertex(event.getVertexID()); ((EventHandler) vertex).handle(event); } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java index 19c24f300c..ce3b62bbd5 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java @@ -846,9 +846,9 @@ public DAGRecoveryData parseRecoveryData() throws IOException { case TASK_STARTED: { TaskStartedEvent taskStartedEvent = (TaskStartedEvent) event; - VertexRecoveryData vertexRecoveryData = recoveredDAGData.vertexRecoveryDataMap.get(taskStartedEvent.getTaskID().getVertexID()); + VertexRecoveryData vertexRecoveryData = recoveredDAGData.vertexRecoveryDataMap.get(taskStartedEvent.getVertexID()); Preconditions.checkArgument(vertexRecoveryData != null, - "Invalid TaskStartedEvent, its vertex does not exist:" + taskStartedEvent.getTaskID().getVertexID()); + "Invalid TaskStartedEvent, its vertex does not exist:" + taskStartedEvent.getVertexID()); TaskRecoveryData taskRecoveryData = vertexRecoveryData.maybeCreateTaskRecoveryData(taskStartedEvent.getTaskID()); taskRecoveryData.taskStartedEvent = taskStartedEvent; break; @@ -856,9 +856,9 @@ public DAGRecoveryData parseRecoveryData() throws IOException { case TASK_FINISHED: { TaskFinishedEvent taskFinishedEvent = (TaskFinishedEvent) event; - VertexRecoveryData vertexRecoveryData = recoveredDAGData.vertexRecoveryDataMap.get(taskFinishedEvent.getTaskID().getVertexID()); + VertexRecoveryData vertexRecoveryData = recoveredDAGData.vertexRecoveryDataMap.get(taskFinishedEvent.getVertexID()); Preconditions.checkArgument(vertexRecoveryData != null, - "Invalid TaskFinishedEvent, its vertex does not exist:" + taskFinishedEvent.getTaskID().getVertexID()); + "Invalid TaskFinishedEvent, its vertex does not exist:" + taskFinishedEvent.getVertexID()); TaskRecoveryData taskRecoveryData = vertexRecoveryData.maybeCreateTaskRecoveryData(taskFinishedEvent.getTaskID()); taskRecoveryData.taskFinishedEvent = taskFinishedEvent; break; @@ -867,7 +867,7 @@ public DAGRecoveryData parseRecoveryData() throws IOException { { TaskAttemptStartedEvent taStartedEvent = (TaskAttemptStartedEvent)event; VertexRecoveryData vertexRecoveryData = recoveredDAGData.vertexRecoveryDataMap.get( - taStartedEvent.getTaskAttemptID().getTaskID().getVertexID()); + taStartedEvent.getVertexID()); Preconditions.checkArgument(vertexRecoveryData != null, "Invalid TaskAttemptStartedEvent, its vertexId does not exist, taId=" + taStartedEvent.getTaskAttemptID()); TaskRecoveryData taskRecoveryData = vertexRecoveryData.taskRecoveryDataMap @@ -882,7 +882,7 @@ public DAGRecoveryData parseRecoveryData() throws IOException { { TaskAttemptFinishedEvent taFinishedEvent = (TaskAttemptFinishedEvent)event; VertexRecoveryData vertexRecoveryData = recoveredDAGData.vertexRecoveryDataMap.get( - taFinishedEvent.getTaskAttemptID().getTaskID().getVertexID()); + taFinishedEvent.getVertexID()); Preconditions.checkArgument(vertexRecoveryData != null, "Invalid TaskAttemtFinishedEvent, its vertexId does not exist, taId=" + taFinishedEvent.getTaskAttemptID()); TaskRecoveryData taskRecoveryData = vertexRecoveryData.taskRecoveryDataMap diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java index ac2f7605ae..51895f4afd 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java @@ -350,14 +350,14 @@ public TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request) } } if (!eventsForVertex.isEmpty()) { - TezVertexID vertexId = taskAttemptID.getTaskID().getVertexID(); + TezVertexID vertexId = taskAttemptID.getVertexID(); sendEvent( new VertexEventRouteEvent(vertexId, Collections.unmodifiableList(eventsForVertex))); } taskHeartbeatHandler.pinged(taskAttemptID); eventInfo = context .getCurrentDAG() - .getVertex(taskAttemptID.getTaskID().getVertexID()) + .getVertex(taskAttemptID.getVertexID()) .getTaskAttemptTezEvents(taskAttemptID, request.getStartIndex(), request.getPreRoutedStartIndex(), request.getMaxEvents()); } @@ -442,7 +442,7 @@ public boolean canCommit(TezTaskAttemptID taskAttemptId) throws IOException { DAG job = context.getCurrentDAG(); Task task = - job.getVertex(taskAttemptId.getTaskID().getVertexID()). + job.getVertex(taskAttemptId.getVertexID()). getTask(taskAttemptId.getTaskID()); return task.canCommit(taskAttemptId); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAGScheduler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAGScheduler.java index 2fa735ef8f..ada7867a6a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAGScheduler.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAGScheduler.java @@ -52,7 +52,7 @@ public void addVertexConcurrencyLimit(TezVertexID vId, int concurrency) { public void scheduleTask(DAGEventSchedulerUpdate event) { VertexInfo vInfo = null; if (vertexInfo != null) { - vInfo = vertexInfo.get(event.getAttempt().getID().getTaskID().getVertexID()); + vInfo = vertexInfo.get(event.getVertexID()); } scheduleTaskWithLimit(event, vInfo); } @@ -60,7 +60,7 @@ public void scheduleTask(DAGEventSchedulerUpdate event) { private void scheduleTaskWithLimit(DAGEventSchedulerUpdate event, VertexInfo vInfo) { if (vInfo != null) { if (vInfo.concurrency >= vInfo.concurrencyLimit) { - vInfo.pendingAttempts.put(event.getAttempt().getID(), event); + vInfo.pendingAttempts.put(event.getTaskAttemptID(), event); return; // already at max concurrency } vInfo.concurrency++; @@ -71,9 +71,9 @@ private void scheduleTaskWithLimit(DAGEventSchedulerUpdate event, VertexInfo vIn public void taskCompleted(DAGEventSchedulerUpdate event) { taskCompletedEx(event); if (vertexInfo != null) { - VertexInfo vInfo = vertexInfo.get(event.getAttempt().getID().getTaskID().getVertexID()); + VertexInfo vInfo = vertexInfo.get(event.getVertexID()); if (vInfo != null) { - if(vInfo.pendingAttempts.remove(event.getAttempt().getID()) == null) { + if(vInfo.pendingAttempts.remove(event.getTaskAttemptID()) == null) { vInfo.concurrency--; if(!vInfo.pendingAttempts.isEmpty()) { Iterator i = vInfo.pendingAttempts.values().iterator(); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java index 3c4a05eb84..cfbdb19e30 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java @@ -457,7 +457,7 @@ public void onTaskSucceeded(String vertexName, TezTaskID taskId, int attemptId) Iterator eventIterator = events.iterator(); while (eventIterator.hasNext()) { TezEvent tezEvent = eventIterator.next(); - int taskIndex = tezEvent.getSourceInfo().getTaskAttemptID().getTaskID().getId(); + int taskIndex = tezEvent.getSourceInfo().getTaskID().getId(); int taskAttemptIndex = tezEvent.getSourceInfo().getTaskAttemptID().getId(); if (taskIndex == taskId.getId()) { // Process only if there's a pending event for the specific succeeded task @@ -476,7 +476,7 @@ public void handleInputInitializerEvents(Collection tezEvents) { List toForwardEvents = new LinkedList(); for (TezEvent tezEvent : tezEvents) { String srcVertexName = tezEvent.getSourceInfo().getTaskVertexName(); - int taskIndex = tezEvent.getSourceInfo().getTaskAttemptID().getTaskID().getId(); + int taskIndex = tezEvent.getSourceInfo().getTaskID().getId(); int taskAttemptIndex = tezEvent.getSourceInfo().getTaskAttemptID().getId(); Map vertexSuccessfulAttemptMap = @@ -496,7 +496,7 @@ public void handleInputInitializerEvents(Collection tezEvents) { Vertex srcVertex = appContext.getCurrentDAG().getVertex(srcVertexName); Task task = srcVertex.getTask(taskIndex); if (task.getState() == TaskState.SUCCEEDED) { - successfulAttemptInteger = task.getSuccessfulAttempt().getID().getId(); + successfulAttemptInteger = task.getSuccessfulAttempt().getTaskAttemptID().getId(); vertexSuccessfulAttemptMap.put(taskIndex, successfulAttemptInteger); } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java index c1fe7f79ad..cb1bc33866 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java @@ -28,16 +28,15 @@ import org.apache.tez.dag.api.TaskLocationHint; import org.apache.tez.dag.api.oldrecords.TaskReport; import org.apache.tez.dag.api.oldrecords.TaskState; +import org.apache.tez.dag.records.TaskIDAware; import org.apache.tez.dag.records.TezTaskAttemptID; -import org.apache.tez.dag.records.TezTaskID; import org.apache.tez.runtime.api.impl.TaskSpec; import org.apache.tez.runtime.api.impl.TezEvent; /** * Read only view of Task. */ -public interface Task { - TezTaskID getTaskId(); +public interface Task extends TaskIDAware { TaskReport getReport(); TaskState getState(); TezCounters getCounters(); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java index 0fc7013fea..563e4c7192 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java @@ -28,17 +28,15 @@ import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.oldrecords.TaskAttemptReport; import org.apache.tez.dag.api.oldrecords.TaskAttemptState; +import org.apache.tez.dag.records.TaskAttemptIDAware; import org.apache.tez.dag.records.TaskAttemptTerminationCause; -import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezTaskAttemptID; -import org.apache.tez.dag.records.TezTaskID; -import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.runtime.api.impl.TezEvent; /** * Read only view of TaskAttempt. */ -public interface TaskAttempt { +public interface TaskAttempt extends TaskAttemptIDAware { public static class TaskAttemptStatus { public TezTaskAttemptID id; @@ -66,11 +64,6 @@ public void setLocalityCounter(DAGCounter localityCounter) { } } } - - TezTaskAttemptID getID(); - TezTaskID getTaskID(); - TezVertexID getVertexID(); - TezDAGID getDAGID(); Task getTask(); TaskAttemptReport getReport(); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEvent.java index a0a8a1af8e..4eed8ff984 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEvent.java @@ -19,13 +19,14 @@ package org.apache.tez.dag.app.dag.event; import org.apache.tez.common.TezAbstractEvent; +import org.apache.tez.dag.records.DAGIDAware; import org.apache.tez.dag.records.TezDAGID; /** * This class encapsulates job related events. * */ -public class DAGEvent extends TezAbstractEvent { +public class DAGEvent extends TezAbstractEvent implements DAGIDAware { private TezDAGID dagId; @@ -34,7 +35,8 @@ public DAGEvent(TezDAGID dagId, DAGEventType type) { this.dagId = dagId; } - public TezDAGID getDAGId() { + @Override + public TezDAGID getDAGID() { return dagId; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventSchedulerUpdate.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventSchedulerUpdate.java index eda02b52da..1dedaecad9 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventSchedulerUpdate.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventSchedulerUpdate.java @@ -19,9 +19,11 @@ package org.apache.tez.dag.app.dag.event; import org.apache.tez.dag.app.dag.TaskAttempt; +import org.apache.tez.dag.records.TaskAttemptIDAware; +import org.apache.tez.dag.records.TezTaskAttemptID; + +public class DAGEventSchedulerUpdate extends DAGEvent implements TaskAttemptIDAware { -public class DAGEventSchedulerUpdate extends DAGEvent { - public enum UpdateType { TA_SCHEDULE, TA_COMPLETED @@ -31,7 +33,7 @@ public enum UpdateType { private final UpdateType updateType; public DAGEventSchedulerUpdate(UpdateType updateType, TaskAttempt attempt) { - super(attempt.getDAGID(), + super(attempt.getDAGID(), DAGEventType.DAG_SCHEDULER_UPDATE); this.attempt = attempt; this.updateType = updateType; @@ -44,4 +46,9 @@ public UpdateType getUpdateType() { public TaskAttempt getAttempt() { return attempt; } + + @Override + public TezTaskAttemptID getTaskAttemptID() { + return attempt.getTaskAttemptID(); + } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexCompleted.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexCompleted.java index e58d46e834..2c18be85dd 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexCompleted.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexCompleted.java @@ -29,7 +29,7 @@ public class DAGEventVertexCompleted extends DAGEvent { private final VertexTerminationCause terminationCause; public DAGEventVertexCompleted(TezVertexID vertexId, VertexState vertexState) { - super(vertexId.getDAGId(), DAGEventType.DAG_VERTEX_COMPLETED); + super(vertexId.getDAGID(), DAGEventType.DAG_VERTEX_COMPLETED); this.vertexId = vertexId; this.vertexState = vertexState; this.terminationCause = null; @@ -37,7 +37,7 @@ public DAGEventVertexCompleted(TezVertexID vertexId, VertexState vertexState) { public DAGEventVertexCompleted(TezVertexID vertexId, VertexState vertexState, VertexTerminationCause terminationCause) { - super(vertexId.getDAGId(), DAGEventType.DAG_VERTEX_COMPLETED); + super(vertexId.getDAGID(), DAGEventType.DAG_VERTEX_COMPLETED); this.vertexId = vertexId; this.vertexState = vertexState; this.terminationCause = terminationCause; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexReRunning.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexReRunning.java index 303d48d375..30e595906b 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexReRunning.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexReRunning.java @@ -25,7 +25,7 @@ public class DAGEventVertexReRunning extends DAGEvent { private TezVertexID vertexId; public DAGEventVertexReRunning(TezVertexID vertexId) { - super(vertexId.getDAGId(), DAGEventType.DAG_VERTEX_RERUNNING); + super(vertexId.getDAGID(), DAGEventType.DAG_VERTEX_RERUNNING); this.vertexId = vertexId; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/SpeculatorEventTaskAttemptStatusUpdate.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/SpeculatorEventTaskAttemptStatusUpdate.java index d5745c4df4..7ab6141c39 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/SpeculatorEventTaskAttemptStatusUpdate.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/SpeculatorEventTaskAttemptStatusUpdate.java @@ -34,7 +34,7 @@ public SpeculatorEventTaskAttemptStatusUpdate(TezTaskAttemptID taId, TaskAttempt public SpeculatorEventTaskAttemptStatusUpdate(TezTaskAttemptID taId, TaskAttemptState state, long timestamp, boolean justStarted) { - super(SpeculatorEventType.S_TASK_ATTEMPT_STATUS_UPDATE, taId.getTaskID().getVertexID()); + super(SpeculatorEventType.S_TASK_ATTEMPT_STATUS_UPDATE, taId.getVertexID()); this.id = taId; this.state = state; this.timestamp = timestamp; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEvent.java index 63ef70feb4..d5d8481a54 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEvent.java @@ -19,13 +19,14 @@ package org.apache.tez.dag.app.dag.event; import org.apache.tez.common.TezAbstractEvent; +import org.apache.tez.dag.records.TaskAttemptIDAware; import org.apache.tez.dag.records.TezTaskAttemptID; /** * This class encapsulates task attempt related events. * */ -public class TaskAttemptEvent extends TezAbstractEvent { +public class TaskAttemptEvent extends TezAbstractEvent implements TaskAttemptIDAware { private TezTaskAttemptID attemptID; @@ -39,6 +40,7 @@ public TaskAttemptEvent(TezTaskAttemptID id, TaskAttemptEventType type) { this.attemptID = id; } + @Override public TezTaskAttemptID getTaskAttemptID() { return attemptID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEvent.java index def9ddfa7c..9e741388fa 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEvent.java @@ -19,13 +19,14 @@ package org.apache.tez.dag.app.dag.event; import org.apache.tez.common.TezAbstractEvent; +import org.apache.tez.dag.records.TaskIDAware; import org.apache.tez.dag.records.TezTaskID; /** * this class encapsulates task related events. * */ -public class TaskEvent extends TezAbstractEvent { +public class TaskEvent extends TezAbstractEvent implements TaskIDAware { private TezTaskID taskId; @@ -34,6 +35,7 @@ public TaskEvent(TezTaskID taskId, TaskEventType type) { this.taskId = taskId; } + @Override public TezTaskID getTaskID() { return taskId; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEvent.java index 33128e4536..6957a50dc7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEvent.java @@ -20,12 +20,13 @@ import org.apache.tez.common.TezAbstractEvent; import org.apache.tez.dag.records.TezVertexID; +import org.apache.tez.dag.records.VertexIDAware; /** * this class encapsulates vertex related events. * */ -public class VertexEvent extends TezAbstractEvent { +public class VertexEvent extends TezAbstractEvent implements VertexIDAware { private TezVertexID vertexId; @@ -34,7 +35,8 @@ public VertexEvent(TezVertexID vertexId, VertexEventType type) { this.vertexId = vertexId; } - public TezVertexID getVertexId() { + @Override + public TezVertexID getVertexID() { return vertexId; } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java index 5b07674422..e23d27cf6e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java @@ -28,7 +28,7 @@ public class VertexEventTaskAttemptCompleted extends VertexEvent { public VertexEventTaskAttemptCompleted(TezTaskAttemptID taskAttemptId, TaskAttemptStateInternal state) { - super(taskAttemptId.getTaskID().getVertexID(), + super(taskAttemptId.getVertexID(), VertexEventType.V_TASK_ATTEMPT_COMPLETED); this.attemptId = taskAttemptId; this.attempState = state; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java index 07715cdfe7..c9337e473a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java @@ -1040,7 +1040,7 @@ public VertexStatusBuilder getVertexStatus(String vertexName, } public TaskAttemptImpl getTaskAttempt(TezTaskAttemptID taId) { - return (TaskAttemptImpl) getVertex(taId.getTaskID().getVertexID()).getTask(taId.getTaskID()) + return (TaskAttemptImpl) getVertex(taId.getVertexID()).getTask(taId.getTaskID()) .getAttempt(taId); } @@ -1206,7 +1206,7 @@ private void abortOutputs() { */ public void handle(DAGEvent event) { if (LOG.isDebugEnabled()) { - LOG.debug("Processing DAGEvent " + event.getDAGId() + " of type " + LOG.debug("Processing DAGEvent " + event.getDAGID() + " of type " + event.getType() + " while in state " + getInternalState() + ". Event: " + event); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrder.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrder.java index 2383db83b1..78860868dd 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrder.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrder.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -52,12 +52,12 @@ public void scheduleTaskEx(DAGEventSchedulerUpdate event) { int priorityHighLimit = getPriorityHighLimit(dag, vertex); if (LOG.isDebugEnabled()) { - LOG.debug("Scheduling " + attempt.getID() + " between priorityLow: " + priorityLowLimit + LOG.debug("Scheduling " + attempt.getTaskAttemptID() + " between priorityLow: " + priorityLowLimit + " and priorityHigh: " + priorityHighLimit); } TaskAttemptEventSchedule attemptEvent = new TaskAttemptEventSchedule( - attempt.getID(), priorityLowLimit, priorityHighLimit); + attempt.getTaskAttemptID(), priorityLowLimit, priorityHighLimit); sendEvent(attemptEvent); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrderControlled.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrderControlled.java index c51783bc2d..3167c9f3b7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrderControlled.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrderControlled.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -62,9 +62,9 @@ public class DAGSchedulerNaturalOrderControlled extends DAGScheduler { LinkedListMultimap.create(); // Tacks vertices for which no additional scheduling checks are required. Once in this list, the // vertex is considered to be fully scheduled. - private final Set scheduledVertices = new HashSet(); + private final Set scheduledVertices = new HashSet<>(); // Tracks tasks scheduled for a vertex. - private final Map vertexScheduledTasks = new HashMap(); + private final Map vertexScheduledTasks = new HashMap<>(); public DAGSchedulerNaturalOrderControlled(DAG dag, EventHandler dispatcher) { this.dag = dag; @@ -82,14 +82,14 @@ public void scheduleTaskEx(DAGEventSchedulerUpdate event) { int priorityHighLimit = getPriorityHighLimit(dag, vertex); TaskAttemptEventSchedule attemptEvent = new TaskAttemptEventSchedule( - attempt.getID(), priorityLowLimit, priorityHighLimit); + attempt.getTaskAttemptID(), priorityLowLimit, priorityHighLimit); - taskAttemptSeen(vertex.getName(), attempt.getID()); + taskAttemptSeen(vertex.getName(), attempt.getTaskAttemptID()); if (vertexAlreadyScheduled(vertex)) { // Vertex previously marked ready for scheduling. if (LOG.isDebugEnabled()) { - LOG.debug("Scheduling " + attempt.getID() + " between priorityLow: " + priorityLowLimit + LOG.debug("Scheduling " + attempt.getTaskAttemptID() + " between priorityLow: " + priorityLowLimit + " and priorityHigh: " + priorityHighLimit); } sendEvent(attemptEvent); @@ -154,8 +154,7 @@ private void processDownstreamVertices(Vertex vertex) { List newlyScheduledVertices = Lists.newLinkedList(); Map outputVertexEdgeMap = vertex.getOutputVertices(); for (Vertex destVertex : outputVertexEdgeMap.keySet()) { - if (vertexAlreadyScheduled(destVertex)) { // Nothing to do if already scheduled. - } else { + if (!vertexAlreadyScheduled(destVertex)) { if (LOG.isDebugEnabled()) { LOG.debug("Attempting to schedule vertex: " + destVertex.getLogIdentifier() + " due to upstream event from " + vertex.getLogIdentifier()); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java index 99b56fbf07..08e1c19cee 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java @@ -413,7 +413,7 @@ public void sendTezEventToSourceTasks(TezEvent tezEvent) throws AMUserCodeExcept " destIndex=" + destTaskIndex + " edgeManager=" + edgeManager.getClass().getName()); } - TezTaskID srcTaskId = srcTask.getTaskId(); + TezTaskID srcTaskId = srcTask.getTaskID(); int srcTaskAttemptIndex = inputReadErrorEvent.getVersion(); TezTaskAttemptID srcTaskAttemptId = TezTaskAttemptID.getInstance(srcTaskId, srcTaskAttemptIndex); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java index e6b9e82000..c8343c834b 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java @@ -104,10 +104,7 @@ import org.apache.tez.dag.history.events.TaskAttemptFinishedEvent; import org.apache.tez.dag.history.events.TaskAttemptStartedEvent; import org.apache.tez.dag.records.TaskAttemptTerminationCause; -import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezTaskAttemptID; -import org.apache.tez.dag.records.TezTaskID; -import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.dag.recovery.records.RecoveryProtos.DataEventDependencyInfoProto; import org.apache.tez.runtime.api.events.InputFailedEvent; import org.apache.tez.runtime.api.events.InputReadErrorEvent; @@ -588,25 +585,10 @@ public TaskAttemptImpl(TezTaskAttemptID attemptId, EventHandler eventHandler, } @Override - public TezTaskAttemptID getID() { + public TezTaskAttemptID getTaskAttemptID() { return attemptId; } - @Override - public TezTaskID getTaskID() { - return attemptId.getTaskID(); - } - - @Override - public TezVertexID getVertexID() { - return attemptId.getTaskID().getVertexID(); - } - - @Override - public TezDAGID getDAGID() { - return getVertexID().getDAGId(); - } - public TezTaskAttemptID getSchedulingCausalTA() { return creationCausalTA; } @@ -884,12 +866,12 @@ public void handle(TaskAttemptEvent event) { LOG.error("Can't handle this event at current state for " + this.attemptId, e); eventHandler.handle(new DAGEventDiagnosticsUpdate( - this.attemptId.getTaskID().getVertexID().getDAGId(), + getDAGID(), "Invalid event " + event.getType() + " on TaskAttempt " + this.attemptId)); eventHandler.handle( new DAGEvent( - this.attemptId.getTaskID().getVertexID().getDAGId(), + getDAGID(), DAGEventType.INTERNAL_ERROR) ); } catch (RuntimeException e) { @@ -897,13 +879,13 @@ public void handle(TaskAttemptEvent event) { + " at current state " + oldState + " for " + this.attemptId, e); eventHandler.handle(new DAGEventDiagnosticsUpdate( - this.attemptId.getTaskID().getVertexID().getDAGId(), + getDAGID(), "Uncaught exception when handling event " + event.getType() + " on TaskAttempt " + this.attemptId + " at state " + oldState + ", error=" + e.getMessage())); eventHandler.handle( new DAGEvent( - this.attemptId.getTaskID().getVertexID().getDAGId(), + getDAGID(), DAGEventType.INTERNAL_ERROR) ); } @@ -1269,7 +1251,7 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl ta, TaskAttemptEvent if (taFinishedEvent == null) { LOG.debug("Only TaskAttemptStartedEvent but no TaskAttemptFinishedEvent, " + "send out TaskAttemptEventAttemptKilled to move it to KILLED"); - ta.sendEvent(new TaskAttemptEventAttemptKilled(ta.getID(), + ta.sendEvent(new TaskAttemptEventAttemptKilled(ta.getTaskAttemptID(), "Task Attempt killed in recovery due to can't recover the running task attempt", TaskAttemptTerminationCause.TERMINATED_AT_RECOVERY, true)); return TaskAttemptStateInternal.NEW; @@ -1280,29 +1262,29 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl ta, TaskAttemptEvent TaskAttemptFinishedEvent taFinishedEvent = ta.recoveryData.getTaskAttemptFinishedEvent(); Preconditions.checkArgument(taFinishedEvent != null, "Both of TaskAttemptStartedEvent and TaskFinishedEvent is null," - + "taskAttemptId=" + ta.getID()); + + "taskAttemptId=" + ta.getTaskAttemptID()); switch (taFinishedEvent.getState()) { case FAILED: LOG.debug("TaskAttemptFinishedEvent is seen with state of FAILED, " + "send TA_FAILED to itself, attemptId={}", ta.attemptId); - ta.sendEvent(new TaskAttemptEventAttemptFailed(ta.getID(), TaskAttemptEventType.TA_FAILED, + ta.sendEvent(new TaskAttemptEventAttemptFailed(ta.getTaskAttemptID(), TaskAttemptEventType.TA_FAILED, taFinishedEvent.getTaskFailureType(), taFinishedEvent.getDiagnostics(), taFinishedEvent.getTaskAttemptError(), true)); break; case KILLED: LOG.debug("TaskAttemptFinishedEvent is seen with state of KILLED, " + "send TA_KILLED to itself, attemptId={}", ta.attemptId); - ta.sendEvent(new TaskAttemptEventAttemptKilled(ta.getID(), + ta.sendEvent(new TaskAttemptEventAttemptKilled(ta.getTaskAttemptID(), taFinishedEvent.getDiagnostics(), taFinishedEvent.getTaskAttemptError(), true)); break; case SUCCEEDED: LOG.debug("TaskAttemptFinishedEvent is seen with state of SUCCEEDED, " + "send TA_DONE to itself, attemptId={}", ta.attemptId); - ta.sendEvent(new TaskAttemptEvent(ta.getID(), TaskAttemptEventType.TA_DONE)); + ta.sendEvent(new TaskAttemptEvent(ta.getTaskAttemptID(), TaskAttemptEventType.TA_DONE)); break; default: throw new TezUncheckedException("Invalid state in TaskAttemptFinishedEvent, state=" - + taFinishedEvent.getState() + ", taId=" + ta.getID()); + + taFinishedEvent.getState() + ", taId=" + ta.getTaskAttemptID()); } return TaskAttemptStateInternal.NEW; } @@ -1428,7 +1410,7 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) { RecoveryEvent rEvent = (RecoveryEvent)event; if (rEvent.isFromRecovery()) { if (LOG.isDebugEnabled()) { - LOG.debug("Faked TerminateEvent from recovery, taskAttemptId=" + ta.getID()); + LOG.debug("Faked TerminateEvent from recovery, taskAttemptId=" + ta.getTaskAttemptID()); } } } @@ -1613,10 +1595,10 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) { // task is hung String diagnostics = "Attempt failed because it appears to make no progress for " + ta.hungIntervalMax + "ms"; - LOG.info(diagnostics + " " + ta.getID()); + LOG.info(diagnostics + " " + ta.getTaskAttemptID()); // send event that will fail this attempt ta.sendEvent( - new TaskAttemptEventAttemptFailed(ta.getID(), + new TaskAttemptEventAttemptFailed(ta.getTaskAttemptID(), TaskAttemptEventType.TA_FAILED, TaskFailureType.NON_FATAL, diagnostics, @@ -1803,8 +1785,8 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl sourceAttempt, InputReadErrorEvent readErrorEvent = (InputReadErrorEvent)inputFailedEvent.getEvent(); int failedInputIndexOnDestTa = readErrorEvent.getIndex(); - if (readErrorEvent.getVersion() != sourceAttempt.getID().getId()) { - throw new TezUncheckedException(sourceAttempt.getID() + if (readErrorEvent.getVersion() != sourceAttempt.getTaskAttemptID().getId()) { + throw new TezUncheckedException(sourceAttempt.getTaskAttemptID() + " incorrectly blamed for read error from " + failedDestTaId + " at inputIndex " + failedInputIndexOnDestTa + " version" + readErrorEvent.getVersion()); @@ -1814,7 +1796,7 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl sourceAttempt, // destination: where the data is tried to be fetched to String dHost = readErrorEvent.getDestinationLocalhostName(); - LOG.info("{} (on {}) blamed for read error from {} (on {}) at inputIndex {}", sourceAttempt.getID(), + LOG.info("{} (on {}) blamed for read error from {} (on {}) at inputIndex {}", sourceAttempt.getTaskAttemptID(), sHost, failedDestTaId, dHost, failedInputIndexOnDestTa); boolean tooManyDownstreamHostsBlamedTheSameUpstreamHost = false; @@ -1856,7 +1838,7 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl sourceAttempt, boolean crossTimeDeadline = readErrorTimespanSec >= maxAllowedTimeForTaskReadErrorSec; int runningTasks = sourceAttempt.appContext.getCurrentDAG().getVertex( - failedDestTaId.getTaskID().getVertexID()).getRunningTasks(); + failedDestTaId.getVertexID()).getRunningTasks(); float failureFraction = runningTasks > 0 ? ((float) sourceAttempt.uniquefailedOutputReports.size()) / runningTasks : 0; boolean withinFailureFractionLimits = @@ -1872,7 +1854,7 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl sourceAttempt, && !tooManyDownstreamHostsBlamedTheSameUpstreamHost) { return sourceAttempt.getInternalState(); } - String message = sourceAttempt.getID() + " being failed for too many output errors. " + String message = sourceAttempt.getTaskAttemptID() + " being failed for too many output errors. " + "failureFraction=" + failureFraction + ", MAX_ALLOWED_OUTPUT_FAILURES_FRACTION=" + maxAllowedOutputFailuresFraction @@ -1929,8 +1911,8 @@ protected void sendInputFailedToConsumers() { tezIfEvents.add(new TezEvent(new InputFailedEvent(), new EventMetaData(EventProducerConsumerType.SYSTEM, vertex.getName(), - edgeVertex.getName(), - getID()), appContext.getClock().getTime())); + edgeVertex.getName(), + getTaskAttemptID()), appContext.getClock().getTime())); } sendEvent(new VertexEventRouteEvent(vertex.getVertexId(), tezIfEvents)); } @@ -2024,7 +2006,7 @@ public TaskFailureType getFailureType(TaskAttemptEvent event) { @Override public String toString() { - return getID().toString(); + return getTaskAttemptID().toString(); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java index b840798c26..d2933c5b86 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java @@ -40,6 +40,10 @@ import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.tez.dag.app.dag.event.TaskEventTAFailed; +import org.apache.tez.dag.records.TaskAttemptTerminationCause; +import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.dag.records.TezTaskID; +import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.runtime.api.TaskFailureType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -93,10 +97,6 @@ import org.apache.tez.dag.history.DAGHistoryEvent; import org.apache.tez.dag.history.events.TaskFinishedEvent; import org.apache.tez.dag.history.events.TaskStartedEvent; -import org.apache.tez.dag.records.TaskAttemptTerminationCause; -import org.apache.tez.dag.records.TezTaskAttemptID; -import org.apache.tez.dag.records.TezTaskID; -import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.dag.utils.TezBuilderUtils; import org.apache.tez.runtime.api.OutputCommitter; import org.apache.tez.runtime.api.impl.TaskSpec; @@ -434,7 +434,7 @@ public Vertex getVertex() { } @Override - public TezTaskID getTaskId() { + public TezTaskID getTaskID() { return taskId; } @@ -523,7 +523,7 @@ public ArrayList getTaskAttemptTezEvents(TezTaskAttemptID attemptID, try { if (!attempts.containsKey(attemptID)) { throw new TezUncheckedException("Unknown TA: " + attemptID - + " asking for events from task:" + getTaskId()); + + " asking for events from task:" + getTaskID()); } if (tezEventsForTaskAttempts.size() > fromEventId) { @@ -775,11 +775,11 @@ public TaskAttempt getSuccessfulAttempt() { private boolean addAndScheduleAttempt(TezTaskAttemptID schedulingCausalTA) { TaskAttempt attempt = createAttempt(attempts.size(), schedulingCausalTA); if (LOG.isDebugEnabled()) { - LOG.debug("Created attempt " + attempt.getID()); + LOG.debug("Created attempt " + attempt.getTaskAttemptID()); } switch (attempts.size()) { case 0: - attempts = Collections.singletonMap(attempt.getID(), attempt); + attempts = Collections.singletonMap(attempt.getTaskAttemptID(), attempt); break; case 1: @@ -787,12 +787,12 @@ private boolean addAndScheduleAttempt(TezTaskAttemptID schedulingCausalTA) { = new LinkedHashMap(maxFailedAttempts); newAttempts.putAll(attempts); attempts = newAttempts; - Preconditions.checkArgument(attempts.put(attempt.getID(), attempt) == null, - attempt.getID() + " already existed"); + Preconditions.checkArgument(attempts.put(attempt.getTaskAttemptID(), attempt) == null, + attempt.getTaskAttemptID() + " already existed"); break; default: - Preconditions.checkArgument(attempts.put(attempt.getID(), attempt) == null, - attempt.getID() + " already existed"); + Preconditions.checkArgument(attempts.put(attempt.getTaskAttemptID(), attempt) == null, + attempt.getTaskAttemptID() + " already existed"); break; } @@ -819,7 +819,7 @@ private boolean addAndScheduleAttempt(TezTaskAttemptID schedulingCausalTA) { } */ - this.taskAttemptStatus.put(attempt.getID().getId(), false); + this.taskAttemptStatus.put(attempt.getTaskAttemptID().getId(), false); //schedule the nextAttemptNumber // send event to DAG to assign priority and schedule the attempt with global // picture in mind @@ -865,17 +865,17 @@ protected void internalError(TaskEventType type) { LOG.error("Invalid event " + type + " on Task " + this.taskId + " in state:" + getInternalState()); eventHandler.handle(new DAGEventDiagnosticsUpdate( - this.taskId.getVertexID().getDAGId(), "Invalid event " + type + + getDAGID(), "Invalid event " + type + " on Task " + this.taskId)); - eventHandler.handle(new DAGEvent(this.taskId.getVertexID().getDAGId(), + eventHandler.handle(new DAGEvent(getDAGID(), DAGEventType.INTERNAL_ERROR)); } protected void internalErrorUncaughtException(TaskEventType type, Exception e) { eventHandler.handle(new DAGEventDiagnosticsUpdate( - this.taskId.getVertexID().getDAGId(), "Uncaught exception when handling event " + type + + getDAGID(), "Uncaught exception when handling event " + type + " on Task " + this.taskId + ", error=" + e.getMessage())); - eventHandler.handle(new DAGEvent(this.taskId.getVertexID().getDAGId(), + eventHandler.handle(new DAGEvent(getDAGID(), DAGEventType.INTERNAL_ERROR)); } @@ -918,7 +918,7 @@ protected void logJobHistoryTaskStartedEvent() { TaskStartedEvent startEvt = new TaskStartedEvent(taskId, getVertex().getName(), scheduledTime, getLaunchTime()); this.appContext.getHistoryHandler().handle( - new DAGHistoryEvent(taskId.getVertexID().getDAGId(), startEvt)); + new DAGHistoryEvent(getDAGID(), startEvt)); } protected void logJobHistoryTaskFinishedEvent() { @@ -930,7 +930,7 @@ protected void logJobHistoryTaskFinishedEvent() { successfulAttempt, TaskState.SUCCEEDED, "", getCounters(), failedAttempts); this.appContext.getHistoryHandler().handle( - new DAGHistoryEvent(taskId.getVertexID().getDAGId(), finishEvt)); + new DAGHistoryEvent(getDAGID(), finishEvt)); } protected void logJobHistoryTaskFailedEvent(TaskState finalState) { @@ -941,7 +941,7 @@ protected void logJobHistoryTaskFailedEvent(TaskState finalState) { StringUtils.join(getDiagnostics(), LINE_SEPARATOR), getCounters(), failedAttempts); this.appContext.getHistoryHandler().handle( - new DAGHistoryEvent(taskId.getVertexID().getDAGId(), finishEvt)); + new DAGHistoryEvent(getDAGID(), finishEvt)); } private void addDiagnosticInfo(String diag) { @@ -994,7 +994,7 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { && tFinishedEvent != null) { Preconditions.checkArgument(tFinishedEvent.getState() == TaskState.KILLED, "TaskStartedEvent is not seen, but TaskFinishedEvent is seen and with invalid state=" - + tFinishedEvent.getState() + ", taskId=" + task.getTaskId()); + + tFinishedEvent.getState() + ", taskId=" + task.getTaskID()); // TODO (TEZ-2938) // use tFinishedEvent.getTerminationCause after adding TaskTerminationCause to TaskFinishedEvent task.eventHandler.handle(new TaskEventTermination(task.taskId, @@ -1043,22 +1043,22 @@ public void transition(TaskImpl task, TaskEvent event) { } else { if (TaskAttemptState.SUCCEEDED.equals(ta.getState())) { LOG.info("Ignore speculation scheduling for task {} since it has succeeded with attempt {}.", - task.getTaskId(), ta.getID()); + task.getTaskID(), ta.getTaskAttemptID()); return; } } } if (earliestUnfinishedAttempt == null) { // no running (or SUCCEEDED) task attempt at this moment, no need to schedule speculative attempt either - LOG.info("Ignore speculation scheduling since there is no running attempt on task {}.", task.getTaskId()); + LOG.info("Ignore speculation scheduling since there is no running attempt on task {}.", task.getTaskID()); return; } if (task.commitAttempt != null) { LOG.info("Ignore speculation scheduling for task {} since commit has started with commitAttempt {}.", - task.getTaskId(), task.commitAttempt); + task.getTaskID(), task.commitAttempt); return; } - task.addAndScheduleAttempt(earliestUnfinishedAttempt.getID()); + task.addAndScheduleAttempt(earliestUnfinishedAttempt.getTaskAttemptID()); } } @@ -1075,26 +1075,26 @@ private String recoverSuccessTaskAttempt(TaskImpl task) { for (Entry entry : task.getVertex().getOutputCommitters().entrySet()) { LOG.info("Recovering data for task from previous DAG attempt" - + ", taskId=" + task.getTaskId() + + ", taskId=" + task.getTaskID() + ", output=" + entry.getKey()); OutputCommitter committer = entry.getValue(); if (!committer.isTaskRecoverySupported()) { errorMsg = "Task recovery not supported by committer" + ", failing task attempt"; LOG.info(errorMsg - + ", taskId=" + task.getTaskId() + + ", taskId=" + task.getTaskID() + ", attemptId=" + task.successfulAttempt + ", output=" + entry.getKey()); break; } try { - committer.recoverTask(task.getTaskId().getId(), + committer.recoverTask(task.getTaskID().getId(), task.appContext.getApplicationAttemptId().getAttemptId()-1); } catch (Exception e) { errorMsg = "Task recovery failed by committer: " + ExceptionUtils.getStackTrace(e); LOG.warn("Task recovery failed by committer" - + ", taskId=" + task.getTaskId() + + ", taskId=" + task.getTaskID() + ", attemptId=" + task.successfulAttempt + ", output=" + entry.getKey(), e); break; @@ -1114,7 +1114,7 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { String errorMsg = recoverSuccessTaskAttempt(task); if (errorMsg != null) { LOG.info("Can not recover the successful task attempt, schedule new task attempt," - + "taskId=" + task.getTaskId()); + + "taskId=" + task.getTaskID()); task.successfulAttempt = null; if (!task.addAndScheduleAttempt(successTaId)) { task.finished(TaskStateInternal.FAILED); @@ -1150,12 +1150,12 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { // issue kill to all other attempts for (TaskAttempt attempt : task.attempts.values()) { - if (!attempt.getID().equals(task.successfulAttempt) && + if (!attempt.getTaskAttemptID().equals(task.successfulAttempt) && // This is okay because it can only talk us out of sending a // TA_KILL message to an attempt that doesn't need one for // other reasons. !attempt.isFinished()) { - LOG.info("Issuing kill to other attempt " + attempt.getID() + " as attempt: " + + LOG.info("Issuing kill to other attempt " + attempt.getTaskAttemptID() + " as attempt: " + task.successfulAttempt + " has succeeded"); String diagnostics = null; TaskAttemptTerminationCause errCause = null; @@ -1169,7 +1169,7 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { errCause = TaskAttemptTerminationCause.TERMINATED_INEFFECTIVE_SPECULATION; } task.eventHandler.handle(new TaskAttemptEventKillRequest(attempt - .getID(), diagnostics, errCause)); + .getTaskAttemptID(), diagnostics, errCause)); } } return task.finished(TaskStateInternal.SUCCEEDED); @@ -1267,7 +1267,7 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { TaskAttempt taskAttempt = task.getAttempt(castEvent.getTaskAttemptID()); if (taskAttempt.getAssignedContainer() != null) { NodeId nodeId = taskAttempt.getAssignedContainer().getNodeId(); - task.appContext.getAppMaster().taskAttemptFailed(taskAttempt.getID(), nodeId); + task.appContext.getAppMaster().taskAttemptFailed(taskAttempt.getTaskAttemptID(), nodeId); } task.taskAttemptStatus.put(castEvent.getTaskAttemptID().getId(), true); @@ -1278,7 +1278,7 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { TaskAttemptStateInternal.FAILED); // we don't need a new event if we already have a spare if (task.shouldScheduleNewAttempt()) { - LOG.info("Scheduling new attempt for task: " + task.getTaskId() + LOG.info("Scheduling new attempt for task: " + task.getTaskID() + ", currentFailedAttempts: " + task.failedAttempts + ", maxFailedAttempts: " + task.maxFailedAttempts + ", maxAttempts: " + task.maxAttempts); if (!task.addAndScheduleAttempt(getSchedulingCausalTA())){ @@ -1289,11 +1289,11 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { if (castEvent.getTaskFailureType() == TaskFailureType.NON_FATAL) { LOG.info( "Failing task: {} due to too many failed attempts. currentFailedAttempts={}, maxFailedAttempts={}", - task.getTaskId(), task.failedAttempts, task.maxFailedAttempts); + task.getTaskID(), task.failedAttempts, task.maxFailedAttempts); } else { LOG.info( "Failing task: {} due to {} error reported by TaskAttempt. CurrentFailedAttempts={}", - task.getTaskId(), TaskFailureType.FATAL, task.failedAttempts); + task.getTaskID(), TaskFailureType.FATAL, task.failedAttempts); } task.handleTaskAttemptCompletion( ((TaskEventTAUpdate) event).getTaskAttemptID(), @@ -1348,7 +1348,7 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { if (task.leafVertex) { LOG.error("Unexpected event for task of leaf vertex " + event.getType() + ", taskId: " - + task.getTaskId()); + + task.getTaskID()); task.internalError(event.getType()); } Preconditions.checkState(castEvent.getCausalEvent() != null); @@ -1418,7 +1418,7 @@ public void transition(TaskImpl task, TaskEvent event) { task.addDiagnosticInfo(terminateEvent.getDiagnosticInfo()); if (terminateEvent.isFromRecovery()) { if (LOG.isDebugEnabled()) { - LOG.debug("Recovered to KILLED, taskId=" + task.getTaskId()); + LOG.debug("Recovered to KILLED, taskId=" + task.getTaskID()); } } else { task.logJobHistoryTaskFailedEvent(TaskState.KILLED); @@ -1453,20 +1453,20 @@ public void onStateChanged(TaskImpl task, TaskStateInternal taskStateInternal) { // With this, recovery will end up failing for DAGs making use of InputInitializerEvents int succesfulAttemptInt = -1; if (successfulAttempt != null) { - succesfulAttemptInt = successfulAttempt.getID().getId(); + succesfulAttemptInt = successfulAttempt.getTaskAttemptID().getId(); } - task.stateChangeNotifier.taskSucceeded(task.getVertex().getName(), task.getTaskId(), + task.stateChangeNotifier.taskSucceeded(task.getVertex().getName(), task.getTaskID(), succesfulAttemptInt); } } private void killUnfinishedAttempt(TaskAttempt attempt, String logMsg, TaskAttemptTerminationCause errorCause) { - if (commitAttempt != null && commitAttempt.equals(attempt.getID())) { + if (commitAttempt != null && commitAttempt.equals(attempt.getTaskAttemptID())) { LOG.info("Unsetting commit attempt: " + commitAttempt + " since attempt is being killed"); commitAttempt = null; } if (attempt != null && !attempt.isFinished()) { - eventHandler.handle(new TaskAttemptEventKillRequest(attempt.getID(), logMsg, errorCause)); + eventHandler.handle(new TaskAttemptEventKillRequest(attempt.getTaskAttemptID(), logMsg, errorCause)); } } @@ -1485,7 +1485,7 @@ public long getFirstAttemptStartTime() { readLock.lock(); try { // The first attempt will always have an index of 0. - return getAttempt(TezTaskAttemptID.getInstance(getTaskId(), 0)).getScheduleTime(); + return getAttempt(TezTaskAttemptID.getInstance(getTaskID(), 0)).getScheduleTime(); } finally { readLock.unlock(); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java index a994359354..be4ee6068e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java @@ -64,7 +64,7 @@ public TezRootInputInitializerContextImpl( @Override public ApplicationId getApplicationId() { - return vertex.getVertexId().getDAGId().getApplicationId(); + return vertex.getVertexId().getDAGID().getApplicationId(); } @Override diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index f22f6de250..934dd4e76e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -1590,7 +1590,7 @@ private void computeProgress() { if (LOG.isDebugEnabled()) { if (!ProgressHelper.isProgressWithinRange(taskProg)) { LOG.debug("progress update: vertex={}, task={} incorrect; range={}", - getName(), task.getTaskId(), taskProg); + getName(), task.getTaskID(), taskProg); } } accProg += ProgressHelper.processProgress(taskProg); @@ -2052,7 +2052,7 @@ public void doneReconfiguringVertex() { */ public void handle(VertexEvent event) { if (LOG.isDebugEnabled()) { - LOG.debug("Processing VertexEvent " + event.getVertexId() + LOG.debug("Processing VertexEvent " + event.getVertexID() + " of type " + event.getType() + " while in state " + getInternalState() + ". Event: " + event); } @@ -2114,7 +2114,7 @@ protected void addTask(Task task) { lazyTasksCopyNeeded = false; } } - tasks.put(task.getTaskId(), task); + tasks.put(task.getTaskID(), task); // TODO Metrics //metrics.waitingTask(task); } @@ -2410,7 +2410,7 @@ void tryEnactKill(VertexTerminationCause trigger, LOG.info(msg); for (Task task : tasks.values()) { eventHandler.handle( // attempt was terminated because the vertex is shutting down - new TaskEventTermination(task.getTaskId(), errCause, msg)); + new TaskEventTermination(task.getTaskID(), errCause, msg)); } } } @@ -2649,7 +2649,7 @@ private void createTasks() { this.addTask(task); if(LOG.isDebugEnabled()) { LOG.debug("Created task for vertex " + logIdentifier + ": " + - task.getTaskId()); + task.getTaskID()); } } } @@ -2663,7 +2663,7 @@ private void addTasks(int newNumTasks) { this.numTasks++; if(LOG.isDebugEnabled()) { LOG.debug("Created task for vertex " + logIdentifier + ": " + - task.getTaskId()); + task.getTaskID()); } } } @@ -2919,7 +2919,7 @@ private static List getTaskAttemptIdentifiers(DAG dag, List attempts = new ArrayList(taIds.size()); String dagName = dag.getName(); for (TezTaskAttemptID taId : taIds) { - String vertexName = dag.getVertex(taId.getTaskID().getVertexID()).getName(); + String vertexName = dag.getVertex(taId.getVertexID()).getName(); attempts.add(getTaskAttemptIdentifier(dagName, vertexName, taId)); } return attempts; @@ -3639,7 +3639,7 @@ public VertexState transition(VertexImpl vertex, VertexEvent event) { TezTaskAttemptID taId = completionEvent.getTaskAttemptId(); vertex.vertexManager.onSourceTaskCompleted( getTaskAttemptIdentifier(vertex.dag.getName(), - vertex.dag.getVertex(taId.getTaskID().getVertexID()).getName(), + vertex.dag.getVertex(taId.getVertexID()).getName(), taId)); } catch (AMUserCodeException e) { String msg = "Exception in " + e.getSource() + ", vertex:" + vertex.getLogIdentifier(); @@ -3692,8 +3692,8 @@ public VertexState transition(VertexImpl vertex, VertexEvent event) { Task task = vertex.tasks.get(taskEvent.getTaskID()); if (taskEvent.getState() == TaskState.SUCCEEDED) { taskSucceeded(vertex, task); - if (!vertex.completedTasksStatsCache.containsTask(task.getTaskId())) { - vertex.completedTasksStatsCache.addTask(task.getTaskId()); + if (!vertex.completedTasksStatsCache.containsTask(task.getTaskID())) { + vertex.completedTasksStatsCache.addTask(task.getTaskID()); vertex.completedTasksStatsCache.mergeFrom(((TaskImpl) task).getStatistics()); } } else if (taskEvent.getState() == TaskState.FAILED) { @@ -3725,7 +3725,7 @@ private void taskSucceeded(VertexImpl vertex, Task task) { private void taskFailed(VertexImpl vertex, Task task) { vertex.failedTaskCount++; vertex.addDiagnostic("Task failed" - + ", taskId=" + task.getTaskId() + + ", taskId=" + task.getTaskID() + ", diagnostics=" + task.getDiagnostics()); // TODO Metrics //vertex.metrics.failedTask(task); @@ -4090,7 +4090,7 @@ private void handleRoutedTezEvents(List tezEvents, boolean isPendingEv pendingTaskEvents.add(tezEvent); } else { // event not from this vertex. must have come from source vertex. - int srcTaskIndex = sourceMeta.getTaskAttemptID().getTaskID().getId(); + int srcTaskIndex = sourceMeta.getTaskID().getId(); Vertex edgeVertex = getDAG().getVertex(sourceMeta.getTaskVertexName()); Edge srcEdge = sourceVertices.get(edgeVertex); if (srcEdge == null) { @@ -4132,7 +4132,7 @@ private void handleRoutedTezEvents(List tezEvents, boolean isPendingEv Preconditions.checkArgument(target != null, "Event sent to unkown vertex: " + vmEvent.getTargetVertexName()); TezTaskAttemptID srcTaId = sourceMeta.getTaskAttemptID(); - if (srcTaId.getTaskID().getVertexID().equals(vertexId)) { + if (srcTaId.getVertexID().equals(vertexId)) { // this is the producer tasks' vertex vmEvent.setProducerAttemptIdentifier( getTaskAttemptIdentifier(dag.getName(), getName(), srcTaId)); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java index 57c8c72be1..c3d49ea840 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java @@ -226,7 +226,7 @@ public synchronized void scheduleTasks(List tasks) { @Override public synchronized void scheduleVertexTasks(List tasks) { checkAndThrowIfDone(); - List schedTasks = new ArrayList(tasks.size()); + List schedTasks = new ArrayList<>(tasks.size()); for (TaskWithLocationHint task : tasks) { schedTasks.add(ScheduleTaskRequest.create( task.getTaskIndex(), task.getTaskLocationHint())); @@ -292,7 +292,7 @@ public void sendEventToProcessor(Collection events, int ta // attempt id won't be used anyway EventMetaData destinationMeta = new EventMetaData(EventProducerConsumerType.PROCESSOR, managedVertex.getName(), managedVertex.getName(), - TezTaskAttemptID.getInstance(managedVertex.getTask(taskId).getTaskId(), -1)); + TezTaskAttemptID.getInstance(managedVertex.getTask(taskId).getTaskID(), -1)); tezEvent.setDestinationInfo(destinationMeta); tezEvents.add(tezEvent); } @@ -571,7 +571,7 @@ private void sendInternalError(Exception e) { // state change must be triggered via an event transition LOG.error("Error after vertex manager callback " + managedVertex.getLogIdentifier(), e); appContext.getEventHandler().handle( - (new DAGEventInternalError(managedVertex.getVertexId().getDAGId(), + (new DAGEventInternalError(managedVertex.getVertexId().getDAGID(), "Error in VertexManager for vertex: " + managedVertex.getLogIdentifier() + ", error=" + ExceptionUtils.getStackTrace(e)))); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java index 7aad8f642b..940bb23025 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java @@ -271,7 +271,7 @@ public void notifyAttemptStatusUpdate(TezTaskAttemptID taId, /** * Absorbs one TaskAttemptStatus * - * @param reportedStatus the status report that we got from a task attempt + * @param reportedState the status report that we got from a task attempt * that we want to fold into the speculation data for this job * @param timestamp the time this status corresponds to. This matters * because statuses contain progress. @@ -331,7 +331,7 @@ public void handle(SpeculatorEvent event) { // with private long speculationValue(Task task, long now, boolean shouldUseTimeout) { Map attempts = task.getAttempts(); - TezTaskID taskID = task.getTaskId(); + TezTaskID taskID = task.getTaskID(); long acceptableRuntime = Long.MIN_VALUE; long result = Long.MIN_VALUE; @@ -359,7 +359,7 @@ private long speculationValue(Task task, long now, boolean shouldUseTimeout) { if (++numberRunningAttempts > 1) { return ALREADY_SPECULATING; } - runningTaskAttemptID = taskAttempt.getID(); + runningTaskAttemptID = taskAttempt.getTaskAttemptID(); long taskAttemptStartTime = estimator.attemptEnrolledTime(runningTaskAttemptID); @@ -404,7 +404,7 @@ private long speculationValue(Task task, long now, boolean shouldUseTimeout) { .hasStagnatedProgress(runningTaskAttemptID, now)) { // Stats have stagnated for a while, simulate heart-beat. // Now simulate the heart-beat - statusUpdate(taskAttempt.getID(), taskAttempt.getState(), + statusUpdate(taskAttempt.getTaskAttemptID(), taskAttempt.getState(), clock.getTime()); } } else { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java index d2587b5190..22c5b26a7c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java @@ -53,8 +53,8 @@ public void run() { BaseHttpConnection httpConnection = null; try { URL baseURL = TezRuntimeUtils.constructBaseURIForShuffleHandlerTaskAttemptFailed( - nodeId.getHost(), shufflePort, taskAttemptID.getTaskID().getVertexID().getDAGId(). - getApplicationId().toString(), taskAttemptID.getTaskID().getVertexID().getDAGId().getId(), + nodeId.getHost(), shufflePort, taskAttemptID.getTaskID().getVertexID().getDAGID(). + getApplicationId().toString(), taskAttemptID.getTaskID().getVertexID().getDAGID().getId(), taskAttemptID.toString(), false); httpConnection = TezRuntimeUtils.getHttpConnection(true, baseURL, httpConnectionParams, "FailedTaskAttemptDelete", jobTokenSecretManager); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java index f7fee3acfe..107fbf62c7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java @@ -42,7 +42,7 @@ public AMSchedulerEventTAEnded(TaskAttempt attempt, ContainerId containerId, } public TezTaskAttemptID getAttemptID() { - return this.attempt.getID(); + return this.attempt.getTaskAttemptID(); } public TaskAttempt getAttempt() { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java index 8e6bfe79a5..8a24ad3426 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java @@ -389,7 +389,7 @@ private void handleTAUnsuccessfulEnd(AMSchedulerEventTAEnded event) { String msg = "Error in TaskScheduler for handling Task De-allocation" + ", eventType=" + event.getType() + ", scheduler=" + Utils.getTaskSchedulerIdentifierString(event.getSchedulerId(), appContext) - + ", taskAttemptId=" + attempt.getID(); + + ", taskAttemptId=" + attempt.getTaskAttemptID(); LOG.error(msg, e); sendEvent( new DAGAppMasterEventUserServiceFatalError( @@ -403,10 +403,10 @@ private void handleTAUnsuccessfulEnd(AMSchedulerEventTAEnded event) { ContainerId attemptContainerId = attempt.getAssignedContainerID(); if(!wasContainerAllocated) { - LOG.info("Task: " + attempt.getID() + + LOG.info("Task: " + attempt.getTaskAttemptID() + " has no container assignment in the scheduler"); if (attemptContainerId != null) { - LOG.error("No container allocated to task: " + attempt.getID() + LOG.error("No container allocated to task: " + attempt.getTaskAttemptID() + " according to scheduler. Task reported container id: " + attemptContainerId); } @@ -425,7 +425,7 @@ private void handleTAUnsuccessfulEnd(AMSchedulerEventTAEnded event) { Container container = amContainer.getContainer(); sendEvent(new AMNodeEventTaskAttemptEnded(container.getNodeId(), event.getSchedulerId(), attemptContainerId, - attempt.getID(), event.getState() == TaskAttemptState.FAILED)); + attempt.getTaskAttemptID(), event.getState() == TaskAttemptState.FAILED)); } } } @@ -458,7 +458,7 @@ private void handleTASucceeded(AMSchedulerEventTAEnded event) { String msg = "Error in TaskScheduler for handling Task De-allocation" + ", eventType=" + event.getType() + ", scheduler=" + Utils.getTaskSchedulerIdentifierString(event.getSchedulerId(), appContext) - + ", taskAttemptId=" + attempt.getID(); + + ", taskAttemptId=" + attempt.getTaskAttemptID(); LOG.error(msg, e); sendEvent( new DAGAppMasterEventUserServiceFatalError( @@ -468,7 +468,7 @@ private void handleTASucceeded(AMSchedulerEventTAEnded event) { } if (!wasContainerAllocated) { - LOG.error("De-allocated successful task: " + attempt.getID() + LOG.error("De-allocated successful task: " + attempt.getTaskAttemptID() + ", but TaskScheduler reported no container assigned to task"); } } @@ -483,15 +483,15 @@ private void handleTaLaunchRequest(AMSchedulerEventTALaunchRequest event) { if (taskAffinity != null) { Vertex vertex = appContext.getCurrentDAG().getVertex(taskAffinity.getVertexName()); Objects.requireNonNull(vertex, "Invalid vertex in task based affinity " + taskAffinity - + " for attempt: " + taskAttempt.getID()); + + " for attempt: " + taskAttempt.getTaskAttemptID()); int taskIndex = taskAffinity.getTaskIndex(); Preconditions.checkState(taskIndex >=0 && taskIndex < vertex.getTotalTasks(), "Invalid taskIndex in task based affinity " + taskAffinity - + " for attempt: " + taskAttempt.getID()); + + " for attempt: " + taskAttempt.getTaskAttemptID()); TaskAttempt affinityAttempt = vertex.getTask(taskIndex).getSuccessfulAttempt(); if (affinityAttempt != null) { Objects.requireNonNull(affinityAttempt.getAssignedContainerID(), - affinityAttempt.getID() == null ? null : affinityAttempt.getID().toString()); + affinityAttempt.getTaskAttemptID() == null ? null : affinityAttempt.getTaskAttemptID().toString()); try { taskSchedulers[event.getSchedulerId()].allocateTask(taskAttempt, event.getCapability(), @@ -503,7 +503,7 @@ private void handleTaLaunchRequest(AMSchedulerEventTALaunchRequest event) { String msg = "Error in TaskScheduler for handling Task Allocation" + ", eventType=" + event.getType() + ", scheduler=" + Utils.getTaskSchedulerIdentifierString(event.getSchedulerId(), appContext) - + ", taskAttemptId=" + taskAttempt.getID(); + + ", taskAttemptId=" + taskAttempt.getTaskAttemptID(); LOG.error(msg, e); sendEvent( new DAGAppMasterEventUserServiceFatalError( @@ -513,7 +513,7 @@ private void handleTaLaunchRequest(AMSchedulerEventTALaunchRequest event) { return; } LOG.info("No attempt for task affinity to " + taskAffinity + " for attempt " - + taskAttempt.getID() + " Ignoring."); + + taskAttempt.getTaskAttemptID() + " Ignoring."); // fall through with null hosts/racks } else { hosts = (locationHint.getHosts() != null) ? locationHint @@ -536,7 +536,7 @@ private void handleTaLaunchRequest(AMSchedulerEventTALaunchRequest event) { String msg = "Error in TaskScheduler for handling Task Allocation" + ", eventType=" + event.getType() + ", scheduler=" + Utils.getTaskSchedulerIdentifierString(event.getSchedulerId(), appContext) - + ", taskAttemptId=" + taskAttempt.getID(); + + ", taskAttemptId=" + taskAttempt.getTaskAttemptID(); LOG.error(msg, e); sendEvent( new DAGAppMasterEventUserServiceFatalError( @@ -552,7 +552,7 @@ private void handleTAStateUpdated(AMSchedulerEventTAStateUpdated event) { String msg = "Error in TaskScheduler for handling Task State Update" + ", eventType=" + event.getType() + ", scheduler=" + Utils.getTaskSchedulerIdentifierString(event.getSchedulerId(), appContext) - + ", taskAttemptId=" + event.getTaskAttempt().getID() + + ", taskAttemptId=" + event.getTaskAttempt().getTaskAttemptID() + ", state=" + event.getState(); LOG.error(msg, e); sendEvent( @@ -763,7 +763,7 @@ public synchronized void taskAllocated(int schedulerId, Object task, event.getContainerContext(), event.getLauncherId(), event.getTaskCommId())); } } - sendEvent(new AMContainerEventAssignTA(containerId, taskAttempt.getID(), + sendEvent(new AMContainerEventAssignTA(containerId, taskAttempt.getTaskAttemptID(), event.getRemoteTaskSpec(), event.getContainerContext().getLocalResources(), event .getContainerContext().getCredentials(), event.getPriority())); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventLaunchRequest.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventLaunchRequest.java index 92e5817243..c3aae35910 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventLaunchRequest.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventLaunchRequest.java @@ -41,7 +41,7 @@ public AMContainerEventLaunchRequest(ContainerId containerId, } public TezDAGID getDAGId() { - return this.vertexId.getDAGId(); + return this.vertexId.getDAGID(); } public TezVertexID getVertexId() { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java b/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java index 08d754d8a0..00cd26e2ce 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java @@ -760,7 +760,7 @@ public void getTasksInfo() { ArrayList> tasksInfo = new ArrayList>(); for(Task t : tasks) { Map taskInfo = new HashMap(); - taskInfo.put("id", t.getTaskId().toString()); + taskInfo.put("id", t.getTaskID().toString()); taskInfo.put("progress", Float.toString(t.getProgress())); taskInfo.put("status", t.getState().toString()); @@ -810,7 +810,7 @@ else if(!attemptIDs.isEmpty()) { } TaskAttempt attempt = task. - getAttempt(TezTaskAttemptID.getInstance(task.getTaskId(), indexes.get(2))); + getAttempt(TezTaskAttemptID.getInstance(task.getTaskID(), indexes.get(2))); if(attempt == null) { continue; } @@ -858,7 +858,7 @@ public void getAttemptsInfo() { ArrayList> attemptsInfo = new ArrayList>(); for(TaskAttempt a : attempts) { Map attemptInfo = new HashMap(); - attemptInfo.put("id", a.getID().toString()); + attemptInfo.put("id", a.getTaskAttemptID().toString()); attemptInfo.put("progress", Float.toString(a.getProgress())); attemptInfo.put("status", a.getState().toString()); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/DAGHistoryEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/DAGHistoryEvent.java index dfa6bbdf91..240d8bd96d 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/DAGHistoryEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/DAGHistoryEvent.java @@ -18,9 +18,10 @@ package org.apache.tez.dag.history; +import org.apache.tez.dag.records.DAGIDAware; import org.apache.tez.dag.records.TezDAGID; -public class DAGHistoryEvent { +public class DAGHistoryEvent implements DAGIDAware { private final HistoryEvent historyEvent; private final TezDAGID dagID; @@ -39,7 +40,8 @@ public HistoryEvent getHistoryEvent() { return historyEvent; } - public TezDAGID getDagID() { + @Override + public TezDAGID getDAGID() { return this.dagID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java index 4fa1926ae9..f4dd789fd9 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java @@ -124,7 +124,7 @@ public void serviceStop() throws Exception { * @throws IOException */ public void handleCriticalEvent(DAGHistoryEvent event) throws IOException { - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); String dagIdStr = "N/A"; if(dagId != null) { dagIdStr = dagId.toString(); @@ -161,7 +161,7 @@ public void handleCriticalEvent(DAGHistoryEvent event) throws IOException { } private boolean shouldLogEvent(DAGHistoryEvent event) { - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); HistoryLogLevel dagLogLevel = null; if (dagId != null) { @@ -207,7 +207,7 @@ private boolean shouldLogTaskAttemptEvents(DAGHistoryEvent event, HistoryLogLeve if (dagLogLevel == HistoryLogLevel.TASK_ATTEMPT && (eventType == HistoryEventType.TASK_ATTEMPT_STARTED || eventType == HistoryEventType.TASK_ATTEMPT_FINISHED)) { - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); Set filters = null; if (dagId != null) { filters = dagIdToTaskAttemptFilters.get(dagId); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java index 0a7ef561f7..6d1d6cd3ff 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java @@ -32,6 +32,7 @@ import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; +import org.apache.tez.dag.records.DAGIDAware; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.recovery.records.RecoveryProtos; import org.apache.tez.dag.recovery.records.RecoveryProtos.DAGFinishedProto; @@ -40,7 +41,7 @@ import com.google.common.primitives.Ints; import com.google.protobuf.ByteString; -public class DAGFinishedEvent implements HistoryEvent, SummaryEvent { +public class DAGFinishedEvent implements HistoryEvent, SummaryEvent, DAGIDAware { private TezDAGID dagID; private long startTime; @@ -179,7 +180,8 @@ public DAGState getState() { return state; } - public TezDAGID getDagID() { + @Override + public TezDAGID getDAGID() { return dagID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java index 9e6c8b282b..4df116ad37 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java @@ -25,12 +25,13 @@ import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; +import org.apache.tez.dag.records.DAGIDAware; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.dag.recovery.records.RecoveryProtos; import org.apache.tez.dag.recovery.records.RecoveryProtos.DAGInitializedProto; -public class DAGInitializedEvent implements HistoryEvent { +public class DAGInitializedEvent implements HistoryEvent, DAGIDAware { private TezDAGID dagID; private long initTime; @@ -101,7 +102,8 @@ public long getInitTime() { return this.initTime; } - public TezDAGID getDagID() { + @Override + public TezDAGID getDAGID() { return dagID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java index f1fdcac809..9e1f6c4a50 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java @@ -25,10 +25,11 @@ import org.apache.tez.dag.app.dag.DAGState; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; +import org.apache.tez.dag.records.DAGIDAware; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.recovery.records.RecoveryProtos.DAGStartedProto; -public class DAGStartedEvent implements HistoryEvent { +public class DAGStartedEvent implements HistoryEvent, DAGIDAware { private TezDAGID dagID; private long startTime; @@ -97,7 +98,8 @@ public long getStartTime() { return this.startTime; } - public TezDAGID getDagID() { + @Override + public TezDAGID getDAGID() { return dagID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java index e04ee80188..bed0f6f25a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java @@ -24,6 +24,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import org.apache.tez.dag.records.DAGIDAware; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -42,7 +43,7 @@ import org.apache.tez.dag.utils.ProtoUtils; -public class DAGSubmittedEvent implements HistoryEvent, SummaryEvent { +public class DAGSubmittedEvent implements HistoryEvent, SummaryEvent, DAGIDAware { private static final Logger LOG = LoggerFactory.getLogger(DAGSubmittedEvent.class); @@ -174,7 +175,8 @@ public DAGProtos.DAGPlan getDAGPlan() { return this.dagPlan; } - public TezDAGID getDagID() { + @Override + public TezDAGID getDAGID() { return dagID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java index 18596495be..ce2eb729f6 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java @@ -27,6 +27,7 @@ import org.apache.tez.common.TezConverterUtils; import org.apache.tez.common.counters.CounterGroup; import org.apache.tez.common.counters.TezCounter; +import org.apache.tez.dag.records.TaskAttemptIDAware; import org.apache.tez.runtime.api.TaskFailureType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,7 +51,7 @@ import org.apache.tez.dag.recovery.records.RecoveryProtos.TezEventProto; import org.apache.tez.runtime.api.impl.TezEvent; -public class TaskAttemptFinishedEvent implements HistoryEvent { +public class TaskAttemptFinishedEvent implements HistoryEvent, TaskAttemptIDAware { private static final Logger LOG = LoggerFactory.getLogger(TaskAttemptFinishedEvent.class); @@ -307,6 +308,7 @@ public String toString() { return sb.toString(); } + @Override public TezTaskAttemptID getTaskAttemptID() { return taskAttemptId; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java index a49e47c97f..10ba439e1c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java @@ -27,10 +27,11 @@ import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; +import org.apache.tez.dag.records.TaskAttemptIDAware; import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.dag.recovery.records.RecoveryProtos.TaskAttemptStartedProto; -public class TaskAttemptStartedEvent implements HistoryEvent { +public class TaskAttemptStartedEvent implements HistoryEvent, TaskAttemptIDAware { private TezTaskAttemptID taskAttemptId; private String inProgressLogsUrl; @@ -113,6 +114,7 @@ public String toString() { + ", nodeId=" + nodeId; } + @Override public TezTaskAttemptID getTaskAttemptID() { return this.taskAttemptId; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java index dd7afdfc7c..215968274c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java @@ -24,6 +24,7 @@ import com.google.protobuf.CodedOutputStream; import org.apache.tez.common.counters.CounterGroup; import org.apache.tez.common.counters.TezCounter; +import org.apache.tez.dag.records.TaskIDAware; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.tez.common.counters.TezCounters; @@ -34,7 +35,7 @@ import org.apache.tez.dag.records.TezTaskID; import org.apache.tez.dag.recovery.records.RecoveryProtos.TaskFinishedProto; -public class TaskFinishedEvent implements HistoryEvent { +public class TaskFinishedEvent implements HistoryEvent, TaskIDAware { private static final Logger LOG = LoggerFactory.getLogger(TaskFinishedEvent.class); @@ -160,6 +161,7 @@ public String toString() { return sb.toString(); } + @Override public TezTaskID getTaskID() { return taskID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java index cc629699a4..577aef963c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java @@ -25,10 +25,11 @@ import org.apache.tez.dag.api.oldrecords.TaskState; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; +import org.apache.tez.dag.records.TaskIDAware; import org.apache.tez.dag.records.TezTaskID; import org.apache.tez.dag.recovery.records.RecoveryProtos.TaskStartedProto; -public class TaskStartedEvent implements HistoryEvent { +public class TaskStartedEvent implements HistoryEvent, TaskIDAware { private TezTaskID taskID; private String vertexName; @@ -104,6 +105,7 @@ public String toString() { + ", launchTime=" + startTime; } + @Override public TezTaskID getTaskID() { return taskID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java index 8ff86b8af1..ca2cafc834 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java @@ -99,7 +99,7 @@ public TezVertexID getVertexID() { @Override public void toSummaryProtoStream(OutputStream outputStream) throws IOException { SummaryEventProto.Builder builder = RecoveryProtos.SummaryEventProto.newBuilder() - .setDagId(vertexID.getDAGId().toString()) + .setDagId(vertexID.getDAGID().toString()) .setTimestamp(commitStartTime) .setEventType(getEventType().ordinal()) .setEventPayload( diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java index a2e20397cf..2f4fac079f 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java @@ -30,6 +30,7 @@ import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.records.TezVertexID; +import org.apache.tez.dag.records.VertexIDAware; import org.apache.tez.dag.recovery.records.RecoveryProtos.EdgeManagerDescriptorProto; import org.apache.tez.dag.recovery.records.RecoveryProtos.RootInputSpecUpdateProto; import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexConfigurationDoneProto; @@ -37,7 +38,7 @@ import com.google.common.collect.Maps; -public class VertexConfigurationDoneEvent implements HistoryEvent { +public class VertexConfigurationDoneEvent implements HistoryEvent, VertexIDAware { private TezVertexID vertexID; private long reconfigureDoneTime; @@ -182,6 +183,7 @@ public String toString() { + ", setParallelismCalledFlag=" + setParallelismCalledFlag; } + @Override public TezVertexID getVertexID() { return this.vertexID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java index 58cb628b28..24612a6a2e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java @@ -25,6 +25,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.app.dag.impl.ServicePluginInfo; +import org.apache.tez.dag.records.VertexIDAware; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.tez.common.counters.TezCounters; @@ -39,7 +40,7 @@ import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexFinishStateProto; import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexFinishedProto; -public class VertexFinishedEvent implements HistoryEvent, SummaryEvent { +public class VertexFinishedEvent implements HistoryEvent, SummaryEvent, VertexIDAware { private static final Logger LOG = LoggerFactory.getLogger(VertexFinishedEvent.class); @@ -157,6 +158,7 @@ public String toString() { + (servicePluginInfo != null ? servicePluginInfo : "null"); } + @Override public TezVertexID getVertexID() { return this.vertexID; } @@ -207,7 +209,7 @@ public void toSummaryProtoStream(OutputStream outputStream) throws IOException { .build(); SummaryEventProto.Builder builder = RecoveryProtos.SummaryEventProto.newBuilder() - .setDagId(vertexID.getDAGId().toString()) + .setDagId(vertexID.getDAGID().toString()) .setTimestamp(finishTime) .setEventType(getEventType().ordinal()) .setEventPayload(finishStateProto.toByteString()); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java index e7452e6770..a019fe3784 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java @@ -35,6 +35,7 @@ import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.utils.TezEventUtils; import org.apache.tez.dag.records.TezVertexID; +import org.apache.tez.dag.records.VertexIDAware; import org.apache.tez.dag.recovery.records.RecoveryProtos; import org.apache.tez.dag.recovery.records.RecoveryProtos.TezEventProto; import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexInitializedProto; @@ -42,7 +43,7 @@ import com.google.common.collect.Lists; -public class VertexInitializedEvent implements HistoryEvent { +public class VertexInitializedEvent implements HistoryEvent, VertexIDAware { private TezVertexID vertexID; private String vertexName; @@ -180,8 +181,9 @@ public String toString() { + (servicePluginInfo != null ? servicePluginInfo : "null"); } + @Override public TezVertexID getVertexID() { - return this.vertexID; + return vertexID; } public long getInitRequestedTime() { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java index 4a3e05f1f4..cdd73b4483 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java @@ -26,9 +26,10 @@ import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.records.TezVertexID; +import org.apache.tez.dag.records.VertexIDAware; import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexStartedProto; -public class VertexStartedEvent implements HistoryEvent { +public class VertexStartedEvent implements HistoryEvent, VertexIDAware { private TezVertexID vertexID; private long startRequestedTime; @@ -94,8 +95,9 @@ public String toString() { + ", startedTime=" + startTime; } + @Override public TezVertexID getVertexID() { - return this.vertexID; + return vertexID; } public long getStartRequestedTime() { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java b/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java index e60575f38e..8f4cd1fa6a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java @@ -341,7 +341,7 @@ private static JSONObject convertContainerStoppedEvent(ContainerStoppedEvent eve private static JSONObject convertDAGFinishedEvent(DAGFinishedEvent event) throws JSONException { JSONObject jsonObject = new JSONObject(); jsonObject.put(ATSConstants.ENTITY, - event.getDagID().toString()); + event.getDAGID().toString()); jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); @@ -384,7 +384,7 @@ private static JSONObject convertDAGFinishedEvent(DAGFinishedEvent event) throws private static JSONObject convertDAGInitializedEvent(DAGInitializedEvent event) throws JSONException { JSONObject jsonObject = new JSONObject(); jsonObject.put(ATSConstants.ENTITY, - event.getDagID().toString()); + event.getDAGID().toString()); jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); @@ -416,7 +416,7 @@ private static JSONObject convertDAGInitializedEvent(DAGInitializedEvent event) private static JSONObject convertDAGStartedEvent(DAGStartedEvent event) throws JSONException { JSONObject jsonObject = new JSONObject(); jsonObject.put(ATSConstants.ENTITY, - event.getDagID().toString()); + event.getDAGID().toString()); jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); @@ -439,7 +439,7 @@ private static JSONObject convertDAGStartedEvent(DAGStartedEvent event) throws J private static JSONObject convertDAGSubmittedEvent(DAGSubmittedEvent event) throws JSONException { JSONObject jsonObject = new JSONObject(); jsonObject.put(ATSConstants.ENTITY, - event.getDagID().toString()); + event.getDAGID().toString()); jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); @@ -602,7 +602,7 @@ private static JSONObject convertTaskAttemptStartedEvent(TaskAttemptStartedEvent containerEntity.put(ATSConstants.ENTITY_TYPE, ATSConstants.CONTAINER_ID); JSONObject taskEntity = new JSONObject(); - taskEntity.put(ATSConstants.ENTITY, event.getTaskAttemptID().getTaskID().toString()); + taskEntity.put(ATSConstants.ENTITY, event.getTaskID().toString()); taskEntity.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_TASK_ID.name()); relatedEntities.put(nodeEntity); @@ -667,7 +667,7 @@ private static JSONObject convertTaskStartedEvent(TaskStartedEvent event) throws // Related entities JSONArray relatedEntities = new JSONArray(); JSONObject vertexEntity = new JSONObject(); - vertexEntity.put(ATSConstants.ENTITY, event.getTaskID().getVertexID().toString()); + vertexEntity.put(ATSConstants.ENTITY, event.getVertexID().toString()); vertexEntity.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_VERTEX_ID.name()); relatedEntities.put(vertexEntity); jsonObject.put(ATSConstants.RELATED_ENTITIES, relatedEntities); @@ -775,7 +775,7 @@ private static JSONObject convertVertexInitializedEvent(VertexInitializedEvent e // Related entities JSONArray relatedEntities = new JSONArray(); JSONObject vertexEntity = new JSONObject(); - vertexEntity.put(ATSConstants.ENTITY, event.getVertexID().getDAGId().toString()); + vertexEntity.put(ATSConstants.ENTITY, event.getDAGID().toString()); vertexEntity.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); relatedEntities.put(vertexEntity); jsonObject.put(ATSConstants.RELATED_ENTITIES, relatedEntities); @@ -815,7 +815,7 @@ private static JSONObject convertVertexStartedEvent(VertexStartedEvent event) // Related entities JSONArray relatedEntities = new JSONArray(); JSONObject vertexEntity = new JSONObject(); - vertexEntity.put(ATSConstants.ENTITY, event.getVertexID().getDAGId().toString()); + vertexEntity.put(ATSConstants.ENTITY, event.getDAGID().toString()); vertexEntity.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); relatedEntities.put(vertexEntity); jsonObject.put(ATSConstants.RELATED_ENTITIES, relatedEntities); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java index 45e7d2fc9c..5b5a9c7261 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java @@ -300,7 +300,7 @@ public void handle(DAGHistoryEvent event) throws IOException { return; } - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); if (eventType.equals(HistoryEventType.DAG_SUBMITTED)) { DAGSubmittedEvent dagSubmittedEvent = (DAGSubmittedEvent) event.getHistoryEvent(); @@ -337,8 +337,8 @@ public void handle(DAGHistoryEvent event) throws IOException { if (summaryEvent.writeToRecoveryImmediately()) { handleRecoveryEvent(event); // outputStream may already be closed and removed - if (outputStreamMap.containsKey(event.getDagID())) { - doFlush(outputStreamMap.get(event.getDagID()), + if (outputStreamMap.containsKey(event.getDAGID())) { + doFlush(outputStreamMap.get(event.getDAGID()), appContext.getClock().getTime()); } } else { @@ -350,7 +350,7 @@ public void handle(DAGHistoryEvent event) throws IOException { } if (eventType.equals(HistoryEventType.DAG_FINISHED)) { LOG.info("DAG completed" - + ", dagId=" + event.getDagID() + + ", dagId=" + event.getDAGID() + ", queueSize=" + eventQueue.size()); completedDAGs.add(dagId); if (outputStreamMap.containsKey(dagId)) { @@ -359,7 +359,7 @@ public void handle(DAGHistoryEvent event) throws IOException { outputStreamMap.remove(dagId); } catch (IOException ioe) { LOG.warn("Error when trying to flush/close recovery file for" - + " dag, dagId=" + event.getDagID()); + + " dag, dagId=" + event.getDAGID()); } } } @@ -436,7 +436,7 @@ protected void handleRecoveryEvent(DAGHistoryEvent event) throws IOException { LOG.debug("Handling recovery event of type " + event.getHistoryEvent().getEventType()); } - TezDAGID dagID = event.getDagID(); + TezDAGID dagID = event.getDAGID(); if (completedDAGs.contains(dagID)) { // no need to recover completed DAGs diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java index aa5d3b9535..6c0231c9b0 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java @@ -261,7 +261,7 @@ public void testBasicEvents() throws Exception { Assert.assertEquals(DAGStatus.State.SUCCEEDED, dagClient.getDAGStatus(null).getState()); VertexImpl vImpl = (VertexImpl) dagImpl.getVertex(vB.getName()); TaskImpl tImpl = (TaskImpl) vImpl.getTask(1); - TezTaskAttemptID taId = TezTaskAttemptID.getInstance(tImpl.getTaskId(), 0); + TezTaskAttemptID taId = TezTaskAttemptID.getInstance(tImpl.getTaskID(), 0); List tEvents = vImpl.getTaskAttemptTezEvents(taId, 0, 0, 1000).getEvents(); Assert.assertEquals(2, tEvents.size()); // 2 from vA Assert.assertEquals(vA.getName(), tEvents.get(0).getDestinationInfo().getEdgeVertexName()); @@ -275,7 +275,7 @@ public void testBasicEvents() throws Exception { (targetIndex1 == 0 && targetIndex2 == 1) || (targetIndex1 == 1 && targetIndex2 == 0)); vImpl = (VertexImpl) dagImpl.getVertex(vC.getName()); tImpl = (TaskImpl) vImpl.getTask(1); - taId = TezTaskAttemptID.getInstance(tImpl.getTaskId(), 0); + taId = TezTaskAttemptID.getInstance(tImpl.getTaskID(), 0); tEvents = vImpl.getTaskAttemptTezEvents(taId, 0, 0, 1000).getEvents(); Assert.assertEquals(2, tEvents.size()); // 2 from vA Assert.assertEquals(vA.getName(), tEvents.get(0).getDestinationInfo().getEdgeVertexName()); @@ -289,7 +289,7 @@ public void testBasicEvents() throws Exception { (targetIndex1 == 0 && targetIndex2 == 1) || (targetIndex1 == 1 && targetIndex2 == 0)); vImpl = (VertexImpl) dagImpl.getVertex(vD.getName()); tImpl = (TaskImpl) vImpl.getTask(1); - taId = TezTaskAttemptID.getInstance(tImpl.getTaskId(), 0); + taId = TezTaskAttemptID.getInstance(tImpl.getTaskID(), 0); tEvents = vImpl.getTaskAttemptTezEvents(taId, 0, 0, 1000).getEvents(); Assert.assertEquals(1, tEvents.size()); // 1 from vA Assert.assertEquals(vA.getName(), tEvents.get(0).getDestinationInfo().getEdgeVertexName()); @@ -398,17 +398,17 @@ public void testMixedEdgeRouting() throws Exception { // vC uses on demand routing and its task does not provide events VertexImpl vImpl = (VertexImpl) dagImpl.getVertex(vC.getName()); TaskImpl tImpl = (TaskImpl) vImpl.getTask(0); - TezTaskAttemptID taId = TezTaskAttemptID.getInstance(tImpl.getTaskId(), 0); + TezTaskAttemptID taId = TezTaskAttemptID.getInstance(tImpl.getTaskID(), 0); Assert.assertEquals(0, tImpl.getTaskAttemptTezEvents(taId, 0, 1000).size()); // vD is mixed mode and only 1 out of 2 edges does legacy routing with task providing events vImpl = (VertexImpl) dagImpl.getVertex(vD.getName()); tImpl = (TaskImpl) vImpl.getTask(0); - taId = TezTaskAttemptID.getInstance(tImpl.getTaskId(), 0); + taId = TezTaskAttemptID.getInstance(tImpl.getTaskID(), 0); Assert.assertEquals(1, tImpl.getTaskAttemptTezEvents(taId, 0, 1000).size()); // vE has single legacy edge and does not use on demand routing and its task provides events vImpl = (VertexImpl) dagImpl.getVertex(vE.getName()); tImpl = (TaskImpl) vImpl.getTask(0); - taId = TezTaskAttemptID.getInstance(tImpl.getTaskId(), 0); + taId = TezTaskAttemptID.getInstance(tImpl.getTaskID(), 0); Assert.assertEquals(1, tImpl.getTaskAttemptTezEvents(taId, 0, 1000).size()); tezClient.stop(); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java index b6f50301c7..302281af9f 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java @@ -326,7 +326,7 @@ public void testSingleTaskSpeculation() throws Exception { Task task = dagImpl.getTask(killedTaId.getTaskID()); Assert.assertEquals(entry.getValue().intValue(), task.getAttempts().size()); if (entry.getValue() > 1) { - Assert.assertEquals(successTaId, task.getSuccessfulAttempt().getID()); + Assert.assertEquals(successTaId, task.getSuccessfulAttempt().getTaskAttemptID()); TaskAttempt killedAttempt = task.getAttempt(killedTaId); Joiner.on(",").join(killedAttempt.getDiagnostics()).contains("Killed as speculative attempt"); Assert.assertEquals(TaskAttemptTerminationCause.TERMINATED_EFFECTIVE_SPECULATION, @@ -369,7 +369,7 @@ public void testBasicSpeculation(boolean withProgress) throws Exception { Task task = dagImpl.getTask(killedTaId.getTaskID()); Assert.assertEquals(ASSERT_SPECULATIONS_COUNT_MSG, 2, task.getAttempts().size()); - Assert.assertEquals(successTaId, task.getSuccessfulAttempt().getID()); + Assert.assertEquals(successTaId, task.getSuccessfulAttempt().getTaskAttemptID()); TaskAttempt killedAttempt = task.getAttempt(killedTaId); Joiner.on(",").join(killedAttempt.getDiagnostics()).contains("Killed as speculative attempt"); Assert.assertEquals(TaskAttemptTerminationCause.TERMINATED_EFFECTIVE_SPECULATION, @@ -380,7 +380,7 @@ public void testBasicSpeculation(boolean withProgress) throws Exception { .getValue()); Assert.assertEquals(1, dagImpl.getAllCounters().findCounter(TaskCounter.NUM_SPECULATIONS) .getValue()); - org.apache.tez.dag.app.dag.Vertex v = dagImpl.getVertex(killedTaId.getTaskID().getVertexID()); + org.apache.tez.dag.app.dag.Vertex v = dagImpl.getVertex(killedTaId.getVertexID()); Assert.assertEquals(1, v.getAllCounters().findCounter(TaskCounter.NUM_SPECULATIONS) .getValue()); } @@ -508,7 +508,7 @@ public void testBasicSpeculationNotUseful() throws Exception { Assert.assertEquals(DAGStatus.State.SUCCEEDED, dagClient.getDAGStatus(null).getState()); Task task = dagImpl.getTask(killedTaId.getTaskID()); Assert.assertEquals(2, task.getAttempts().size()); - Assert.assertEquals(successTaId, task.getSuccessfulAttempt().getID()); + Assert.assertEquals(successTaId, task.getSuccessfulAttempt().getTaskAttemptID()); TaskAttempt killedAttempt = task.getAttempt(killedTaId); Joiner.on(",").join(killedAttempt.getDiagnostics()).contains("Killed speculative attempt as"); Assert.assertEquals(TaskAttemptTerminationCause.TERMINATED_INEFFECTIVE_SPECULATION, @@ -517,7 +517,7 @@ public void testBasicSpeculationNotUseful() throws Exception { .getValue()); Assert.assertEquals(1, dagImpl.getAllCounters().findCounter(TaskCounter.NUM_SPECULATIONS) .getValue()); - org.apache.tez.dag.app.dag.Vertex v = dagImpl.getVertex(killedTaId.getTaskID().getVertexID()); + org.apache.tez.dag.app.dag.Vertex v = dagImpl.getVertex(killedTaId.getVertexID()); Assert.assertEquals(1, v.getAllCounters().findCounter(TaskCounter.NUM_SPECULATIONS) .getValue()); tezClient.stop(); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestCommit.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestCommit.java index 7611f1c44e..436c522ef7 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestCommit.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestCommit.java @@ -167,7 +167,7 @@ private class VertexEventDispatcher implements EventHandler { @SuppressWarnings("unchecked") @Override public void handle(VertexEvent event) { - Vertex vertex = dag.getVertex(event.getVertexId()); + Vertex vertex = dag.getVertex(event.getVertexID()); ((EventHandler) vertex).handle(event); } } @@ -176,7 +176,7 @@ private class TaskEventDispatcher implements EventHandler { @SuppressWarnings("unchecked") @Override public void handle(TaskEvent event) { - Vertex vertex = dag.getVertex(event.getTaskID().getVertexID()); + Vertex vertex = dag.getVertex(event.getVertexID()); Task task = vertex.getTask(event.getTaskID()); ((EventHandler) task).handle(event); } @@ -563,7 +563,7 @@ public void testVertexCommit_OnDAGSuccess() throws Exception { initDAG(dag); startDAG(dag); VertexImpl v1 = (VertexImpl) dag.getVertex("vertex1"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.SUCCEEDED, v1.getState()); Assert.assertNull(v1.getTerminationCause()); @@ -594,7 +594,7 @@ public void testVertexCommit_OnVertexSuccess() throws Exception { initDAG(dag); startDAG(dag); VertexImpl v1 = (VertexImpl) dag.getVertex("vertex1"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v1.getState()); CountingOutputCommitter v1OutputCommitter_1 = (CountingOutputCommitter) v1 @@ -633,7 +633,7 @@ public void testVertexCommitFail1_OnVertexSuccess() throws Exception { initDAG(dag); startDAG(dag); VertexImpl v1 = (VertexImpl) dag.getVertex("vertex1"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v1.getState()); CountingOutputCommitter v1OutputCommitter_1 = (CountingOutputCommitter) v1 @@ -669,7 +669,7 @@ public void testVertexCommitFail2_OnVertexSuccess() throws Exception { initDAG(dag); startDAG(dag); VertexImpl v1 = (VertexImpl) dag.getVertex("vertex1"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v1.getState()); CountingOutputCommitter v1OutputCommitter_1 = (CountingOutputCommitter) v1 @@ -707,7 +707,7 @@ public void testVertexKilledWhileCommitting() throws Exception { initDAG(dag); startDAG(dag); VertexImpl v1 = (VertexImpl) dag.getVertex("vertex1"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v1.getState()); // kill dag which will trigger the vertex killed event @@ -746,7 +746,7 @@ public void testVertexRescheduleWhileCommitting() throws Exception { initDAG(dag); startDAG(dag); VertexImpl v1 = (VertexImpl) dag.getVertex("vertex1"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v1.getState()); // reschedule task @@ -786,7 +786,7 @@ public void testVertexRouteEventErrorWhileCommitting() throws Exception { initDAG(dag); startDAG(dag); VertexImpl v1 = (VertexImpl) dag.getVertex("vertex1"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v1.getState()); // reschedule task @@ -831,7 +831,7 @@ public void testVertexInternalErrorWhileCommiting() throws Exception { initDAG(dag); startDAG(dag); VertexImpl v1 = (VertexImpl) dag.getVertex("vertex1"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v1.getState()); // internal error @@ -875,11 +875,11 @@ public void testDAGCommitSucceeded_OnDAGSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.COMMITTING); CountingOutputCommitter v12OutputCommitter = (CountingOutputCommitter) v1 @@ -931,11 +931,11 @@ public void testDAGCommitFail1_OnDAGSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.COMMITTING); @@ -992,11 +992,11 @@ public void testDAGCommitFail2_OnDAGSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.COMMITTING); @@ -1052,11 +1052,11 @@ public void testDAGCommitSucceeded1_OnVertexSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.SUCCEEDED, v1.getState()); Assert.assertEquals(VertexState.SUCCEEDED, v2.getState()); @@ -1110,11 +1110,11 @@ public void testDAGCommitSucceeded2_OnVertexSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.SUCCEEDED, v1.getState()); Assert.assertEquals(VertexState.SUCCEEDED, v2.getState()); @@ -1170,11 +1170,11 @@ public void testDAGCommitSucceeded3_OnVertexSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.SUCCEEDED, v1.getState()); Assert.assertEquals(VertexState.SUCCEEDED, v2.getState()); @@ -1235,11 +1235,11 @@ public void testDAGCommitFail1_OnVertexSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.SUCCEEDED, v1.getState()); Assert.assertEquals(VertexState.SUCCEEDED, v2.getState()); @@ -1294,11 +1294,11 @@ public void testDAGCommitFail2_OnVertexSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.SUCCEEDED, v1.getState()); Assert.assertEquals(VertexState.SUCCEEDED, v2.getState()); @@ -1353,11 +1353,11 @@ public void testDAGCommitFail3_OnVertexSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.SUCCEEDED, v1.getState()); Assert.assertEquals(VertexState.SUCCEEDED, v2.getState()); @@ -1416,11 +1416,11 @@ public void testDAGCommitFail4_OnVertexSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.SUCCEEDED, v1.getState()); Assert.assertEquals(VertexState.SUCCEEDED, v2.getState()); @@ -1475,11 +1475,11 @@ public void testDAGInternalErrorWhileCommiting_OnDAGSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.COMMITTING); dag.handle(new DAGEvent(dag.getID(), DAGEventType.INTERNAL_ERROR)); @@ -1537,11 +1537,11 @@ private void _testDAGTerminatedWhileCommitting1_OnDAGSuccess(DAGTerminationCause VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.COMMITTING); dag.handle(new DAGEventTerminateDag(dag.getID(), terminationCause, null)); @@ -1602,11 +1602,11 @@ private void _testDAGTerminatedWhileCommitting1_OnVertexSuccess(DAGTerminationCa VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v3.getState()); // dag is still in RUNNING because v3 has not completed @@ -1672,11 +1672,11 @@ private void _testDAGKilledWhileRunning_OnVertexSuccess(DAGTerminationCause term VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v3.getState()); // dag is still in RUNNING because v3 has not completed @@ -1731,11 +1731,11 @@ public void testDAGCommitVertexRerunWhileCommitting_OnDAGSuccess() throws Except VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.COMMITTING); TezTaskID newTaskId = TezTaskID.getInstance(v1.getVertexId(), 1); @@ -1790,11 +1790,11 @@ public void testDAGCommitInternalErrorWhileCommiting_OnDAGSuccess() throws Excep VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.COMMITTING); dag.handle(new DAGEvent(dag.getID(), DAGEventType.INTERNAL_ERROR)); @@ -1840,11 +1840,11 @@ public void testVertexGroupCommitFinishedEventFail_OnVertexSuccess() throws Exce VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); CountingOutputCommitter v12OutputCommitter = (CountingOutputCommitter) v1 .getOutputCommitter("v12Out"); @@ -1895,11 +1895,11 @@ public void testDAGCommitStartedEventFail_OnDAGSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.FAILED); Assert.assertEquals(DAGTerminationCause.RECOVERY_FAILURE, dag.getTerminationCause()); @@ -1959,11 +1959,11 @@ private void _testCommitCanceled_OnDAGSuccess(DAGTerminationCause terminationCau VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.COMMITTING); // mean the commits have been submitted to ThreadPool @@ -2110,7 +2110,7 @@ public void verifyDAGFinishedEvent(TezDAGID dagId, int expectedTimes) { for (HistoryEvent event : historyEvents) { if (event.getEventType() == HistoryEventType.DAG_FINISHED) { DAGFinishedEvent startedEvent = (DAGFinishedEvent)event; - if (startedEvent.getDagID().equals(dagId)) { + if (startedEvent.getDAGID().equals(dagId)) { actualTimes ++; } } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java index 2f2b3b8b8f..f2b16a9094 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java @@ -221,7 +221,7 @@ private DAGImpl chooseDAG(TezDAGID curDAGId) { private class DagEventDispatcher implements EventHandler { @Override public void handle(DAGEvent event) { - DAGImpl dag = chooseDAG(event.getDAGId()); + DAGImpl dag = chooseDAG(event.getDAGID()); dag.handle(event); } } @@ -230,9 +230,9 @@ private class TaskEventDispatcher implements EventHandler { @SuppressWarnings("unchecked") @Override public void handle(TaskEvent event) { - TezDAGID id = event.getTaskID().getVertexID().getDAGId(); + TezDAGID id = event.getDAGID(); DAGImpl handler = chooseDAG(id); - Vertex vertex = handler.getVertex(event.getTaskID().getVertexID()); + Vertex vertex = handler.getVertex(event.getVertexID()); Task task = vertex.getTask(event.getTaskID()); ((EventHandler)task).handle(event); } @@ -249,10 +249,10 @@ public void handle(TaskAttemptEvent event) { private class TaskAttemptEventDisptacher2 implements EventHandler { @Override public void handle(TaskAttemptEvent event) { - TezDAGID id = event.getTaskAttemptID().getTaskID().getVertexID().getDAGId(); + TezDAGID id = event.getDAGID(); DAGImpl handler = chooseDAG(id); - Vertex vertex = handler.getVertex(event.getTaskAttemptID().getTaskID().getVertexID()); - Task task = vertex.getTask(event.getTaskAttemptID().getTaskID()); + Vertex vertex = handler.getVertex(event.getVertexID()); + Task task = vertex.getTask(event.getTaskID()); TaskAttempt ta = task.getAttempt(event.getTaskAttemptID()); ((EventHandler)ta).handle(event); } @@ -264,9 +264,9 @@ private class VertexEventDispatcher @SuppressWarnings("unchecked") @Override public void handle(VertexEvent event) { - TezDAGID id = event.getVertexId().getDAGId(); + TezDAGID id = event.getDAGID(); DAGImpl handler = chooseDAG(id); - Vertex vertex = handler.getVertex(event.getVertexId()); + Vertex vertex = handler.getVertex(event.getVertexID()); ((EventHandler) vertex).handle(event); } } @@ -1174,14 +1174,14 @@ public void testEdgeManager_RouteDataMovementEventToDestination() { VertexImpl v2 = (VertexImpl)dagWithCustomEdge.getVertex("vertex2"); dispatcher.await(); Task t1= v2.getTask(0); - TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskId(), 0)); + TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskID(), 0)); DataMovementEvent daEvent = DataMovementEvent.create(ByteBuffer.wrap(new byte[0])); TezEvent tezEvent = new TezEvent(daEvent, - new EventMetaData(EventProducerConsumerType.INPUT, "vertex1", "vertex2", ta1.getID())); + new EventMetaData(EventProducerConsumerType.INPUT, "vertex1", "vertex2", ta1.getTaskAttemptID())); dispatcher.getEventHandler().handle(new VertexEventRouteEvent(v2.getVertexId(), Lists.newArrayList(tezEvent))); dispatcher.await(); - v2.getTaskAttemptTezEvents(ta1.getID(), 0, 0, 1000); + v2.getTaskAttemptTezEvents(ta1.getTaskAttemptID(), 0, 0, 1000); dispatcher.await(); Assert.assertEquals(VertexState.FAILED, v2.getState()); @@ -1207,11 +1207,11 @@ public void testEdgeManager_RouteDataMovementEventToDestinationWithLegacyRouting dispatcher.await(); Task t1= v2.getTask(0); - TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskId(), 0)); + TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskID(), 0)); DataMovementEvent daEvent = DataMovementEvent.create(ByteBuffer.wrap(new byte[0])); TezEvent tezEvent = new TezEvent(daEvent, - new EventMetaData(EventProducerConsumerType.INPUT, "vertex1", "vertex2", ta1.getID())); + new EventMetaData(EventProducerConsumerType.INPUT, "vertex1", "vertex2", ta1.getTaskAttemptID())); dispatcher.getEventHandler().handle( new VertexEventRouteEvent(v2.getVertexId(), Lists.newArrayList(tezEvent))); dispatcher.await(); @@ -1239,13 +1239,13 @@ public void testEdgeManager_RouteInputSourceTaskFailedEventToDestinationLegacyRo dispatcher.await(); Task t1= v2.getTask(0); - TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskId(), 0)); + TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskID(), 0)); InputFailedEvent ifEvent = InputFailedEvent.create(0, 1); TezEvent tezEvent = new TezEvent(ifEvent, - new EventMetaData(EventProducerConsumerType.INPUT,"vertex1", "vertex2", ta1.getID())); + new EventMetaData(EventProducerConsumerType.INPUT,"vertex1", "vertex2", ta1.getTaskAttemptID())); dispatcher.getEventHandler().handle(new VertexEventRouteEvent(v2.getVertexId(), Lists.newArrayList(tezEvent))); dispatcher.await(); - v2.getTaskAttemptTezEvents(ta1.getID(), 0, 0, 1000); + v2.getTaskAttemptTezEvents(ta1.getTaskAttemptID(), 0, 0, 1000); dispatcher.await(); Assert.assertEquals(VertexState.FAILED, v2.getState()); @@ -1270,11 +1270,11 @@ public void testEdgeManager_GetNumDestinationConsumerTasks() { dispatcher.await(); Task t1= v2.getTask(0); - TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskId(), 0)); + TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskID(), 0)); InputReadErrorEvent ireEvent = InputReadErrorEvent.create("", 0, 0); TezEvent tezEvent = new TezEvent(ireEvent, - new EventMetaData(EventProducerConsumerType.INPUT,"vertex2", "vertex1", ta1.getID())); + new EventMetaData(EventProducerConsumerType.INPUT,"vertex2", "vertex1", ta1.getTaskAttemptID())); dispatcher.getEventHandler().handle( new VertexEventRouteEvent(v2.getVertexId(), Lists.newArrayList(tezEvent))); dispatcher.await(); @@ -1301,10 +1301,10 @@ public void testEdgeManager_RouteInputErrorEventToSource() { dispatcher.await(); Task t1= v2.getTask(0); - TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskId(), 0)); + TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskID(), 0)); InputReadErrorEvent ireEvent = InputReadErrorEvent.create("", 0, 0); TezEvent tezEvent = new TezEvent(ireEvent, - new EventMetaData(EventProducerConsumerType.INPUT,"vertex2", "vertex1", ta1.getID())); + new EventMetaData(EventProducerConsumerType.INPUT,"vertex2", "vertex1", ta1.getTaskAttemptID())); dispatcher.getEventHandler().handle(new VertexEventRouteEvent(v2.getVertexId(), Lists.newArrayList(tezEvent))); dispatcher.await(); // diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java index 95ea8a030e..acbef67dc0 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java @@ -209,7 +209,7 @@ private class TaskEventDispatcher implements EventHandler { @SuppressWarnings("unchecked") @Override public void handle(TaskEvent event) { - TaskImpl task = (TaskImpl) dag.getVertex(event.getTaskID().getVertexID()) + TaskImpl task = (TaskImpl) dag.getVertex(event.getVertexID()) .getTask(event.getTaskID()); task.handle(event); } @@ -220,8 +220,7 @@ private class TaskAttemptEventDispatcher implements EventHandler { @Override public void handle(TaskAttemptEvent event) { - Vertex vertex = dag.getVertex(event.getTaskAttemptID().getTaskID() - .getVertexID()); + Vertex vertex = dag.getVertex(event.getVertexID()); Task task = vertex.getTask(event.getTaskAttemptID().getTaskID()); TaskAttempt ta = task.getAttempt(event.getTaskAttemptID()); ((EventHandler) ta).handle(event); @@ -233,7 +232,7 @@ private class VertexEventDispatcher implements EventHandler { @SuppressWarnings("unchecked") @Override public void handle(VertexEvent event) { - VertexImpl vertex = (VertexImpl) dag.getVertex(event.getVertexId()); + VertexImpl vertex = (VertexImpl) dag.getVertex(event.getVertexID()); vertex.handle(event); } } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGScheduler.java index 07c361aded..e2df050799 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGScheduler.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGScheduler.java @@ -39,7 +39,7 @@ public class TestDAGScheduler { - class MockEventHandler implements EventHandler { + static class MockEventHandler implements EventHandler { TaskAttemptEventSchedule event; List events = Lists.newLinkedList(); @Override @@ -114,17 +114,17 @@ public void testConcurrencyLimit() { // schedule beyond limit and it gets scheduled mockAttempt = mock(TaskAttempt.class); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId0, 0)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId0, 0)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(1, mockEventHandler.events.size()); mockAttempt = mock(TaskAttempt.class); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId0, 1)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId0, 1)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(2, mockEventHandler.events.size()); mockAttempt = mock(TaskAttempt.class); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId0, 2)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId0, 2)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(3, mockEventHandler.events.size()); @@ -139,34 +139,34 @@ public void testConcurrencyLimit() { // schedule beyond limit and it gets buffered mockAttempt = mock(TaskAttempt.class); mockAttempts.add(mockAttempt); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(scheduled+1, mockEventHandler.events.size()); // scheduled - Assert.assertEquals(mockAttempts.get(scheduled).getID(), + Assert.assertEquals(mockAttempts.get(scheduled).getTaskAttemptID(), mockEventHandler.events.get(scheduled).getTaskAttemptID()); // matches order scheduled++; mockAttempt = mock(TaskAttempt.class); mockAttempts.add(mockAttempt); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(scheduled+1, mockEventHandler.events.size()); // scheduled - Assert.assertEquals(mockAttempts.get(scheduled).getID(), + Assert.assertEquals(mockAttempts.get(scheduled).getTaskAttemptID(), mockEventHandler.events.get(scheduled).getTaskAttemptID()); // matches order scheduled++; mockAttempt = mock(TaskAttempt.class); mockAttempts.add(mockAttempt); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(scheduled, mockEventHandler.events.size()); // buffered mockAttempt = mock(TaskAttempt.class); mockAttempts.add(mockAttempt); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(scheduled, mockEventHandler.events.size()); // buffered @@ -174,14 +174,14 @@ public void testConcurrencyLimit() { scheduler.taskCompleted(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_COMPLETED, mockAttempts.get(completed++))); Assert.assertEquals(scheduled+1, mockEventHandler.events.size()); // scheduled - Assert.assertEquals(mockAttempts.get(scheduled).getID(), + Assert.assertEquals(mockAttempts.get(scheduled).getTaskAttemptID(), mockEventHandler.events.get(scheduled).getTaskAttemptID()); // matches order scheduled++; scheduler.taskCompleted(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_COMPLETED, mockAttempts.get(completed++))); Assert.assertEquals(scheduled+1, mockEventHandler.events.size()); // scheduled - Assert.assertEquals(mockAttempts.get(scheduled).getID(), + Assert.assertEquals(mockAttempts.get(scheduled).getTaskAttemptID(), mockEventHandler.events.get(scheduled).getTaskAttemptID()); // matches order scheduled++; @@ -191,14 +191,12 @@ public void testConcurrencyLimit() { mockAttempt = mock(TaskAttempt.class); mockAttempts.add(mockAttempt); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(scheduled+1, mockEventHandler.events.size()); // scheduled - Assert.assertEquals(mockAttempts.get(scheduled).getID(), + Assert.assertEquals(mockAttempts.get(scheduled).getTaskAttemptID(), mockEventHandler.events.get(scheduled).getTaskAttemptID()); // matches order - scheduled++; - } @Test(timeout=5000) @@ -220,7 +218,6 @@ public void testConcurrencyLimitWithKilledNonRunningTask() { mockEventHandler); List mockAttempts = Lists.newArrayList(); - int completed = 0; int requested = 0; int scheduled = 0; scheduler.addVertexConcurrencyLimit(vId0, 1); // effective @@ -228,24 +225,24 @@ public void testConcurrencyLimitWithKilledNonRunningTask() { // schedule beyond limit and it gets buffered mockAttempt = mock(TaskAttempt.class); mockAttempts.add(mockAttempt); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId0, requested++)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId0, requested++)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(scheduled+1, mockEventHandler.events.size()); // scheduled - Assert.assertEquals(mockAttempts.get(scheduled).getID(), + Assert.assertEquals(mockAttempts.get(scheduled).getTaskAttemptID(), mockEventHandler.events.get(scheduled).getTaskAttemptID()); // matches order scheduled++; mockAttempt = mock(TaskAttempt.class); mockAttempts.add(mockAttempt); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId0, requested++)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId0, requested++)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(scheduled, mockEventHandler.events.size()); // buffered mockAttempt = mock(TaskAttempt.class); mockAttempts.add(mockAttempt); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId0, requested++)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId0, requested++)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(scheduled, mockEventHandler.events.size()); // buffered @@ -253,7 +250,7 @@ public void testConcurrencyLimitWithKilledNonRunningTask() { scheduler.taskCompleted(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_COMPLETED, mockAttempts.get(1))); Assert.assertEquals(scheduled, mockEventHandler.events.size()); // buffered - Assert.assertEquals(mockAttempts.get(0).getID(), + Assert.assertEquals(mockAttempts.get(0).getTaskAttemptID(), mockEventHandler.events.get(0).getTaskAttemptID()); // matches order } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGSchedulerNaturalOrderControlled.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGSchedulerNaturalOrderControlled.java index 63137c716a..3b1db4c6d1 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGSchedulerNaturalOrderControlled.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGSchedulerNaturalOrderControlled.java @@ -164,9 +164,9 @@ public void testSourceRequestDelayed() { for (Event raw : args.getAllValues()) { TaskAttemptEventSchedule event = (TaskAttemptEventSchedule) raw; if (count < vertices[2].getTotalTasks() - 3) { - assertEquals(2, event.getTaskAttemptID().getTaskID().getVertexID().getId()); + assertEquals(2, event.getVertexID().getId()); } else { - assertEquals(4, event.getTaskAttemptID().getTaskID().getVertexID().getId()); + assertEquals(4, event.getVertexID().getId()); } count++; } @@ -364,7 +364,7 @@ private TaskAttempt createTaskAttempt(TezVertexID vertexId, int taskIdInt, int a TaskAttempt taskAttempt = mock(TaskAttempt.class); TezTaskID taskId = TezTaskID.getInstance(vertexId, taskIdInt); TezTaskAttemptID taskAttemptId = TezTaskAttemptID.getInstance(taskId, attemptIdInt); - doReturn(taskAttemptId).when(taskAttempt).getID(); + doReturn(taskAttemptId).when(taskAttempt).getTaskAttemptID(); doReturn(vertexId).when(taskAttempt).getVertexID(); return taskAttempt; } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestEdge.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestEdge.java index 1143395204..c9c99f5630 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestEdge.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestEdge.java @@ -232,9 +232,9 @@ private void verifyEvents(TezTaskAttemptID srcTAID, LinkedHashMap tasks) { for (Task task : tasks) { - TezTaskID taskID = task.getTaskId(); + TezTaskID taskID = task.getTaskID(); reset(task); - doReturn(taskID).when(task).getTaskId(); + doReturn(taskID).when(task).getTaskID(); } } @@ -243,7 +243,7 @@ private LinkedHashMap mockTasks(TezVertexID vertexID, int numTa for (int i = 0 ; i < numTasks ; i++) { Task task = mock(Task.class); TezTaskID taskID = TezTaskID.getInstance(vertexID, i); - doReturn(taskID).when(task).getTaskId(); + doReturn(taskID).when(task).getTaskID(); tasks.put(taskID, task); } return tasks; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java index 7a2a05fb26..dc59b4615c 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java @@ -425,7 +425,7 @@ taListener, taskConf, new SystemClock(), taImpl.handle(new TaskAttemptEventKillRequest(taskAttemptID, null, TaskAttemptTerminationCause.TERMINATED_BY_CLIENT)); assertEquals(TaskAttemptStateInternal.KILL_IN_PROGRESS, taImpl.getInternalState()); - taImpl.handle(new TaskAttemptEventTezEventUpdate(taImpl.getID(), Collections.EMPTY_LIST)); + taImpl.handle(new TaskAttemptEventTezEventUpdate(taImpl.getTaskAttemptID(), Collections.EMPTY_LIST)); assertFalse( "InternalError occurred trying to handle TA_TEZ_EVENT_UPDATE in KILL_IN_PROGRESS state", eventHandler.internalError); @@ -480,7 +480,7 @@ public void testContainerTerminationWhileRunning() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -585,7 +585,7 @@ public void testContainerTerminatedWhileRunning() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); taImpl.handle(new TaskAttemptEventSubmitted(taskAttemptID, contId)); @@ -674,7 +674,7 @@ public void testContainerTerminatedAfterSuccess() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -764,7 +764,7 @@ public void testLastDataEventRecording() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); taImpl.handle(new TaskAttemptEventSubmitted(taskAttemptID, contId)); @@ -849,7 +849,7 @@ public void testFailure() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -879,7 +879,7 @@ taListener, taskConf, new SystemClock(), assertEquals(TaskAttemptTerminationCause.APPLICATION_ERROR, taImpl.getTerminationCause()); assertEquals(TaskAttemptStateInternal.FAIL_IN_PROGRESS, taImpl.getInternalState()); - taImpl.handle(new TaskAttemptEventTezEventUpdate(taImpl.getID(), Collections.EMPTY_LIST)); + taImpl.handle(new TaskAttemptEventTezEventUpdate(taImpl.getTaskAttemptID(), Collections.EMPTY_LIST)); assertFalse( "InternalError occurred trying to handle TA_TEZ_EVENT_UPDATE in FAIL_IN_PROGRESS state", eventHandler.internalError); @@ -956,7 +956,7 @@ public void testFailureFatalError() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -986,7 +986,7 @@ taListener, taskConf, new SystemClock(), assertEquals(TaskAttemptTerminationCause.APPLICATION_ERROR, taImpl.getTerminationCause()); assertEquals(TaskAttemptStateInternal.FAIL_IN_PROGRESS, taImpl.getInternalState()); - taImpl.handle(new TaskAttemptEventTezEventUpdate(taImpl.getID(), Collections.EMPTY_LIST)); + taImpl.handle(new TaskAttemptEventTezEventUpdate(taImpl.getTaskAttemptID(), Collections.EMPTY_LIST)); assertFalse( "InternalError occurred trying to handle TA_TEZ_EVENT_UPDATE in FAIL_IN_PROGRESS state", eventHandler.internalError); @@ -1064,7 +1064,7 @@ public void testProgressTimeStampUpdate() throws Exception { taListener, taskConf, mockClock, mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -1080,7 +1080,7 @@ public void testProgressTimeStampUpdate() throws Exception { verify(eventHandler, atLeast(1)).handle(arg.capture()); if (arg.getValue() instanceof TaskAttemptEventAttemptFailed) { TaskAttemptEventAttemptFailed fEvent = (TaskAttemptEventAttemptFailed) arg.getValue(); - assertEquals(taImpl.getID(), fEvent.getTaskAttemptID()); + assertEquals(taImpl.getTaskAttemptID(), fEvent.getTaskAttemptID()); assertEquals(TaskAttemptTerminationCause.NO_PROGRESS, fEvent.getTerminationCause()); taImpl.handle(fEvent); fail("Should not fail since the timestamps do not differ by progress interval config"); @@ -1135,7 +1135,7 @@ public void testStatusUpdateWithNullCounters() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); taImpl.handle(new TaskAttemptEventSubmitted(taskAttemptID, contId)); @@ -1203,7 +1203,7 @@ public void testProgressAfterSubmit() throws Exception { taListener, taskConf, mockClock, mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); mockClock.incrementTime(20L); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -1262,7 +1262,7 @@ public void testNoProgressFail() throws Exception { taListener, taskConf, mockClock, mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -1295,7 +1295,7 @@ public void testNoProgressFail() throws Exception { // failed event sent to self verify(eventHandler, atLeast(1)).handle(arg.capture()); TaskAttemptEventAttemptFailed fEvent = (TaskAttemptEventAttemptFailed) arg.getValue(); - assertEquals(taImpl.getID(), fEvent.getTaskAttemptID()); + assertEquals(taImpl.getTaskAttemptID(), fEvent.getTaskAttemptID()); assertEquals(TaskAttemptTerminationCause.NO_PROGRESS, fEvent.getTerminationCause()); assertEquals(TaskFailureType.NON_FATAL, fEvent.getTaskFailureType()); taImpl.handle(fEvent); @@ -1377,7 +1377,7 @@ public void testCompletedAtSubmitted() throws ServicePluginException { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -1460,7 +1460,7 @@ public void testSuccess() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -1548,7 +1548,7 @@ public void testContainerPreemptedAfterSuccess() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); @@ -1640,7 +1640,7 @@ public void testNodeFailedNonLeafVertex() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); @@ -1699,7 +1699,7 @@ taListener, taskConf, new SystemClock(), assertEquals(TaskAttemptTerminationCause.NODE_FAILED, taImpl.getTerminationCause()); assertEquals(TaskAttemptStateInternal.KILLED, taImpl.getInternalState()); - taImpl.handle(new TaskAttemptEventTezEventUpdate(taImpl.getID(), Collections.EMPTY_LIST)); + taImpl.handle(new TaskAttemptEventTezEventUpdate(taImpl.getTaskAttemptID(), Collections.EMPTY_LIST)); assertFalse( "InternalError occurred trying to handle TA_TEZ_EVENT_UPDATE in KILLED state", eventHandler.internalError); @@ -1747,7 +1747,7 @@ public void testNodeFailedLeafVertex() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), true); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); @@ -1847,7 +1847,7 @@ public void testMultipleOutputFailed() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); taImpl.handle(new TaskAttemptEventSubmitted(taskAttemptID, contId)); @@ -1876,6 +1876,7 @@ taListener, taskConf, new SystemClock(), TezTaskID destTaskID = mock(TezTaskID.class); TezVertexID destVertexID = mock(TezVertexID.class); when(mockDestId1.getTaskID()).thenReturn(destTaskID); + when(mockDestId1.getVertexID()).thenReturn(destVertexID); when(destTaskID.getVertexID()).thenReturn(destVertexID); Vertex destVertex = mock(VertexImpl.class); when(destVertex.getRunningTasks()).thenReturn(11); @@ -1901,14 +1902,14 @@ taListener, taskConf, new SystemClock(), destTaskID = mock(TezTaskID.class); destVertexID = mock(TezVertexID.class); when(mockDestId2.getTaskID()).thenReturn(destTaskID); + when(mockDestId2.getVertexID()).thenReturn(destVertexID); when(destTaskID.getVertexID()).thenReturn(destVertexID); destVertex = mock(VertexImpl.class); when(destVertex.getRunningTasks()).thenReturn(11); when(mockDAG.getVertex(destVertexID)).thenReturn(destVertex); taImpl.handle(new TaskAttemptEventOutputFailed(taskAttemptID, tzEvent, 11)); - assertEquals("Task attempt is not in FAILED state", taImpl.getState(), - TaskAttemptState.FAILED); + assertEquals("Task attempt is not in FAILED state", TaskAttemptState.FAILED, taImpl.getState()); assertEquals(TaskAttemptTerminationCause.OUTPUT_LOST, taImpl.getTerminationCause()); // verify unregister is not invoked again verify(mockHeartbeatHandler, times(1)).unregister(taskAttemptID); @@ -1949,7 +1950,7 @@ taListener, taskConf, new SystemClock(), taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID2 = taImpl2.getID(); + TezTaskAttemptID taskAttemptID2 = taImpl2.getTaskAttemptID(); taImpl2.handle(new TaskAttemptEventSchedule(taskAttemptID2, 0, 0)); taImpl2.handle(new TaskAttemptEventSubmitted(taskAttemptID2, contId)); @@ -1964,6 +1965,7 @@ taListener, taskConf, new SystemClock(), mockMeta = mock(EventMetaData.class); mockDestId1 = mock(TezTaskAttemptID.class); when(mockDestId1.getTaskID()).thenReturn(destTaskID); + when(mockDestId1.getVertexID()).thenReturn(destVertexID); when(mockMeta.getTaskAttemptID()).thenReturn(mockDestId1); tzEvent = new TezEvent(mockReEvent, mockMeta); //This should fail even when MAX_ALLOWED_OUTPUT_FAILURES_FRACTION is within limits, as @@ -1991,7 +1993,7 @@ taListener, taskConf, new SystemClock(), taListener, taskConf, mockClock, mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID3 = taImpl3.getID(); + TezTaskAttemptID taskAttemptID3 = taImpl3.getTaskAttemptID(); taImpl3.handle(new TaskAttemptEventSchedule(taskAttemptID3, 0, 0)); taImpl3.handle(new TaskAttemptEventSubmitted(taskAttemptID3, contId)); @@ -2006,6 +2008,7 @@ taListener, taskConf, new SystemClock(), mockMeta = mock(EventMetaData.class); mockDestId1 = mock(TezTaskAttemptID.class); when(mockDestId1.getTaskID()).thenReturn(destTaskID); + when(mockDestId1.getVertexID()).thenReturn(destVertexID); when(mockMeta.getTaskAttemptID()).thenReturn(mockDestId1); tzEvent = new TezEvent(mockReEvent, mockMeta); when(mockClock.getTime()).thenReturn(1000L); @@ -2073,7 +2076,7 @@ public void testTAFailureBasedOnRunningTasks() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); taImpl.handle(new TaskAttemptEventSubmitted(taskAttemptID, contId)); @@ -2102,6 +2105,7 @@ taListener, taskConf, new SystemClock(), TezTaskID destTaskID = mock(TezTaskID.class); TezVertexID destVertexID = mock(TezVertexID.class); when(mockDestId1.getTaskID()).thenReturn(destTaskID); + when(mockDestId1.getVertexID()).thenReturn(destVertexID); when(destTaskID.getVertexID()).thenReturn(destVertexID); Vertex destVertex = mock(VertexImpl.class); when(destVertex.getRunningTasks()).thenReturn(5); @@ -2111,8 +2115,7 @@ taListener, taskConf, new SystemClock(), taImpl.handle(new TaskAttemptEventOutputFailed(taskAttemptID, tzEvent, 11)); // failure threshold is met due to running tasks. state is FAILED - assertEquals("Task attempt is not in FAILED state", taImpl.getState(), - TaskAttemptState.FAILED); + assertEquals("Task attempt is not in FAILED state", TaskAttemptState.FAILED, taImpl.getState()); } @SuppressWarnings("deprecation") @@ -2157,7 +2160,7 @@ taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), true); Assert.assertEquals(TaskAttemptStateInternal.NEW, taImpl.getInternalState()); - taImpl.handle(new TaskAttemptEventKillRequest(taImpl.getID(), "kill it", + taImpl.handle(new TaskAttemptEventKillRequest(taImpl.getTaskAttemptID(), "kill it", TaskAttemptTerminationCause.TERMINATED_BY_CLIENT)); Assert.assertEquals(TaskAttemptStateInternal.KILLED, taImpl.getInternalState()); @@ -2190,7 +2193,7 @@ private void testMapTaskFailingForFetchFailureType(boolean isLocalFetch, InputReadErrorEvent.create("", 0, 1, 1, isLocalFetch, isDiskErrorAtSource, null); TezTaskAttemptID destTaskAttemptId = mock(TezTaskAttemptID.class); when(destTaskAttemptId.getTaskID()).thenReturn(mock(TezTaskID.class)); - when(destTaskAttemptId.getTaskID().getVertexID()).thenReturn(mock(TezVertexID.class)); + when(destTaskAttemptId.getVertexID()).thenReturn(mock(TezVertexID.class)); when(appCtx.getCurrentDAG()).thenReturn(mock(DAG.class)); when(appCtx.getCurrentDAG().getVertex(Mockito.any(TezVertexID.class))) .thenReturn(mock(Vertex.class)); @@ -2203,7 +2206,7 @@ private void testMapTaskFailingForFetchFailureType(boolean isLocalFetch, // the event is propagated to map task's event handler TaskAttemptEventOutputFailed outputFailedEvent = - new TaskAttemptEventOutputFailed(sourceAttempt.getID(), tezEvent, 11); + new TaskAttemptEventOutputFailed(sourceAttempt.getTaskAttemptID(), tezEvent, 11); Assert.assertEquals(TaskAttemptStateInternal.NEW, sourceAttempt.getInternalState()); TaskAttemptStateInternal resultState = new TaskAttemptImpl.OutputReportedFailedTransition() @@ -2236,13 +2239,13 @@ public void testMapTaskIsBlamedByDownstreamAttemptsFromDifferentHosts() { when(mockMeta.getTaskAttemptID()).thenReturn(destTaskAttemptId); // mapper task succeeded earlier - sourceAttempt.handle(new TaskAttemptEvent(sourceAttempt.getID(), TaskAttemptEventType.TA_DONE)); + sourceAttempt.handle(new TaskAttemptEvent(sourceAttempt.getTaskAttemptID(), TaskAttemptEventType.TA_DONE)); Assert.assertEquals(TaskAttemptStateInternal.SUCCEEDED, sourceAttempt.getInternalState()); // the event is propagated to map task's event handler TezEvent tezEvent = new TezEvent(inputReadErrorEvent1, mockMeta); TaskAttemptEventOutputFailed outputFailedEvent = - new TaskAttemptEventOutputFailed(sourceAttempt.getID(), tezEvent, 11); + new TaskAttemptEventOutputFailed(sourceAttempt.getTaskAttemptID(), tezEvent, 11); TaskAttemptStateInternal resultState = new TaskAttemptImpl.OutputReportedFailedTransition().transition(sourceAttempt, outputFailedEvent); // SUCCEEDED, as we haven't reached the host limit fraction @@ -2252,7 +2255,7 @@ public void testMapTaskIsBlamedByDownstreamAttemptsFromDifferentHosts() { // the second event is propagated to map task's event handler TezEvent tezEvent2 = new TezEvent(inputReadErrorEvent2, mockMeta); TaskAttemptEventOutputFailed outputFailedEvent2 = - new TaskAttemptEventOutputFailed(sourceAttempt.getID(), tezEvent2, 11); + new TaskAttemptEventOutputFailed(sourceAttempt.getTaskAttemptID(), tezEvent2, 11); TaskAttemptStateInternal resultState2 = new TaskAttemptImpl.OutputReportedFailedTransition().transition(sourceAttempt, outputFailedEvent2); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java index a28e786c6e..f4e40f7333 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java @@ -386,9 +386,9 @@ public void testKillRunningTask() { LOG.info("--- START: testKillRunningTask ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); killTask(taskId); - mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertEquals(TaskStateInternal.KILLED, mockTask.getInternalState()); verifyOutgoingEvents(eventHandler.events, VertexEventType.V_TASK_COMPLETED); } @@ -398,20 +398,20 @@ public void testTooManyFailedAttempts() { LOG.info("--- START: testTooManyFailedAttempts ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId, TaskState.SCHEDULED); - launchTaskAttempt(mockTask.getLastAttempt().getID()); - failRunningTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); + failRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); scheduleTaskAttempt(taskId, TaskState.RUNNING); - launchTaskAttempt(mockTask.getLastAttempt().getID()); - failRunningTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); + failRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); scheduleTaskAttempt(taskId, TaskState.RUNNING); - launchTaskAttempt(mockTask.getLastAttempt().getID()); - failRunningTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); + failRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); scheduleTaskAttempt(taskId, TaskState.RUNNING); - launchTaskAttempt(mockTask.getLastAttempt().getID()); - failRunningTaskAttempt(mockTask.getLastAttempt().getID(), false); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); + failRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID(), false); assertEquals(TaskStateInternal.FAILED, mockTask.getInternalState()); verifyOutgoingEvents(eventHandler.events, VertexEventType.V_TASK_COMPLETED); @@ -431,11 +431,11 @@ public void testTooManyAttempts() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId, TaskState.SCHEDULED); // attempt_0 - launchTaskAttempt(mockTask.getLastAttempt().getID()); - killRunningTaskAttempt(mockTask.getLastAttempt().getID(), TaskState.RUNNING, 1); // attempt_1 + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); + killRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID(), TaskState.RUNNING, 1); // attempt_1 - launchTaskAttempt(mockTask.getLastAttempt().getID()); - killRunningTaskAttempt(mockTask.getLastAttempt().getID(), TaskState.FAILED, 2); // attempt_2 -> reached 3 + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); + killRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID(), TaskState.FAILED, 2); // attempt_2 -> reached 3 assertEquals(TaskStateInternal.FAILED, mockTask.getInternalState()); verifyOutgoingEvents(eventHandler.events, VertexEventType.V_TASK_COMPLETED); @@ -446,9 +446,9 @@ public void testFailedAttemptWithFatalError() { LOG.info("--- START: testFailedAttemptWithFatalError ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId, TaskState.SCHEDULED); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); mockTask.handle( - createTaskTAFailedEvent(mockTask.getLastAttempt().getID(), TaskFailureType.FATAL, null)); + createTaskTAFailedEvent(mockTask.getLastAttempt().getTaskAttemptID(), TaskFailureType.FATAL, null)); assertEquals(TaskStateInternal.FAILED, mockTask.getInternalState()); assertEquals(1, mockTask.failedAttempts); @@ -460,11 +460,11 @@ public void testKillRunningTaskPreviousKilledAttempts() { LOG.info("--- START: testKillRunningTaskPreviousKilledAttempts ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); - killRunningTaskAttempt(mockTask.getLastAttempt().getID(), TaskState.RUNNING); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); + killRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID(), TaskState.RUNNING); assertEquals(TaskStateInternal.RUNNING, mockTask.getInternalState()); killTask(taskId); - mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertEquals(TaskStateInternal.KILLED, mockTask.getInternalState()); verifyOutgoingEvents(eventHandler.events, VertexEventType.V_TASK_COMPLETED); @@ -478,9 +478,9 @@ public void testKillRunningTaskButAttemptSucceeds() { LOG.info("--- START: testKillRunningTaskButAttemptSucceeds ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); killTask(taskId); - mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertEquals(TaskStateInternal.KILLED, mockTask.getInternalState()); } @@ -492,9 +492,9 @@ public void testKillRunningTaskButAttemptFails() { LOG.info("--- START: testKillRunningTaskButAttemptFails ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); killTask(taskId); - mockTask.handle(createTaskTAFailedEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAFailedEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertEquals(TaskStateInternal.KILLED, mockTask.getInternalState()); } @@ -507,8 +507,8 @@ public void testKillScheduledTaskAttempt() { LOG.info("--- START: testKillScheduledTaskAttempt ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getID(); - killScheduledTaskAttempt(mockTask.getLastAttempt().getID()); + TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getTaskAttemptID(); + killScheduledTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); // last killed attempt should be causal TA of next attempt Assert.assertEquals(lastTAId, mockTask.getLastAttempt().getSchedulingCausalTA()); } @@ -522,7 +522,7 @@ public void testLaunchTaskAttempt() { LOG.info("--- START: testLaunchTaskAttempt ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); } @Test(timeout = 5000) @@ -534,9 +534,9 @@ public void testKillRunningTaskAttempt() { LOG.info("--- START: testKillRunningTaskAttempt ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getID(); - launchTaskAttempt(mockTask.getLastAttempt().getID()); - killRunningTaskAttempt(mockTask.getLastAttempt().getID(), TaskState.RUNNING); + TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getTaskAttemptID(); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); + killRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID(), TaskState.RUNNING); // last killed attempt should be causal TA of next attempt Assert.assertEquals(lastTAId, mockTask.getLastAttempt().getSchedulingCausalTA()); } @@ -550,9 +550,9 @@ public void testKillTaskAttemptServiceBusy() { LOG.info("--- START: testKillTaskAttemptServiceBusy ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); mockTask.handle(createTaskTAKilledEvent( - mockTask.getLastAttempt().getID(), new ServiceBusyEvent())); + mockTask.getLastAttempt().getTaskAttemptID(), new ServiceBusyEvent())); assertTaskRunningState(); verify(mockTask.getVertex(), times(0)).incrementKilledTaskAttemptCount(); verify(mockTask.getVertex(), times(1)).incrementRejectedTaskAttemptCount(); @@ -566,15 +566,15 @@ public void testKilledAttemptAtTaskKilled() { LOG.info("--- START: testKilledAttemptAtTaskKilled ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); killTask(taskId); - mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertEquals(TaskStateInternal.KILLED, mockTask.getInternalState()); // Send duplicate kill for same attempt // This will not happen in practice but this is to simulate handling // of killed attempts in killed state. - mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertEquals(TaskStateInternal.KILLED, mockTask.getInternalState()); } @@ -588,12 +588,12 @@ public void testKilledAttemptAtTaskFailed() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); for (int i = 0; i < mockTask.maxFailedAttempts; ++i) { - mockTask.handle(createTaskTAFailedEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAFailedEvent(mockTask.getLastAttempt().getTaskAttemptID())); } assertEquals(TaskStateInternal.FAILED, mockTask.getInternalState()); // Send kill for an attempt - mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertEquals(TaskStateInternal.FAILED, mockTask.getInternalState()); } @@ -609,7 +609,7 @@ public void testFetchedEventsModifyUnderlyingList() { scheduleTaskAttempt(taskId); sendTezEventsToTask(taskId, 2); TezTaskAttemptID attemptID = mockTask.getAttemptList().iterator().next() - .getID(); + .getTaskAttemptID(); fetchedList = mockTask.getTaskAttemptTezEvents(attemptID, 0, 100); assertEquals(2, fetchedList.size()); @@ -631,7 +631,7 @@ public void testTaskProgress() { scheduleTaskAttempt(taskId); float progress = 0f; assert (mockTask.getProgress() == progress); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); // update attempt1 progress = 50f; @@ -649,13 +649,13 @@ public void testTaskProgress() { // kill first attempt // should trigger a new attempt // as no successful attempts - failRunningTaskAttempt(mockTask.getLastAttempt().getID()); + failRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); assert (mockTask.getAttemptList().size() == 2); assertEquals(1, mockTask.failedAttempts); verify(mockTask.getVertex(), times(1)).incrementFailedTaskAttemptCount(); assert (mockTask.getProgress() == 0f); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); progress = 50f; updateAttemptProgress(mockTask.getLastAttempt(), progress); assert (mockTask.getProgress() == progress); @@ -665,17 +665,17 @@ public void testTaskProgress() { public void testFailureDuringTaskAttemptCommit() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING); assertTrue("First attempt should commit", - mockTask.canCommit(mockTask.getLastAttempt().getID())); + mockTask.canCommit(mockTask.getLastAttempt().getTaskAttemptID())); // During the task attempt commit there is an exception which causes // the attempt to fail - TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getID(); + TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getTaskAttemptID(); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.FAILED); assertEquals(1, mockTask.getAttemptList().size()); - failRunningTaskAttempt(mockTask.getLastAttempt().getID()); + failRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); assertEquals(2, mockTask.getAttemptList().size()); assertEquals(1, mockTask.failedAttempts); @@ -683,13 +683,13 @@ public void testFailureDuringTaskAttemptCommit() { Assert.assertEquals(lastTAId, mockTask.getLastAttempt().getSchedulingCausalTA()); assertFalse("First attempt should not commit", - mockTask.canCommit(mockTask.getAttemptList().get(0).getID())); + mockTask.canCommit(mockTask.getAttemptList().get(0).getTaskAttemptID())); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING); assertTrue("Second attempt should commit", - mockTask.canCommit(mockTask.getLastAttempt().getID())); + mockTask.canCommit(mockTask.getLastAttempt().getTaskAttemptID())); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.SUCCEEDED); - mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertTaskSucceededState(); } @@ -703,14 +703,14 @@ public void testEventBacklogDuringTaskAttemptCommit() { // simulate // task in scheduled state due to event backlog - real task done and calling canCommit assertFalse("Commit should return false to make running task wait", - mockTask.canCommit(mockTask.getLastAttempt().getID())); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + mockTask.canCommit(mockTask.getLastAttempt().getTaskAttemptID())); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING); assertTrue("Task state in AM is running now. Can commit.", - mockTask.canCommit(mockTask.getLastAttempt().getID())); + mockTask.canCommit(mockTask.getLastAttempt().getTaskAttemptID())); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.SUCCEEDED); - mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertTaskSucceededState(); } @@ -720,13 +720,13 @@ public void testEventBacklogDuringTaskAttemptCommit() { public void testChangeCommitTaskAttempt() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING); - TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getID(); + TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getTaskAttemptID(); // Add a speculative task attempt that succeeds - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING); assertEquals(2, mockTask.getAttemptList().size()); @@ -735,24 +735,24 @@ public void testChangeCommitTaskAttempt() { Assert.assertEquals(lastTAId, mockTask.getLastAttempt().getSchedulingCausalTA()); assertTrue("Second attempt should commit", - mockTask.canCommit(mockTask.getAttemptList().get(1).getID())); + mockTask.canCommit(mockTask.getAttemptList().get(1).getTaskAttemptID())); assertFalse("First attempt should not commit", - mockTask.canCommit(mockTask.getAttemptList().get(0).getID())); + mockTask.canCommit(mockTask.getAttemptList().get(0).getTaskAttemptID())); // During the task attempt commit there is an exception which causes // the second attempt to fail updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.FAILED); - failRunningTaskAttempt(mockTask.getLastAttempt().getID()); + failRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); assertEquals(2, mockTask.getAttemptList().size()); assertFalse("Second attempt should not commit", - mockTask.canCommit(mockTask.getAttemptList().get(1).getID())); + mockTask.canCommit(mockTask.getAttemptList().get(1).getTaskAttemptID())); assertTrue("First attempt should commit", - mockTask.canCommit(mockTask.getAttemptList().get(0).getID())); + mockTask.canCommit(mockTask.getAttemptList().get(0).getTaskAttemptID())); updateAttemptState(mockTask.getAttemptList().get(0), TaskAttemptState.SUCCEEDED); - mockTask.handle(createTaskTASucceededEvent(mockTask.getAttemptList().get(0).getID())); + mockTask.handle(createTaskTASucceededEvent(mockTask.getAttemptList().get(0).getTaskAttemptID())); assertTaskSucceededState(); } @@ -762,15 +762,15 @@ public void testChangeCommitTaskAttempt() { public void testTaskSucceedAndRetroActiveFailure() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING); - mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getTaskAttemptID())); // The task should now have succeeded assertTaskSucceededState(); verify(mockTask.stateChangeNotifier).taskSucceeded(any(String.class), eq(taskId), - eq(mockTask.getLastAttempt().getID().getId())); + eq(mockTask.getLastAttempt().getTaskAttemptID().getId())); ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(DAGHistoryEvent.class); verify(mockHistoryHandler).handle(argumentCaptor.capture()); @@ -789,7 +789,7 @@ public void testTaskSucceedAndRetroActiveFailure() { TaskAttemptEventOutputFailed outputFailedEvent = new TaskAttemptEventOutputFailed(mockDestId, mockTezEvent, 1); mockTask.handle( - createTaskTAFailedEvent(mockTask.getLastAttempt().getID(), TaskFailureType.NON_FATAL, + createTaskTAFailedEvent(mockTask.getLastAttempt().getTaskAttemptID(), TaskFailureType.NON_FATAL, outputFailedEvent)); // The task should still be in the scheduled state @@ -811,19 +811,19 @@ public void testTaskSucceedAndRetroActiveFailure() { public void testTaskSucceedAndRetroActiveKilled() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING); - mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getTaskAttemptID())); // The task should now have succeeded assertTaskSucceededState(); verify(mockTask.stateChangeNotifier).taskSucceeded(any(String.class), eq(taskId), - eq(mockTask.getLastAttempt().getID().getId())); + eq(mockTask.getLastAttempt().getTaskAttemptID().getId())); eventHandler.events.clear(); // Now kill the attempt after it has succeeded - mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getTaskAttemptID())); // The task should still be in the scheduled state assertTaskScheduledState(); @@ -862,25 +862,25 @@ public void testFailedThenSpeculativeFailed() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstAttempt.getID()); + launchTaskAttempt(firstAttempt.getTaskAttemptID()); updateAttemptState(firstAttempt, TaskAttemptState.RUNNING); // Add a speculative task attempt - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(specAttempt.getID()); + launchTaskAttempt(specAttempt.getTaskAttemptID()); updateAttemptState(specAttempt, TaskAttemptState.RUNNING); assertEquals(2, mockTask.getAttemptList().size()); // Fail the first attempt updateAttemptState(firstAttempt, TaskAttemptState.FAILED); - mockTask.handle(createTaskTAFailedEvent(firstAttempt.getID())); + mockTask.handle(createTaskTAFailedEvent(firstAttempt.getTaskAttemptID())); assertEquals(TaskState.FAILED, mockTask.getState()); assertEquals(2, mockTask.getAttemptList().size()); // Now fail the speculative attempt updateAttemptState(specAttempt, TaskAttemptState.FAILED); - mockTask.handle(createTaskTAFailedEvent(specAttempt.getID())); + mockTask.handle(createTaskTAFailedEvent(specAttempt.getTaskAttemptID())); assertEquals(TaskState.FAILED, mockTask.getState()); assertEquals(2, mockTask.getAttemptList().size()); } @@ -897,25 +897,25 @@ public void testFailedThenSpeculativeSucceeded() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstAttempt.getID()); + launchTaskAttempt(firstAttempt.getTaskAttemptID()); updateAttemptState(firstAttempt, TaskAttemptState.RUNNING); // Add a speculative task attempt - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(specAttempt.getID()); + launchTaskAttempt(specAttempt.getTaskAttemptID()); updateAttemptState(specAttempt, TaskAttemptState.RUNNING); assertEquals(2, mockTask.getAttemptList().size()); // Fail the first attempt updateAttemptState(firstAttempt, TaskAttemptState.FAILED); - mockTask.handle(createTaskTAFailedEvent(firstAttempt.getID())); + mockTask.handle(createTaskTAFailedEvent(firstAttempt.getTaskAttemptID())); assertEquals(TaskState.FAILED, mockTask.getState()); assertEquals(2, mockTask.getAttemptList().size()); // Now succeed the speculative attempt updateAttemptState(specAttempt, TaskAttemptState.SUCCEEDED); - mockTask.handle(createTaskTASucceededEvent(specAttempt.getID())); + mockTask.handle(createTaskTASucceededEvent(specAttempt.getTaskAttemptID())); assertEquals(TaskState.FAILED, mockTask.getState()); assertEquals(2, mockTask.getAttemptList().size()); } @@ -932,10 +932,10 @@ public void testKilledBeforeSpeculatedSucceeded() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstAttempt.getID()); + launchTaskAttempt(firstAttempt.getTaskAttemptID()); updateAttemptState(firstAttempt, TaskAttemptState.RUNNING); - mockTask.handle(createTaskTAKilledEvent(firstAttempt.getID())); + mockTask.handle(createTaskTAKilledEvent(firstAttempt.getTaskAttemptID())); assertEquals(TaskStateInternal.RUNNING, mockTask.getInternalState()); // We need to manually override the current node id @@ -945,16 +945,16 @@ public void testKilledBeforeSpeculatedSucceeded() { mockNodeId = null; // Add a speculative task attempt - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); mockNodeId = nodeId; MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(specAttempt.getID()); + launchTaskAttempt(specAttempt.getTaskAttemptID()); updateAttemptState(specAttempt, TaskAttemptState.RUNNING); assertEquals(3, mockTask.getAttemptList().size()); // Now succeed the speculative attempt updateAttemptState(specAttempt, TaskAttemptState.SUCCEEDED); - mockTask.handle(createTaskTASucceededEvent(specAttempt.getID())); + mockTask.handle(createTaskTASucceededEvent(specAttempt.getTaskAttemptID())); assertEquals(TaskState.SUCCEEDED, mockTask.getState()); assertEquals(3, mockTask.getAttemptList().size()); } @@ -964,35 +964,35 @@ public void testKilledAttemptUpdatesDAGScheduler() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstAttempt.getID()); + launchTaskAttempt(firstAttempt.getTaskAttemptID()); updateAttemptState(firstAttempt, TaskAttemptState.RUNNING); // Add a speculative task attempt - mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getID())); + mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getTaskAttemptID())); MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(specAttempt.getID()); + launchTaskAttempt(specAttempt.getTaskAttemptID()); updateAttemptState(specAttempt, TaskAttemptState.RUNNING); assertEquals(2, mockTask.getAttemptList().size()); // Have the first task succeed eventHandler.events.clear(); - mockTask.handle(createTaskTASucceededEvent(firstAttempt.getID())); + mockTask.handle(createTaskTASucceededEvent(firstAttempt.getTaskAttemptID())); verifyOutgoingEvents(eventHandler.events, DAGEventType.DAG_SCHEDULER_UPDATE, VertexEventType.V_TASK_COMPLETED, VertexEventType.V_TASK_ATTEMPT_COMPLETED); // The task should now have succeeded and sent kill to other attempt assertTaskSucceededState(); verify(mockTask.stateChangeNotifier).taskSucceeded(any(String.class), eq(taskId), - eq(firstAttempt.getID().getId())); + eq(firstAttempt.getTaskAttemptID().getId())); @SuppressWarnings("rawtypes") Event event = eventHandler.events.get(eventHandler.events.size()-1); assertEquals(TaskAttemptEventType.TA_KILL_REQUEST, event.getType()); - assertEquals(specAttempt.getID(), + assertEquals(specAttempt.getTaskAttemptID(), ((TaskAttemptEventKillRequest) event).getTaskAttemptID()); eventHandler.events.clear(); // Emulate the spec attempt being killed - mockTask.handle(createTaskTAKilledEvent(specAttempt.getID())); + mockTask.handle(createTaskTAKilledEvent(specAttempt.getTaskAttemptID())); assertTaskSucceededState(); verifyOutgoingEvents(eventHandler.events, DAGEventType.DAG_SCHEDULER_UPDATE, VertexEventType.V_TASK_ATTEMPT_COMPLETED); @@ -1003,32 +1003,32 @@ public void testSpeculatedThenRetroactiveFailure() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstAttempt.getID()); + launchTaskAttempt(firstAttempt.getTaskAttemptID()); updateAttemptState(firstAttempt, TaskAttemptState.RUNNING); // Add a speculative task attempt - mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getID())); + mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getTaskAttemptID())); MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(specAttempt.getID()); + launchTaskAttempt(specAttempt.getTaskAttemptID()); updateAttemptState(specAttempt, TaskAttemptState.RUNNING); assertEquals(2, mockTask.getAttemptList().size()); // Have the first task succeed eventHandler.events.clear(); - mockTask.handle(createTaskTASucceededEvent(firstAttempt.getID())); + mockTask.handle(createTaskTASucceededEvent(firstAttempt.getTaskAttemptID())); // The task should now have succeeded and sent kill to other attempt assertTaskSucceededState(); verify(mockTask.stateChangeNotifier).taskSucceeded(any(String.class), eq(taskId), - eq(firstAttempt.getID().getId())); + eq(firstAttempt.getTaskAttemptID().getId())); @SuppressWarnings("rawtypes") Event event = eventHandler.events.get(eventHandler.events.size()-1); assertEquals(TaskAttemptEventType.TA_KILL_REQUEST, event.getType()); - assertEquals(specAttempt.getID(), + assertEquals(specAttempt.getTaskAttemptID(), ((TaskAttemptEventKillRequest) event).getTaskAttemptID()); // Emulate the spec attempt being killed - mockTask.handle(createTaskTAKilledEvent(specAttempt.getID())); + mockTask.handle(createTaskTAKilledEvent(specAttempt.getTaskAttemptID())); assertTaskSucceededState(); // Now fail the attempt after it has succeeded @@ -1039,7 +1039,7 @@ public void testSpeculatedThenRetroactiveFailure() { TaskAttemptEventOutputFailed outputFailedEvent = new TaskAttemptEventOutputFailed(mockDestId, mockTezEvent, 1); eventHandler.events.clear(); - mockTask.handle(createTaskTAFailedEvent(firstAttempt.getID(), TaskFailureType.NON_FATAL, outputFailedEvent)); + mockTask.handle(createTaskTAFailedEvent(firstAttempt.getTaskAttemptID(), TaskFailureType.NON_FATAL, outputFailedEvent)); // The task should still be in the scheduled state assertTaskScheduledState(); @@ -1059,15 +1059,15 @@ public void testIgnoreSpeculationOnSuccessfulOriginalAttempt() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstAttempt.getID()); + launchTaskAttempt(firstAttempt.getTaskAttemptID()); // Mock success of the first task attempt updateAttemptState(firstAttempt, TaskAttemptState.SUCCEEDED); - firstAttempt.handle(new TaskAttemptEvent(firstAttempt.getID(), TaskAttemptEventType.TA_DONE)); + firstAttempt.handle(new TaskAttemptEvent(firstAttempt.getTaskAttemptID(), TaskAttemptEventType.TA_DONE)); // Verify the speculation scheduling is ignored and no speculative attempt was added to the task - mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getID())); + mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getTaskAttemptID())); MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(specAttempt.getID()); + launchTaskAttempt(specAttempt.getTaskAttemptID()); assertEquals(1, mockTask.getAttemptList().size()); } @@ -1076,15 +1076,15 @@ public void testIgnoreSpeculationAfterOriginalAttemptCommit() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstAttempt.getID()); + launchTaskAttempt(firstAttempt.getTaskAttemptID()); updateAttemptState(firstAttempt, TaskAttemptState.RUNNING); // Mock commit of the first task attempt - mockTask.canCommit(firstAttempt.getID()); + mockTask.canCommit(firstAttempt.getTaskAttemptID()); // Verify the speculation scheduling is ignored and no speculative attempt was added to the task - mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getID())); + mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getTaskAttemptID())); MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(specAttempt.getID()); + launchTaskAttempt(specAttempt.getTaskAttemptID()); assertEquals(1, mockTask.getAttemptList().size()); } @@ -1094,10 +1094,10 @@ public void testSucceededAttemptStatusWithRetroActiveFailures() throws Interrupt TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstMockTaskAttempt = mockTask.getAttemptList().get(0); - launchTaskAttempt(firstMockTaskAttempt.getID()); - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + launchTaskAttempt(firstMockTaskAttempt.getTaskAttemptID()); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl secondMockTaskAttempt = mockTask.getAttemptList().get(1); - launchTaskAttempt(secondMockTaskAttempt.getID()); + launchTaskAttempt(secondMockTaskAttempt.getTaskAttemptID()); firstMockTaskAttempt.handle(new TaskAttemptEventSchedule( TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()), 10, 10)); @@ -1119,8 +1119,8 @@ public void testSucceededAttemptStatusWithRetroActiveFailures() throws Interrupt new TaskAttemptEvent(TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()), TaskAttemptEventType.TA_DONE)); - mockTask.handle(new TaskEventTASucceeded(secondMockTaskAttempt.getID())); - mockTask.handle(new TaskEventTASucceeded(firstMockTaskAttempt.getID())); + mockTask.handle(new TaskEventTASucceeded(secondMockTaskAttempt.getTaskAttemptID())); + mockTask.handle(new TaskEventTASucceeded(firstMockTaskAttempt.getTaskAttemptID())); assertTrue("Attempts should have succeeded!", firstMockTaskAttempt.getInternalState() == TaskAttemptStateInternal.SUCCEEDED && secondMockTaskAttempt.getInternalState() == TaskAttemptStateInternal.SUCCEEDED); @@ -1148,10 +1148,10 @@ public void testFailedAttemptStatus() throws InterruptedException { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstMockTaskAttempt = mockTask.getAttemptList().get(0); - launchTaskAttempt(firstMockTaskAttempt.getID()); - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + launchTaskAttempt(firstMockTaskAttempt.getTaskAttemptID()); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl secondMockTaskAttempt = mockTask.getAttemptList().get(1); - launchTaskAttempt(secondMockTaskAttempt.getID()); + launchTaskAttempt(secondMockTaskAttempt.getTaskAttemptID()); firstMockTaskAttempt.handle(new TaskAttemptEventSchedule( TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()), 10, 10)); @@ -1176,12 +1176,12 @@ public void testFailedAttemptStatus() throws InterruptedException { TaskAttemptTerminationCause.NO_PROGRESS)); firstMockTaskAttempt.handle(new TaskAttemptEventContainerTerminated(mockContainerId, - firstMockTaskAttempt.getID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); + firstMockTaskAttempt.getTaskAttemptID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); secondMockTaskAttempt.handle(new TaskAttemptEventContainerTerminated(mockContainerId, - secondMockTaskAttempt.getID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); - mockTask.handle(new TaskEventTAFailed(secondMockTaskAttempt.getID(), TaskFailureType.NON_FATAL, + secondMockTaskAttempt.getTaskAttemptID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); + mockTask.handle(new TaskEventTAFailed(secondMockTaskAttempt.getTaskAttemptID(), TaskFailureType.NON_FATAL, mock(TaskAttemptEvent.class))); - mockTask.handle(new TaskEventTAFailed(firstMockTaskAttempt.getID(), TaskFailureType.NON_FATAL, + mockTask.handle(new TaskEventTAFailed(firstMockTaskAttempt.getTaskAttemptID(), TaskFailureType.NON_FATAL, mock(TaskAttemptEvent.class))); assertTrue("Attempts should have failed!", firstMockTaskAttempt.getInternalState() == TaskAttemptStateInternal.FAILED @@ -1204,10 +1204,10 @@ public void testSucceededLeafTaskWithRetroFailures() throws InterruptedException TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstMockTaskAttempt = mockTask.getAttemptList().get(0); - launchTaskAttempt(firstMockTaskAttempt.getID()); - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + launchTaskAttempt(firstMockTaskAttempt.getTaskAttemptID()); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl secondMockTaskAttempt = mockTask.getAttemptList().get(1); - launchTaskAttempt(secondMockTaskAttempt.getID()); + launchTaskAttempt(secondMockTaskAttempt.getTaskAttemptID()); firstMockTaskAttempt.handle(new TaskAttemptEventSchedule( TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()), 10, 10)); @@ -1230,18 +1230,18 @@ public void testSucceededLeafTaskWithRetroFailures() throws InterruptedException TaskAttemptEventType.TA_FAILED, TaskFailureType.NON_FATAL, "test", TaskAttemptTerminationCause.CONTAINER_EXITED)); - mockTask.handle(new TaskEventTASucceeded(secondMockTaskAttempt.getID())); + mockTask.handle(new TaskEventTASucceeded(secondMockTaskAttempt.getTaskAttemptID())); firstMockTaskAttempt.handle(new TaskAttemptEventContainerTerminated(mockContainerId, - firstMockTaskAttempt.getID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); + firstMockTaskAttempt.getTaskAttemptID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); InputReadErrorEvent mockReEvent = InputReadErrorEvent.create("", 0, 0); - TezTaskAttemptID mockDestId = firstMockTaskAttempt.getID(); + TezTaskAttemptID mockDestId = firstMockTaskAttempt.getTaskAttemptID(); EventMetaData meta = new EventMetaData(EventProducerConsumerType.INPUT, "Vertex", "Edge", mockDestId); TezEvent tzEvent = new TezEvent(mockReEvent, meta); TaskAttemptEventOutputFailed outputFailedEvent = new TaskAttemptEventOutputFailed(mockDestId, tzEvent, 1); firstMockTaskAttempt.handle(outputFailedEvent); - mockTask.handle(new TaskEventTAFailed(firstMockTaskAttempt.getID(), TaskFailureType.NON_FATAL, + mockTask.handle(new TaskEventTAFailed(firstMockTaskAttempt.getTaskAttemptID(), TaskFailureType.NON_FATAL, mock(TaskAttemptEvent.class))); Assert.assertEquals(mockTask.getInternalState(), TaskStateInternal.SUCCEEDED); } @@ -1255,7 +1255,7 @@ private void failAttempt(MockTaskAttemptImpl taskAttempt, int index, int expecte new TaskAttemptEventOutputFailed(mockDestId, tzEvent, 1); taskAttempt.handle( outputFailedEvent); - TaskEvent tEventFail1 = new TaskEventTAFailed(taskAttempt.getID(), TaskFailureType.NON_FATAL, outputFailedEvent); + TaskEvent tEventFail1 = new TaskEventTAFailed(taskAttempt.getTaskAttemptID(), TaskFailureType.NON_FATAL, outputFailedEvent); mockTask.handle(tEventFail1); assertEquals("Unexpected number of incomplete attempts!", expectedIncompleteAttempts, mockTask.getUncompletedAttemptsCount()); @@ -1274,10 +1274,10 @@ public void testFailedTaskTransitionWithLaunchedAttempt() throws InterruptedExce TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstMockTaskAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstMockTaskAttempt.getID()); - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + launchTaskAttempt(firstMockTaskAttempt.getTaskAttemptID()); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl secondMockTaskAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(secondMockTaskAttempt.getID()); + launchTaskAttempt(secondMockTaskAttempt.getTaskAttemptID()); firstMockTaskAttempt.handle(new TaskAttemptEventSchedule( TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()), 10, 10)); @@ -1302,21 +1302,21 @@ public void testFailedTaskTransitionWithLaunchedAttempt() throws InterruptedExce TaskAttemptTerminationCause.NO_PROGRESS)); firstMockTaskAttempt.handle(new TaskAttemptEventContainerTerminated(mockContainerId, - firstMockTaskAttempt.getID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); + firstMockTaskAttempt.getTaskAttemptID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); secondMockTaskAttempt.handle(new TaskAttemptEventContainerTerminated(mockContainerId, - secondMockTaskAttempt.getID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); - mockTask.handle(new TaskEventTAFailed(secondMockTaskAttempt.getID(), TaskFailureType.NON_FATAL, + secondMockTaskAttempt.getTaskAttemptID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); + mockTask.handle(new TaskEventTAFailed(secondMockTaskAttempt.getTaskAttemptID(), TaskFailureType.NON_FATAL, mock(TaskAttemptEvent.class))); - mockTask.handle(new TaskEventTAFailed(firstMockTaskAttempt.getID(), TaskFailureType.NON_FATAL, + mockTask.handle(new TaskEventTAFailed(firstMockTaskAttempt.getTaskAttemptID(), TaskFailureType.NON_FATAL, mock(TaskAttemptEvent.class))); assertTrue("Attempts should have failed!", firstMockTaskAttempt.getInternalState() == TaskAttemptStateInternal.FAILED && secondMockTaskAttempt.getInternalState() == TaskAttemptStateInternal.FAILED); assertEquals("Task should have no uncompleted attempts!", 0, mockTask.getUncompletedAttemptsCount()); assertTrue("Task should have failed!", mockTask.getState() == TaskState.FAILED); - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl thirdMockTaskAttempt = mockTask.getLastAttempt(); - mockTask.handle(createTaskTALauncherEvent(thirdMockTaskAttempt.getID())); + mockTask.handle(createTaskTALauncherEvent(thirdMockTaskAttempt.getTaskAttemptID())); } @Test (timeout = 30000) @@ -1324,10 +1324,10 @@ public void testKilledTaskTransitionWithLaunchedAttempt() throws InterruptedExce TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstMockTaskAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstMockTaskAttempt.getID()); - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + launchTaskAttempt(firstMockTaskAttempt.getTaskAttemptID()); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl secondMockTaskAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(secondMockTaskAttempt.getID()); + launchTaskAttempt(secondMockTaskAttempt.getTaskAttemptID()); firstMockTaskAttempt.handle(new TaskAttemptEventSchedule( TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()), 10, 10)); @@ -1342,30 +1342,30 @@ public void testKilledTaskTransitionWithLaunchedAttempt() throws InterruptedExce new TaskAttemptEventStartedRemotely(TezTaskAttemptID.fromString(secondMockTaskAttempt.toString()))); firstMockTaskAttempt.handle( new TaskAttemptEventStartedRemotely(TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()))); - mockTask.handle(new TaskEventTermination(mockTask.getTaskId(), + mockTask.handle(new TaskEventTermination(mockTask.getTaskID(), TaskAttemptTerminationCause.FRAMEWORK_ERROR, "test")); secondMockTaskAttempt.handle( new TaskAttemptEventAttemptKilled(TezTaskAttemptID.fromString(secondMockTaskAttempt.toString()),"test", TaskAttemptTerminationCause.FRAMEWORK_ERROR)); - mockTask.handle(new TaskEventTAKilled(secondMockTaskAttempt.getID(), - new TaskAttemptEvent(secondMockTaskAttempt.getID(), TaskAttemptEventType.TA_KILLED))); + mockTask.handle(new TaskEventTAKilled(secondMockTaskAttempt.getTaskAttemptID(), + new TaskAttemptEvent(secondMockTaskAttempt.getTaskAttemptID(), TaskAttemptEventType.TA_KILLED))); firstMockTaskAttempt.handle( new TaskAttemptEventAttemptKilled(TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()),"test", TaskAttemptTerminationCause.FRAMEWORK_ERROR)); - mockTask.handle(new TaskEventTAKilled(firstMockTaskAttempt.getID(), - new TaskAttemptEvent(firstMockTaskAttempt.getID(), TaskAttemptEventType.TA_KILLED))); + mockTask.handle(new TaskEventTAKilled(firstMockTaskAttempt.getTaskAttemptID(), + new TaskAttemptEvent(firstMockTaskAttempt.getTaskAttemptID(), TaskAttemptEventType.TA_KILLED))); firstMockTaskAttempt.handle( new TaskAttemptEventAttemptKilled(TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()),"test", TaskAttemptTerminationCause.FRAMEWORK_ERROR)); assertEquals("Task should have been killed!", mockTask.getInternalState(), TaskStateInternal.KILLED); - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl thirdMockTaskAttempt = mockTask.getLastAttempt(); - mockTask.handle(createTaskTALauncherEvent(thirdMockTaskAttempt.getID())); - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTALauncherEvent(thirdMockTaskAttempt.getTaskAttemptID())); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl fourthMockTaskAttempt = mockTask.getLastAttempt(); - mockTask.handle(createTaskTASucceededEvent(fourthMockTaskAttempt.getID())); + mockTask.handle(createTaskTASucceededEvent(fourthMockTaskAttempt.getTaskAttemptID())); MockTaskAttemptImpl fifthMockTaskAttempt = mockTask.getLastAttempt(); - mockTask.handle(createTaskTAFailedEvent(fifthMockTaskAttempt.getID())); + mockTask.handle(createTaskTAFailedEvent(fifthMockTaskAttempt.getTaskAttemptID())); } // TODO Add test to validate the correct commit attempt. @@ -1418,7 +1418,7 @@ public MockTaskImpl(TezVertexID vertexId, int partition, @Override protected TaskAttemptImpl createAttempt(int attemptNumber, TezTaskAttemptID schedCausalTA) { MockTaskAttemptImpl attempt = new MockTaskAttemptImpl( - TezBuilderUtils.newTaskAttemptId(getTaskId(), attemptNumber), + TezBuilderUtils.newTaskAttemptId(getTaskID(), attemptNumber), eventHandler, taskCommunicatorManagerInterface, conf, clock, taskHeartbeatHandler, appContext, true, taskResource, containerContext, schedCausalTA); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java index fb7872ff9a..5cdcf49206 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java @@ -386,7 +386,7 @@ private class TaskAttemptEventDispatcher implements EventHandler)task.getAttempt( event.getTaskAttemptID())).handle(event); @@ -399,7 +399,7 @@ private class TaskEventDispatcher implements EventHandler { @Override public void handle(TaskEvent event) { events.add(event); - VertexImpl vertex = vertexIdMap.get(event.getTaskID().getVertexID()); + VertexImpl vertex = vertexIdMap.get(event.getVertexID()); Task task = vertex.getTask(event.getTaskID()); if (task != null) { ((EventHandler)task).handle(event); @@ -437,7 +437,7 @@ private class VertexEventDispatcher @Override public void handle(VertexEvent event) { - VertexImpl vertex = vertexIdMap.get(event.getVertexId()); + VertexImpl vertex = vertexIdMap.get(event.getVertexID()); ((EventHandler) vertex).handle(event); } } @@ -3155,7 +3155,7 @@ private void checkTasks(Vertex v, int numTasks) { int i = 0; // iteration maintains order due to linked hash map for(Task task : tasks.values()) { - Assert.assertEquals(i, task.getTaskId().getId()); + Assert.assertEquals(i, task.getTaskID().getId()); i++; } } @@ -3638,7 +3638,7 @@ public void testVertexTaskAttemptProcessorFailure() throws Exception { startVertex(v); dispatcher.await(); TaskAttemptImpl ta = (TaskAttemptImpl) v.getTask(0).getAttempts().values().iterator().next(); - ta.handle(new TaskAttemptEventSchedule(ta.getID(), 2, 2)); + ta.handle(new TaskAttemptEventSchedule(ta.getTaskAttemptID(), 2, 2)); NodeId nid = NodeId.newInstance("127.0.0.1", 0); ContainerId contId = ContainerId.newInstance(appAttemptId, 3); @@ -3652,10 +3652,10 @@ public void testVertexTaskAttemptProcessorFailure() throws Exception { containers.addContainerIfNew(container, 0, 0, 0); doReturn(containers).when(appContext).getAllContainers(); - ta.handle(new TaskAttemptEventSubmitted(ta.getID(), contId)); - ta.handle(new TaskAttemptEventStartedRemotely(ta.getID())); + ta.handle(new TaskAttemptEventSubmitted(ta.getTaskAttemptID(), contId)); + ta.handle(new TaskAttemptEventStartedRemotely(ta.getTaskAttemptID())); Assert.assertEquals(TaskAttemptStateInternal.RUNNING, ta.getInternalState()); - ta.handle(new TaskAttemptEventAttemptFailed(ta.getID(), TaskAttemptEventType.TA_FAILED, + ta.handle(new TaskAttemptEventAttemptFailed(ta.getTaskAttemptID(), TaskAttemptEventType.TA_FAILED, TaskFailureType.NON_FATAL, "diag", TaskAttemptTerminationCause.APPLICATION_ERROR)); dispatcher.await(); @@ -3673,7 +3673,7 @@ public void testVertexTaskAttemptInputFailure() throws Exception { startVertex(v); dispatcher.await(); TaskAttemptImpl ta = (TaskAttemptImpl) v.getTask(0).getAttempts().values().iterator().next(); - ta.handle(new TaskAttemptEventSchedule(ta.getID(), 2, 2)); + ta.handle(new TaskAttemptEventSchedule(ta.getTaskAttemptID(), 2, 2)); NodeId nid = NodeId.newInstance("127.0.0.1", 0); ContainerId contId = ContainerId.newInstance(appAttemptId, 3); @@ -3687,11 +3687,11 @@ public void testVertexTaskAttemptInputFailure() throws Exception { containers.addContainerIfNew(container, 0, 0, 0); doReturn(containers).when(appContext).getAllContainers(); - ta.handle(new TaskAttemptEventSubmitted(ta.getID(), contId)); - ta.handle(new TaskAttemptEventStartedRemotely(ta.getID())); + ta.handle(new TaskAttemptEventSubmitted(ta.getTaskAttemptID(), contId)); + ta.handle(new TaskAttemptEventStartedRemotely(ta.getTaskAttemptID())); Assert.assertEquals(TaskAttemptStateInternal.RUNNING, ta.getInternalState()); - ta.handle(new TaskAttemptEventAttemptFailed(ta.getID(), TaskAttemptEventType.TA_FAILED, + ta.handle(new TaskAttemptEventAttemptFailed(ta.getTaskAttemptID(), TaskAttemptEventType.TA_FAILED, TaskFailureType.NON_FATAL, "diag", TaskAttemptTerminationCause.INPUT_READ_ERROR)); dispatcher.await(); @@ -3710,7 +3710,7 @@ public void testVertexTaskAttemptOutputFailure() throws Exception { startVertex(v); dispatcher.await(); TaskAttemptImpl ta = (TaskAttemptImpl) v.getTask(0).getAttempts().values().iterator().next(); - ta.handle(new TaskAttemptEventSchedule(ta.getID(), 2, 2)); + ta.handle(new TaskAttemptEventSchedule(ta.getTaskAttemptID(), 2, 2)); NodeId nid = NodeId.newInstance("127.0.0.1", 0); ContainerId contId = ContainerId.newInstance(appAttemptId, 3); @@ -3724,11 +3724,11 @@ public void testVertexTaskAttemptOutputFailure() throws Exception { containers.addContainerIfNew(container, 0, 0, 0); doReturn(containers).when(appContext).getAllContainers(); - ta.handle(new TaskAttemptEventSubmitted(ta.getID(), contId)); - ta.handle(new TaskAttemptEventStartedRemotely(ta.getID())); + ta.handle(new TaskAttemptEventSubmitted(ta.getTaskAttemptID(), contId)); + ta.handle(new TaskAttemptEventStartedRemotely(ta.getTaskAttemptID())); Assert.assertEquals(TaskAttemptStateInternal.RUNNING, ta.getInternalState()); - ta.handle(new TaskAttemptEventAttemptFailed(ta.getID(), TaskAttemptEventType.TA_FAILED, + ta.handle(new TaskAttemptEventAttemptFailed(ta.getTaskAttemptID(), TaskAttemptEventType.TA_FAILED, TaskFailureType.NON_FATAL, "diag", TaskAttemptTerminationCause.OUTPUT_WRITE_ERROR)); dispatcher.await(); @@ -4800,7 +4800,7 @@ public void testInputInitializerEventsMultipleSources() throws Exception { Assert.assertEquals(2, v2.getTotalTasks()); // Generate events from v2 to v3's initializer. 1 from task 0, 2 from task 1 for (Task task : v2.getTasks().values()) { - TezTaskID taskId = task.getTaskId(); + TezTaskID taskId = task.getTaskID(); TezTaskAttemptID attemptId = TezTaskAttemptID.getInstance(taskId, 0); int numEventsFromTask = taskId.getId() + 1; for (int i = 0; i < numEventsFromTask; i++) { @@ -5499,7 +5499,7 @@ public void testVertexWithInitializerSuccess() throws Exception { for (int i=0; i()); //Vertex 1, Task 4, Attempt 1, host1 TezTaskAttemptID taID114 = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexID11, 4), 1); TaskAttempt ta114 = mock(TaskAttempt.class); - doReturn(taID114).when(ta114).getID(); + doReturn(taID114).when(ta114).getTaskAttemptID(); doReturn("Mock for TA " + taID114.toString()).when(ta114).toString(); AMSchedulerEventTALaunchRequest lrEvent14 = createLaunchRequestEvent( taID114, ta114, resource1, host1, racks, priority1, new HashMap()); @@ -1261,7 +1261,7 @@ public void testReuseConflictLocalResources() throws IOException, InterruptedExc //Vertex 2, Task 1, Attempt 1, host1, lr2 TezTaskAttemptID taID211 = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexID21, 1), 1); TaskAttempt ta211 = mock(TaskAttempt.class); - doReturn(taID211).when(ta211).getID(); + doReturn(taID211).when(ta211).getTaskAttemptID(); doReturn("Mock for TA " + taID211.toString()).when(ta211).toString(); AMSchedulerEventTALaunchRequest lrEvent21 = createLaunchRequestEvent(taID211, ta211, resource1, host1, racks, priority1, v21LR); @@ -1297,7 +1297,7 @@ public void testReuseConflictLocalResources() throws IOException, InterruptedExc TezTaskAttemptID taID311 = TezTaskAttemptID.getInstance( TezTaskID.getInstance(TezVertexID.getInstance(dagID3, 1), 1), 1); TaskAttempt ta311 = mock(TaskAttempt.class); - doReturn(taID311).when(ta311).getID(); + doReturn(taID311).when(ta311).getTaskAttemptID(); doReturn("Mock for TA " + taID311).when(ta311).toString(); AMSchedulerEventTALaunchRequest lrEvent31 = createLaunchRequestEvent(taID311, ta311, resource1, host1, racks, priority1, v31LR); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java index dcf9a5dd69..3ddd1861d4 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java @@ -189,7 +189,7 @@ public void testSimpleAllocate() throws Exception { TaskAttemptImpl mockTaskAttempt = mock(TaskAttemptImpl.class); TezTaskAttemptID mockAttemptId = mock(TezTaskAttemptID.class); when(mockAttemptId.getId()).thenReturn(0); - when(mockTaskAttempt.getID()).thenReturn(mockAttemptId); + when(mockTaskAttempt.getTaskAttemptID()).thenReturn(mockAttemptId); Resource resource = Resource.newInstance(1024, 1); ContainerContext containerContext = new ContainerContext(new HashMap(), new Credentials(), @@ -228,7 +228,7 @@ public void testTASucceededAfterContainerCleanup() throws Exception { TaskAttemptImpl mockTaskAttempt = mock(TaskAttemptImpl.class); TezTaskAttemptID mockAttemptId = mock(TezTaskAttemptID.class); when(mockAttemptId.getId()).thenReturn(0); - when(mockTaskAttempt.getID()).thenReturn(mockAttemptId); + when(mockTaskAttempt.getTaskAttemptID()).thenReturn(mockAttemptId); Resource resource = Resource.newInstance(1024, 1); ContainerContext containerContext = new ContainerContext(new HashMap(), new Credentials(), @@ -268,7 +268,7 @@ public void testTAUnsuccessfulAfterContainerCleanup() throws Exception { TaskAttemptImpl mockTaskAttempt = mock(TaskAttemptImpl.class); TezTaskAttemptID mockAttemptId = mock(TezTaskAttemptID.class); when(mockAttemptId.getId()).thenReturn(0); - when(mockTaskAttempt.getID()).thenReturn(mockAttemptId); + when(mockTaskAttempt.getTaskAttemptID()).thenReturn(mockAttemptId); ContainerId mockCId = mock(ContainerId.class); Container container = mock(Container.class); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java b/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java index 16b391b55f..94c8df6233 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java @@ -640,7 +640,7 @@ Map getTasksTestHelper(List tasks, List > t //Creating mock tasks and attaching to mock vertex Map taskMap = Maps.newHashMap(); for(Task task : tasks) { - TezTaskID taskId = task.getTaskId(); + TezTaskID taskId = task.getTaskID(); int taskIndex = taskId.getId(); doReturn(task).when(mockVertex).getTask(taskIndex); taskMap.put(taskId, task); @@ -688,7 +688,7 @@ private List createMockTasks() { private Task createMockTask(String taskIDStr, TaskState status, float progress) { Task mockTask = mock(Task.class); - doReturn(TezTaskID.fromString(taskIDStr)).when(mockTask).getTaskId(); + doReturn(TezTaskID.fromString(taskIDStr)).when(mockTask).getTaskID(); doReturn(status).when(mockTask).getState(); doReturn(progress).when(mockTask).getProgress(); @@ -711,7 +711,7 @@ private Task createMockTask(String taskIDStr, TaskState status, float progress) private void verifySingleTaskResult(Task mockTask, Map taskResult) { Assert.assertEquals(3, taskResult.size()); - Assert.assertEquals(mockTask.getTaskId().toString(), taskResult.get("id")); + Assert.assertEquals(mockTask.getTaskID().toString(), taskResult.get("id")); Assert.assertEquals(mockTask.getState().toString(), taskResult.get("status")); Assert.assertEquals(Float.toString(mockTask.getProgress()), taskResult.get("progress")); } @@ -777,7 +777,7 @@ Map getAttemptsTestHelper(List attempts, List getAttemptsTestHelper(List attempts, List attemptsMap = Maps.newHashMap(); for(TaskAttempt attempt : attempts) { - TezTaskAttemptID attemptId = attempt.getID(); + TezTaskAttemptID attemptId = attempt.getTaskAttemptID(); doReturn(attempt).when(mockTask).getAttempt(attemptId); attemptsMap.put(attemptId, attempt); } @@ -835,7 +835,7 @@ private List createMockAttempts() { private TaskAttempt createMockAttempt(String attemptIDStr, TaskAttemptState status, float progress) { TaskAttempt mockAttempt = mock(TaskAttempt.class); - doReturn(TezTaskAttemptID.fromString(attemptIDStr)).when(mockAttempt).getID(); + doReturn(TezTaskAttemptID.fromString(attemptIDStr)).when(mockAttempt).getTaskAttemptID(); doReturn(status).when(mockAttempt).getState(); doReturn(progress).when(mockAttempt).getProgress(); @@ -858,7 +858,7 @@ private TaskAttempt createMockAttempt(String attemptIDStr, TaskAttemptState stat private void verifySingleAttemptResult(TaskAttempt mockTask, Map taskResult) { Assert.assertEquals(3, taskResult.size()); - Assert.assertEquals(mockTask.getID().toString(), taskResult.get("id")); + Assert.assertEquals(mockTask.getTaskAttemptID().toString(), taskResult.get("id")); Assert.assertEquals(mockTask.getState().toString(), taskResult.get("status")); Assert.assertEquals(Float.toString(mockTask.getProgress()), taskResult.get("progress")); } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java index 50a80cb5f5..51b4bf9f92 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java @@ -186,8 +186,8 @@ private void testDAGSubmittedEvent() throws Exception { testProtoConversion(event); Assert.assertEquals(event.getApplicationAttemptId(), deserializedEvent.getApplicationAttemptId()); - Assert.assertEquals(event.getDagID(), - deserializedEvent.getDagID()); + Assert.assertEquals(event.getDAGID(), + deserializedEvent.getDAGID()); Assert.assertEquals(event.getDAGName(), deserializedEvent.getDAGName()); Assert.assertEquals(event.getSubmitTime(), @@ -204,8 +204,8 @@ private void testDAGInitializedEvent() throws Exception { "user", "dagName", null); DAGInitializedEvent deserializedEvent = (DAGInitializedEvent) testProtoConversion(event); - Assert.assertEquals(event.getDagID(), - deserializedEvent.getDagID()); + Assert.assertEquals(event.getDAGID(), + deserializedEvent.getDAGID()); Assert.assertEquals(event.getInitTime(), deserializedEvent.getInitTime()); logEvents(event, deserializedEvent); } @@ -216,8 +216,8 @@ private void testDAGStartedEvent() throws Exception { "user", "dagName"); DAGStartedEvent deserializedEvent = (DAGStartedEvent) testProtoConversion(event); - Assert.assertEquals(event.getDagID(), - deserializedEvent.getDagID()); + Assert.assertEquals(event.getDAGID(), + deserializedEvent.getDAGID()); Assert.assertEquals(event.getStartTime(), deserializedEvent.getStartTime()); logEvents(event, deserializedEvent); } @@ -243,8 +243,8 @@ private void testDAGFinishedEvent() throws Exception { DAGFinishedEvent deserializedEvent = (DAGFinishedEvent) testProtoConversion(event); Assert.assertEquals( - event.getDagID(), - deserializedEvent.getDagID()); + event.getDAGID(), + deserializedEvent.getDAGID()); Assert.assertEquals(event.getState(), deserializedEvent.getState()); Assert.assertNotEquals(event.getStartTime(), deserializedEvent.getStartTime()); Assert.assertEquals(event.getFinishTime(), deserializedEvent.getFinishTime()); @@ -264,8 +264,8 @@ private void testDAGFinishedEvent() throws Exception { DAGFinishedEvent deserializedEvent = (DAGFinishedEvent) testProtoConversion(event); Assert.assertEquals( - event.getDagID(), - deserializedEvent.getDagID()); + event.getDAGID(), + deserializedEvent.getDAGID()); Assert.assertEquals(event.getState(), deserializedEvent.getState()); Assert.assertNotEquals(event.getStartTime(), deserializedEvent.getStartTime()); Assert.assertEquals(event.getFinishTime(), deserializedEvent.getFinishTime()); diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java index eef73a0d0a..7864e1c852 100644 --- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java +++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java @@ -556,12 +556,12 @@ private void checkAndThrowExceptionForTests(SubmitWorkRequestProto request) thro } TaskSpec taskSpec = ProtoConverters.getTaskSpecfromProto(request.getTaskSpec()); - if (taskSpec.getTaskAttemptID().getTaskID().getId() == 0 && + if (taskSpec.getTaskID().getId() == 0 && taskSpec.getTaskAttemptID().getId() == 0) { LOG.info("Simulating Rejected work"); throw new RejectedExecutionException( "Simulating Rejected work for taskAttemptId=" + taskSpec.getTaskAttemptID()); - } else if (taskSpec.getTaskAttemptID().getTaskID().getId() == 1 && + } else if (taskSpec.getTaskID().getId() == 1 && taskSpec.getTaskAttemptID().getId() == 0) { LOG.info("Simulating Task Setup Failure during launch"); throw new TezException("Simulating Task Setup Failure during launch for taskAttemptId=" + diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/IDConverter.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/IDConverter.java index 0f1b56d3d2..2fddd00522 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/IDConverter.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/IDConverter.java @@ -39,7 +39,7 @@ public static JobID toMRJobId(TezDAGID dagId) { public static TaskID toMRTaskId(TezTaskID taskid) { return new TaskID( - toMRJobId(taskid.getVertexID().getDAGId()), + toMRJobId(taskid.getDAGID()), taskid.getVertexID().getId() == 0 ? TaskType.MAP : TaskType.REDUCE, taskid.getId()); } diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java index e5e7022064..941c8732bd 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java @@ -53,8 +53,8 @@ public static org.apache.hadoop.mapred.TaskAttemptID createMockTaskAttemptID( public static org.apache.hadoop.mapred.TaskAttemptID createMockTaskAttemptIDFromTezTaskAttemptId(TezTaskAttemptID tezTaId, boolean isMap) { - TezVertexID vId = tezTaId.getTaskID().getVertexID(); - ApplicationId appId = vId.getDAGId().getApplicationId(); + TezVertexID vId = tezTaId.getVertexID(); + ApplicationId appId = vId.getApplicationId(); return new org.apache.hadoop.mapred.TaskAttemptID( new org.apache.hadoop.mapred.TaskID(String.valueOf(appId.getClusterTimestamp()) + String.valueOf(vId.getId()), appId.getId(), @@ -65,7 +65,7 @@ public static org.apache.hadoop.mapred.TaskAttemptID createMockTaskAttemptID( public static org.apache.hadoop.mapred.TaskID createMockTaskAttemptIDFromTezTaskId(TezTaskID tezTaId, boolean isMap) { TezVertexID vId = tezTaId.getVertexID(); - ApplicationId appId = vId.getDAGId().getApplicationId(); + ApplicationId appId = vId.getApplicationId(); return new org.apache.hadoop.mapred.TaskID(String.valueOf(appId.getClusterTimestamp()) + String.valueOf(vId.getId()), appId.getId(), isMap ? TaskType.MAP : TaskType.REDUCE, tezTaId.getId()); diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java index 1a13168cb7..55e5709c3b 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java @@ -36,7 +36,6 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FileSystem.Statistics; import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.DataInputBuffer; @@ -321,8 +320,7 @@ public void initTask(LogicalOutput output) throws IOException, } this.mrReporter = new MRTaskReporter(processorContext); this.useNewApi = jobConf.getUseNewMapper(); - TezDAGID dagId = IDConverter.fromMRTaskAttemptId(taskAttemptId).getTaskID() - .getVertexID().getDAGId(); + TezDAGID dagId = IDConverter.fromMRTaskAttemptId(taskAttemptId).getDAGID(); this.jobContext = new JobContextImpl(jobConf, dagId, mrReporter); this.taskAttemptContext = diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java index c1711ce2cb..b57f4f0e65 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java @@ -264,8 +264,8 @@ protected void readEventsFromSource(String dagId, JSONObjectSource source, case Constants.TEZ_VERTEX_ID: String vertexName = entity; TezVertexID tezVertexID = TezVertexID.fromString(vertexName); - if (!tezDAGID.equals(tezVertexID.getDAGId())) { - LOG.warn("{} does not belong to {} ('{}' != '{}')}", vertexName, tezDAGID, tezDAGID, tezVertexID.getDAGId()); + if (!tezDAGID.equals(tezVertexID.getDAGID())) { + LOG.warn("{} does not belong to {} ('{}' != '{}')}", vertexName, tezDAGID, tezDAGID, tezVertexID.getDAGID()); continue; } if (!vertexJsonMap.containsKey(vertexName)) { @@ -278,9 +278,9 @@ protected void readEventsFromSource(String dagId, JSONObjectSource source, case Constants.TEZ_TASK_ID: String taskName = entity; TezTaskID tezTaskID = TezTaskID.fromString(taskName); - if (!tezDAGID.equals(tezTaskID.getVertexID().getDAGId())) { + if (!tezDAGID.equals(tezTaskID.getDAGID())) { LOG.warn("{} does not belong to {} ('{}' != '{}')}", taskName, tezDAGID, tezDAGID, - tezTaskID.getVertexID().getDAGId()); + tezTaskID.getDAGID()); continue; } if (!taskJsonMap.containsKey(taskName)) { @@ -293,9 +293,9 @@ protected void readEventsFromSource(String dagId, JSONObjectSource source, case Constants.TEZ_TASK_ATTEMPT_ID: String taskAttemptName = entity; TezTaskAttemptID tezAttemptId = TezTaskAttemptID.fromString(taskAttemptName); - if (!tezDAGID.equals(tezAttemptId.getTaskID().getVertexID().getDAGId())) { + if (!tezDAGID.equals(tezAttemptId.getDAGID())) { LOG.warn("{} does not belong to {} ('{}' != '{}')}", taskAttemptName, tezDAGID, tezDAGID, - tezAttemptId.getTaskID().getVertexID().getDAGId()); + tezAttemptId.getDAGID()); continue; } if (!attemptJsonMap.containsKey(taskAttemptName)) { diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseParser.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseParser.java index af8e292b24..9f3881c8b2 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseParser.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseParser.java @@ -90,9 +90,8 @@ protected void linkParsedContents() { //Link task to task attempt TezTaskAttemptID taskAttemptId = TezTaskAttemptID.fromString(attemptInfo .getTaskAttemptId()); - VertexInfo vertexInfo = dagInfo.getVertexFromId(taskAttemptId.getTaskID() - .getVertexID().toString()); - Preconditions.checkState(vertexInfo != null, "Vertex " + taskAttemptId.getTaskID() + VertexInfo vertexInfo = dagInfo.getVertexFromId(taskAttemptId.getVertexID().toString()); + Preconditions.checkState(vertexInfo != null, "Vertex " + taskAttemptId .getVertexID().toString() + " is not present in DAG"); TaskInfo taskInfo = vertexInfo.getTask(taskAttemptId.getTaskID().toString()); attemptInfo.setTaskInfo(taskInfo); diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java index 09079bdf1c..f5f436eb9d 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -150,7 +150,7 @@ private HistoryEventProto.Builder makeBuilderForEvent(HistoryEvent event, long t } if (vertexId != null) { builder.setVertexId(vertexId.toString()); - dagId = vertexId.getDAGId(); + dagId = vertexId.getDAGID(); } if (dagId != null) { builder.setDagId(dagId.toString()); @@ -240,7 +240,7 @@ private HistoryEventProto convertContainerStoppedEvent(ContainerStoppedEvent eve private HistoryEventProto convertDAGSubmittedEvent(DAGSubmittedEvent event) { HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getSubmitTime(), - event.getDagID(), null, event.getApplicationAttemptId(), null, null, null, + event.getDAGID(), null, event.getApplicationAttemptId(), null, null, null, event.getUser()); addEventData(builder, ATSConstants.DAG_NAME, event.getDAGName()); if (event.getDAGPlan().hasCallerContext() && @@ -267,11 +267,11 @@ private HistoryEventProto convertDAGSubmittedEvent(DAGSubmittedEvent event) { private HistoryEventProto convertDAGInitializedEvent(DAGInitializedEvent event) { HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getInitTime(), - event.getDagID(), null, null, null, null, null, event.getUser()); + event.getDAGID(), null, null, null, null, null, event.getUser()); addEventData(builder, ATSConstants.DAG_NAME, event.getDagName()); if (event.getVertexNameIDMap() != null) { - Map nameIdStrMap = new TreeMap(); + Map nameIdStrMap = new TreeMap<>(); for (Entry entry : event.getVertexNameIDMap().entrySet()) { nameIdStrMap.put(entry.getKey(), entry.getValue().toString()); } @@ -282,7 +282,7 @@ private HistoryEventProto convertDAGInitializedEvent(DAGInitializedEvent event) private HistoryEventProto convertDAGStartedEvent(DAGStartedEvent event) { HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getStartTime(), - event.getDagID(), null, null, null, null, null, event.getUser()); + event.getDAGID(), null, null, null, null, null, event.getUser()); addEventData(builder, ATSConstants.DAG_NAME, event.getDagName()); addEventData(builder, ATSConstants.STATUS, event.getDagState().name()); @@ -292,7 +292,7 @@ private HistoryEventProto convertDAGStartedEvent(DAGStartedEvent event) { private HistoryEventProto convertDAGFinishedEvent(DAGFinishedEvent event) { HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getFinishTime(), - event.getDagID(), null, event.getApplicationAttemptId(), null, null, null, + event.getDAGID(), null, event.getApplicationAttemptId(), null, null, null, event.getUser()); addEventData(builder, ATSConstants.DAG_NAME, event.getDagName()); if (event.getDAGPlan().hasCallerContext()) { @@ -473,7 +473,7 @@ private HistoryEventProto convertVertexReconfigureDoneEvent(VertexConfigurationD HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getReconfigureDoneTime(), null, null, null, event.getVertexID(), null, null, null); if (event.getSourceEdgeProperties() != null && !event.getSourceEdgeProperties().isEmpty()) { - Map updatedEdgeManagers = new HashMap(); + Map updatedEdgeManagers = new HashMap<>(); for (Entry entry : event.getSourceEdgeProperties().entrySet()) { updatedEdgeManagers.put(entry.getKey(), DAGUtils.convertEdgeProperty(entry.getValue())); } diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java index 008b05d616..d675a659a0 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -73,7 +73,7 @@ public ProtoHistoryLoggingService() { } @Override - protected void serviceInit(Configuration conf) throws Exception { + protected void serviceInit(Configuration conf) { LOG.info("Initing ProtoHistoryLoggingService"); setConfig(conf); loggingDisabled = !conf.getBoolean(TezConfiguration.TEZ_AM_HISTORY_LOGGING_ENABLED, @@ -146,7 +146,7 @@ private void loop() { } catch (InterruptedException e) { LOG.info("EventQueue poll interrupted, ignoring it.", e); } catch (IOException e) { - TezDAGID dagid = evt.getDagID(); + TezDAGID dagid = evt.getDAGID(); HistoryEventType type = evt.getHistoryEvent().getEventType(); // Retry is hard, because there are several places where this exception can happen // the state will get messed up a lot. @@ -160,7 +160,7 @@ private void handleEvent(DAGHistoryEvent event) throws IOException { return; } HistoryEvent historyEvent = event.getHistoryEvent(); - if (event.getDagID() == null) { + if (event.getDAGID() == null) { if (historyEvent.getEventType() == HistoryEventType.APP_LAUNCHED) { appEventsFile = appEventsWriter.getPath().toString(); appLaunchedEventOffset = appEventsWriter.getOffset(); @@ -168,7 +168,7 @@ private void handleEvent(DAGHistoryEvent event) throws IOException { appEventsWriter.writeProto(converter.convert(historyEvent)); } else { HistoryEventType type = historyEvent.getEventType(); - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); if (type == HistoryEventType.DAG_FINISHED) { finishCurrentDag((DAGFinishedEvent)historyEvent); } else if (type == HistoryEventType.DAG_SUBMITTED) { @@ -220,7 +220,7 @@ private void finishCurrentDag(DAGFinishedEvent event) throws IOException { .setAppLaunchedEventOffset(appLaunchedEventOffset) .setWriteTime(System.currentTimeMillis()); if (event != null) { - entry.setDagId(event.getDagID().toString()); + entry.setDagId(event.getDAGID().toString()); } manifestEventsWriter.writeProto(entry.build()); manifestEventsWriter.hflush(); diff --git a/tez-plugins/tez-yarn-timeline-cache-plugin/src/main/java/org/apache/tez/dag/history/logging/ats/TimelineCachePluginImpl.java b/tez-plugins/tez-yarn-timeline-cache-plugin/src/main/java/org/apache/tez/dag/history/logging/ats/TimelineCachePluginImpl.java index d211feb0e3..2ee2c21b79 100644 --- a/tez-plugins/tez-yarn-timeline-cache-plugin/src/main/java/org/apache/tez/dag/history/logging/ats/TimelineCachePluginImpl.java +++ b/tez-plugins/tez-yarn-timeline-cache-plugin/src/main/java/org/apache/tez/dag/history/logging/ats/TimelineCachePluginImpl.java @@ -94,18 +94,18 @@ private Set convertToTimelineEntityGroupIds(String entity } else if (entityType.equals(EntityTypes.TEZ_VERTEX_ID.name())) { TezVertexID vertexID = TezVertexID.fromString(entityId); if (vertexID != null) { - return createTimelineEntityGroupIds(vertexID.getDAGId()); + return createTimelineEntityGroupIds(vertexID.getDAGID()); } } else if (entityType.equals(EntityTypes.TEZ_TASK_ID.name())) { TezTaskID taskID = TezTaskID.fromString(entityId); if (taskID != null) { - return createTimelineEntityGroupIds(taskID.getVertexID().getDAGId()); + return createTimelineEntityGroupIds(taskID.getDAGID()); } } else if (entityType.equals(EntityTypes.TEZ_TASK_ATTEMPT_ID.name())) { TezTaskAttemptID taskAttemptID = TezTaskAttemptID.fromString(entityId); if (taskAttemptID != null) { - return createTimelineEntityGroupIds(taskAttemptID.getTaskID().getVertexID().getDAGId()); + return createTimelineEntityGroupIds(taskAttemptID.getDAGID()); } } else if (entityType.equals(EntityTypes.TEZ_CONTAINER_ID.name())) { String cId = entityId; diff --git a/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java b/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java index f79a78b9fa..f284fe4049 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java +++ b/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java @@ -451,7 +451,7 @@ public void testDagLoggingDisabled() throws Exception { DAGHistoryEvent event = new DAGHistoryEvent(tezDAGID, submittedEvent); historyLoggingService.handle(new DAGHistoryEvent(tezDAGID, submittedEvent)); Thread.sleep(1000l); - String url = "http://" + timelineAddress + "/ws/v1/timeline/TEZ_DAG_ID/"+event.getDagID(); + String url = "http://" + timelineAddress + "/ws/v1/timeline/TEZ_DAG_ID/"+event.getDAGID(); Client client = new Client(); WebResource resource = client.resource(url); @@ -496,7 +496,7 @@ public void testDagLoggingEnabled() throws Exception { DAGHistoryEvent event = new DAGHistoryEvent(tezDAGID, submittedEvent); historyLoggingService.handle(new DAGHistoryEvent(tezDAGID, submittedEvent)); Thread.sleep(1000l); - String url = "http://" + timelineAddress + "/ws/v1/timeline/TEZ_DAG_ID/"+event.getDagID(); + String url = "http://" + timelineAddress + "/ws/v1/timeline/TEZ_DAG_ID/"+event.getDAGID(); Client client = new Client(); WebResource resource = client.resource(url); diff --git a/tez-plugins/tez-yarn-timeline-history-with-fs/src/main/java/org/apache/tez/dag/history/logging/ats/ATSV15HistoryLoggingService.java b/tez-plugins/tez-yarn-timeline-history-with-fs/src/main/java/org/apache/tez/dag/history/logging/ats/ATSV15HistoryLoggingService.java index a71f0d8db0..5068fb77db 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-fs/src/main/java/org/apache/tez/dag/history/logging/ats/ATSV15HistoryLoggingService.java +++ b/tez-plugins/tez-yarn-timeline-history-with-fs/src/main/java/org/apache/tez/dag/history/logging/ats/ATSV15HistoryLoggingService.java @@ -311,9 +311,9 @@ public TimelineEntityGroupId getGroupId(DAGHistoryEvent event) { case VERTEX_GROUP_COMMIT_FINISHED: case DAG_RECOVERED: String entityGroupId = numDagsPerGroup > 1 - ? event.getDagID().getGroupId(numDagsPerGroup) - : event.getDagID().toString(); - return TimelineEntityGroupId.newInstance(event.getDagID().getApplicationId(), entityGroupId); + ? event.getDAGID().getGroupId(numDagsPerGroup) + : event.getDAGID().toString(); + return TimelineEntityGroupId.newInstance(event.getApplicationId(), entityGroupId); case APP_LAUNCHED: case AM_LAUNCHED: case AM_STARTED: @@ -333,7 +333,7 @@ public void handle(DAGHistoryEvent event) { private boolean isValidEvent(DAGHistoryEvent event) { HistoryEventType eventType = event.getHistoryEvent().getEventType(); - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); if (eventType.equals(HistoryEventType.DAG_SUBMITTED)) { DAGSubmittedEvent dagSubmittedEvent = @@ -373,7 +373,7 @@ private boolean isValidEvent(DAGHistoryEvent event) { private void handleEvents(DAGHistoryEvent event) { String domainId = getDomainForEvent(event); // skippedDags is updated in the above call so check again. - if (event.getDagID() != null && skippedDAGs.contains(event.getDagID())) { + if (event.getDAGID() != null && skippedDAGs.contains(event.getDAGID())) { return; } TimelineEntityGroupId groupId = getGroupId(event); @@ -417,7 +417,7 @@ private String getDomainForEvent(DAGHistoryEvent event) { return domainId; } - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); HistoryEvent historyEvent = event.getHistoryEvent(); if (dagId == null || !HistoryEventType.isDAGSpecificEvent(historyEvent.getEventType())) { return domainId; diff --git a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/ATSHistoryLoggingService.java b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/ATSHistoryLoggingService.java index 6d035cce2c..c1883a9216 100644 --- a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/ATSHistoryLoggingService.java +++ b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/ATSHistoryLoggingService.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -291,7 +291,7 @@ public void handle(DAGHistoryEvent event) { private boolean isValidEvent(DAGHistoryEvent event) { HistoryEventType eventType = event.getHistoryEvent().getEventType(); - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); if (eventType.equals(HistoryEventType.DAG_SUBMITTED)) { DAGSubmittedEvent dagSubmittedEvent = @@ -333,7 +333,7 @@ private void handleEvents(List events) { for (DAGHistoryEvent event : events) { String domainId = getDomainForEvent(event); // skippedDags is updated in the above call so check again. - if (event.getDagID() != null && skippedDAGs.contains(event.getDagID())) { + if (event.getDAGID() != null && skippedDAGs.contains(event.getDAGID())) { continue; } List eventEntities = HistoryEventTimelineConversion.convertToTimelineEntities( @@ -377,7 +377,7 @@ private String getDomainForEvent(DAGHistoryEvent event) { return domainId; } - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); HistoryEvent historyEvent = event.getHistoryEvent(); if (dagId == null || !HistoryEventType.isDAGSpecificEvent(historyEvent.getEventType())) { return domainId; diff --git a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java index c40d3a8f78..b9457a0761 100644 --- a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java +++ b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java @@ -32,7 +32,6 @@ import org.apache.tez.dag.api.EdgeProperty; import org.apache.tez.dag.api.TezUncheckedException; import org.apache.tez.dag.api.oldrecords.TaskAttemptState; -import org.apache.tez.dag.api.oldrecords.TaskState; import org.apache.tez.dag.api.records.DAGProtos.CallerContextProto; import org.apache.tez.dag.app.web.AMWebController; import org.apache.tez.dag.history.HistoryEvent; @@ -288,7 +287,7 @@ private static TimelineEntity convertContainerStoppedEvent(ContainerStoppedEvent private static TimelineEntity convertDAGFinishedEvent(DAGFinishedEvent event) { TimelineEntity atsEntity = new TimelineEntity(); - atsEntity.setEntityId(event.getDagID().toString()); + atsEntity.setEntityId(event.getDAGID().toString()); atsEntity.setEntityType(EntityTypes.TEZ_DAG_ID.name()); TimelineEvent finishEvt = new TimelineEvent(); @@ -298,7 +297,7 @@ private static TimelineEntity convertDAGFinishedEvent(DAGFinishedEvent event) { atsEntity.addPrimaryFilter(ATSConstants.USER, event.getUser()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getDagID().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(ATSConstants.DAG_NAME, event.getDagName()); atsEntity.addPrimaryFilter(ATSConstants.STATUS, event.getState().name()); if (event.getDAGPlan().hasCallerContext() @@ -327,10 +326,10 @@ private static TimelineEntity convertDAGFinishedEvent(DAGFinishedEvent event) { private static TimelineEntity convertDAGFinishedToDAGExtraInfoEntity(DAGFinishedEvent event) { TimelineEntity atsEntity = new TimelineEntity(); - atsEntity.setEntityId(event.getDagID().toString()); + atsEntity.setEntityId(event.getDAGID().toString()); atsEntity.setEntityType(EntityTypes.TEZ_DAG_EXTRA_INFO.name()); - atsEntity.addRelatedEntity(EntityTypes.TEZ_DAG_ID.name(), event.getDagID().toString()); + atsEntity.addRelatedEntity(EntityTypes.TEZ_DAG_ID.name(), event.getDAGID().toString()); TimelineEvent submitEvt = new TimelineEvent(); submitEvt.setEventType(HistoryEventType.DAG_FINISHED.name()); @@ -345,7 +344,7 @@ private static TimelineEntity convertDAGFinishedToDAGExtraInfoEntity(DAGFinished private static TimelineEntity convertDAGInitializedEvent(DAGInitializedEvent event) { TimelineEntity atsEntity = new TimelineEntity(); - atsEntity.setEntityId(event.getDagID().toString()); + atsEntity.setEntityId(event.getDAGID().toString()); atsEntity.setEntityType(EntityTypes.TEZ_DAG_ID.name()); TimelineEvent initEvt = new TimelineEvent(); @@ -355,13 +354,13 @@ private static TimelineEntity convertDAGInitializedEvent(DAGInitializedEvent eve atsEntity.addPrimaryFilter(ATSConstants.USER, event.getUser()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getDagID().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(ATSConstants.DAG_NAME, event.getDagName()); atsEntity.addOtherInfo(ATSConstants.INIT_TIME, event.getInitTime()); if (event.getVertexNameIDMap() != null) { - Map nameIdStrMap = new TreeMap(); + Map nameIdStrMap = new TreeMap<>(); for (Entry entry : event.getVertexNameIDMap().entrySet()) { nameIdStrMap.put(entry.getKey(), entry.getValue().toString()); } @@ -373,7 +372,7 @@ private static TimelineEntity convertDAGInitializedEvent(DAGInitializedEvent eve private static TimelineEntity convertDAGStartedEvent(DAGStartedEvent event) { TimelineEntity atsEntity = new TimelineEntity(); - atsEntity.setEntityId(event.getDagID().toString()); + atsEntity.setEntityId(event.getDAGID().toString()); atsEntity.setEntityType(EntityTypes.TEZ_DAG_ID.name()); TimelineEvent startEvt = new TimelineEvent(); @@ -383,7 +382,7 @@ private static TimelineEntity convertDAGStartedEvent(DAGStartedEvent event) { atsEntity.addPrimaryFilter(ATSConstants.USER, event.getUser()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getDagID().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(ATSConstants.DAG_NAME, event.getDagName()); atsEntity.addOtherInfo(ATSConstants.START_TIME, event.getStartTime()); @@ -394,7 +393,7 @@ private static TimelineEntity convertDAGStartedEvent(DAGStartedEvent event) { private static TimelineEntity convertDAGSubmittedEvent(DAGSubmittedEvent event) { TimelineEntity atsEntity = new TimelineEntity(); - atsEntity.setEntityId(event.getDagID().toString()); + atsEntity.setEntityId(event.getDAGID().toString()); atsEntity.setEntityType(EntityTypes.TEZ_DAG_ID.name()); atsEntity.addRelatedEntity(EntityTypes.TEZ_APPLICATION.name(), @@ -412,7 +411,7 @@ private static TimelineEntity convertDAGSubmittedEvent(DAGSubmittedEvent event) atsEntity.addPrimaryFilter(ATSConstants.USER, event.getUser()); atsEntity.addPrimaryFilter(ATSConstants.DAG_NAME, event.getDAGName()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getDagID().getApplicationId().toString()); + event.getApplicationId().toString()); if (event.getDAGPlan().hasCallerContext() && event.getDAGPlan().getCallerContext().hasCallerId()) { @@ -450,10 +449,10 @@ private static TimelineEntity convertDAGSubmittedEvent(DAGSubmittedEvent event) private static TimelineEntity convertDAGSubmittedToDAGExtraInfoEntity(DAGSubmittedEvent event) { TimelineEntity atsEntity = new TimelineEntity(); - atsEntity.setEntityId(event.getDagID().toString()); + atsEntity.setEntityId(event.getDAGID().toString()); atsEntity.setEntityType(EntityTypes.TEZ_DAG_EXTRA_INFO.name()); - atsEntity.addRelatedEntity(EntityTypes.TEZ_DAG_ID.name(), event.getDagID().toString()); + atsEntity.addRelatedEntity(EntityTypes.TEZ_DAG_ID.name(), event.getDAGID().toString()); TimelineEvent submitEvt = new TimelineEvent(); submitEvt.setEventType(HistoryEventType.DAG_SUBMITTED.name()); @@ -477,13 +476,13 @@ private static TimelineEntity convertTaskAttemptFinishedEvent(TaskAttemptFinishe atsEntity.setEntityType(EntityTypes.TEZ_TASK_ATTEMPT_ID.name()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getTaskAttemptID().getTaskID().getVertexID().getDAGId().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(), - event.getTaskAttemptID().getTaskID().getVertexID().getDAGId().toString()); + event.getDAGID().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_VERTEX_ID.name(), - event.getTaskAttemptID().getTaskID().getVertexID().toString()); + event.getVertexID().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_TASK_ID.name(), - event.getTaskAttemptID().getTaskID().toString()); + event.getTaskID().toString()); TimelineEvent finishEvt = new TimelineEvent(); finishEvt.setEventType(HistoryEventType.TASK_ATTEMPT_FINISHED.name()); @@ -542,16 +541,16 @@ private static TimelineEntity convertTaskAttemptStartedEvent(TaskAttemptStartedE atsEntity.setStartTime(event.getStartTime()); atsEntity.addRelatedEntity(EntityTypes.TEZ_TASK_ID.name(), - event.getTaskAttemptID().getTaskID().toString()); + event.getTaskID().toString()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getTaskAttemptID().getTaskID().getVertexID().getDAGId().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(), - event.getTaskAttemptID().getTaskID().getVertexID().getDAGId().toString()); + event.getDAGID().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_VERTEX_ID.name(), - event.getTaskAttemptID().getTaskID().getVertexID().toString()); + event.getVertexID().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_TASK_ID.name(), - event.getTaskAttemptID().getTaskID().toString()); + event.getTaskID().toString()); TimelineEvent startEvt = new TimelineEvent(); startEvt.setEventType(HistoryEventType.TASK_ATTEMPT_STARTED.name()); @@ -579,11 +578,11 @@ private static TimelineEntity convertTaskFinishedEvent(TaskFinishedEvent event) atsEntity.setEntityType(EntityTypes.TEZ_TASK_ID.name()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getTaskID().getVertexID().getDAGId().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(), - event.getTaskID().getVertexID().getDAGId().toString()); + event.getDAGID().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_VERTEX_ID.name(), - event.getTaskID().getVertexID().toString()); + event.getVertexID().toString()); TimelineEvent finishEvt = new TimelineEvent(); finishEvt.setEventType(HistoryEventType.TASK_FINISHED.name()); @@ -614,14 +613,14 @@ private static TimelineEntity convertTaskStartedEvent(TaskStartedEvent event) { atsEntity.setEntityType(EntityTypes.TEZ_TASK_ID.name()); atsEntity.addRelatedEntity(EntityTypes.TEZ_VERTEX_ID.name(), - event.getTaskID().getVertexID().toString()); + event.getVertexID().toString()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getTaskID().getVertexID().getDAGId().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(), - event.getTaskID().getVertexID().getDAGId().toString()); + event.getDAGID().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_VERTEX_ID.name(), - event.getTaskID().getVertexID().toString()); + event.getVertexID().toString()); TimelineEvent startEvt = new TimelineEvent(); startEvt.setEventType(HistoryEventType.TASK_STARTED.name()); @@ -643,9 +642,9 @@ private static TimelineEntity convertVertexFinishedEvent(VertexFinishedEvent eve atsEntity.setEntityType(EntityTypes.TEZ_VERTEX_ID.name()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getVertexID().getDAGId().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(), - event.getVertexID().getDAGId().toString()); + event.getDAGID().toString()); TimelineEvent finishEvt = new TimelineEvent(); finishEvt.setEventType(HistoryEventType.VERTEX_FINISHED.name()); @@ -685,12 +684,12 @@ private static TimelineEntity convertVertexInitializedEvent(VertexInitializedEve atsEntity.setEntityType(EntityTypes.TEZ_VERTEX_ID.name()); atsEntity.addRelatedEntity(EntityTypes.TEZ_DAG_ID.name(), - event.getVertexID().getDAGId().toString()); + event.getDAGID().toString()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getVertexID().getDAGId().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(), - event.getVertexID().getDAGId().toString()); + event.getDAGID().toString()); TimelineEvent initEvt = new TimelineEvent(); initEvt.setEventType(HistoryEventType.VERTEX_INITIALIZED.name()); @@ -718,9 +717,9 @@ private static TimelineEntity convertVertexStartedEvent(VertexStartedEvent event atsEntity.setEntityType(EntityTypes.TEZ_VERTEX_ID.name()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getVertexID().getDAGId().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(), - event.getVertexID().getDAGId().toString()); + event.getDAGID().toString()); TimelineEvent startEvt = new TimelineEvent(); startEvt.setEventType(HistoryEventType.VERTEX_STARTED.name()); @@ -741,17 +740,17 @@ private static TimelineEntity convertVertexReconfigureDoneEvent( atsEntity.setEntityType(EntityTypes.TEZ_VERTEX_ID.name()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getVertexID().getDAGId().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(), - event.getVertexID().getDAGId().toString()); + event.getDAGID().toString()); TimelineEvent updateEvt = new TimelineEvent(); updateEvt.setEventType(HistoryEventType.VERTEX_CONFIGURE_DONE.name()); updateEvt.setTimestamp(event.getReconfigureDoneTime()); - Map eventInfo = new HashMap(); + Map eventInfo = new HashMap<>(); if (event.getSourceEdgeProperties() != null && !event.getSourceEdgeProperties().isEmpty()) { - Map updatedEdgeManagers = new HashMap(); + Map updatedEdgeManagers = new HashMap<>(); for (Entry entry : event.getSourceEdgeProperties().entrySet()) { updatedEdgeManagers.put(entry.getKey(), diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java index 0542b33d03..abe8c9dd53 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java @@ -28,6 +28,7 @@ import javax.annotation.Nullable; import org.apache.hadoop.io.Writable; +import org.apache.tez.dag.records.TaskAttemptIDAware; import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.util.StringInterner; @@ -35,7 +36,7 @@ * Class that encapsulates all the information to identify the unique * object that either generated an Event or is the recipient of an Event. */ -public class EventMetaData implements Writable { +public class EventMetaData implements Writable, TaskAttemptIDAware { public static enum EventProducerConsumerType { INPUT, @@ -83,6 +84,7 @@ public EventProducerConsumerType getEventGenerator() { return producerConsumerType; } + @Override public TezTaskAttemptID getTaskAttemptID() { return taskAttemptID; } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java index 63c251c054..15e756b853 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java @@ -30,12 +30,13 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Writable; import org.apache.tez.dag.api.ProcessorDescriptor; +import org.apache.tez.dag.records.TaskAttemptIDAware; import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.util.StringInterner; import com.google.common.collect.Lists; -public class TaskSpec implements Writable { +public class TaskSpec implements Writable, TaskAttemptIDAware { private TezTaskAttemptID taskAttemptId; private String dagName; @@ -128,7 +129,7 @@ public String getDAGName() { } public int getDagIdentifier() { - return taskAttemptId.getTaskID().getVertexID().getDAGId().getId(); + return taskAttemptId.getDAGID().getId(); } public int getVertexParallelism() { @@ -139,6 +140,7 @@ public String getVertexName() { return vertexName; } + @Override public TezTaskAttemptID getTaskAttemptID() { return taskAttemptId; } @@ -266,5 +268,4 @@ public String toString() { } return sb.toString(); } - } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java index a47dac1e0a..b645346654 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java @@ -118,8 +118,7 @@ public TezTaskContextImpl(Configuration conf, String[] workDirs, int appAttemptN @Override public ApplicationId getApplicationId() { - return taskAttemptID.getTaskID().getVertexID().getDAGId() - .getApplicationId(); + return taskAttemptID.getApplicationId(); } @Override @@ -154,12 +153,12 @@ public String getTaskVertexName() { @Override public int getTaskVertexIndex() { - return taskAttemptID.getTaskID().getVertexID().getId(); + return taskAttemptID.getVertexID().getId(); } @Override public int getDagIdentifier() { - return taskAttemptID.getTaskID().getVertexID().getDAGId().getId(); + return taskAttemptID.getDAGID().getId(); } @Override diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java index c82355a9fa..bc8c2d8391 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java @@ -61,7 +61,6 @@ import org.apache.tez.common.security.JobTokenIdentifier; import org.apache.tez.common.security.TokenCache; import org.apache.tez.dag.api.TezConfiguration; -import org.apache.tez.dag.api.TezConstants; import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.api.records.DAGProtos; import org.apache.tez.dag.records.TezVertexID; @@ -372,13 +371,12 @@ public URI apply(TezLocalResource input) { private void cleanupOnTaskChanged(ContainerTask containerTask) { Preconditions.checkState(!containerTask.shouldDie()); Preconditions.checkState(containerTask.getTaskSpec() != null); - TezVertexID newVertexID = containerTask.getTaskSpec().getTaskAttemptID().getTaskID() - .getVertexID(); + TezVertexID newVertexID = containerTask.getTaskSpec().getTaskAttemptID().getVertexID(); if (lastVertexID != null) { if (!lastVertexID.equals(newVertexID)) { objectRegistry.clearCache(ObjectRegistryImpl.ObjectLifeCycle.VERTEX); } - if (!lastVertexID.getDAGId().equals(newVertexID.getDAGId())) { + if (!lastVertexID.getDAGID().equals(newVertexID.getDAGID())) { objectRegistry.clearCache(ObjectRegistryImpl.ObjectLifeCycle.DAG); startedInputsMap = HashMultimap.create(); } diff --git a/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java b/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java index 3ed386e346..186ab7e659 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java +++ b/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java @@ -334,7 +334,7 @@ public boolean match(HistoryEvent incomingEvent) { TaskStartedEvent otherEvent = (TaskStartedEvent) incomingEvent; TaskStartedEvent conditionEvent = (TaskStartedEvent) event; // compare vertexId and taskId - return otherEvent.getTaskID().getVertexID().getId() == conditionEvent.getTaskID().getVertexID().getId() + return otherEvent.getVertexID().getId() == conditionEvent.getVertexID().getId() && otherEvent.getTaskID().getId() == conditionEvent.getTaskID().getId(); } break; @@ -344,7 +344,7 @@ public boolean match(HistoryEvent incomingEvent) { TaskFinishedEvent otherEvent = (TaskFinishedEvent) incomingEvent; TaskFinishedEvent conditionEvent = (TaskFinishedEvent) event; // compare vertexId and taskId - return otherEvent.getTaskID().getVertexID().getId() == conditionEvent.getTaskID().getVertexID().getId() + return otherEvent.getVertexID().getId() == conditionEvent.getVertexID().getId() && otherEvent.getTaskID().getId() == conditionEvent.getTaskID().getId(); } break; @@ -354,9 +354,9 @@ public boolean match(HistoryEvent incomingEvent) { TaskAttemptStartedEvent otherEvent = (TaskAttemptStartedEvent) incomingEvent; TaskAttemptStartedEvent conditionEvent = (TaskAttemptStartedEvent) event; // compare vertexId, taskId & taskAttemptId - return otherEvent.getTaskAttemptID().getTaskID().getVertexID().getId() - == conditionEvent.getTaskAttemptID().getTaskID().getVertexID().getId() - && otherEvent.getTaskAttemptID().getTaskID().getId() == conditionEvent.getTaskAttemptID().getTaskID().getId() + return otherEvent.getVertexID().getId() + == conditionEvent.getVertexID().getId() + && otherEvent.getTaskID().getId() == conditionEvent.getTaskID().getId() && otherEvent.getTaskAttemptID().getId() == conditionEvent.getTaskAttemptID().getId(); } break; @@ -366,9 +366,9 @@ public boolean match(HistoryEvent incomingEvent) { TaskAttemptFinishedEvent otherEvent = (TaskAttemptFinishedEvent) incomingEvent; TaskAttemptFinishedEvent conditionEvent = (TaskAttemptFinishedEvent) event; // compare vertexId, taskId & taskAttemptId - return otherEvent.getTaskAttemptID().getTaskID().getVertexID().getId() - == conditionEvent.getTaskAttemptID().getTaskID().getVertexID().getId() - && otherEvent.getTaskAttemptID().getTaskID().getId() == conditionEvent.getTaskAttemptID().getTaskID().getId() + return otherEvent.getVertexID().getId() + == conditionEvent.getVertexID().getId() + && otherEvent.getTaskID().getId() == conditionEvent.getTaskID().getId() && otherEvent.getTaskAttemptID().getId() == conditionEvent.getTaskAttemptID().getId(); } break; diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java b/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java index 6d3ab1c711..7fe3b3acc1 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java @@ -463,9 +463,9 @@ private List findTaskAttemptFinishedEvent( if (historyEvent.getEventType() == HistoryEventType.TASK_ATTEMPT_FINISHED) { TaskAttemptFinishedEvent taFinishedEvent = (TaskAttemptFinishedEvent) historyEvent; - if (taFinishedEvent.getTaskAttemptID().getTaskID().getVertexID() + if (taFinishedEvent.getVertexID() .getId() == vertexId - && taFinishedEvent.getTaskAttemptID().getTaskID().getId() == taskId) { + && taFinishedEvent.getTaskID().getId() == taskId) { resultEvents.add(taFinishedEvent); } } From 9f4cb313980d91e0661fc3bdc568815f470b4bba Mon Sep 17 00:00:00 2001 From: ghanko <54805928+ghanko@users.noreply.github.com> Date: Mon, 14 Feb 2022 10:23:54 +0100 Subject: [PATCH 358/512] TEZ-4379 Tez Hive Queries query page rows does not go over (#183) (Csaba Juhasz, Gergely Hanko reviewed by Laszlo Bodor) --- tez-ui/src/main/webapp/app/routes/server-side-ops.js | 2 +- .../src/main/webapp/tests/unit/routes/server-side-ops-test.js | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tez-ui/src/main/webapp/app/routes/server-side-ops.js b/tez-ui/src/main/webapp/app/routes/server-side-ops.js index 85b4d033c7..161b7b7498 100644 --- a/tez-ui/src/main/webapp/app/routes/server-side-ops.js +++ b/tez-ui/src/main/webapp/app/routes/server-side-ops.js @@ -74,7 +74,7 @@ export default AbstractRoute.extend({ return this.load(null, query).then(function (data) { if(that.get("controller.loadingMore")) { that.set("controller.loadingMore", false); - that.get("loadedValue").pushObjects(data); + that.get("loadedValue").pushObjects(data.content); return data; } }); diff --git a/tez-ui/src/main/webapp/tests/unit/routes/server-side-ops-test.js b/tez-ui/src/main/webapp/tests/unit/routes/server-side-ops-test.js index 59e04f75ba..055da433eb 100644 --- a/tez-ui/src/main/webapp/tests/unit/routes/server-side-ops-test.js +++ b/tez-ui/src/main/webapp/tests/unit/routes/server-side-ops-test.js @@ -136,7 +136,7 @@ test('loadNewPage test', function(assert) { let currentQuery = { val: {} }, - data = [], + data = {content: []}, fromId = "id1", route = this.subject({ controller: Ember.Object.create(), @@ -144,7 +144,7 @@ test('loadNewPage test', function(assert) { fromId: fromId, loadedValue: { pushObjects: function (objs) { - assert.equal(data, objs); + assert.equal(data.content, objs); } }, load: function (value, query) { From f62bf12cea38c9db0e56bd527a2b2a294aa10f66 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Attila=20T=C3=B3th?= Date: Tue, 15 Feb 2022 10:42:03 +0100 Subject: [PATCH 359/512] TEZ-4384: Remove unused EnvironmentUpdateUtils remaining from TEZ-1837 (#186) (Laszlo Attila Toth reviewed by Laszlo Bodor) --- .../tez/common/EnvironmentUpdateUtils.java | 127 ------------------ .../common/TestEnvironmentUpdateUtils.java | 96 ------------- 2 files changed, 223 deletions(-) delete mode 100644 tez-common/src/main/java/org/apache/tez/common/EnvironmentUpdateUtils.java delete mode 100644 tez-common/src/test/java/org/apache/tez/common/TestEnvironmentUpdateUtils.java diff --git a/tez-common/src/main/java/org/apache/tez/common/EnvironmentUpdateUtils.java b/tez-common/src/main/java/org/apache/tez/common/EnvironmentUpdateUtils.java deleted file mode 100644 index 0e597b3779..0000000000 --- a/tez-common/src/main/java/org/apache/tez/common/EnvironmentUpdateUtils.java +++ /dev/null @@ -1,127 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tez.common; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.util.Shell; - -import java.lang.reflect.Field; -import java.util.HashMap; -import java.util.Map; - -/** - * A utility class which allows one to dynamically update/change Environment variables - */ -@InterfaceAudience.Private -public class EnvironmentUpdateUtils { - - /** - * Allows dynamic update to the environment variables. After calling put, - * System.getenv(key) will then return value. - * - * @param key System environment variable - * @param value Value to assign to system environment variable - */ - public synchronized static void put(String key, String value){ - Map environment = new HashMap(System.getenv()); - environment.put(key, value); - if (!Shell.WINDOWS) { - updateEnvironment(environment); - } else { - updateEnvironmentOnWindows(environment); - } - } - - /** - * Allows dynamic update to a collection of environment variables. After - * calling putAll, System.getenv(key) will then return value for each entry - * in the map - * - * @param additionalEnvironment Collection where the key is the System - * environment variable and the value is the value to assign the system - * environment variable - */ - public synchronized static void putAll(Map additionalEnvironment) { - Map environment = new HashMap(System.getenv()); - environment.putAll(additionalEnvironment); - if (!Shell.WINDOWS) { - updateEnvironment(environment); - } else { - updateEnvironmentOnWindows(environment); - } - } - - /** - * Finds and modifies internal storage for system environment variables using - * reflection - * - * @param environment Collection where the key is the System - * environment variable and the value is the value to assign the system - * environment variable - */ - @SuppressWarnings("unchecked") - private static void updateEnvironment(Map environment) { - final Map currentEnv = System.getenv(); - copyMapValuesToPrivateField(currentEnv.getClass(), currentEnv, "m", environment); - } - - /** - * Finds and modifies internal storage for system environment variables using reflection. This - * method works only on windows. Note that the actual env is not modified, rather the copy of env - * which the JVM creates at the beginning of execution is. - * - * @param environment Collection where the key is the System - * environment variable and the value is the value to assign the system - * environment variable - */ - @SuppressWarnings("unchecked") - private static void updateEnvironmentOnWindows(Map environment) { - try { - Class processEnvironmentClass = Class.forName("java.lang.ProcessEnvironment"); - copyMapValuesToPrivateField(processEnvironmentClass, null, "theEnvironment", environment); - copyMapValuesToPrivateField(processEnvironmentClass, null, "theCaseInsensitiveEnvironment", - environment); - } catch (ClassNotFoundException e) { - throw new IllegalStateException("Failed to update Environment variables", e); - } - } - - /** - * Copies the given map values to the field specified by {@code fieldName} - * @param klass The {@code Class} of the object - * @param object The object to modify or null if the field is static - * @param fieldName The name of the field to set - * @param newMapValues The values to replace the current map. - */ - @SuppressWarnings("unchecked") - private static void copyMapValuesToPrivateField(Class klass, Object object, String fieldName, - Map newMapValues) { - try { - Field field = klass.getDeclaredField(fieldName); - field.setAccessible(true); - Map currentMap = (Map) field.get(object); - currentMap.clear(); - currentMap.putAll(newMapValues); - } catch (NoSuchFieldException e) { - throw new IllegalStateException("Failed to update Environment variables", e); - } catch (IllegalAccessException e) { - throw new IllegalStateException("Failed to update Environment variables", e); - } - } -} diff --git a/tez-common/src/test/java/org/apache/tez/common/TestEnvironmentUpdateUtils.java b/tez-common/src/test/java/org/apache/tez/common/TestEnvironmentUpdateUtils.java deleted file mode 100644 index a9cecc216a..0000000000 --- a/tez-common/src/test/java/org/apache/tez/common/TestEnvironmentUpdateUtils.java +++ /dev/null @@ -1,96 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tez.common; - -import static org.junit.Assert.assertEquals; - -import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; -import com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; - -public class TestEnvironmentUpdateUtils { - - @Test(timeout = 5000) - public void testMultipleUpdateEnvironment() { - EnvironmentUpdateUtils.put("test.environment1", "test.value1"); - EnvironmentUpdateUtils.put("test.environment2", "test.value2"); - assertEquals("Environment was not set propertly", "test.value1", System.getenv("test.environment1")); - assertEquals("Environment was not set propertly", "test.value2", System.getenv("test.environment2")); - } - - @Test(timeout = 5000) - public void testConcurrentRequests() throws InterruptedException { - int timeoutSecond = 5; - int concurThread = 10; - int exceptionCount = 0; - List> tasks = new ArrayList>(); - List> pendingTasks = new ArrayList>(); - final ExecutorService callbackExecutor = Executors.newFixedThreadPool(concurThread, - new ThreadFactoryBuilder().setDaemon(false).setNameFormat("CallbackExecutor").build()); - ListeningExecutorService taskExecutorService = - MoreExecutors.listeningDecorator(callbackExecutor); - while(concurThread > 0){ - ListenableFuture runningTaskFuture = - taskExecutorService.submit(new EnvironmentRequest()); - pendingTasks.add(runningTaskFuture); - concurThread--; - } - - //waiting for all threads submitted to thread pool - for (ListenableFuture future : pendingTasks) { - try { - future.get(); - } catch (ExecutionException e) { - exceptionCount++; - } - } - - //stop accepting new threads and shutdown threadpool - taskExecutorService.shutdown(); - try { - if(!taskExecutorService.awaitTermination(timeoutSecond, TimeUnit.SECONDS)) { - taskExecutorService.shutdownNow(); - } - } catch (InterruptedException ie) { - taskExecutorService.shutdownNow(); - } - - assertEquals(0, exceptionCount); - } - - private class EnvironmentRequest implements Callable { - - @Override - public Object call() throws Exception { - EnvironmentUpdateUtils.put("test.environment.concurrent" - +Thread.currentThread().getId(), "test.evironment.concurrent"); - return null; - } - } - } From a495fb69bb809007e5be507d321b9e5a876a952b Mon Sep 17 00:00:00 2001 From: Mark Bathori <57758037+mbathori-cloudera@users.noreply.github.com> Date: Fri, 18 Feb 2022 09:25:36 +0100 Subject: [PATCH 360/512] TEZ-4389: TestSecureShuffle fails if localhost name is different than "localhost" (#191) (Mark Bathori reviewed by Laszlo Bodor) --- .../test/java/org/apache/tez/test/TestSecureShuffle.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java b/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java index 7857c1ca2f..f9d4469779 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java @@ -302,10 +302,15 @@ public static X509Certificate generateCertificate(String dn, KeyPair pair, int d KeyPair keyPair = pair; X509V3CertificateGenerator certGen = new X509V3CertificateGenerator(); + String hostName = InetAddress.getLocalHost().getHostName(); String hostAddress = InetAddress.getLocalHost().getHostAddress(); certGen.addExtension(X509Extensions.SubjectAlternativeName, false, - new GeneralNames(new GeneralName[] { new GeneralName(GeneralName.iPAddress, hostAddress), - new GeneralName(GeneralName.dNSName, "localhost") })); + new GeneralNames(new GeneralName[] { + new GeneralName(GeneralName.iPAddress, hostAddress), + new GeneralName(GeneralName.dNSName, hostName), + new GeneralName(GeneralName.dNSName, "localhost") + }) + ); X500Principal dnName = new X500Principal(dn); certGen.setSerialNumber(sn); From 5241f58a5945dfce87a5b4e7ec81caf0c6f021ac Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Fri, 18 Feb 2022 20:12:32 +0100 Subject: [PATCH 361/512] TEZ-4300: Download protoc automatically compile/development time (#115) --- BUILDING.txt | 6 +- build-tools/.gitignore | 2 + build-tools/install-protobuf.sh | 76 +++++++++++++++++-- build-tools/protocw | 32 ++++++++ pom.xml | 22 ++++++ tez-api/pom.xml | 22 ++++++ .../tez-protobuf-history-plugin/pom.xml | 15 +++- 7 files changed, 167 insertions(+), 8 deletions(-) create mode 100644 build-tools/.gitignore create mode 100755 build-tools/protocw diff --git a/BUILDING.txt b/BUILDING.txt index 875bf3e097..ae81d9c6c8 100644 --- a/BUILDING.txt +++ b/BUILDING.txt @@ -117,8 +117,10 @@ It's important to note that maven will still include tez-ui project, but all of ---------------------------------------------------------------------------------- Protocol Buffer compiler: -The version of Protocol Buffer compiler, protoc, must be 2.5.0 and match the -version of the protobuf JAR. +The version of Protocol Buffer compiler, protoc, can be defined on-the-fly as: + $ mvn clean install -DskipTests -pl ./tez-api -Dprotobuf.version=3.7.1 + +The default version is defined in the root pom.xml. If you have multiple versions of protoc in your system, you can set in your build shell the PROTOC_PATH environment variable to point to the one you diff --git a/build-tools/.gitignore b/build-tools/.gitignore new file mode 100644 index 0000000000..adfc42ea48 --- /dev/null +++ b/build-tools/.gitignore @@ -0,0 +1,2 @@ +protobuf + diff --git a/build-tools/install-protobuf.sh b/build-tools/install-protobuf.sh index 902049dab1..c28729a8a7 100755 --- a/build-tools/install-protobuf.sh +++ b/build-tools/install-protobuf.sh @@ -1,4 +1,4 @@ -#!/bin/sh +#!/bin/bash # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -16,7 +16,73 @@ # See the License for the specific language governing permissions and # limitations under the License. -set -ex -wget https://github.com/google/protobuf/releases/download/v2.5.0/protobuf-2.5.0.tar.gz -tar -xzvf protobuf-2.5.0.tar.gz -cd protobuf-2.5.0 && ./configure --prefix=/usr && make && sudo make install +# This script attempts to install an arbitrary version of protobuf if needed. +# The desired version should be the first parameter: $1. +# Typical usage: the script is automatically called from tez-api (by maven) during the build process. + +# This script runs from build-tools folder. The user can remove +# the dynamically installed protobuf anytime like: +# rm -rf ./build-tools/protobuf/ #from root folder + +set -x +PROTOBUF_VERSION=${1:-2.5.0} +PROTOBUF_MAJOR_VERSION=$(echo "$PROTOBUF_VERSION" | cut -d. -f1) +if [ -n "$ZSH_VERSION" ]; then + SCRIPT_DIR="${0:a:h}" +else + SCRIPT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" +fi + +function install_protobuf { + # before protobuf 3, there is no pre-compiled executables are host on github, let's try to build and make it + if (( PROTOBUF_MAJOR_VERSION < 3 )); then + wget "https://github.com/google/protobuf/releases/download/v$PROTOBUF_VERSION/protobuf-$PROTOBUF_VERSION.tar.gz" + tar -xzvf "protobuf-$PROTOBUF_VERSION.tar.gz" + rm "protobuf-$PROTOBUF_VERSION.tar.gz" + cd "protobuf-$PROTOBUF_VERSION" && ./configure --prefix=/usr && make && sudo make install + # since protobuf 3, there are precompiled protoc executables on github, let's quickly download and use it + else + ARCH=`uname -m` + case "$(uname -s)" in + Darwin) + FILE_NAME="protoc-$PROTOBUF_VERSION-osx-$ARCH" + ;; + Linux) + if test $ARCH = "aarch64"; then + ARCH="aarch_64" + fi + FILE_NAME="protoc-$PROTOBUF_VERSION-linux-$ARCH" + ;; + *) + echo "Unsupported OS returned by uname -s, you'll have to install protobuf 3.x manually" + exit 1 + ;; + esac + rm -f "$FILE_NAME.zip" #cleanup unfinished file if any + wget "https://github.com/google/protobuf/releases/download/v$PROTOBUF_VERSION/$FILE_NAME.zip" + mkdir "$SCRIPT_DIR/protobuf" + unzip -o "$FILE_NAME.zip" -d "$SCRIPT_DIR/protobuf" + rm "$FILE_NAME.zip" + fi +} + +if test -f "$SCRIPT_DIR/protobuf/bin/protoc"; then + PROTOBUF_INSTALLED_VERSION=$("$SCRIPT_DIR/protobuf/bin/protoc" --version) +else + PROTOBUF_INSTALLED_VERSION=$(protoc --version) +fi + +PROTOC_EXIT_CODE=$? + +if [ $PROTOC_EXIT_CODE -eq 0 ]; then + PROTOBUF_INSTALLED_VERSION=$(echo "$PROTOBUF_INSTALLED_VERSION" | tr -s ' ' | cut -d ' ' -f 2) + if [ "$PROTOBUF_INSTALLED_VERSION" == "$PROTOBUF_VERSION" ]; then + echo "Current protobuf version is equal to the requested ($PROTOBUF_INSTALLED_VERSION), exiting..." + else + echo "Current protobuf version ($PROTOBUF_INSTALLED_VERSION) is not equal to the requested ($PROTOBUF_VERSION), installing $PROTOBUF_VERSION" + install_protobuf + fi +else + echo "protoc --version command had non-zero return value, need to install probuf" + install_protobuf +fi diff --git a/build-tools/protocw b/build-tools/protocw new file mode 100755 index 0000000000..6196071815 --- /dev/null +++ b/build-tools/protocw @@ -0,0 +1,32 @@ +#!/bin/bash + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +### This is a protoc wrapper for tez, which can dinamically call protoc from a downloaded protobuf. + +if [ -n "$ZSH_VERSION" ]; then + SCRIPT_DIR="${0:a:h}" +else + SCRIPT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" +fi + +if test -f "$SCRIPT_DIR/protobuf/bin/protoc"; then + "$SCRIPT_DIR/protobuf/bin/protoc" "$@" +else + protoc "$@" +fi +exit $? diff --git a/pom.xml b/pom.xml index aff586bad9..9499fe62b6 100644 --- a/pom.xml +++ b/pom.xml @@ -1310,6 +1310,28 @@ + + + protoc-path-env-variable-not-defined + + + !env.PROTOC_PATH + + + + ${basedir}/../build-tools/protocw + + diff --git a/tez-api/pom.xml b/tez-api/pom.xml index 8cab40c172..4e1825b09d 100644 --- a/tez-api/pom.xml +++ b/tez-api/pom.xml @@ -145,6 +145,28 @@ org.apache.rat apache-rat-plugin + + + exec-maven-plugin + org.codehaus.mojo + 1.6.0 + + + Install protobuf + initialize + + exec + + + ${basedir}/../build-tools/install-protobuf.sh + ${protobuf.version} + + + + org.apache.hadoop hadoop-maven-plugins diff --git a/tez-plugins/tez-protobuf-history-plugin/pom.xml b/tez-plugins/tez-protobuf-history-plugin/pom.xml index ab06981f3d..9d82ed3965 100644 --- a/tez-plugins/tez-protobuf-history-plugin/pom.xml +++ b/tez-plugins/tez-protobuf-history-plugin/pom.xml @@ -85,8 +85,21 @@ - + + + + protoc-path-env-variable-not-defined + + + !env.PROTOC_PATH + + + + ${basedir}/../../build-tools/protocw + + + From fa74b95876e1074b5ba2eec6ddb0eea2699c9149 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Tue, 22 Feb 2022 10:32:30 +0100 Subject: [PATCH 362/512] TEZ-4388: Asynchttpclient can cause stuck TezChild processes (#189) (Laszlo Bodor reviewed by Rajesh Balamohan) --- .../tez/util/TezRuntimeShutdownHandler.java | 44 +++++++++++++++++++ .../org/apache/tez/runtime/task/TezChild.java | 10 ++++- .../http/async/netty/AsyncHttpConnection.java | 11 +++++ 3 files changed, 63 insertions(+), 2 deletions(-) create mode 100644 tez-common/src/main/java/org/apache/tez/util/TezRuntimeShutdownHandler.java diff --git a/tez-common/src/main/java/org/apache/tez/util/TezRuntimeShutdownHandler.java b/tez-common/src/main/java/org/apache/tez/util/TezRuntimeShutdownHandler.java new file mode 100644 index 0000000000..4881e08ab9 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/util/TezRuntimeShutdownHandler.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.util; + +import java.util.ArrayList; +import java.util.List; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public final class TezRuntimeShutdownHandler { + private static final Logger LOG = LoggerFactory.getLogger(TezRuntimeShutdownHandler.class); + + private static final List shutdownTasks = new ArrayList<>(); + + private TezRuntimeShutdownHandler() { + } + + public static void addShutdownTask(Runnable r) { + shutdownTasks.add(r); + } + + public static synchronized void shutdown() { + LOG.info("Handling {} shutdown tasks", shutdownTasks.size()); + for (Runnable shutdownTask : shutdownTasks) { + shutdownTask.run(); + } + } +} diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java index bc8c2d8391..b89b12db2b 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java @@ -71,6 +71,7 @@ import org.apache.tez.runtime.api.impl.ExecutionContextImpl; import org.apache.tez.runtime.common.objectregistry.ObjectRegistryImpl; import org.apache.tez.runtime.internals.api.TaskReporterInterface; +import org.apache.tez.util.TezRuntimeShutdownHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -390,8 +391,10 @@ public void shutdown() { LOG.info("Shutting down container {}", containerIdString); // It's possible that there's pending tasks on the executor. Those should be cancelled. List pendingRunnables = executor.shutdownNow(); + LOG.info("There are {} runnables in shared executor, cancelling those...", pendingRunnables.size()); for (Runnable r : pendingRunnables) { - LOG.info("Cancelling pending runnables during TezChild shutdown for containerId={}", containerIdString); + LOG.info("Cancelling pending runnable ({}) during TezChild shutdown for containerId={}", r.hashCode(), + containerIdString); ((FutureTask)r).cancel(false); } if (taskReporter != null) { @@ -401,6 +404,8 @@ public void shutdown() { RPC.stopProxy(umbilical); } } + TezRuntimeShutdownHandler.shutdown(); + LOG.info("TezChild shutdown finished"); } public static class ContainerExecutionResult { @@ -522,7 +527,8 @@ public static void main(String[] args) throws IOException, InterruptedException, System.getenv(), pid, new ExecutionContextImpl(System.getenv(Environment.NM_HOST.name())), credentials, Runtime.getRuntime().maxMemory(), System .getenv(ApplicationConstants.Environment.USER.toString()), null, true, hadoopShim); - tezChild.run(); + ContainerExecutionResult result = tezChild.run(); + LOG.info("TezChild is about to exit from main(), run() returned result: {}", result.toString()); } private void handleError(Throwable t) { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java b/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java index 63b8934821..215e63af58 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java @@ -35,6 +35,7 @@ import org.apache.tez.runtime.library.common.security.SecureShuffleUtils; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.ShuffleHeader; import org.apache.tez.util.StopWatch; +import org.apache.tez.util.TezRuntimeShutdownHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -102,6 +103,16 @@ private void initClient(HttpConnectionParams httpConnParams) throws IOException .build(); DefaultAsyncHttpClientConfig config = builder.build(); httpAsyncClient = new DefaultAsyncHttpClient(config); + TezRuntimeShutdownHandler.addShutdownTask(() -> { + try { + if (httpAsyncClient != null) { + httpAsyncClient.close(); + httpAsyncClient = null; + } + } catch (IOException e) { + LOG.warn("Error while closing async client (this won't block shutdown)", e); + } + }); } } } From 132ea4c1b3505a066d300173bd967d1bc04b142f Mon Sep 17 00:00:00 2001 From: Mark Bathori <57758037+mbathori-cloudera@users.noreply.github.com> Date: Thu, 10 Mar 2022 20:54:24 +0100 Subject: [PATCH 363/512] TEZ-4359: ShuffleHandler: Make sure of properly releasing netty reference counted objects (#194) (Mark Bathori reviewed by Laszlo Bodor) --- .../java/org/apache/tez/auxservices/ShuffleHandler.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java index 9730be1869..7e6fd750f8 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java @@ -1499,7 +1499,6 @@ protected void sendError(ChannelHandlerContext ctx, HttpResponseStatus status) { protected void sendError(ChannelHandlerContext ctx, String message, HttpResponseStatus status) { FullHttpResponse response = new DefaultFullHttpResponse(HTTP_1_1, status); sendError(ctx, message, response); - response.release(); } protected void sendError(ChannelHandlerContext ctx, String message, FullHttpResponse response) { @@ -1517,7 +1516,6 @@ private void sendFakeShuffleHeaderWithError(ChannelHandlerContext ctx, String me header.write(out); sendError(ctx, wrappedBuffer(out.getData(), 0, out.getLength()), fullResponse); - fullResponse.release(); } protected void sendError(ChannelHandlerContext ctx, ByteBuf content, @@ -1532,6 +1530,11 @@ protected void sendError(ChannelHandlerContext ctx, ByteBuf content, // Close the connection as soon as the error message is sent. ctx.channel().writeAndFlush(response).addListener(ChannelFutureListener.CLOSE); + /* + * The general rule of thumb is that the party that accesses a reference-counted object last + * is also responsible for the destruction of that reference-counted object. + */ + content.release(); } @Override From 20873a3a1008d3572f0d0cfefe1c146f8f42449e Mon Sep 17 00:00:00 2001 From: Syed Shameerur Rahman Date: Wed, 16 Mar 2022 23:48:39 +0530 Subject: [PATCH 364/512] TEZ-3363: Delete intermediate data at the vertex level for Shuffle Handler (#60) (Syed Shameerur Rahman reviewed by Laszlo Bodor) --- .../apache/tez/dag/api/TezConfiguration.java | 20 +- .../tez/common/DagContainerLauncher.java | 6 + .../org/apache/tez/dag/app/DAGAppMaster.java | 4 + .../dag/app/dag/event/VertexEventType.java | 1 + .../dag/event/VertexShuffleDataDeletion.java | 43 ++ .../apache/tez/dag/app/dag/impl/DAGImpl.java | 8 + .../tez/dag/app/dag/impl/VertexImpl.java | 72 ++- .../VertexShuffleDataDeletionContext.java | 96 ++++ .../launcher/ContainerLauncherManager.java | 8 + .../launcher/ContainerLauncherWrapper.java | 9 + .../tez/dag/app/launcher/DeletionTracker.java | 7 + .../dag/app/launcher/DeletionTrackerImpl.java | 23 + .../app/launcher/LocalContainerLauncher.java | 15 +- .../launcher/TezContainerLauncherImpl.java | 16 +- .../app/launcher/VertexDeleteRunnable.java | 82 +++ .../tez/dag/app/dag/impl/TestVertexImpl.java | 529 ++++++++++++++++-- .../TestContainerLauncherWrapper.java | 2 +- .../tez/auxservices/ShuffleHandler.java | 48 ++ .../tez/auxservices/TestShuffleHandler.java | 78 +++ .../library/common/TezRuntimeUtils.java | 19 + 20 files changed, 1020 insertions(+), 66 deletions(-) create mode 100644 tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexShuffleDataDeletion.java create mode 100644 tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexShuffleDataDeletionContext.java create mode 100644 tez-dag/src/main/java/org/apache/tez/dag/app/launcher/VertexDeleteRunnable.java diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 16d1dfcb70..71ebfee10b 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -883,6 +883,22 @@ public TezConfiguration(boolean loadDefaults) { + "dag.cleanup.on.completion"; public static final boolean TEZ_AM_DAG_CLEANUP_ON_COMPLETION_DEFAULT = false; + /** + * Integer value. Instructs AM to delete vertex shuffle data if a vertex and all its + * child vertices at a certain depth are completed. Value less than or equal to 0 indicates the feature + * is disabled. + * Let's say we have a dag Map1 - Reduce2 - Reduce3 - Reduce4. + * case:1 height = 1 + * when Reduce 2 completes all the shuffle data of Map1 will be deleted and so on for other vertex. + * case: 2 height = 2 + * when Reduce 3 completes all the shuffle data of Map1 will be deleted and so on for other vertex. + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty(type="integer") + public static final String TEZ_AM_VERTEX_CLEANUP_HEIGHT = TEZ_AM_PREFIX + + "vertex.cleanup.height"; + public static final int TEZ_AM_VERTEX_CLEANUP_HEIGHT_DEFAULT = 0; + /** * Boolean value. Instructs AM to delete intermediate attempt data for failed task attempts. */ @@ -893,8 +909,8 @@ public TezConfiguration(boolean loadDefaults) { public static final boolean TEZ_AM_TASK_ATTEMPT_CLEANUP_ON_FAILURE_DEFAULT = false; /** - * Int value. Upper limit on the number of threads used to delete DAG directories and failed task attempts - * directories on nodes. + * Int value. Upper limit on the number of threads used to delete DAG directories, + * Vertex directories and failed task attempts directories on nodes. */ @ConfigurationScope(Scope.AM) @ConfigurationProperty(type="integer") diff --git a/tez-common/src/main/java/org/apache/tez/common/DagContainerLauncher.java b/tez-common/src/main/java/org/apache/tez/common/DagContainerLauncher.java index 6bda0a8da3..c2337af77e 100644 --- a/tez-common/src/main/java/org/apache/tez/common/DagContainerLauncher.java +++ b/tez-common/src/main/java/org/apache/tez/common/DagContainerLauncher.java @@ -24,9 +24,12 @@ import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.serviceplugins.api.ContainerLauncher; import org.apache.tez.serviceplugins.api.ContainerLauncherContext; +import java.util.Set; + /** * Plugin to allow custom container launchers to be written to launch containers that want to * support cleanup of DAG level directories upon DAG completion in session mode. The directories are created by @@ -43,6 +46,9 @@ public DagContainerLauncher(ContainerLauncherContext containerLauncherContext) { public abstract void dagComplete(TezDAGID dag, JobTokenSecretManager jobTokenSecretManager); + public abstract void vertexComplete(TezVertexID vertex, JobTokenSecretManager jobTokenSecretManager, + Set nodeIdList); + public abstract void taskAttemptFailed(TezTaskAttemptID taskAttemptID, JobTokenSecretManager jobTokenSecretManager, NodeId nodeId); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index 972fadf854..58288612bb 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -2739,6 +2739,10 @@ String buildPluginComponentLog(List namedEntityDescriptor return sb.toString(); } + public void vertexComplete(TezVertexID completedVertexID, Set nodesList) { + getContainerLauncherManager().vertexComplete(completedVertexID, jobTokenSecretManager, nodesList); + } + public void taskAttemptFailed(TezTaskAttemptID attemptID, NodeId nodeId) { getContainerLauncherManager().taskAttemptFailed(attemptID, jobTokenSecretManager, nodeId); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventType.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventType.java index 15be94dfa1..ed325290cb 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventType.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventType.java @@ -34,6 +34,7 @@ public enum VertexEventType { V_START, V_SOURCE_TASK_ATTEMPT_COMPLETED, V_SOURCE_VERTEX_STARTED, + V_DELETE_SHUFFLE_DATA, //Producer:Task V_TASK_COMPLETED, diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexShuffleDataDeletion.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexShuffleDataDeletion.java new file mode 100644 index 0000000000..8ea3a154b3 --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexShuffleDataDeletion.java @@ -0,0 +1,43 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.app.dag.event; + +import org.apache.tez.dag.app.dag.Vertex; + + +public class VertexShuffleDataDeletion extends VertexEvent { + // child vertex + private Vertex sourceVertex; + // parent vertex + private Vertex targetVertex; + + public VertexShuffleDataDeletion(Vertex sourceVertex, Vertex targetVertex) { + super(targetVertex.getVertexId(), VertexEventType.V_DELETE_SHUFFLE_DATA); + this.sourceVertex = sourceVertex; + this.targetVertex = targetVertex; + } + + public Vertex getSourceVertex() { + return sourceVertex; + } + + public Vertex getTargetVertex() { + return targetVertex; + } +} \ No newline at end of file diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java index c9337e473a..aa28e02441 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java @@ -51,6 +51,7 @@ import org.apache.tez.common.counters.LimitExceededException; import org.apache.tez.dag.app.dag.event.DAGEventTerminateDag; import org.apache.tez.dag.app.dag.event.DiagnosableEvent; +import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.state.OnStateChangedCallback; import org.apache.tez.state.StateMachineTez; import org.slf4j.Logger; @@ -1772,6 +1773,13 @@ private static void parseVertexEdges(DAGImpl dag, Map edgePlan vertex.setInputVertices(inVertices); vertex.setOutputVertices(outVertices); + boolean cleanupShuffleDataAtVertexLevel = dag.dagConf.getInt(TezConfiguration.TEZ_AM_VERTEX_CLEANUP_HEIGHT, + TezConfiguration.TEZ_AM_VERTEX_CLEANUP_HEIGHT_DEFAULT) > 0 && ShuffleUtils.isTezShuffleHandler(dag.dagConf); + if (cleanupShuffleDataAtVertexLevel) { + int deletionHeight = dag.dagConf.getInt(TezConfiguration.TEZ_AM_VERTEX_CLEANUP_HEIGHT, + TezConfiguration.TEZ_AM_VERTEX_CLEANUP_HEIGHT_DEFAULT); + ((VertexImpl) vertex).initShuffleDeletionContext(deletionHeight); + } } /** diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index 934dd4e76e..e55b10a659 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -114,6 +114,8 @@ import org.apache.tez.dag.app.dag.RootInputInitializerManager; import org.apache.tez.dag.app.dag.StateChangeNotifier; import org.apache.tez.dag.app.dag.Task; +import org.apache.tez.dag.app.dag.TaskAttempt; +import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.tez.dag.app.dag.TaskAttemptStateInternal; import org.apache.tez.dag.app.dag.TaskTerminationCause; import org.apache.tez.dag.app.dag.Vertex; @@ -130,6 +132,7 @@ import org.apache.tez.dag.app.dag.event.TaskEventScheduleTask; import org.apache.tez.dag.app.dag.event.TaskEventTermination; import org.apache.tez.dag.app.dag.event.TaskEventType; +import org.apache.tez.dag.app.dag.event.VertexShuffleDataDeletion; import org.apache.tez.dag.app.dag.event.VertexEvent; import org.apache.tez.dag.app.dag.event.VertexEventCommitCompleted; import org.apache.tez.dag.app.dag.event.VertexEventInputDataInformation; @@ -187,6 +190,7 @@ import org.apache.tez.runtime.api.impl.TaskSpec; import org.apache.tez.runtime.api.impl.TaskStatistics; import org.apache.tez.runtime.api.impl.TezEvent; +import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.state.OnStateChangedCallback; import org.apache.tez.state.StateMachineTez; import org.apache.tez.util.StringInterner; @@ -556,7 +560,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl VertexEventType.V_ROUTE_EVENT, VertexEventType.V_SOURCE_TASK_ATTEMPT_COMPLETED, VertexEventType.V_TASK_ATTEMPT_COMPLETED, - VertexEventType.V_TASK_RESCHEDULED)) + VertexEventType.V_TASK_RESCHEDULED, + VertexEventType.V_DELETE_SHUFFLE_DATA)) // Transitions from SUCCEEDED state .addTransition( @@ -592,6 +597,9 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl .addTransition(VertexState.SUCCEEDED, VertexState.SUCCEEDED, VertexEventType.V_TASK_ATTEMPT_COMPLETED, new TaskAttemptCompletedEventTransition()) + .addTransition(VertexState.SUCCEEDED, VertexState.SUCCEEDED, + VertexEventType.V_DELETE_SHUFFLE_DATA, + new VertexShuffleDeleteTransition()) // Transitions from FAILED state @@ -613,7 +621,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl VertexEventType.V_ROOT_INPUT_INITIALIZED, VertexEventType.V_SOURCE_TASK_ATTEMPT_COMPLETED, VertexEventType.V_NULL_EDGE_INITIALIZED, - VertexEventType.V_INPUT_DATA_INFORMATION)) + VertexEventType.V_INPUT_DATA_INFORMATION, + VertexEventType.V_DELETE_SHUFFLE_DATA)) // Transitions from KILLED state .addTransition( @@ -635,7 +644,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl VertexEventType.V_TASK_COMPLETED, VertexEventType.V_ROOT_INPUT_INITIALIZED, VertexEventType.V_NULL_EDGE_INITIALIZED, - VertexEventType.V_INPUT_DATA_INFORMATION)) + VertexEventType.V_INPUT_DATA_INFORMATION, + VertexEventType.V_DELETE_SHUFFLE_DATA)) // No transitions from INTERNAL_ERROR state. Ignore all. .addTransition( @@ -655,7 +665,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl VertexEventType.V_INTERNAL_ERROR, VertexEventType.V_ROOT_INPUT_INITIALIZED, VertexEventType.V_NULL_EDGE_INITIALIZED, - VertexEventType.V_INPUT_DATA_INFORMATION)) + VertexEventType.V_INPUT_DATA_INFORMATION, + VertexEventType.V_DELETE_SHUFFLE_DATA)) // create the topology tables .installTopology(); @@ -729,6 +740,9 @@ private void augmentStateMachine() { @VisibleForTesting Map sourceVertices; private Map targetVertices; + private boolean cleanupShuffleDataAtVertexLevel; + @VisibleForTesting + VertexShuffleDataDeletionContext vShuffleDeletionContext; Set uninitializedEdges = Sets.newHashSet(); // using a linked hash map to conveniently map edge names to a contiguous index LinkedHashMap ioIndices = Maps.newLinkedHashMap(); @@ -1151,7 +1165,9 @@ public VertexImpl(TezVertexID vertexId, VertexPlan vertexPlan, .append(", ContainerLauncher=").append(containerLauncherIdentifier).append(":").append(containerLauncherName) .append(", TaskCommunicator=").append(taskCommunicatorIdentifier).append(":").append(taskCommName); LOG.info(sb.toString()); - + cleanupShuffleDataAtVertexLevel = vertexConf.getInt(TezConfiguration.TEZ_AM_VERTEX_CLEANUP_HEIGHT, + TezConfiguration.TEZ_AM_VERTEX_CLEANUP_HEIGHT_DEFAULT) > 0 && + ShuffleUtils.isTezShuffleHandler(vertexConf); stateMachine = new StateMachineTez( stateMachineFactory.make(this), this); augmentStateMachine(); @@ -2306,6 +2322,12 @@ static VertexState checkTasksForCompletion(final VertexImpl vertex) { if((vertexSucceeded || vertexFailuresBelowThreshold) && vertex.terminationCause == null) { if(vertexSucceeded) { LOG.info("All tasks have succeeded, vertex:" + vertex.logIdentifier); + if (vertex.cleanupShuffleDataAtVertexLevel) { + + for (Vertex v : vertex.vShuffleDeletionContext.getAncestors()) { + vertex.eventHandler.handle(new VertexShuffleDataDeletion(vertex, v)); + } + } } else { LOG.info("All tasks in the vertex " + vertex.logIdentifier + " have completed and the percentage of failed tasks (failed/total) (" + vertex.failedTaskCount + "/" + vertex.numTasks + ") is less that the threshold of " + vertex.maxFailuresPercent); vertex.addDiagnostic("Vertex succeeded as percentage of failed tasks (failed/total) (" + vertex.failedTaskCount + "/" + vertex.numTasks + ") is less that the threshold of " + vertex.maxFailuresPercent); @@ -3758,6 +3780,36 @@ public VertexState transition(VertexImpl vertex, VertexEvent event) { } } + private static class VertexShuffleDeleteTransition implements + SingleArcTransition { + + @Override + public void transition(VertexImpl vertex, VertexEvent event) { + int incompleteChildrenVertices = vertex.vShuffleDeletionContext.getIncompleteChildrenVertices(); + incompleteChildrenVertices = incompleteChildrenVertices - 1; + vertex.vShuffleDeletionContext.setIncompleteChildrenVertices(incompleteChildrenVertices); + // check if all the child vertices are completed + if (incompleteChildrenVertices == 0) { + LOG.info("Vertex shuffle data deletion for vertex name: " + + vertex.getName() + " with vertex id: " + vertex.getVertexId()); + // Get nodes of all the task attempts in vertex + Set nodes = Sets.newHashSet(); + Map tasksMap = vertex.getTasks(); + tasksMap.keySet().forEach(taskId -> { + Map taskAttemptMap = tasksMap.get(taskId).getAttempts(); + taskAttemptMap.keySet().forEach(attemptId -> { + nodes.add(taskAttemptMap.get(attemptId).getNodeId()); + }); + }); + vertex.appContext.getAppMaster().vertexComplete( + vertex.vertexId, nodes); + } else { + LOG.debug("The number of incomplete child vertex are {} for the vertex {}", + incompleteChildrenVertices, vertex.vertexId); + } + } + } + private static class TaskCompletedAfterVertexSuccessTransition implements MultipleArcTransition { @Override @@ -4930,4 +4982,14 @@ public boolean getTaskRescheduleRelaxedLocality() { public Map> getDownstreamBlamingHosts(){ return downstreamBlamingHosts; } + + /** + * Initialize context from vertex shuffle deletion. + * @param deletionHeight + */ + public void initShuffleDeletionContext(int deletionHeight) { + VertexShuffleDataDeletionContext vShuffleDeletionContext = new VertexShuffleDataDeletionContext(deletionHeight); + vShuffleDeletionContext.setSpannedVertices(this); + this.vShuffleDeletionContext = vShuffleDeletionContext; + } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexShuffleDataDeletionContext.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexShuffleDataDeletionContext.java new file mode 100644 index 0000000000..4ffdf11231 --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexShuffleDataDeletionContext.java @@ -0,0 +1,96 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.app.dag.impl; + +import org.apache.tez.dag.app.dag.Vertex; + +import java.util.HashSet; +import java.util.Set; + +public class VertexShuffleDataDeletionContext { + private int deletionHeight; + private int incompleteChildrenVertices; + private Set ancestors; + private Set children; + + public VertexShuffleDataDeletionContext(int deletionHeight) { + this.deletionHeight = deletionHeight; + this.incompleteChildrenVertices = 0; + this.ancestors = new HashSet<>(); + this.children = new HashSet<>(); + } + + public void setSpannedVertices(Vertex vertex) { + getSpannedVerticesAncestors(vertex, ancestors, deletionHeight); + getSpannedVerticesChildren(vertex, children, deletionHeight); + setIncompleteChildrenVertices(children.size()); + } + + /** + * get all the ancestor vertices at a particular depth. + */ + private static void getSpannedVerticesAncestors(Vertex vertex, Set ancestorVertices, int level) { + if (level == 0) { + ancestorVertices.add(vertex); + return; + } + + if (level == 1) { + ancestorVertices.addAll(vertex.getInputVertices().keySet()); + return; + } + + vertex.getInputVertices().forEach((inVertex, edge) -> getSpannedVerticesAncestors(inVertex, ancestorVertices, + level - 1)); + } + + /** + * get all the child vertices at a particular depth. + */ + private static void getSpannedVerticesChildren(Vertex vertex, Set childVertices, int level) { + if (level == 0) { + childVertices.add(vertex); + return; + } + + if (level == 1) { + childVertices.addAll(vertex.getOutputVertices().keySet()); + return; + } + + vertex.getOutputVertices().forEach((outVertex, edge) -> getSpannedVerticesChildren(outVertex, childVertices, + level - 1)); + } + + public void setIncompleteChildrenVertices(int incompleteChildrenVertices) { + this.incompleteChildrenVertices = incompleteChildrenVertices; + } + + public int getIncompleteChildrenVertices() { + return this.incompleteChildrenVertices; + } + + public Set getAncestors() { + return this.ancestors; + } + + public Set getChildren() { + return this.children; + } +} diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherManager.java index b0e0f0cf0d..65360d6d01 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherManager.java @@ -16,6 +16,7 @@ import java.net.UnknownHostException; import java.util.List; +import java.util.Set; import com.google.common.annotations.VisibleForTesting; import org.apache.tez.common.Preconditions; @@ -37,6 +38,7 @@ import org.apache.tez.dag.app.dag.event.DAGAppMasterEventUserServiceFatalError; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.serviceplugins.api.ContainerLaunchRequest; import org.apache.tez.serviceplugins.api.ContainerLauncher; import org.apache.tez.serviceplugins.api.ContainerLauncherContext; @@ -202,6 +204,12 @@ public void dagComplete(TezDAGID dag, JobTokenSecretManager secretManager) { } } + public void vertexComplete(TezVertexID vertex, JobTokenSecretManager secretManager, Set nodeIdList) { + for (int i = 0; i < containerLaunchers.length; i++) { + containerLaunchers[i].vertexComplete(vertex, secretManager, nodeIdList); + } + } + public void taskAttemptFailed(TezTaskAttemptID taskAttemptID, JobTokenSecretManager secretManager, NodeId nodeId) { for (int i = 0; i < containerLaunchers.length; i++) { containerLaunchers[i].taskAttemptFailed(taskAttemptID, secretManager, nodeId); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherWrapper.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherWrapper.java index 5d262bdab4..4703abe100 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherWrapper.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherWrapper.java @@ -14,11 +14,14 @@ package org.apache.tez.dag.app.launcher; +import java.util.Set; + import org.apache.tez.common.DagContainerLauncher; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.serviceplugins.api.ContainerLaunchRequest; import org.apache.tez.serviceplugins.api.ContainerLauncher; import org.apache.tez.serviceplugins.api.ContainerStopRequest; @@ -49,6 +52,12 @@ public void dagComplete(TezDAGID dag, JobTokenSecretManager jobTokenSecretManage } } + public void vertexComplete(TezVertexID vertex, JobTokenSecretManager jobTokenSecretManager, Set nodeIdList) { + if (real instanceof DagContainerLauncher) { + ((DagContainerLauncher) real).vertexComplete(vertex, jobTokenSecretManager, nodeIdList); + } + } + public void taskAttemptFailed(TezTaskAttemptID taskAttemptID, JobTokenSecretManager jobTokenSecretManager, NodeId nodeId) { if (real instanceof DagContainerLauncher) { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTracker.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTracker.java index 87b7366bfc..56760c86be 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTracker.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTracker.java @@ -18,11 +18,14 @@ package org.apache.tez.dag.app.launcher; +import java.util.Set; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.dag.records.TezVertexID; public abstract class DeletionTracker { @@ -36,6 +39,10 @@ public void dagComplete(TezDAGID dag, JobTokenSecretManager jobTokenSecretManage //do nothing } + public void vertexComplete(TezVertexID vertex, JobTokenSecretManager jobTokenSecretManager, Set nodeIdList) { + //do nothing + } + public void taskAttemptFailed(TezTaskAttemptID taskAttemptID, JobTokenSecretManager jobTokenSecretManager, NodeId nodeId) { //do nothing diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTrackerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTrackerImpl.java index e4204bfc71..73eaa68e72 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTrackerImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTrackerImpl.java @@ -21,6 +21,7 @@ import java.util.HashMap; import java.util.Map; +import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.RejectedExecutionException; @@ -35,6 +36,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -113,4 +115,25 @@ public void shutdown() { } nodeIdShufflePortMap = null; } + + @Override + public void vertexComplete(TezVertexID vertex, JobTokenSecretManager jobTokenSecretManager, Set nodeIdList) { + super.vertexComplete(vertex, jobTokenSecretManager, nodeIdList); + String vertexId = String.format("%02d", vertex.getId()); + for (NodeId nodeId : nodeIdList) { + Integer shufflePort = null; + if (nodeIdShufflePortMap != null) { + shufflePort = nodeIdShufflePortMap.get(nodeId); + } + if (shufflePort != null) { + VertexDeleteRunnable vertexDeleteRunnable = new VertexDeleteRunnable(vertex, jobTokenSecretManager, nodeId, + shufflePort, vertexId, TezRuntimeUtils.getHttpConnectionParams(conf)); + try { + dagCleanupService.submit(vertexDeleteRunnable); + } catch (RejectedExecutionException rejectedException) { + LOG.info("Ignoring deletion request for " + vertexDeleteRunnable); + } + } + } + } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java index ebc8f95566..47cc9f1325 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java @@ -24,6 +24,7 @@ import java.nio.ByteBuffer; import java.util.HashMap; import java.util.Map; +import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.Callable; import java.util.concurrent.CancellationException; @@ -50,6 +51,7 @@ import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.serviceplugins.api.ContainerLaunchRequest; @@ -96,6 +98,7 @@ public class LocalContainerLauncher extends DagContainerLauncher { int shufflePort = TezRuntimeUtils.INVALID_PORT; private DeletionTracker deletionTracker; private boolean dagDelete; + private boolean vertexDelete; private boolean failedTaskAttemptDelete; private final ConcurrentHashMap> @@ -162,11 +165,14 @@ public LocalContainerLauncher(ContainerLauncherContext containerLauncherContext, dagDelete = ShuffleUtils.isTezShuffleHandler(conf) && conf.getBoolean(TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION, TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION_DEFAULT); + vertexDelete = ShuffleUtils.isTezShuffleHandler(conf) && + conf.getInt(TezConfiguration.TEZ_AM_VERTEX_CLEANUP_HEIGHT, + TezConfiguration.TEZ_AM_VERTEX_CLEANUP_HEIGHT_DEFAULT) > 0; failedTaskAttemptDelete = ShuffleUtils.isTezShuffleHandler(conf) && conf.getBoolean(TezConfiguration.TEZ_AM_TASK_ATTEMPT_CLEANUP_ON_FAILURE, TezConfiguration.TEZ_AM_TASK_ATTEMPT_CLEANUP_ON_FAILURE_DEFAULT); - if (dagDelete || failedTaskAttemptDelete) { + if (dagDelete || vertexDelete || failedTaskAttemptDelete) { String deletionTrackerClassName = conf.get(TezConfiguration.TEZ_AM_DELETION_TRACKER_CLASS, TezConfiguration.TEZ_AM_DELETION_TRACKER_CLASS_DEFAULT); deletionTracker = ReflectionUtils.createClazzInstance( @@ -454,6 +460,13 @@ public void dagComplete(TezDAGID dag, JobTokenSecretManager jobTokenSecretManage } } + @Override + public void vertexComplete(TezVertexID dag, JobTokenSecretManager jobTokenSecretManager, Set nodeIdList) { + if (vertexDelete && deletionTracker != null) { + deletionTracker.vertexComplete(dag, jobTokenSecretManager, nodeIdList); + } + } + @Override public void taskAttemptFailed(TezTaskAttemptID taskAttemptID, JobTokenSecretManager jobTokenSecretManager, NodeId nodeId) { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezContainerLauncherImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezContainerLauncherImpl.java index 88ed4f7b89..654224adb5 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezContainerLauncherImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezContainerLauncherImpl.java @@ -21,8 +21,8 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Collections; -import java.util.HashMap; import java.util.Map; +import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.LinkedBlockingQueue; @@ -43,6 +43,7 @@ import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.serviceplugins.api.ContainerLaunchRequest; @@ -93,6 +94,7 @@ public class TezContainerLauncherImpl extends DagContainerLauncher { private AtomicBoolean serviceStopped = new AtomicBoolean(false); private DeletionTracker deletionTracker = null; private boolean dagDelete; + private boolean vertexDelete; private boolean failedTaskAttemptDelete; private Container getContainer(ContainerOp event) { @@ -339,11 +341,14 @@ public void run() { dagDelete = ShuffleUtils.isTezShuffleHandler(conf) && conf.getBoolean(TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION, TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION_DEFAULT); + vertexDelete = ShuffleUtils.isTezShuffleHandler(conf) && + conf.getInt(TezConfiguration.TEZ_AM_VERTEX_CLEANUP_HEIGHT, + TezConfiguration.TEZ_AM_VERTEX_CLEANUP_HEIGHT_DEFAULT) > 0; failedTaskAttemptDelete = ShuffleUtils.isTezShuffleHandler(conf) && conf.getBoolean(TezConfiguration.TEZ_AM_TASK_ATTEMPT_CLEANUP_ON_FAILURE, TezConfiguration.TEZ_AM_TASK_ATTEMPT_CLEANUP_ON_FAILURE_DEFAULT); - if (dagDelete || failedTaskAttemptDelete) { + if (dagDelete || vertexDelete || failedTaskAttemptDelete) { String deletionTrackerClassName = conf.get(TezConfiguration.TEZ_AM_DELETION_TRACKER_CLASS, TezConfiguration.TEZ_AM_DELETION_TRACKER_CLASS_DEFAULT); deletionTracker = ReflectionUtils.createClazzInstance( @@ -454,6 +459,13 @@ public void dagComplete(TezDAGID dag, JobTokenSecretManager jobTokenSecretManage } } + @Override + public void vertexComplete(TezVertexID vertex, JobTokenSecretManager jobTokenSecretManager, Set nodeIdList) { + if (vertexDelete && deletionTracker != null) { + deletionTracker.vertexComplete(vertex, jobTokenSecretManager, nodeIdList); + } + } + @Override public void taskAttemptFailed(TezTaskAttemptID taskAttemptID, JobTokenSecretManager jobTokenSecretManager, NodeId nodeId) { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/VertexDeleteRunnable.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/VertexDeleteRunnable.java new file mode 100644 index 0000000000..a8d25379a5 --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/VertexDeleteRunnable.java @@ -0,0 +1,82 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.app.launcher; + +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.tez.common.security.JobTokenSecretManager; +import org.apache.tez.dag.records.TezVertexID; +import org.apache.tez.http.BaseHttpConnection; +import org.apache.tez.http.HttpConnectionParams; +import org.apache.tez.runtime.library.common.TezRuntimeUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.URL; + +public class VertexDeleteRunnable implements Runnable { + private static final Logger LOG = LoggerFactory.getLogger(VertexDeleteRunnable.class); + final private TezVertexID vertex; + final private JobTokenSecretManager jobTokenSecretManager; + final private NodeId nodeId; + final private int shufflePort; + final private String vertexId; + final private HttpConnectionParams httpConnectionParams; + + VertexDeleteRunnable(TezVertexID vertex, JobTokenSecretManager jobTokenSecretManager, + NodeId nodeId, int shufflePort, String vertexId, + HttpConnectionParams httpConnectionParams) { + this.vertex = vertex; + this.jobTokenSecretManager = jobTokenSecretManager; + this.nodeId = nodeId; + this.shufflePort = shufflePort; + this.vertexId = vertexId; + this.httpConnectionParams = httpConnectionParams; + } + + @Override + public void run() { + BaseHttpConnection httpConnection = null; + try { + URL baseURL = TezRuntimeUtils.constructBaseURIForShuffleHandlerVertexComplete( + nodeId.getHost(), shufflePort, + vertex.getDAGID().getApplicationId().toString(), vertex.getDAGID().getId(), vertexId, false); + httpConnection = TezRuntimeUtils.getHttpConnection(true, baseURL, httpConnectionParams, + "VertexDelete", jobTokenSecretManager); + httpConnection.connect(); + httpConnection.getInputStream(); + } catch (Exception e) { + LOG.warn("Could not setup HTTP Connection to the node %s " + nodeId.getHost() + + " for vertex shuffle delete. ", e); + } finally { + try { + if (httpConnection != null) { + httpConnection.cleanup(true); + } + } catch (IOException e) { + LOG.warn("Encountered IOException for " + nodeId.getHost() + " during close. ", e); + } + } + } + + @Override + public String toString() { + return "VertexDeleteRunnable nodeId=" + nodeId + ", shufflePort=" + shufflePort + ", vertexId=" + vertexId; + } +} diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java index 5cdcf49206..c118110948 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java @@ -20,6 +20,15 @@ import java.nio.ByteBuffer; +import org.apache.tez.common.TezUtils; +import org.apache.tez.common.security.JobTokenSecretManager; +import org.apache.tez.dag.api.NamedEntityDescriptor; +import org.apache.tez.dag.app.DAGAppMaster; +import org.apache.tez.dag.app.dag.TaskAttempt; +import org.apache.tez.dag.app.launcher.ContainerLauncherManager; +import org.apache.tez.dag.app.launcher.TezContainerLauncherImpl; +import org.apache.tez.dag.app.rm.container.AMContainer; +import org.apache.tez.serviceplugins.api.ContainerLauncherDescriptor; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -2395,12 +2404,254 @@ private DAGPlan createDAGPlanForGraceParallelism() throws IOException { .build(); } - private void setupVertices() { + /** + * The dag is of the following structure. + * vertex1 vertex2 + * \ / + * vertex 3 + * / \ + * vertex4 vertex5 + * \ / + * vertex6 + * @return dagPlan + */ + + public DAGPlan createDAGPlanVertexShuffleDelete() { + LOG.info("Setting up dag plan"); + DAGPlan dag = DAGPlan.newBuilder() + .setName("testverteximpl") + .setDagConf(DAGProtos.ConfigurationProto.newBuilder() + .addConfKeyValues(DAGProtos.PlanKeyValuePair.newBuilder() + .setKey(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS) + .setValue(3 + ""))) + .addVertex( + VertexPlan.newBuilder() + .setName("vertex1") + .setType(PlanVertexType.NORMAL) + .addTaskLocationHint( + PlanTaskLocationHint.newBuilder() + .addHost("host1") + .addRack("rack1") + .build() + ) + .setTaskConfig( + PlanTaskConfiguration.newBuilder() + .setNumTasks(1) + .setVirtualCores(4) + .setMemoryMb(1024) + .setJavaOpts("") + .setTaskModule("x1.y1") + .build() + ) + .setVertexConf(DAGProtos.ConfigurationProto.newBuilder() + .addConfKeyValues(DAGProtos.PlanKeyValuePair.newBuilder() + .setKey(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS) + .setValue(2+""))) + .addOutEdgeId("e1") + .build() + ) + .addVertex( + VertexPlan.newBuilder() + .setName("vertex2") + .setType(PlanVertexType.NORMAL) + .addTaskLocationHint( + PlanTaskLocationHint.newBuilder() + .addHost("host2") + .addRack("rack2") + .build() + ) + .setTaskConfig( + PlanTaskConfiguration.newBuilder() + .setNumTasks(2) + .setVirtualCores(4) + .setMemoryMb(1024) + .setJavaOpts("") + .setTaskModule("x2.y2") + .build() + ) + .addOutEdgeId("e2") + .build() + ) + .addVertex( + VertexPlan.newBuilder() + .setName("vertex3") + .setType(PlanVertexType.NORMAL) + .setProcessorDescriptor(TezEntityDescriptorProto.newBuilder().setClassName("x3.y3")) + .addTaskLocationHint( + PlanTaskLocationHint.newBuilder() + .addHost("host3") + .addRack("rack3") + .build() + ) + .setTaskConfig( + PlanTaskConfiguration.newBuilder() + .setNumTasks(2) + .setVirtualCores(4) + .setMemoryMb(1024) + .setJavaOpts("foo") + .setTaskModule("x3.y3") + .build() + ) + .addInEdgeId("e1") + .addInEdgeId("e2") + .addOutEdgeId("e3") + .addOutEdgeId("e4") + .build() + ) + .addVertex( + VertexPlan.newBuilder() + .setName("vertex4") + .setType(PlanVertexType.NORMAL) + .addTaskLocationHint( + PlanTaskLocationHint.newBuilder() + .addHost("host4") + .addRack("rack4") + .build() + ) + .setTaskConfig( + PlanTaskConfiguration.newBuilder() + .setNumTasks(2) + .setVirtualCores(4) + .setMemoryMb(1024) + .setJavaOpts("") + .setTaskModule("x4.y4") + .build() + ) + .addInEdgeId("e3") + .addOutEdgeId("e5") + .build() + ) + .addVertex( + VertexPlan.newBuilder() + .setName("vertex5") + .setType(PlanVertexType.NORMAL) + .addTaskLocationHint( + PlanTaskLocationHint.newBuilder() + .addHost("host5") + .addRack("rack5") + .build() + ) + .setTaskConfig( + PlanTaskConfiguration.newBuilder() + .setNumTasks(2) + .setVirtualCores(4) + .setMemoryMb(1024) + .setJavaOpts("") + .setTaskModule("x5.y5") + .build() + ) + .addInEdgeId("e4") + .addOutEdgeId("e6") + .build() + ) + .addVertex( + VertexPlan.newBuilder() + .setName("vertex6") + .setType(PlanVertexType.NORMAL) + .addTaskLocationHint( + PlanTaskLocationHint.newBuilder() + .addHost("host6") + .addRack("rack6") + .build() + ) + .setTaskConfig( + PlanTaskConfiguration.newBuilder() + .setNumTasks(2) + .setVirtualCores(4) + .setMemoryMb(1024) + .setJavaOpts("") + .setTaskModule("x6.y6") + .build() + ) + .addInEdgeId("e5") + .addInEdgeId("e6") + .build() + ) + .addEdge( + EdgePlan.newBuilder() + .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("i3_v1")) + .setInputVertexName("vertex1") + .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("o1")) + .setOutputVertexName("vertex3") + .setDataMovementType(PlanEdgeDataMovementType.SCATTER_GATHER) + .setId("e1") + .setDataSourceType(PlanEdgeDataSourceType.PERSISTED) + .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL) + .build() + ) + .addEdge( + EdgePlan.newBuilder() + .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("i3_v2")) + .setInputVertexName("vertex2") + .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("o2")) + .setOutputVertexName("vertex3") + .setDataMovementType(PlanEdgeDataMovementType.SCATTER_GATHER) + .setId("e2") + .setDataSourceType(PlanEdgeDataSourceType.PERSISTED) + .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL) + .build() + ) + .addEdge( + EdgePlan.newBuilder() + .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("i4_v3")) + .setInputVertexName("vertex3") + .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("o3_v4")) + .setOutputVertexName("vertex4") + .setDataMovementType(PlanEdgeDataMovementType.SCATTER_GATHER) + .setId("e3") + .setDataSourceType(PlanEdgeDataSourceType.PERSISTED) + .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL) + .build() + ) + .addEdge( + EdgePlan.newBuilder() + .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("i5_v3")) + .setInputVertexName("vertex3") + .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("o3_v5")) + .setOutputVertexName("vertex5") + .setDataMovementType(PlanEdgeDataMovementType.SCATTER_GATHER) + .setId("e4") + .setDataSourceType(PlanEdgeDataSourceType.PERSISTED) + .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL) + .build() + ) + .addEdge( + EdgePlan.newBuilder() + .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("i6_v4")) + .setInputVertexName("vertex4") + .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("o4")) + .setOutputVertexName("vertex6") + .setDataMovementType(PlanEdgeDataMovementType.SCATTER_GATHER) + .setId("e5") + .setDataSourceType(PlanEdgeDataSourceType.PERSISTED) + .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL) + .build() + ) + .addEdge( + EdgePlan.newBuilder() + .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("i6_v5")) + .setInputVertexName("vertex5") + .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("o5")) + .setOutputVertexName("vertex6") + .setDataMovementType(PlanEdgeDataMovementType.SCATTER_GATHER) + .setId("e6") + .setDataSourceType(PlanEdgeDataSourceType.PERSISTED) + .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL) + .build() + ) + .build(); + + return dag; + } + + private void setupVertices(boolean cleanupShuffleDataAtVertexLevel) { int vCnt = dagPlan.getVertexCount(); LOG.info("Setting up vertices from dag plan, verticesCnt=" + vCnt); vertices = new HashMap(); vertexIdMap = new HashMap(); Configuration dagConf = new Configuration(false); + dagConf.setBoolean(TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION, true); + conf.setInt(TezConfiguration.TEZ_AM_VERTEX_CLEANUP_HEIGHT, cleanupShuffleDataAtVertexLevel ? 1 : 0); dagConf.set("abc", "foobar"); for (int i = 0; i < vCnt; ++i) { VertexPlan vPlan = dagPlan.getVertex(i); @@ -2447,7 +2698,6 @@ private void parseVertexEdges() { Map outVertices = new HashMap(); - for(String inEdgeId : vertexPlan.getInEdgeIdList()){ EdgePlan edgePlan = edgePlans.get(inEdgeId); Vertex inVertex = this.vertices.get(edgePlan.getInputVertexName()); @@ -2472,8 +2722,14 @@ private void parseVertexEdges() { + ", outputVerticesCnt=" + outVertices.size()); vertex.setOutputVertices(outVertices); } + + for (Map.Entry vertex : vertices.entrySet()) { + VertexImpl vertexImpl = vertex.getValue(); + vertexImpl.initShuffleDeletionContext(2); + } } + public void setupPreDagCreation() { LOG.info("____________ RESETTING CURRENT DAG ____________"); conf = new Configuration(); @@ -2488,8 +2744,9 @@ public void setupPreDagCreation() { } @SuppressWarnings({ "unchecked", "rawtypes" }) - public void setupPostDagCreation() throws TezException { + public void setupPostDagCreation(boolean cleanupShuffleDataAtVertexLevel) throws TezException { String dagName = "dag0"; + taskCommunicatorManagerInterface = mock(TaskCommunicatorManagerInterface.class); // dispatcher may be created multiple times (setupPostDagCreation may be called multiples) if (dispatcher != null) { dispatcher.stop(); @@ -2499,6 +2756,40 @@ public void setupPostDagCreation() throws TezException { when(appContext.getHadoopShim()).thenReturn(new DefaultHadoopShim()); when(appContext.getContainerLauncherName(anyInt())).thenReturn( TezConstants.getTezYarnServicePluginName()); + DAGAppMaster mockDagAppMaster = mock(DAGAppMaster.class); + when(appContext.getAppMaster()).thenReturn(mockDagAppMaster); + doCallRealMethod().when(mockDagAppMaster).vertexComplete(any(TezVertexID.class), any(Set.class)); + List containerDescriptors = new ArrayList<>(); + ContainerLauncherDescriptor containerLaunchers = + ContainerLauncherDescriptor.create("ContainerLaunchers", + TezContainerLauncherImpl.class.getName()); + conf.setBoolean(TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION, true); + conf.set(TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID, "tez_shuffle"); + conf.setInt(TezConfiguration.TEZ_AM_VERTEX_CLEANUP_HEIGHT, 0); + try { + containerLaunchers.setUserPayload(UserPayload.create( + TezUtils.createByteStringFromConf(conf).asReadOnlyByteBuffer())); + } catch (IOException e) { + e.printStackTrace(); + } + containerDescriptors.add(containerLaunchers); + ContainerLauncherManager mockContainerLauncherManager = spy(new ContainerLauncherManager(appContext, + taskCommunicatorManagerInterface, "test", containerDescriptors, false)); + doCallRealMethod().when(mockContainerLauncherManager).vertexComplete(any( + TezVertexID.class), any(JobTokenSecretManager.class + ), any(Set.class)); + when(appContext.getAppMaster().getContainerLauncherManager()).thenReturn( + mockContainerLauncherManager); + mockContainerLauncherManager.init(conf); + mockContainerLauncherManager.start(); + AMContainerMap amContainerMap = mock(AMContainerMap.class); + AMContainer amContainer = mock(AMContainer.class); + Container mockContainer = mock(Container.class); + when(amContainer.getContainer()).thenReturn(mockContainer); + when(mockContainer.getNodeId()).thenReturn(mock(NodeId.class)); + when(mockContainer.getNodeHttpAddress()).thenReturn("localhost:12345"); + when(amContainerMap.get(any(ContainerId.class))).thenReturn(amContainer); + when(appContext.getAllContainers()).thenReturn(amContainerMap); thh = mock(TaskHeartbeatHandler.class); historyEventHandler = mock(HistoryEventHandler.class); @@ -2557,7 +2848,7 @@ public ListenableFuture answer(InvocationOnMock invocation) { updateTracker.stop(); } updateTracker = new StateChangeNotifierForTest(appContext.getCurrentDAG()); - setupVertices(); + setupVertices(cleanupShuffleDataAtVertexLevel); when(dag.getVertex(any(TezVertexID.class))).thenAnswer(new Answer() { @Override public Vertex answer(InvocationOnMock invocation) throws Throwable { @@ -2622,7 +2913,7 @@ public void setup() throws TezException { setupPreDagCreation(); dagPlan = createTestDAGPlan(); invalidDagPlan = createInvalidDAGPlan(); - setupPostDagCreation(); + setupPostDagCreation(false); } @After @@ -2750,7 +3041,7 @@ public void testVertexInit() throws AMUserCodeException { public void testNonExistVertexManager() throws TezException { setupPreDagCreation(); dagPlan = createDAGPlanWithNonExistVertexManager(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl v1 = vertices.get("vertex1"); v1.handle(new VertexEvent(v1.getVertexId(), VertexEventType.V_INIT)); Assert.assertEquals(VertexState.FAILED, v1.getState()); @@ -2763,7 +3054,7 @@ public void testNonExistVertexManager() throws TezException { public void testNonExistInputInitializer() throws TezException { setupPreDagCreation(); dagPlan = createDAGPlanWithNonExistInputInitializer(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl v1 = vertices.get("vertex1"); v1.handle(new VertexEvent(v1.getVertexId(), VertexEventType.V_INIT)); Assert.assertEquals(VertexState.FAILED, v1.getState()); @@ -2776,7 +3067,7 @@ public void testNonExistInputInitializer() throws TezException { public void testNonExistOutputCommitter() throws TezException { setupPreDagCreation(); dagPlan = createDAGPlanWithNonExistOutputCommitter(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl v1 = vertices.get("vertex1"); v1.handle(new VertexEvent(v1.getVertexId(), VertexEventType.V_INIT)); Assert.assertEquals(VertexState.FAILED, v1.getState()); @@ -2815,7 +3106,7 @@ public void testVertexConfigureEventWithReconfigure() throws Exception { setupPreDagCreation(); // initialize() will make VM call planned() and started() will make VM call done() dagPlan = createDAGPlanWithVMException("TestVMStateUpdate", VMExceptionLocation.NoExceptionDoReconfigure); - setupPostDagCreation(); + setupPostDagCreation(false); TestUpdateListener listener = new TestUpdateListener(); updateTracker @@ -3824,7 +4115,7 @@ public void testFailuresMaxPercentSourceTaskAttemptCompletionEvents() throws Tez conf.setFloat(TezConfiguration.TEZ_VERTEX_FAILURES_MAXPERCENT, 50.0f); conf.setInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS, 1); dagPlan = createTestDAGPlan(); - setupPostDagCreation(); + setupPostDagCreation(false); initAllVertices(VertexState.INITED); VertexImpl v4 = vertices.get("vertex4"); @@ -3879,7 +4170,7 @@ public void testFailuresMaxPercentExceededSourceTaskAttemptCompletionEvents() th conf.setFloat(TezConfiguration.TEZ_VERTEX_FAILURES_MAXPERCENT, 50.0f); conf.setInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS, 1); dagPlan = createTestDAGPlan(); - setupPostDagCreation(); + setupPostDagCreation(false); initAllVertices(VertexState.INITED); VertexImpl v4 = vertices.get("vertex4"); @@ -3978,7 +4269,7 @@ public void testTaskReschedule() { public void testTerminatingVertexForTaskComplete() throws Exception { setupPreDagCreation(); dagPlan = createSamplerDAGPlan(false); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl vertex = spy(vertices.get("A")); initVertex(vertex); startVertex(vertex); @@ -3996,7 +4287,7 @@ public void testTerminatingVertexForTaskComplete() throws Exception { public void testTerminatingVertexForVComplete() throws Exception { setupPreDagCreation(); dagPlan = createSamplerDAGPlan(false); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl vertex = spy(vertices.get("A")); initVertex(vertex); startVertex(vertex); @@ -4251,7 +4542,7 @@ public void testBadCommitter2() throws Exception { public void testVertexInitWithCustomVertexManager() throws Exception { setupPreDagCreation(); dagPlan = createDAGWithCustomVertexManager(); - setupPostDagCreation(); + setupPostDagCreation(false); int numTasks = 3; VertexImpl v1 = vertices.get("v1"); @@ -4305,7 +4596,7 @@ public void testVertexInitWithCustomVertexManager() throws Exception { public void testVertexManagerHeuristic() throws TezException { setupPreDagCreation(); dagPlan = createDAGPlanWithMixedEdges(); - setupPostDagCreation(); + setupPostDagCreation(false); initAllVertices(VertexState.INITED); Assert.assertEquals(ImmediateStartVertexManager.class, vertices.get("vertex1").getVertexManager().getPlugin().getClass()); @@ -4330,7 +4621,7 @@ public void testVertexWithOneToOneSplit() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanForOneToOneSplit("TestInputInitializer", -1, true); - setupPostDagCreation(); + setupPostDagCreation(false); int numTasks = 5; VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices @@ -4397,7 +4688,7 @@ public void testVertexWithOneToOneSplitWhileRunning() throws Exception { // create a diamond shaped dag with 1-1 edges. setupPreDagCreation(); dagPlan = createDAGPlanForOneToOneSplit(null, numTasks, false); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl v1 = vertices.get("vertex1"); v1.vertexReconfigurationPlanned(); initAllVertices(VertexState.INITED); @@ -4436,7 +4727,7 @@ public void testVertexWithOneToOneSplitWhileInited() throws Exception { // create a diamond shaped dag with 1-1 edges. setupPreDagCreation(); dagPlan = createDAGPlanForOneToOneSplit(null, numTasks, false); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl v1 = vertices.get("vertex1"); v1.vertexReconfigurationPlanned(); initAllVertices(VertexState.INITED); @@ -4478,7 +4769,7 @@ public void testVertexVMErrorReport() throws Exception { // create a diamond shaped dag with 1-1 edges. setupPreDagCreation(); dagPlan = createDAGPlanForOneToOneSplit(null, numTasks, false); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl v1 = vertices.get("vertex1"); initAllVertices(VertexState.INITED); @@ -4522,7 +4813,7 @@ public void testVertexWithInitializerFailure() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithInputInitializer("TestInputInitializer"); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices .get("vertex1"); @@ -4567,7 +4858,7 @@ public void testVertexWithInitializerParallelismSetTo0() throws InterruptedExcep setupPreDagCreation(); dagPlan = createDAGPlanWithInitializer0Tasks(RootInitializerSettingParallelismTo0.class.getName()); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl v1 = vertices.get("vertex1"); VertexImpl v2 = vertices.get("vertex2"); @@ -4615,7 +4906,7 @@ public void testInputInitializerVertexStateUpdates() throws Exception { initializer.setNumVertexStateUpdateEvents(3); setupPreDagCreation(); dagPlan = createDAGPlanWithRunningInitializer(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithRunningInputInitializer v1 = (VertexImplWithRunningInputInitializer) vertices.get("vertex1"); @@ -4650,7 +4941,7 @@ public void testInputInitializerEventMultipleAttempts() throws Exception { (EventHandlingRootInputInitializer) customInitializer; setupPreDagCreation(); dagPlan = createDAGPlanWithRunningInitializer4(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithRunningInputInitializer v1 = (VertexImplWithRunningInputInitializer) vertices.get("vertex1"); @@ -4738,7 +5029,7 @@ public void testInputInitializerEventsMultipleSources() throws Exception { initializer.setNumExpectedEvents(4); setupPreDagCreation(); dagPlan = createDAGPlanWithRunningInitializer4(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithRunningInputInitializer v1 = (VertexImplWithRunningInputInitializer) vertices.get("vertex1"); @@ -4861,7 +5152,7 @@ public void testInputInitializerEventNoDirectConnection() throws Exception { (EventHandlingRootInputInitializer) customInitializer; setupPreDagCreation(); dagPlan = createDAGPlanWithRunningInitializer4(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithRunningInputInitializer v1 = (VertexImplWithRunningInputInitializer) vertices.get("vertex1"); @@ -4941,7 +5232,7 @@ public void testInputInitializerEventsAtNew() throws Exception { (EventHandlingRootInputInitializer) customInitializer; setupPreDagCreation(); dagPlan = createDAGPlanWithRunningInitializer3(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithRunningInputInitializer v1 = (VertexImplWithRunningInputInitializer) vertices.get("vertex1"); @@ -5027,7 +5318,7 @@ public void testInputInitializerEvents() throws Exception { (EventHandlingRootInputInitializer) customInitializer; setupPreDagCreation(); dagPlan = createDAGPlanWithRunningInitializer(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithRunningInputInitializer v1 = (VertexImplWithRunningInputInitializer) vertices.get("vertex1"); @@ -5104,7 +5395,7 @@ public void testInputInitializerEvents() throws Exception { public void testTaskSchedulingWithCustomEdges() throws TezException { setupPreDagCreation(); dagPlan = createCustomDAGWithCustomEdges(); - setupPostDagCreation(); + setupPostDagCreation(false); /** * @@ -5402,7 +5693,7 @@ public void testVertexWithMultipleInitializers1() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithMultipleInitializers("TestInputInitializer"); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices .get("vertex1"); @@ -5432,7 +5723,7 @@ public void testVertexWithMultipleInitializers2() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithMultipleInitializers("TestInputInitializer"); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices .get("vertex1"); @@ -5462,7 +5753,7 @@ public void testVertexWithInitializerSuccess() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithInputInitializer("TestInputInitializer"); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices .get("vertex1"); @@ -5563,7 +5854,7 @@ public void testVertexWithInitializerSuccessLegacyRouting() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithInputInitializer("TestInputInitializer"); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices .get("vertex1"); @@ -5638,7 +5929,7 @@ public void testVertexWithInputDistributor() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithInputDistributor("TestInputInitializer"); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices .get("vertex1"); @@ -5673,7 +5964,7 @@ public void testVertexRootInputSpecUpdateAll() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithInputInitializer("TestInputInitializer"); - setupPostDagCreation(); + setupPostDagCreation(false); int expectedNumTasks = RootInputSpecUpdaterVertexManager.NUM_TASKS; VertexImplWithControlledInitializerManager v3 = (VertexImplWithControlledInitializerManager) vertices @@ -5703,7 +5994,7 @@ public void testVertexRootInputSpecUpdatePerTask() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithInputInitializer("TestInputInitializer"); - setupPostDagCreation(); + setupPostDagCreation(false); int expectedNumTasks = RootInputSpecUpdaterVertexManager.NUM_TASKS; VertexImplWithControlledInitializerManager v4 = (VertexImplWithControlledInitializerManager) vertices @@ -6015,7 +6306,7 @@ public void completeInputInitialization(int initializerIndex, int targetTasks, public void testVertexGroupInput() throws TezException { setupPreDagCreation(); dagPlan = createVertexGroupDAGPlan(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl vA = vertices.get("A"); VertexImpl vB = vertices.get("B"); @@ -6044,7 +6335,7 @@ public void testStartWithUninitializedCustomEdge() throws Exception { // been initialized setupPreDagCreation(); dagPlan = createSamplerDAGPlan(true); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl vA = vertices.get("A"); VertexImpl vB = vertices.get("B"); @@ -6093,7 +6384,7 @@ public void testVertexConfiguredDoneByVMBeforeEdgeDefined() throws Exception { // been initialized setupPreDagCreation(); dagPlan = createSamplerDAGPlan(true); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl vA = vertices.get("A"); VertexImpl vB = vertices.get("B"); @@ -6167,7 +6458,7 @@ public void testInitStartRace() throws TezException { // been initialized setupPreDagCreation(); dagPlan = createSamplerDAGPlan(false); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl vA = vertices.get("A"); VertexImpl vB = vertices.get("B"); @@ -6190,7 +6481,7 @@ public void testInitStartRace2() throws TezException { // been initialized setupPreDagCreation(); dagPlan = createSamplerDAGPlan2(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl vA = vertices.get("A"); VertexImpl vB = vertices.get("B"); @@ -6215,7 +6506,7 @@ public void testInitStartRace2() throws TezException { public void testTez2684() throws IOException, TezException { setupPreDagCreation(); dagPlan = createSamplerDAGPlan2(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl vA = vertices.get("A"); VertexImpl vB = vertices.get("B"); @@ -6255,7 +6546,7 @@ public void testTez2684() throws IOException, TezException { public void testVertexGraceParallelism() throws IOException, TezException { setupPreDagCreation(); dagPlan = createDAGPlanForGraceParallelism(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl vA = vertices.get("A"); VertexImpl vB = vertices.get("B"); @@ -6323,7 +6614,7 @@ public void testVMEventBeforeVertexInitialized() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithCountingVM(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl v1 = vertices.get("vertex1"); VertexImpl v2 = vertices.get("vertex2"); @@ -6380,7 +6671,7 @@ public void testExceptionFromVM_Initialize() throws TezException { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithVMException("TestInputInitializer", VMExceptionLocation.Initialize); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices .get("vertex1"); @@ -6399,7 +6690,7 @@ public void testExceptionFromVM_OnRootVertexInitialized() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithVMException("TestInputInitializer", VMExceptionLocation.OnRootVertexInitialized); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices .get("vertex1"); @@ -6423,7 +6714,7 @@ public void testExceptionFromVM_OnVertexStarted() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithVMException("TestInputInitializer", VMExceptionLocation.OnVertexStarted); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices .get("vertex1"); @@ -6450,7 +6741,7 @@ public void testExceptionFromVM_OnSourceTaskCompleted() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithVMException("TestInputInitializer", VMExceptionLocation.OnSourceTaskCompleted); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices .get("vertex1"); @@ -6486,7 +6777,7 @@ public void testExceptionFromVM_OnVertexManagerEventReceived() throws Exception useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithVMException("TestInputInitializer", VMExceptionLocation.OnVertexManagerEventReceived); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices .get("vertex1"); @@ -6514,7 +6805,7 @@ public void testExceptionFromVM_OnVertexManagerVertexStateUpdated() throws Excep useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithVMException("TestVMStateUpdate", VMExceptionLocation.OnVertexManagerVertexStateUpdated); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices .get("vertex1"); @@ -6543,7 +6834,7 @@ public void testExceptionFromII_Initialize() throws InterruptedException, TezExc (EventHandlingRootInputInitializer) customInitializer; setupPreDagCreation(); dagPlan = createDAGPlanWithIIException(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithRunningInputInitializer v1 = (VertexImplWithRunningInputInitializer) vertices.get("vertex1"); @@ -6564,7 +6855,7 @@ public void testExceptionFromII_InitFailedAfterInitialized() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithIIException(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager)vertices.get("vertex1"); @@ -6588,7 +6879,7 @@ public void testExceptionFromII_InitFailedAfterRunning() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithIIException(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager)vertices.get("vertex1"); @@ -6616,7 +6907,7 @@ public void testExceptionFromII_HandleInputInitializerEvent() throws Exception { (EventHandlingRootInputInitializer) customInitializer; setupPreDagCreation(); dagPlan = createDAGPlanWithRunningInitializer(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithRunningInputInitializer v1 = (VertexImplWithRunningInputInitializer) vertices.get("vertex1"); @@ -6666,7 +6957,7 @@ public void testExceptionFromII_OnVertexStateUpdated() throws InterruptedExcepti (EventHandlingRootInputInitializer) customInitializer; setupPreDagCreation(); dagPlan = createDAGPlanWithRunningInitializer(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithRunningInputInitializer v1 = (VertexImplWithRunningInputInitializer) vertices.get("vertex1"); @@ -6695,7 +6986,7 @@ public void testExceptionFromII_InitSucceededAfterInitFailure() throws Interrupt (EventHandlingRootInputInitializer) customInitializer; setupPreDagCreation(); dagPlan = createDAGPlanWithRunningInitializer(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithRunningInputInitializer v1 = (VertexImplWithRunningInputInitializer) vertices.get("vertex1"); @@ -7212,4 +7503,132 @@ public void testPickupDagLocalResourceOnScheduleTask() { Assert.assertTrue(localResourceMap.containsKey("dag lr")); Assert.assertTrue(localResourceMap.containsKey("vertex lr")); } + + @Test + public void testVertexShuffleDelete() throws Exception { + setupPreDagCreation(); + dagPlan = createDAGPlanVertexShuffleDelete(); + setupPostDagCreation(true); + checkSpannedVertices(); + runVertices(); + Mockito.verify(appContext.getAppMaster().getContainerLauncherManager(), + times(3)).vertexComplete(any(TezVertexID.class), + any(JobTokenSecretManager.class), any(Set.class)); + } + + private void checkSpannedVertices() { + // vertex1 should have 0 ancestor and 2 children at height = 2 + VertexImpl v1 = vertices.get("vertex1"); + checkResults(v1.vShuffleDeletionContext.getAncestors(), new ArrayList<>()); + checkResults(v1.vShuffleDeletionContext.getChildren(), Arrays.asList("vertex5", "vertex4")); + + // vertex2 should have 0 ancestor and 2 children at height = 2 + VertexImpl v2 = vertices.get("vertex2"); + checkResults(v2.vShuffleDeletionContext.getAncestors(), new ArrayList<>()); + checkResults(v2.vShuffleDeletionContext.getChildren(), Arrays.asList("vertex5", "vertex4")); + + // vertex3 should have 0 ancestor and 1 children at height = 2 + VertexImpl v3 = vertices.get("vertex3"); + checkResults(v3.vShuffleDeletionContext.getAncestors(), new ArrayList<>()); + checkResults(v3.vShuffleDeletionContext.getChildren(), Arrays.asList("vertex6")); + + // vertex4 should have 2 ancestor and 0 children at height = 2 + VertexImpl v4 = vertices.get("vertex4"); + checkResults(v4.vShuffleDeletionContext.getAncestors(), Arrays.asList("vertex1", "vertex2")); + checkResults(v4.vShuffleDeletionContext.getChildren(), new ArrayList<>()); + + // vertex5 should have 2 ancestor and 0 children at height = 2 + VertexImpl v5 = vertices.get("vertex5"); + checkResults(v5.vShuffleDeletionContext.getAncestors(), Arrays.asList("vertex1", "vertex2")); + checkResults(v5.vShuffleDeletionContext.getChildren(), new ArrayList<>()); + + // vertex6 should have 1 ancestor and 0 children at height = 2 + VertexImpl v6 = vertices.get("vertex6"); + checkResults(v6.vShuffleDeletionContext.getAncestors(), Arrays.asList("vertex3")); + checkResults(v6.vShuffleDeletionContext.getChildren(), new ArrayList<>()); + } + + private void checkResults(Set actual, List expected) { + assertEquals(actual.size(), expected.size()); + for (Vertex vertex : actual) { + assertTrue(expected.contains(vertex.getName())); + } + } + + private void runVertices() { + VertexImpl v1 = vertices.get("vertex1"); + VertexImpl v2 = vertices.get("vertex2"); + VertexImpl v3 = vertices.get("vertex3"); + VertexImpl v4 = vertices.get("vertex4"); + VertexImpl v5 = vertices.get("vertex5"); + VertexImpl v6 = vertices.get("vertex6"); + dispatcher.getEventHandler().handle(new VertexEvent(v1.getVertexId(), VertexEventType.V_INIT)); + dispatcher.getEventHandler().handle(new VertexEvent(v2.getVertexId(), VertexEventType.V_INIT)); + dispatcher.await(); + dispatcher.getEventHandler().handle(new VertexEvent(v1.getVertexId(), VertexEventType.V_START)); + dispatcher.getEventHandler().handle(new VertexEvent(v2.getVertexId(), VertexEventType.V_START)); + dispatcher.await(); + + TezTaskID v1t1 = TezTaskID.getInstance(v1.getVertexId(), 0); + Map attempts = v1.getTask(v1t1).getAttempts(); + startAttempts(attempts); + v1.handle(new VertexEventTaskCompleted(v1t1, TaskState.SUCCEEDED)); + TezTaskID v2t1 = TezTaskID.getInstance(v2.getVertexId(), 0); + attempts = v2.getTask(v2t1).getAttempts(); + startAttempts(attempts); + v2.handle(new VertexEventTaskCompleted(v2t1, TaskState.SUCCEEDED)); + TezTaskID v2t2 = TezTaskID.getInstance(v2.getVertexId(), 1); + attempts = v2.getTask(v2t2).getAttempts(); + startAttempts(attempts); + v2.handle(new VertexEventTaskCompleted(v2t2, TaskState.SUCCEEDED)); + TezTaskID v3t1 = TezTaskID.getInstance(v3.getVertexId(), 0); + v3.scheduleTasks(Lists.newArrayList(ScheduleTaskRequest.create(0, null))); + dispatcher.await(); + attempts = v3.getTask(v3t1).getAttempts(); + startAttempts(attempts); + v3.handle(new VertexEventTaskCompleted(v3t1, TaskState.SUCCEEDED)); + TezTaskID v3t2 = TezTaskID.getInstance(v3.getVertexId(), 1); + attempts = v3.getTask(v3t2).getAttempts(); + startAttempts(attempts); + v3.handle(new VertexEventTaskCompleted(v3t2, TaskState.SUCCEEDED)); + dispatcher.await(); + TezTaskID v4t1 = TezTaskID.getInstance(v4.getVertexId(), 0); + attempts = v4.getTask(v4t1).getAttempts(); + startAttempts(attempts); + v4.handle(new VertexEventTaskCompleted(v4t1, TaskState.SUCCEEDED)); + TezTaskID v4t2 = TezTaskID.getInstance(v4.getVertexId(), 1); + attempts = v4.getTask(v4t2).getAttempts(); + startAttempts(attempts); + v4.handle(new VertexEventTaskCompleted(v4t2, TaskState.SUCCEEDED)); + TezTaskID v5t1 = TezTaskID.getInstance(v5.getVertexId(), 0); + attempts = v5.getTask(v5t1).getAttempts(); + startAttempts(attempts); + v5.handle(new VertexEventTaskCompleted(v5t1, TaskState.SUCCEEDED)); + TezTaskID v5t2 = TezTaskID.getInstance(v5.getVertexId(), 1); + attempts = v5.getTask(v5t2).getAttempts(); + startAttempts(attempts); + v5.handle(new VertexEventTaskCompleted(v5t2, TaskState.SUCCEEDED)); + TezTaskID v6t1 = TezTaskID.getInstance(v6.getVertexId(), 0); + attempts = v6.getTask(v6t1).getAttempts(); + startAttempts(attempts); + v6.handle(new VertexEventTaskCompleted(v6t1, TaskState.SUCCEEDED)); + TezTaskID v6t2 = TezTaskID.getInstance(v6.getVertexId(), 1); + attempts = v6.getTask(v6t2).getAttempts(); + startAttempts(attempts); + v6.handle(new VertexEventTaskCompleted(v6t2, TaskState.SUCCEEDED)); + dispatcher.await(); + } + + private void startAttempts(Map attempts) { + for (Map.Entry entry : attempts.entrySet()) { + TezTaskAttemptID id = entry.getKey(); + TaskAttemptImpl taskAttempt = (TaskAttemptImpl)entry.getValue(); + taskAttempt.handle(new TaskAttemptEventSchedule(id, 10, 10)); + dispatcher.await(); + ContainerId mockContainer = mock(ContainerId.class, RETURNS_DEEP_STUBS); + taskAttempt.handle(new TaskAttemptEventSubmitted(id, mockContainer)); + taskAttempt.handle(new TaskAttemptEventStartedRemotely(id)); + dispatcher.await(); + } + } } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherWrapper.java b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherWrapper.java index c4f4eff0cf..cb7d62dca0 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherWrapper.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherWrapper.java @@ -24,7 +24,7 @@ public class TestContainerLauncherWrapper { @Test(timeout = 5000) public void testDelegation() throws Exception { PluginWrapperTestHelpers.testDelegation(ContainerLauncherWrapper.class, ContainerLauncher.class, - Sets.newHashSet("getContainerLauncher", "dagComplete", "taskAttemptFailed")); + Sets.newHashSet("getContainerLauncher", "dagComplete", "vertexComplete", "taskAttemptFailed")); } } diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java index 7e6fd750f8..0fa1c03a96 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java @@ -18,6 +18,7 @@ package org.apache.tez.auxservices; +import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.util.DiskChecker; import static org.fusesource.leveldbjni.JniDBFactory.asString; import static org.fusesource.leveldbjni.JniDBFactory.bytes; @@ -1009,6 +1010,7 @@ private void handleRequest(ChannelHandlerContext ctx, FullHttpRequest request) final Map> q = new QueryStringDecoder(request.getUri()).parameters(); final List keepAliveList = q.get("keepAlive"); final List dagCompletedQ = q.get("dagAction"); + final List vertexCompletedQ = q.get("vertexAction"); final List taskAttemptFailedQ = q.get("taskAttemptAction"); boolean keepAliveParam = false; if (keepAliveList != null && keepAliveList.size() == 1) { @@ -1019,6 +1021,7 @@ private void handleRequest(ChannelHandlerContext ctx, FullHttpRequest request) final Range reduceRange = splitReduces(q.get("reduce")); final List jobQ = q.get("job"); final List dagIdQ = q.get("dag"); + final List vertexIdQ = q.get("vertex"); if (LOG.isDebugEnabled()) { LOG.debug("RECV: " + request.getUri() + "\n mapId: " + mapIds + @@ -1031,6 +1034,9 @@ private void handleRequest(ChannelHandlerContext ctx, FullHttpRequest request) if (deleteDagDirectories(ctx.channel(), dagCompletedQ, jobQ, dagIdQ)) { return; } + if (deleteVertexDirectories(ctx.channel(), vertexCompletedQ, jobQ, dagIdQ, vertexIdQ)) { + return; + } if (deleteTaskAttemptDirectories(ctx.channel(), taskAttemptFailedQ, jobQ, dagIdQ, mapIds)) { return; } @@ -1155,6 +1161,25 @@ private boolean deleteDagDirectories(Channel channel, return false; } + private boolean deleteVertexDirectories(Channel channel, List vertexCompletedQ, + List jobQ, List dagIdQ, + List vertexIdQ) { + if (jobQ == null || jobQ.isEmpty()) { + return false; + } + if (notEmptyAndContains(vertexCompletedQ, "delete") && !isNullOrEmpty(vertexIdQ)) { + try { + deleteTaskDirsOfVertex(jobQ.get(0), dagIdQ.get(0), vertexIdQ.get(0), userRsrc.get(jobQ.get(0))); + } catch (IOException e) { + LOG.warn("Encountered exception during vertex delete " + e); + } + channel.writeAndFlush(new DefaultHttpResponse(HTTP_1_1, OK)) + .addListener(ChannelFutureListener.CLOSE); + return true; + } + return false; + } + private boolean deleteTaskAttemptDirectories(Channel channel, List taskAttemptFailedQ, List jobQ, List dagIdQ, List taskAttemptIdQ) { if (jobQ == null || jobQ.isEmpty()) { @@ -1256,6 +1281,29 @@ private String getBaseLocation(String jobId, String dagId, String user) { return baseStr; } + /** + * Delete shuffle data in task directories belonging to a vertex. + */ + private void deleteTaskDirsOfVertex(String jobId, String dagId, String vertexId, String user) throws IOException { + String baseStr = getBaseLocation(jobId, dagId, user); + FileContext lfc = FileContext.getLocalFSFileContext(); + for(Path dagPath : lDirAlloc.getAllLocalPathsToRead(baseStr, conf)) { + RemoteIterator status = lfc.listStatus(dagPath); + final JobID jobID = JobID.forName(jobId); + String taskDirPrefix = String.format("attempt%s_%s_%s_", + jobID.toString().replace("job", ""), dagId, vertexId); + while (status.hasNext()) { + FileStatus fileStatus = status.next(); + Path attemptPath = fileStatus.getPath(); + if (attemptPath.getName().startsWith(taskDirPrefix)) { + if(lfc.delete(attemptPath, true)) { + LOG.debug("deleted shuffle data in task directory: {}", attemptPath); + } + } + } + } + } + private String getDagLocation(String jobId, String dagId, String user) { final JobID jobID = JobID.forName(jobId); final ApplicationId appID = diff --git a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java index 45dd0adb06..b91e0eb203 100644 --- a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java @@ -24,6 +24,7 @@ import static org.junit.Assert.assertTrue; import static io.netty.buffer.Unpooled.wrappedBuffer; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; import static org.junit.Assume.assumeTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -1312,6 +1313,83 @@ protected void sendError(ChannelHandlerContext ctx, String message, } } + @Test + public void testVertexShuffleDelete() throws Exception { + final ArrayList failures = new ArrayList(1); + Configuration conf = new Configuration(); + conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, 3); + conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); + conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, + "simple"); + UserGroupInformation.setConfiguration(conf); + File absLogDir = new File("target", TestShuffleHandler.class. + getSimpleName() + "LocDir").getAbsoluteFile(); + conf.set(YarnConfiguration.NM_LOCAL_DIRS, absLogDir.getAbsolutePath()); + ApplicationId appId = ApplicationId.newInstance(12345L, 1); + String appAttemptId = "attempt_12345_0001_1_00_000000_0_10003_0"; + String user = "randomUser"; + List fileMap = new ArrayList(); + String vertexDirStr = StringUtils.join(Path.SEPARATOR, new String[] { absLogDir.getAbsolutePath(), + ShuffleHandler.USERCACHE, user, ShuffleHandler.APPCACHE, appId.toString(), "dag_1/output/" + appAttemptId}); + File vertexDir = new File(vertexDirStr); + Assert.assertFalse("vertex directory should not be present", vertexDir.exists()); + createShuffleHandlerFiles(absLogDir, user, appId.toString(), appAttemptId, + conf, fileMap); + ShuffleHandler shuffleHandler = new ShuffleHandler() { + @Override + protected Shuffle getShuffle(Configuration conf) { + // replace the shuffle handler with one stubbed for testing + return new Shuffle(conf) { + @Override + protected void sendError(ChannelHandlerContext ctx, String message, + HttpResponseStatus status) { + if (failures.size() == 0) { + failures.add(new Error(message)); + ctx.channel().close(); + } + } + }; + } + }; + shuffleHandler.init(conf); + try { + shuffleHandler.start(); + DataOutputBuffer outputBuffer = new DataOutputBuffer(); + outputBuffer.reset(); + Token jt = + new Token("identifier".getBytes(), + "password".getBytes(), new Text(user), new Text("shuffleService")); + jt.write(outputBuffer); + shuffleHandler + .initializeApplication(new ApplicationInitializationContext(user, + appId, ByteBuffer.wrap(outputBuffer.getData(), 0, + outputBuffer.getLength()))); + URL url = + new URL( + "http://127.0.0.1:" + + shuffleHandler.getConfig().get( + ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY) + + "/mapOutput?vertexAction=delete&job=job_12345_0001&dag=1&vertex=00"); + HttpURLConnection conn = (HttpURLConnection) url.openConnection(); + conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME, + ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); + conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION, + ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); + Assert.assertTrue("Attempt Directory does not exist!", vertexDir.exists()); + conn.connect(); + try { + DataInputStream is = new DataInputStream(conn.getInputStream()); + is.close(); + Assert.assertFalse("Vertex Directory was not deleted", vertexDir.exists()); + } catch (EOFException e) { + fail("Encountered Exception!" + e.getMessage()); + } + } finally { + shuffleHandler.stop(); + FileUtil.fullyDelete(absLogDir); + } + } + @Test(timeout = 5000) public void testFailedTaskAttemptDelete() throws Exception { final ArrayList failures = new ArrayList(1); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java index 48b23bc694..a75925c5fe 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java @@ -187,6 +187,25 @@ public static URL constructBaseURIForShuffleHandlerDagComplete( return new URL(sb.toString()); } + public static URL constructBaseURIForShuffleHandlerVertexComplete( + String host, int port, String appId, int dagIdentifier, String vertexIndentifier, boolean sslShuffle) + throws MalformedURLException { + String httpProtocol = (sslShuffle) ? "https://" : "http://"; + StringBuilder sb = new StringBuilder(httpProtocol); + sb.append(host); + sb.append(":"); + sb.append(port); + sb.append("/"); + sb.append("mapOutput?vertexAction=delete"); + sb.append("&job="); + sb.append(appId.replace("application", "job")); + sb.append("&dag="); + sb.append(String.valueOf(dagIdentifier)); + sb.append("&vertex="); + sb.append(String.valueOf(vertexIndentifier)); + return new URL(sb.toString()); + } + public static URL constructBaseURIForShuffleHandlerTaskAttemptFailed( String host, int port, String appId, int dagIdentifier, String taskAttemptIdentifier, boolean sslShuffle) throws MalformedURLException { From f724c546069885e29e6446813805bb63bf0d5d9d Mon Sep 17 00:00:00 2001 From: Ramesh Kumar Date: Wed, 23 Mar 2022 21:34:47 -0700 Subject: [PATCH 365/512] TEZ-4397 Open Tez Input splits asynchronously Contributed by Ramesh Kumar Thangarajan --- .../split/TezGroupedSplitsInputFormat.java | 72 +++++++++++++++++-- .../mapreduce/grouper/TezSplitGrouper.java | 11 +++ 2 files changed, 79 insertions(+), 4 deletions(-) diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java index 61ba560300..6266ec1bcf 100644 --- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java +++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java @@ -19,8 +19,16 @@ package org.apache.hadoop.mapred.split; import java.io.IOException; +import java.util.LinkedList; import java.util.Objects; +import java.util.Queue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicInteger; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.tez.mapreduce.grouper.TezSplitGrouper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience.Public; @@ -129,14 +137,58 @@ public class TezGroupedSplitsRecordReader implements RecordReader { int idx = 0; long progress; RecordReader curReader; - + final AtomicInteger initIndex; + final int numReaders; + final ExecutorService initReaderExecService; + final Queue>> initedReaders; + public TezGroupedSplitsRecordReader(TezGroupedSplit split, JobConf job, Reporter reporter) throws IOException { this.groupedSplit = split; this.job = job; this.reporter = reporter; + this.initIndex = new AtomicInteger(0); + int numThreads = conf.getInt(TezSplitGrouper.TEZ_GROUPING_SPLIT_INIT_THREADS, + TezSplitGrouper.TEZ_GROUPING_SPLIT_INIT_THREADS_DEFAULT); + this.numReaders = conf.getInt(TezSplitGrouper.TEZ_GROUPING_SPLIT_INIT_NUM_RECORDREADERS, + TezSplitGrouper.TEZ_GROUPING_SPLIT_INIT_NUM_RECORDREADERS_DEFAULT); + this.initReaderExecService = Executors.newFixedThreadPool(numThreads, + new ThreadFactoryBuilder() + .setDaemon(true) + .setPriority(Thread.MAX_PRIORITY) + .setNameFormat("TEZ-Split-Init-Thread-%d") + .build()); + this.initedReaders = new LinkedList<>(); + preInitReaders(); initNextRecordReader(); } + + private void preInitReaders() { + if (initReaderExecService == null) { + LOG.info("Init record reader threadpool is not initialized"); + return; + } + for (int i = 0; i < numReaders; i++) { + initedReaders.offer(this.initReaderExecService.submit(() -> { + try { + int index = initIndex.getAndIncrement(); + if (index >= groupedSplit.wrappedSplits.size()) { + return null; + } + InputSplit s = groupedSplit.wrappedSplits.get(index); + RecordReader reader = wrappedInputFormat.getRecordReader(s, job, reporter); + LOG.debug("Init Thread processed reader number {} initialization", index); + return reader; + } catch (Exception e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + cancelsFutures(); + throw new RuntimeException(e); + } + })); + } + } @Override public boolean next(K key, V value) throws IOException { @@ -183,6 +235,8 @@ protected boolean initNextRecordReader() throws IOException { // if all chunks have been processed, nothing more to do. if (idx == groupedSplit.wrappedSplits.size()) { + LOG.info("Shutting down the init record reader threadpool"); + initReaderExecService.shutdownNow(); return false; } @@ -193,15 +247,25 @@ protected boolean initNextRecordReader() throws IOException { // get a record reader for the idx-th chunk try { - curReader = wrappedInputFormat.getRecordReader( - groupedSplit.wrappedSplits.get(idx), job, reporter); + curReader = initedReaders.poll().get(); + preInitReaders(); } catch (Exception e) { - throw new RuntimeException (e); + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + cancelsFutures(); + throw new RuntimeException(e); } idx++; return true; } + private void cancelsFutures() { + for (Future> f : initedReaders) { + f.cancel(true); + } + } + @Override public long getPos() throws IOException { long subprogress = 0; // bytes processed in current split diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java index a1d6b6c806..3b2f17d1ff 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java @@ -102,6 +102,17 @@ public abstract class TezSplitGrouper { public static final String TEZ_GROUPING_NODE_LOCAL_ONLY = "tez.grouping.node.local.only"; public static final boolean TEZ_GROUPING_NODE_LOCAL_ONLY_DEFAULT = false; + /** + * Number of threads used to initialize the grouped splits, to asynchronously open the readers. + */ + public static final String TEZ_GROUPING_SPLIT_INIT_THREADS = "tez.grouping.split.init-threads"; + public static final int TEZ_GROUPING_SPLIT_INIT_THREADS_DEFAULT = 4; + + /** + * Number of record readers to asynchronously and proactively init. + */ + public static final String TEZ_GROUPING_SPLIT_INIT_NUM_RECORDREADERS = "tez.grouping.split.init.num-recordreaders"; + public static final int TEZ_GROUPING_SPLIT_INIT_NUM_RECORDREADERS_DEFAULT = 10; static class LocationHolder { List splits; From 3e452e9859c507f089c6d6c01cbfabe24fd7b1d9 Mon Sep 17 00:00:00 2001 From: Himanshu Mishra Date: Fri, 1 Apr 2022 14:15:03 +0530 Subject: [PATCH 366/512] TEZ-4399: ShuffleHandler fails with SSLHandshakeException not found when SSL is enabled (#196) (Himanshu Mishra reviewed by Laszlo Bodor) --- .../org/apache/tez/dag/app/launcher/DagDeleteRunnable.java | 2 +- .../tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java | 2 +- .../org/apache/tez/dag/app/launcher/VertexDeleteRunnable.java | 3 ++- tez-plugins/tez-aux-services/pom.xml | 1 + 4 files changed, 5 insertions(+), 3 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DagDeleteRunnable.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DagDeleteRunnable.java index eac745e1ff..cbea36b86d 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DagDeleteRunnable.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DagDeleteRunnable.java @@ -54,7 +54,7 @@ public void run() { try { URL baseURL = TezRuntimeUtils.constructBaseURIForShuffleHandlerDagComplete( nodeId.getHost(), shufflePort, - dag.getApplicationId().toString(), dag.getId(), false); + dag.getApplicationId().toString(), dag.getId(), httpConnectionParams.isSslShuffle()); httpConnection = TezRuntimeUtils.getHttpConnection(true, baseURL, httpConnectionParams, "DAGDelete", jobTokenSecretManager); httpConnection.connect(); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java index 22c5b26a7c..3a9f5345a0 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java @@ -55,7 +55,7 @@ public void run() { URL baseURL = TezRuntimeUtils.constructBaseURIForShuffleHandlerTaskAttemptFailed( nodeId.getHost(), shufflePort, taskAttemptID.getTaskID().getVertexID().getDAGID(). getApplicationId().toString(), taskAttemptID.getTaskID().getVertexID().getDAGID().getId(), - taskAttemptID.toString(), false); + taskAttemptID.toString(), httpConnectionParams.isSslShuffle()); httpConnection = TezRuntimeUtils.getHttpConnection(true, baseURL, httpConnectionParams, "FailedTaskAttemptDelete", jobTokenSecretManager); httpConnection.connect(); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/VertexDeleteRunnable.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/VertexDeleteRunnable.java index a8d25379a5..3bfec9663a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/VertexDeleteRunnable.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/VertexDeleteRunnable.java @@ -56,7 +56,8 @@ public void run() { try { URL baseURL = TezRuntimeUtils.constructBaseURIForShuffleHandlerVertexComplete( nodeId.getHost(), shufflePort, - vertex.getDAGID().getApplicationId().toString(), vertex.getDAGID().getId(), vertexId, false); + vertex.getDAGID().getApplicationId().toString(), vertex.getDAGID().getId(), vertexId, + httpConnectionParams.isSslShuffle()); httpConnection = TezRuntimeUtils.getHttpConnection(true, baseURL, httpConnectionParams, "VertexDelete", jobTokenSecretManager); httpConnection.connect(); diff --git a/tez-plugins/tez-aux-services/pom.xml b/tez-plugins/tez-aux-services/pom.xml index 76f431b3ac..9493bd51dd 100644 --- a/tez-plugins/tez-aux-services/pom.xml +++ b/tez-plugins/tez-aux-services/pom.xml @@ -247,6 +247,7 @@ javax.crypto.* javax.security.** + javax.net.** From 627f33077480afdcefc0611fbde87d6be0010176 Mon Sep 17 00:00:00 2001 From: csjuhasz-c Date: Thu, 14 Apr 2022 11:33:56 +0200 Subject: [PATCH 367/512] TEZ-4398: Gitignore pyc files (#199) (Csaba Juhasz reviewed by Laszlo Bodor) --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index 01c99576d6..85d660672c 100644 --- a/.gitignore +++ b/.gitignore @@ -2,6 +2,7 @@ *.ipr *.iws *.DS_Store +*.pyc .idea .svn .classpath From 9f8d6fbd34bc99756e9115373917a05fad2ab8fc Mon Sep 17 00:00:00 2001 From: guptanikhil007 Date: Tue, 26 Apr 2022 04:29:03 +0530 Subject: [PATCH 368/512] Create a FileSystem for given Path (#201) Co-authored-by: Nikhil Gupta --- tez-api/src/main/java/org/apache/tez/client/TezClient.java | 5 +++-- .../client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java | 5 +++-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java index 378017b364..c37f0c181d 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java @@ -688,14 +688,15 @@ private DAGClient submitDAGSession(DAG dag) throws TezException, IOException { sessionAppId.toString()), TezConstants.TEZ_PB_PLAN_BINARY_NAME + serializedSubmitDAGPlanRequestCounter.incrementAndGet()); - try (FSDataOutputStream fsDataOutputStream = stagingFs.create(dagPlanPath, false)) { + FileSystem fs = dagPlanPath.getFileSystem(stagingFs.getConf()); + try (FSDataOutputStream fsDataOutputStream = fs.create(dagPlanPath, false)) { LOG.info("Send dag plan using YARN local resources since it's too large" + ", dag plan size=" + request.getSerializedSize() + ", max dag plan size through IPC=" + maxSubmitDAGRequestSizeThroughIPC + ", max IPC message size= " + amConfig.getTezConfiguration().getInt( CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH, CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH_DEFAULT)); request.writeTo(fsDataOutputStream); - request = requestBuilder.clear().setSerializedRequestPath(stagingFs.resolvePath(dagPlanPath).toString()).build(); + request = requestBuilder.clear().setSerializedRequestPath(fs.resolvePath(dagPlanPath).toString()).build(); } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java index 72cf0d5642..4bdb468859 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java @@ -166,7 +166,8 @@ public SubmitDAGResponseProto submitDAG(RpcController controller, if (request.hasSerializedRequestPath()) { // need to deserialize large request from hdfs Path requestPath = new Path(request.getSerializedRequestPath()); - try (FSDataInputStream fsDataInputStream = stagingFs.open(requestPath)) { + FileSystem fs = requestPath.getFileSystem(stagingFs.getConf()); + try (FSDataInputStream fsDataInputStream = fs.open(requestPath)) { CodedInputStream in = CodedInputStream.newInstance(fsDataInputStream); in.setSizeLimit(Integer.MAX_VALUE); @@ -183,7 +184,7 @@ public SubmitDAGResponseProto submitDAG(RpcController controller, } String dagId = real.submitDAG(dagPlan, additionalResources); return SubmitDAGResponseProto.newBuilder().setDagId(dagId).build(); - } catch(TezException e) { + } catch(IOException | TezException e) { throw wrapException(e); } } From 2a9495afe1ef9080941cb903fba0189abb7d3c8f Mon Sep 17 00:00:00 2001 From: guptanikhil007 Date: Fri, 29 Apr 2022 12:26:19 +0530 Subject: [PATCH 369/512] TEZ-4411: Update FileSaver dependency (#206) (Nikhil Gupta reviewed by Laszlo Bodor, Deependra Patel) --- tez-ui/src/main/resources/META-INF/LICENSE.txt | 2 +- tez-ui/src/main/webapp/bower-shrinkwrap.json | 8 ++++---- tez-ui/src/main/webapp/bower.json | 4 ++-- tez-ui/src/main/webapp/ember-cli-build.js | 2 +- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/tez-ui/src/main/resources/META-INF/LICENSE.txt b/tez-ui/src/main/resources/META-INF/LICENSE.txt index 608dc614c8..833a2b00c6 100644 --- a/tez-ui/src/main/resources/META-INF/LICENSE.txt +++ b/tez-ui/src/main/resources/META-INF/LICENSE.txt @@ -224,7 +224,7 @@ The Apache TEZ tez-ui bundles the following files under the MIT License: - jquery-ui v1.11.4 (http://jqueryui.com/) - Copyright 2014 jQuery Foundation and other contributors - jquery-mousewheel v3.1.13 (https://github.com/jquery/jquery-mousewheel) - Copyright 2006, 2014 jQuery Foundation and other contributors, https://jquery.org/ - CodeMirror 5.11.0 (https://codemirror.net/) - Copyright (C) 2015 by Marijn Haverbeke and others - - file-saver.js v1.20150507.2 (https://github.com/Teleborder/FileSaver.js) - Authored by Eli Grey + - file-saver v1.3.4 (https://github.com/eligrey/FileSaver.js) - Authored by Eli Grey - moment v2.12.0 (http://momentjs.com/) - Copyright (c) 2011-2015 Tim Wood, Iskren Chernev, Moment.js contributors - moment-timezone v0.5.0 (http://momentjs.com/timezone/) - Copyright (c) 2014 Tim Wood - font-awesome css/less files v4.5.0 (http://fontawesome.io/) - Created by Dave Gandy diff --git a/tez-ui/src/main/webapp/bower-shrinkwrap.json b/tez-ui/src/main/webapp/bower-shrinkwrap.json index 357d57691a..c8c506edce 100644 --- a/tez-ui/src/main/webapp/bower-shrinkwrap.json +++ b/tez-ui/src/main/webapp/bower-shrinkwrap.json @@ -2,9 +2,6 @@ "https://github.com/FortAwesome/Font-Awesome.git": { "4.5.0": "593ad563a987977f14102be935d0abc2a172903e" }, - "https://github.com/Teleborder/FileSaver.js.git": { - "1.20150507.2": "b7cf622909258086bc63ad764d08fcaed780ab42" - }, "https://github.com/adamwdraper/Numeral-js.git": { "1.5.3": "f97f14bb8bab988f28f1d854525b4cfeff8ec9e1" }, @@ -26,6 +23,9 @@ "https://github.com/dockyard/qunit-notifications.git": { "0.1.1": "7a13f6dba5a340e1cb9e0b64c1c711e4d7edaca1" }, + "https://github.com/eligrey/FileSaver.js.git": { + "1.2.0": "a6d11998e279e94f2926b2a897231355dfab48ab" + }, "https://github.com/ember-cli/ember-cli-shims.git": { "0.0.6": "dcab43b58d5698690050bb9a46ead5c8663c7da1" }, @@ -69,4 +69,4 @@ "https://github.com/twbs/bootstrap.git": { "3.3.6": "81df608a40bf0629a1dc08e584849bb1e43e0b7a" } -} \ No newline at end of file +} diff --git a/tez-ui/src/main/webapp/bower.json b/tez-ui/src/main/webapp/bower.json index cca56d817a..fb0498f11e 100644 --- a/tez-ui/src/main/webapp/bower.json +++ b/tez-ui/src/main/webapp/bower.json @@ -22,7 +22,7 @@ "snippet-ss": "1.11.0", "jquery-mousewheel": "3.1.13", "codemirror": "5.11.0", - "file-saver.js": "1.20150507.2", - "zip-js": "1.0.0" + "zip-js": "1.0.0", + "file-saver": "v1.2.0" } } diff --git a/tez-ui/src/main/webapp/ember-cli-build.js b/tez-ui/src/main/webapp/ember-cli-build.js index e4217e9591..f34092c52a 100644 --- a/tez-ui/src/main/webapp/ember-cli-build.js +++ b/tez-ui/src/main/webapp/ember-cli-build.js @@ -64,7 +64,7 @@ module.exports = function(defaults) { app.import('bower_components/more-js/dist/more.js'); - app.import('bower_components/file-saver.js/FileSaver.js'); + app.import('bower_components/file-saver/FileSaver.js'); app.import('bower_components/zip-js/WebContent/zip.js'); app.import('bower_components/codemirror/lib/codemirror.js'); From 4d1a86070316a1b85196d06df94787268084a5da Mon Sep 17 00:00:00 2001 From: Syed Shameerur Rahman Date: Sat, 30 Apr 2022 13:37:04 +0530 Subject: [PATCH 370/512] TEZ-4403: Upgrade SLF4J Version To 1.7.36 (#198) (Syed Shameerur Rahman reviewed by Laszlo Bodor) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 9499fe62b6..10c4200312 100644 --- a/pom.xml +++ b/pom.xml @@ -62,7 +62,7 @@ 4.1.72.Final 0.13.0 1.19 - 1.7.30 + 1.7.36 2.5.0 0.7.45 ${env.PROTOC_PATH} From 75876fcc76cff736a6321c2e85ca1a3fdcc9dd42 Mon Sep 17 00:00:00 2001 From: Murali Krishna Date: Mon, 2 May 2022 12:09:54 +0530 Subject: [PATCH 371/512] TEZ-4405: Replace log4j 1.x with reload4j (#200) (D M Murali Krishna Reddy reviewed by Laszlo Bodor) --- pom.xml | 2 +- tez-common/pom.xml | 2 +- tez-ext-service-tests/pom.xml | 2 +- tez-plugins/tez-aux-services/pom.xml | 2 +- tez-runtime-internals/pom.xml | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/pom.xml b/pom.xml index 10c4200312..9ed00c9974 100644 --- a/pom.xml +++ b/pom.xml @@ -253,7 +253,7 @@ org.slf4j - slf4j-log4j12 + slf4j-reload4j ${slf4j.version} diff --git a/tez-common/pom.xml b/tez-common/pom.xml index 5fd6910b18..0fd4255abe 100644 --- a/tez-common/pom.xml +++ b/tez-common/pom.xml @@ -27,7 +27,7 @@ org.slf4j - slf4j-log4j12 + slf4j-reload4j com.google.guava diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml index 327d457187..8c2d84ee7c 100644 --- a/tez-ext-service-tests/pom.xml +++ b/tez-ext-service-tests/pom.xml @@ -36,7 +36,7 @@ org.slf4j - slf4j-log4j12 + slf4j-reload4j com.google.guava diff --git a/tez-plugins/tez-aux-services/pom.xml b/tez-plugins/tez-aux-services/pom.xml index 9493bd51dd..b9204d2a9d 100644 --- a/tez-plugins/tez-aux-services/pom.xml +++ b/tez-plugins/tez-aux-services/pom.xml @@ -207,7 +207,7 @@ true - log4j:log4j + ch.qos.reload4j:reload4j org.slf4j:* diff --git a/tez-runtime-internals/pom.xml b/tez-runtime-internals/pom.xml index f7e14dae59..65841cf666 100644 --- a/tez-runtime-internals/pom.xml +++ b/tez-runtime-internals/pom.xml @@ -76,7 +76,7 @@ org.slf4j - slf4j-log4j12 + slf4j-reload4j From 798ddda06dbd4353f1088b7b4f2200831ff28910 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Wed, 4 May 2022 06:44:54 +0200 Subject: [PATCH 372/512] TEZ-4347: Add some diagnostic endpoints to TezAM's WebUIService (#160) (Laszlo Bodor reviewed by Rajesh Balamohan) --- .../apache/tez/dag/api/TezConfiguration.java | 12 ++ .../apache/tez/dag/api/client/DAGClient.java | 9 ++ .../tez/dag/api/client/DAGClientImpl.java | 7 +- .../tez/dag/api/client/DAGClientInternal.java | 2 + .../dag/api/client/DAGClientTimelineImpl.java | 5 + .../dag/api/client/rpc/DAGClientRPCImpl.java | 12 ++ .../src/main/proto/DAGClientAMProtocol.proto | 8 + .../AbstractServletToControllerAdapter.java | 104 +++++++++++++ .../web/ServletToControllerAdapters.java | 45 ++++++ .../apache/tez/common/web/package-info.java | 22 +++ .../tez/dag/api/client/DAGClientHandler.java | 3 + ...GClientAMProtocolBlockingPBServerImpl.java | 8 + .../org/apache/tez/dag/app/DAGAppMaster.java | 4 + .../apache/tez/dag/app/web/WebUIService.java | 23 ++- .../tez/dag/api/client/MRDAGClient.java | 6 + .../test/java/org/apache/tez/test/TestAM.java | 146 ++++++++++++++++++ 16 files changed, 414 insertions(+), 2 deletions(-) create mode 100644 tez-common/src/main/java/org/apache/tez/common/web/AbstractServletToControllerAdapter.java create mode 100644 tez-common/src/main/java/org/apache/tez/common/web/ServletToControllerAdapters.java create mode 100644 tez-common/src/main/java/org/apache/tez/common/web/package-info.java create mode 100644 tez-tests/src/test/java/org/apache/tez/test/TestAM.java diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 71ebfee10b..1ffd70a3ff 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -1980,6 +1980,18 @@ public TezConfiguration(boolean loadDefaults) { + "tez-ui.webservice.enable"; public static final boolean TEZ_AM_WEBSERVICE_ENABLE_DEFAULT = true; + /** + * String value. Range of ports that the AM can use for the WebUIService. Leave blank + * to use all possible ports. Expert level setting. It's hadoop standard range configuration. + * For example 50000-50050,50100-50200 + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty(type="boolean") + public static final String TEZ_AM_WEBSERVICE_PORT_RANGE = TEZ_AM_PREFIX + + "tez-ui.webservice.port-range"; + + public static final String TEZ_AM_WEBSERVICE_PORT_RANGE_DEFAULT = "50000-50050"; + // TODO only validate property here, value can also be validated if necessary public static void validateProperty(String property, Scope usedScope) { Scope validScope = PropertyScope.get(property); diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java index ec20ef1de4..944bff3fbd 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java @@ -140,4 +140,13 @@ public DAGStatus waitForCompletion(long timeMs) throws IOException, TezException public abstract DAGStatus waitForCompletionWithStatusUpdates(@Nullable Set statusGetOpts) throws IOException, TezException, InterruptedException; + /** + * Returns the Tez AM's web ui address if any. + * + * @return The http web UI address + * @throws IOException + * @throws TezException + */ + public abstract String getWebUIAddress() throws IOException, TezException; + } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java index 3c0de00819..bfea96b998 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java @@ -30,10 +30,10 @@ import java.util.concurrent.TimeUnit; import com.google.common.annotations.VisibleForTesting; + import org.apache.hadoop.security.UserGroupInformation; import org.apache.tez.common.CachedEntity; import org.apache.tez.common.Preconditions; - import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -670,6 +670,11 @@ public DAGClientInternal getRealClient() { return realClient; } + @Override + public String getWebUIAddress() throws IOException, TezException { + return realClient.getWebUIAddress(); + } + private double getProgress(Progress progress) { return (progress.getTotalTaskCount() == 0 ? 0.0 : (double) (progress.getSucceededTaskCount()) / progress.getTotalTaskCount()); diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientInternal.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientInternal.java index a3c898a855..8346d53da7 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientInternal.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientInternal.java @@ -125,4 +125,6 @@ public abstract VertexStatus getVertexStatus(String vertexName, */ public abstract DAGStatus waitForCompletionWithStatusUpdates(@Nullable Set statusGetOpts) throws IOException, TezException, InterruptedException; + + public abstract String getWebUIAddress() throws IOException, TezException; } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientTimelineImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientTimelineImpl.java index 17d2386860..4ec9c94354 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientTimelineImpl.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientTimelineImpl.java @@ -523,4 +523,9 @@ public DAGStatus getDAGStatus(@Nullable Set statusOptions, return getDAGStatus(statusOptions); } + @Override + public String getWebUIAddress() throws IOException, TezException { + throw new TezException("DAGClientTimelineImpl.getWebUIAddress is not supported"); + } + } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java index 5d5752e6e2..798160a4ab 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java @@ -48,6 +48,7 @@ import org.apache.tez.dag.api.client.VertexStatus; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetDAGStatusRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetVertexStatusRequestProto; +import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetWebUIAddressRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.TryKillDAGRequestProto; import com.google.common.annotations.VisibleForTesting; @@ -303,4 +304,15 @@ public DAGStatus waitForCompletionWithStatusUpdates(@Nullable Set throw new TezException("not supported"); } + @Override + public String getWebUIAddress() throws IOException, TezException { + LOG.debug("getWebUIAddress via AM for app: {} dag: {}", appId, dagId); + GetWebUIAddressRequestProto.Builder requestProtoBuilder = GetWebUIAddressRequestProto.newBuilder(); + try { + return proxy.getWebUIAddress(null, requestProtoBuilder.build()).getWebUiAddress(); + } catch (ServiceException e) { + RPCUtil.unwrapAndThrowException(e); + throw new TezException(e); + } + } } diff --git a/tez-api/src/main/proto/DAGClientAMProtocol.proto b/tez-api/src/main/proto/DAGClientAMProtocol.proto index 113c9ccfce..f0ff3916ea 100644 --- a/tez-api/src/main/proto/DAGClientAMProtocol.proto +++ b/tez-api/src/main/proto/DAGClientAMProtocol.proto @@ -90,6 +90,13 @@ message GetAMStatusResponseProto { required TezAppMasterStatusProto status = 1; } +message GetWebUIAddressRequestProto { +} + +message GetWebUIAddressResponseProto { + required string web_ui_address = 1; +} + service DAGClientAMProtocol { rpc getAllDAGs (GetAllDAGsRequestProto) returns (GetAllDAGsResponseProto); rpc getDAGStatus (GetDAGStatusRequestProto) returns (GetDAGStatusResponseProto); @@ -98,4 +105,5 @@ service DAGClientAMProtocol { rpc submitDAG (SubmitDAGRequestProto) returns (SubmitDAGResponseProto); rpc shutdownSession (ShutdownSessionRequestProto) returns (ShutdownSessionResponseProto); rpc getAMStatus (GetAMStatusRequestProto) returns (GetAMStatusResponseProto); + rpc getWebUIAddress (GetWebUIAddressRequestProto) returns (GetWebUIAddressResponseProto); } diff --git a/tez-common/src/main/java/org/apache/tez/common/web/AbstractServletToControllerAdapter.java b/tez-common/src/main/java/org/apache/tez/common/web/AbstractServletToControllerAdapter.java new file mode 100644 index 0000000000..b79b5d5d9c --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/common/web/AbstractServletToControllerAdapter.java @@ -0,0 +1,104 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.common.web; + +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.Enumeration; +import java.util.concurrent.atomic.AtomicBoolean; + +import javax.servlet.ServletConfig; +import javax.servlet.ServletContext; +import javax.servlet.ServletException; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; + +import org.apache.hadoop.yarn.webapp.Controller; + +/** + * AbstractServletToControllerAdapter is a common ancestor for classes + * that wish to adapt servlets to yarn webapp controllers. + * The adapter is responsible for: + * 1. creating a servlet instance + * 2. creating a dummy ServletConfig + * 3. delegating calls to the servlet instance's doGet method + */ +public abstract class AbstractServletToControllerAdapter extends Controller { + private AtomicBoolean initialized = new AtomicBoolean(false); + protected HttpServlet servlet; + + @Override + public void index() { + if (initialized.compareAndSet(false, true)) { + initServlet(); + } + try { + /* + * This reflection workaround is needed because HttpServlet.doGet is protected + * (even if subclasses have it public). + */ + Method doGetMethod = + this.servlet.getClass().getMethod("doGet", HttpServletRequest.class, HttpServletResponse.class); + doGetMethod.setAccessible(true); + doGetMethod.invoke(this.servlet, request(), response()); + } catch (IllegalAccessException | IllegalArgumentException | InvocationTargetException | NoSuchMethodException + | SecurityException e) { + throw new RuntimeException(e); + } + } + + /** + * Creates a dummy servlet config which is suitable for initializing a servlet instance. + * @param servletName + * @return a ServletConfig instance initialized with a ServletContext + */ + private ServletConfig getDummyServletConfig(String servletName) { + return new ServletConfig() { + + @Override + public String getServletName() { + return servletName; + } + + @Override + public ServletContext getServletContext() { + return request().getServletContext(); + } + + @Override + public Enumeration getInitParameterNames() { + return null; + } + + @Override + public String getInitParameter(String name) { + return null; + } + }; + } + + private void initServlet() { + try { + servlet.init(getDummyServletConfig(this.servlet.getClass().getSimpleName())); + } catch (ServletException e) { + throw new RuntimeException(e); + } + } +} diff --git a/tez-common/src/main/java/org/apache/tez/common/web/ServletToControllerAdapters.java b/tez-common/src/main/java/org/apache/tez/common/web/ServletToControllerAdapters.java new file mode 100644 index 0000000000..35ca1b6408 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/common/web/ServletToControllerAdapters.java @@ -0,0 +1,45 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.common.web; + +import javax.servlet.ServletException; + +import org.apache.hadoop.conf.ConfServlet; +import org.apache.hadoop.http.HttpServer2.StackServlet; +import org.apache.hadoop.jmx.JMXJsonServlet; + +public class ServletToControllerAdapters { + public static class JMXJsonServletController extends AbstractServletToControllerAdapter { + public JMXJsonServletController() throws ServletException { + this.servlet = new JMXJsonServlet(); + } + } + + public static class ConfServletController extends AbstractServletToControllerAdapter { + public ConfServletController() throws ServletException { + this.servlet = new ConfServlet(); + } + } + + public static class StackServletController extends AbstractServletToControllerAdapter { + public StackServletController() throws ServletException { + this.servlet = new StackServlet(); + } + } +} diff --git a/tez-common/src/main/java/org/apache/tez/common/web/package-info.java b/tez-common/src/main/java/org/apache/tez/common/web/package-info.java new file mode 100644 index 0000000000..2fbda31fda --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/common/web/package-info.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +@Private +package org.apache.tez.common.web; + +import org.apache.hadoop.classification.InterfaceAudience.Private; \ No newline at end of file diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java index 4cdd1ec9d1..1de62012e7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java @@ -186,4 +186,7 @@ public long getLastHeartbeatTime() { return lastHeartbeatTime.get(); } + public String getWebUIAddress() { + return dagAppMaster.getWebUIAddress(); + } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java index 4bdb468859..5c24a27908 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java @@ -45,6 +45,8 @@ import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetDAGStatusResponseProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetVertexStatusRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetVertexStatusResponseProto; +import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetWebUIAddressRequestProto; +import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetWebUIAddressResponseProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.ShutdownSessionRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.ShutdownSessionResponseProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto; @@ -227,4 +229,10 @@ public GetAMStatusResponseProto getAMStatus(RpcController controller, } } + @Override + public GetWebUIAddressResponseProto getWebUIAddress(RpcController controller, GetWebUIAddressRequestProto request) + throws ServiceException { + String address = real.getWebUIAddress(); + return GetWebUIAddressResponseProto.newBuilder().setWebUiAddress(address).build(); + } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index 58288612bb..ca50b2ac6b 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -2618,6 +2618,10 @@ private boolean enableWebUIService() { TezConfiguration.TEZ_AM_WEBSERVICE_ENABLE_DEFAULT); } + public String getWebUIAddress() { + return webUIService == null ? null : webUIService.getBaseUrl(); + } + @VisibleForTesting static void parseAllPlugins( List taskSchedulerDescriptors, BiMap taskSchedulerPluginMap, diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java index 1670370187..bf94a73002 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java @@ -23,6 +23,10 @@ import java.net.InetSocketAddress; import org.apache.tez.common.Preconditions; +import org.apache.tez.common.web.ServletToControllerAdapters.ConfServletController; +import org.apache.tez.common.web.ServletToControllerAdapters.JMXJsonServletController; +import org.apache.tez.common.web.ServletToControllerAdapters.StackServletController; + import com.google.inject.name.Names; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,6 +36,7 @@ import org.apache.hadoop.yarn.webapp.WebApp; import org.apache.hadoop.yarn.webapp.WebApps; import org.apache.hadoop.yarn.webapp.YarnWebParams; +import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.TezUncheckedException; import org.apache.tez.dag.app.AppContext; @@ -51,6 +56,7 @@ public class WebUIService extends AbstractService { private final AppContext context; private TezAMWebApp tezAMWebApp; private WebApp webApp; + private String baseUrl = ""; //url without paths, like http://host:port private String trackingUrl = ""; private String historyUrl = ""; @@ -88,9 +94,16 @@ protected void serviceStart() throws Exception { // certificates, however AM user is not trusted. // ideally the withHttpPolicy should be used, however hadoop 2.2 does not have the api conf.set("yarn.http.policy", "HTTP_ONLY"); + if (conf.get(TezConfiguration.TEZ_AM_WEBSERVICE_PORT_RANGE) == null) { + conf.set(TezConfiguration.TEZ_AM_WEBSERVICE_PORT_RANGE, + TezConfiguration.TEZ_AM_WEBSERVICE_PORT_RANGE_DEFAULT); + LOG.info( + "Using default port range for WebUIService: " + conf.get(TezConfiguration.TEZ_AM_WEBSERVICE_PORT_RANGE)); + } this.webApp = WebApps .$for(this.tezAMWebApp) .with(conf) + .withPortRange(conf, TezConfiguration.TEZ_AM_WEBSERVICE_PORT_RANGE) .start(this.tezAMWebApp); InetSocketAddress address = webApp.getListenerAddress(); if (address != null) { @@ -105,7 +118,8 @@ protected void serviceStart() throws Exception { LOG.warn("Failed to resolve canonical hostname for " + context.getAppMaster().getAppNMHost()); } - trackingUrl = "http://" + hostname + ":" + port + "/ui/"; + baseUrl = "http://" + hostname + ":" + port; + trackingUrl = baseUrl + "/ui/"; LOG.info("Instantiated WebUIService at " + trackingUrl); } } catch (Exception e) { @@ -125,6 +139,10 @@ protected void serviceStop() throws Exception { super.serviceStop(); } + public String getBaseUrl() { + return baseUrl; + } + public String getTrackingURL() { return trackingUrl; } @@ -214,6 +232,9 @@ public void setup() { "getTasksInfo"); route(WS_PREFIX_V2 + pajoin("attemptsInfo", ATTEMPT_ID, DAG_ID), AMWebController.class, "getAttemptsInfo"); + route("/jmx", JMXJsonServletController.class); + route("/conf", ConfServletController.class); + route("/stacks", StackServletController.class); } } } diff --git a/tez-mapreduce/src/main/java/org/apache/tez/dag/api/client/MRDAGClient.java b/tez-mapreduce/src/main/java/org/apache/tez/dag/api/client/MRDAGClient.java index 16dc2f8c01..c2646bd819 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/dag/api/client/MRDAGClient.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/dag/api/client/MRDAGClient.java @@ -106,4 +106,10 @@ public DAGStatus getDAGStatus(@Nullable Set statusOptions, long timeout) throws IOException, TezException { return getDAGStatus(statusOptions); } + + @Override + public String getWebUIAddress() throws IOException, TezException { + throw new TezException("MRDAGClient.getWebUIAddress is not supported"); + } + } diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestAM.java b/tez-tests/src/test/java/org/apache/tez/test/TestAM.java new file mode 100644 index 0000000000..3e8a58cf7a --- /dev/null +++ b/tez-tests/src/test/java/org/apache/tez/test/TestAM.java @@ -0,0 +1,146 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.test; + +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.net.HttpURLConnection; +import java.net.URL; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.conf.Configuration.IntegerRanges; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.tez.client.TezClient; +import org.apache.tez.dag.api.DAG; +import org.apache.tez.dag.api.ProcessorDescriptor; +import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.api.TezException; +import org.apache.tez.dag.api.Vertex; +import org.apache.tez.dag.api.client.DAGClient; +import org.apache.tez.dag.api.client.DAGStatus; +import org.apache.tez.runtime.library.processor.SleepProcessor; +import org.apache.tez.runtime.library.processor.SleepProcessor.SleepProcessorConfig; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TestAM { + + private static final Logger LOG = LoggerFactory.getLogger(TestAM.class); + + private static MiniTezCluster tezCluster; + private static MiniDFSCluster dfsCluster; + + private static Configuration conf = new Configuration(); + private static FileSystem remoteFs; + + private static final String TEST_ROOT_DIR = "target" + Path.SEPARATOR + TestAM.class.getName() + "-tmpDir"; + + @BeforeClass + public static void setup() throws IOException { + try { + conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, TEST_ROOT_DIR); + dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).format(true).racks(null).build(); + remoteFs = dfsCluster.getFileSystem(); + } catch (IOException io) { + throw new RuntimeException("problem starting mini dfs cluster", io); + } + + if (tezCluster == null) { + tezCluster = new MiniTezCluster(TestAM.class.getName(), 1, 1, 1); + Configuration tezClusterConf = new Configuration(); + tezClusterConf.set("fs.defaultFS", remoteFs.getUri().toString()); // use HDFS + tezClusterConf.setInt("yarn.nodemanager.delete.debug-delay-sec", 20000); + tezClusterConf.setLong(TezConfiguration.TEZ_AM_SLEEP_TIME_BEFORE_EXIT_MILLIS, 1000); + tezClusterConf.set(YarnConfiguration.PROXY_ADDRESS, "localhost"); + tezCluster.init(tezClusterConf); + tezCluster.start(); + } + } + + @AfterClass + public static void tearDown() { + if (tezCluster != null) { + tezCluster.stop(); + tezCluster = null; + } + if (dfsCluster != null) { + dfsCluster.shutdown(); + dfsCluster = null; + } + } + + @Test(timeout = 60000) + public void testAMWebUIService() throws TezException, IOException, InterruptedException { + SleepProcessorConfig spConf = new SleepProcessorConfig(1); + + DAG dag = DAG.create("TezSleepProcessor"); + Vertex vertex = Vertex.create("SleepVertex", + ProcessorDescriptor.create(SleepProcessor.class.getName()).setUserPayload(spConf.toUserPayload()), 1, + Resource.newInstance(1024, 1)); + dag.addVertex(vertex); + + TezConfiguration tezConf = new TezConfiguration(tezCluster.getConfig()); + TezClient tezSession = TezClient.create("TezSleepProcessor", tezConf, false); + tezSession.start(); + + DAGClient dagClient = tezSession.submitDAG(dag); + + DAGStatus dagStatus = dagClient.getDAGStatus(null); + while (!dagStatus.isCompleted()) { + Thread.sleep(500L); + dagStatus = dagClient.getDAGStatus(null); + } + + String webUIAddress = dagClient.getWebUIAddress(); + assertNotNull("getWebUIAddress should return TezAM's web UI address", webUIAddress); + LOG.info("TezAM webUI address: " + webUIAddress); + + checkAddress(webUIAddress + "/jmx"); + checkAddress(webUIAddress + "/conf"); + checkAddress(webUIAddress + "/stacks"); + + URL url = new URL(webUIAddress); + IntegerRanges portRange = conf.getRange(TezConfiguration.TEZ_AM_WEBSERVICE_PORT_RANGE, + TezConfiguration.TEZ_AM_WEBSERVICE_PORT_RANGE_DEFAULT); + assertTrue("WebUIService port should be in the defined range (got: " + url.getPort() + ")", + portRange.getRangeStart() <= url.getPort()); + + tezSession.stop(); + } + + private void checkAddress(String url) { + boolean success = false; + try { + HttpURLConnection connection = (HttpURLConnection) new URL(url).openConnection(); + connection.connect(); + success = (connection.getResponseCode() == 200); + } catch (Exception e) { + LOG.error("Error while checking url: " + url, e); + } + assertTrue(url + " should be available", success); + } +} From 98a81b1b66ee86273865ff4a77699d728ebb4786 Mon Sep 17 00:00:00 2001 From: skysiders <64545691+skysiders@users.noreply.github.com> Date: Tue, 10 May 2022 13:30:59 +0800 Subject: [PATCH 373/512] TEZ-4412 ensure mkDirForAM create directory with special permissions (#209) (Zhang Dongsheng reviewed by Laszlo Bodor) --- .../java/org/apache/tez/common/TezCommonUtils.java | 9 ++++++++- .../org/apache/tez/common/TestTezCommonUtils.java | 14 ++++++++++++++ 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java index 3163968908..5c2876444c 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java @@ -291,7 +291,14 @@ public static Path getSummaryRecoveryPath(Path attemptRecoverPath) { * @throws IOException */ public static void mkDirForAM(FileSystem fs, Path dir) throws IOException { - fs.mkdirs(dir, new FsPermission(TEZ_AM_DIR_PERMISSION)); + FsPermission perm = new FsPermission(TEZ_AM_DIR_PERMISSION); + fs.mkdirs(dir, perm); + if (!fs.getFileStatus(dir).getPermission().equals(perm)) { + LOG.warn("Directory " + dir.toString() + " created with unexpected permissions : " + + fs.getFileStatus(dir).getPermission() + ". Fixing permissions to correct value : " + + perm.toString()); + fs.setPermission(dir, perm); + } } /** diff --git a/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java b/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java index d7bd397386..d5dc6fd6b5 100644 --- a/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java +++ b/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java @@ -26,6 +26,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.MiniDFSCluster; @@ -413,4 +414,17 @@ public void testGetDAGSessionTimeout() { } + @Test + public void testMkDirForAM() throws IOException { + Configuration remoteConf = new Configuration(); + remoteConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, TEST_ROOT_DIR); + remoteConf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "777"); + MiniDFSCluster miniDFS = new MiniDFSCluster.Builder(remoteConf).numDataNodes(3).format(true).racks(null) + .build(); + FileSystem remoteFileSystem = miniDFS.getFileSystem(); + Path path = new Path(TEST_ROOT_DIR + "/testMkDirForAM"); + TezCommonUtils.mkDirForAM(remoteFileSystem, path); + Assert.assertEquals(TezCommonUtils.TEZ_AM_DIR_PERMISSION, remoteFileSystem.getFileStatus(path).getPermission()); + miniDFS.shutdown(); + } } From ba851fa06e4f57584b938706276797db9807e1d8 Mon Sep 17 00:00:00 2001 From: Mark Bathori <57758037+mbathori-cloudera@users.noreply.github.com> Date: Sun, 29 May 2022 18:58:04 +0200 Subject: [PATCH 374/512] TEZ-4417: Missing wget command in docker image (#210) (Mark Bathori reviewed by Laszlo Bodor) --- build-tools/docker/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/build-tools/docker/Dockerfile b/build-tools/docker/Dockerfile index f51cc3bd2b..158408c85e 100644 --- a/build-tools/docker/Dockerfile +++ b/build-tools/docker/Dockerfile @@ -55,6 +55,7 @@ RUN apt-get -q update && apt-get -q install --no-install-recommends -y \ rsync \ software-properties-common \ ssh-client \ + wget \ xz-utils \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* From 24a77c93f2f3d7933fbd6a3fcbcb9cb5f9d58845 Mon Sep 17 00:00:00 2001 From: Murali Krishna Date: Sun, 29 May 2022 22:37:36 +0530 Subject: [PATCH 375/512] TEZ-4409: Upgrade async-http-client to 2.12.3 to resolve CVE (#204) (D M Murali Krishna Reddy reviewed by Laszlo Bodor) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 9ed00c9974..1a51c6a9a2 100644 --- a/pom.xml +++ b/pom.xml @@ -243,7 +243,7 @@ org.asynchttpclient async-http-client - 2.12.1 + 2.12.3 io.netty From cf9e3ff308402904cd0d4eb0ece313104270de8a Mon Sep 17 00:00:00 2001 From: Mark Bathori <57758037+mbathori-cloudera@users.noreply.github.com> Date: Wed, 1 Jun 2022 19:54:15 +0200 Subject: [PATCH 376/512] TEZ-4421: Missing sudo command in docker image (#214) (Mark Bathori reviewed by Laszlo Bodor) --- build-tools/docker/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/build-tools/docker/Dockerfile b/build-tools/docker/Dockerfile index 158408c85e..3e792604ec 100644 --- a/build-tools/docker/Dockerfile +++ b/build-tools/docker/Dockerfile @@ -55,6 +55,7 @@ RUN apt-get -q update && apt-get -q install --no-install-recommends -y \ rsync \ software-properties-common \ ssh-client \ + sudo \ wget \ xz-utils \ && apt-get clean \ From e5a5578025409cbc18f5e727ea35ec03ec4bcdac Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Attila=20T=C3=B3th?= Date: Mon, 6 Jun 2022 21:50:39 +0200 Subject: [PATCH 377/512] TEZ-4383: upgrade to mockito 4.3.1 (#190) (Laszlo Attila Toth reviewed by Laszlo Bodor) --- pom.xml | 4 +- tez-api/pom.xml | 2 +- .../org/apache/tez/client/TestTezClient.java | 68 +++++----- .../tez/common/security/TestTokenCache.java | 4 +- .../tez/dag/api/TestEntityDescriptor.java | 13 +- .../tez/dag/api/client/rpc/TestDAGClient.java | 48 +++---- tez-common/pom.xml | 2 +- tez-dag/pom.xml | 6 +- .../dag/api/client/TestDAGClientHandler.java | 4 +- ...GClientAMProtocolBlockingPBServerImpl.java | 7 +- .../tez/dag/app/TestRecoveryParser.java | 8 +- .../dag/app/TestTaskCommunicatorManager.java | 8 +- .../dag/app/TestTaskCommunicatorManager1.java | 4 +- .../dag/app/TestTaskCommunicatorManager2.java | 10 +- .../dag/TestRootInputInitializerManager.java | 14 +- .../dag/app/dag/TestStateChangeNotifier.java | 6 +- .../tez/dag/app/dag/impl/TestDAGImpl.java | 19 +-- .../tez/dag/app/dag/impl/TestDAGRecovery.java | 13 +- ...estDAGSchedulerNaturalOrderControlled.java | 38 +++--- .../apache/tez/dag/app/dag/impl/TestEdge.java | 2 +- .../impl/TestImmediateStartVertexManager.java | 2 +- .../dag/impl/TestRootInputVertexManager.java | 12 +- .../tez/dag/app/dag/impl/TestTaskAttempt.java | 25 ++-- .../tez/dag/app/dag/impl/TestTaskImpl.java | 8 +- .../tez/dag/app/dag/impl/TestVertexImpl.java | 24 ++-- .../dag/app/dag/impl/TestVertexManager.java | 6 +- .../TestContainerLauncherManager.java | 6 +- .../tez/dag/app/rm/TestContainerReuse.java | 90 ++++++------- .../app/rm/TestDagAwareYarnTaskScheduler.java | 120 +++++++++--------- .../app/rm/TestLocalTaskSchedulerService.java | 4 +- .../tez/dag/app/rm/TestTaskScheduler.java | 41 +++--- .../dag/app/rm/TestTaskSchedulerManager.java | 8 +- .../dag/app/rm/container/TestAMContainer.java | 8 +- .../tez/dag/app/web/TestAMWebController.java | 6 +- .../history/recovery/TestRecoveryService.java | 4 +- tez-ext-service-tests/pom.xml | 6 +- tez-mapreduce/pom.xml | 6 +- .../mapred/split/TestGroupedSplits.java | 10 +- .../tez/mapreduce/combine/TestMRCombiner.java | 27 ++-- tez-plugins/tez-aux-services/pom.xml | 2 +- .../tez/auxservices/TestShuffleHandler.java | 20 +-- tez-plugins/tez-history-parser/pom.xml | 2 +- .../tez-protobuf-history-plugin/pom.xml | 2 +- .../pom.xml | 2 +- .../tez-yarn-timeline-history-with-fs/pom.xml | 7 +- .../ats/TestATSV15HistoryLoggingService.java | 15 +-- tez-plugins/tez-yarn-timeline-history/pom.xml | 2 +- .../ats/TestATSHistoryLoggingService.java | 110 +++++++--------- tez-runtime-internals/pom.xml | 6 +- .../TestLogicalIOProcessorRuntimeTask.java | 9 +- .../tez/runtime/task/TestTaskReporter.java | 8 +- tez-runtime-library/pom.xml | 2 +- .../TestFairShuffleVertexManager.java | 3 +- .../TestInputReadyVertexManager.java | 7 +- .../TestShuffleVertexManager.java | 6 +- .../TestShuffleVertexManagerBase.java | 37 +++--- .../TestShuffleVertexManagerUtils.java | 11 +- ...tesianProductVertexManagerPartitioned.java | 19 ++- ...TestFairCartesianProductVertexManager.java | 43 +++---- .../library/common/shuffle/TestFetcher.java | 11 +- .../common/shuffle/TestShuffleUtils.java | 44 +++---- .../TestShuffleInputEventHandlerImpl.java | 18 +-- .../shuffle/impl/TestShuffleManager.java | 10 +- .../shuffle/orderedgrouped/TestFetcher.java | 38 +++--- .../shuffle/orderedgrouped/TestShuffle.java | 10 +- ...huffleInputEventHandlerOrderedGrouped.java | 24 ++-- .../orderedgrouped/TestShuffleScheduler.java | 38 +++--- .../common/sort/impl/TestPipelinedSorter.java | 2 +- .../sort/impl/dflt/TestDefaultSorter.java | 9 +- .../TestUnorderedPartitionedKVWriter.java | 27 ++-- .../input/TestOrderedGroupedKVInput.java | 2 +- .../library/output/OutputTestHelpers.java | 7 +- .../output/TestOnFileSortedOutput.java | 7 +- .../output/TestOnFileUnorderedKVOutput.java | 7 +- .../runtime/library/utils/TestCodecUtils.java | 11 +- tez-tools/analyzers/job-analyzer/pom.xml | 10 +- 76 files changed, 638 insertions(+), 643 deletions(-) diff --git a/pom.xml b/pom.xml index 1a51c6a9a2..c496e0ce87 100644 --- a/pom.xml +++ b/pom.xml @@ -704,8 +704,8 @@ org.mockito - mockito-all - 1.10.8 + mockito-core + 4.3.1 org.apache.commons diff --git a/tez-api/pom.xml b/tez-api/pom.xml index 4e1825b09d..cd5abdb03a 100644 --- a/tez-api/pom.xml +++ b/tez-api/pom.xml @@ -107,7 +107,7 @@ org.mockito - mockito-all + mockito-core test diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java index 67c4a604b6..4f33348f89 100644 --- a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java +++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java @@ -38,14 +38,14 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.isNull; import static org.mockito.Mockito.RETURNS_DEEP_STUBS; -import static org.mockito.Mockito.times; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doCallRealMethod; +import static org.mockito.Mockito.isNull; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -89,9 +89,7 @@ import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolBlockingPB; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetAMStatusRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetAMStatusResponseProto; -import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetDAGStatusRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetDAGStatusResponseProto; -import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.ShutdownSessionRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.TezAppMasterStatusProto; import org.apache.tez.dag.api.records.DAGProtos.DAGStatusProto; @@ -188,10 +186,11 @@ TezClientForTest configureAndCreateTezClient(Map lrs, boo YarnClient yarnClient = mock(YarnClient.class, RETURNS_DEEP_STUBS); when(yarnClient.createApplication().getNewApplicationResponse().getApplicationId()).thenReturn(appId1); when(yarnClient.getApplicationReport(appId1).getYarnApplicationState()).thenReturn(YarnApplicationState.NEW); - when(yarnClient.submitApplication(any(ApplicationSubmissionContext.class))).thenReturn(appId1); + when(yarnClient.submitApplication(any())).thenReturn(appId1); DAGClientAMProtocolBlockingPB sessionAmProxy = mock(DAGClientAMProtocolBlockingPB.class, RETURNS_DEEP_STUBS); - when(sessionAmProxy.getAMStatus(any(RpcController.class), any(GetAMStatusRequestProto.class))) + when(sessionAmProxy.getAMStatus(any(), any())) + .thenReturn(GetAMStatusResponseProto.newBuilder().setStatus(TezAppMasterStatusProto.RUNNING).build()) .thenReturn(GetAMStatusResponseProto.newBuilder().setStatus(TezAppMasterStatusProto.RUNNING).build()); client.sessionAmProxy = sessionAmProxy; @@ -255,7 +254,7 @@ private void _testTezClientSessionLargeDAGPlan(int maxIPCMsgSize, int payloadSiz client.stop(); ArgumentCaptor captor = ArgumentCaptor.forClass(SubmitDAGRequestProto.class); - verify(client.sessionAmProxy).submitDAG((RpcController)any(), captor.capture()); + verify(client.sessionAmProxy).submitDAG(any(), captor.capture()); SubmitDAGRequestProto request = captor.getValue(); if (shouldSerialize) { @@ -308,12 +307,12 @@ public void testGetClient() throws Exception { verify(client2.mockYarnClient, times(0)).submitApplication(captor.capture()); // Validate dag submission from second TezClient as normal */ - verify(client2.sessionAmProxy, times(1)).submitDAG((RpcController)any(), (SubmitDAGRequestProto) any()); + verify(client2.sessionAmProxy, times(1)).submitDAG(any(), any()); // Validate stop from new TezClient as normal */ client2.stop(); - verify(client2.sessionAmProxy, times(1)).shutdownSession((RpcController) any(), - (ShutdownSessionRequestProto) any()); + verify(client2.sessionAmProxy, times(1)).shutdownSession(any(), + any()); verify(client2.mockYarnClient, times(1)).stop(); /* END reuse of AM from new TezClient */ } @@ -330,7 +329,7 @@ public TezClientForTest testTezClient(boolean isSession, boolean shouldStop) thr when(client.mockYarnClient.getApplicationReport(client.mockAppId).getYarnApplicationState()) .thenReturn(YarnApplicationState.RUNNING); client.start(); - verify(client.mockYarnClient, times(1)).init((Configuration)any()); + verify(client.mockYarnClient, times(1)).init(any()); verify(client.mockYarnClient, times(1)).start(); if (isSession) { verify(client.mockYarnClient, times(1)).submitApplication(captor.capture()); @@ -353,6 +352,10 @@ public TezClientForTest testTezClient(boolean isSession, boolean shouldStop) thr Vertex vertex = Vertex.create("Vertex", ProcessorDescriptor.create("P"), 1, Resource.newInstance(1, 1)); DAG dag = DAG.create("DAG").addVertex(vertex).addTaskLocalFiles(lrDAG); + if (!isSession) { + when(client.sessionAmProxy.getAMStatus(any(), any())) + .thenReturn(GetAMStatusResponseProto.newBuilder().setStatus(TezAppMasterStatusProto.SHUTDOWN).build()); + } DAGClient dagClient = client.submitDAG(dag); assertTrue(dagClient.getExecutionContext().contains(client.mockAppId.toString())); @@ -360,7 +363,7 @@ public TezClientForTest testTezClient(boolean isSession, boolean shouldStop) thr if (isSession) { verify(client.mockYarnClient, times(1)).submitApplication(captor.capture()); - verify(client.sessionAmProxy, times(1)).submitDAG((RpcController)any(), (SubmitDAGRequestProto) any()); + verify(client.sessionAmProxy, times(1)).submitDAG(any(), any()); } else { verify(client.mockYarnClient, times(1)).submitApplication(captor.capture()); ApplicationSubmissionContext context = captor.getValue(); @@ -399,7 +402,7 @@ public TezClientForTest testTezClient(boolean isSession, boolean shouldStop) thr assertEquals(dagClient.getSessionIdentifierString(), client.mockAppId.toString()); // additional resource is sent ArgumentCaptor captor1 = ArgumentCaptor.forClass(SubmitDAGRequestProto.class); - verify(client.sessionAmProxy, times(2)).submitDAG((RpcController)any(), captor1.capture()); + verify(client.sessionAmProxy, times(2)).submitDAG(any(), captor1.capture()); SubmitDAGRequestProto proto = captor1.getValue(); Assert.assertEquals(1, proto.getAdditionalAmResources().getLocalResourcesCount()); Assert.assertEquals(lrName2, proto.getAdditionalAmResources().getLocalResources(0).getName()); @@ -426,8 +429,8 @@ public TezClientForTest testTezClient(boolean isSession, boolean shouldStop) thr if(shouldStop) { client.stop(); if (isSession) { - verify(client.sessionAmProxy, times(1)).shutdownSession((RpcController) any(), - (ShutdownSessionRequestProto) any()); + verify(client.sessionAmProxy, times(1)).shutdownSession(any(), + any()); } verify(client.mockYarnClient, times(1)).stop(); } @@ -443,14 +446,14 @@ public void testPreWarm() throws Exception { .thenReturn(YarnApplicationState.RUNNING); when( - client.sessionAmProxy.getAMStatus((RpcController) any(), (GetAMStatusRequestProto) any())) + client.sessionAmProxy.getAMStatus(any(), any())) .thenReturn(GetAMStatusResponseProto.newBuilder().setStatus(TezAppMasterStatusProto.READY).build()); PreWarmVertex vertex = PreWarmVertex.create("PreWarm", 1, Resource.newInstance(1, 1)); client.preWarm(vertex); ArgumentCaptor captor1 = ArgumentCaptor.forClass(SubmitDAGRequestProto.class); - verify(client.sessionAmProxy, times(1)).submitDAG((RpcController)any(), captor1.capture()); + verify(client.sessionAmProxy, times(1)).submitDAG(any(), captor1.capture()); SubmitDAGRequestProto proto = captor1.getValue(); assertTrue(proto.getDAGPlan().getName().startsWith(TezConstants.TEZ_PREWARM_DAG_NAME_PREFIX)); @@ -467,7 +470,7 @@ public void testPreWarmCloseStuck() throws Exception { when(client.mockYarnClient.getApplicationReport(client.mockAppId).getYarnApplicationState()) .thenReturn(YarnApplicationState.RUNNING); - when(client.sessionAmProxy.getAMStatus((RpcController) any(), (GetAMStatusRequestProto) any())) + when(client.sessionAmProxy.getAMStatus(any(), any())) .thenReturn(GetAMStatusResponseProto.newBuilder().setStatus(TezAppMasterStatusProto.READY).build()); PreWarmVertex vertex = PreWarmVertex.create("PreWarm", 1, Resource.newInstance(1, 1)); @@ -480,7 +483,7 @@ public void testPreWarmCloseStuck() throws Exception { private void setClientToReportStoppedDags(TezClientForTest client) throws Exception { when(client.mockYarnClient.getApplicationReport(client.mockAppId).getYarnApplicationState()) .thenReturn(YarnApplicationState.FINISHED); - when(client.sessionAmProxy.getDAGStatus(isNull(RpcController.class), any(GetDAGStatusRequestProto.class))) + when(client.sessionAmProxy.getDAGStatus(isNull(), any())) .thenReturn(GetDAGStatusResponseProto.newBuilder().setDagStatus(DAGStatusProto.newBuilder() .addDiagnostics("Diagnostics_0").setState(DAGStatusStateProto.DAG_SUCCEEDED) .setDAGProgress(ProgressProto.newBuilder() @@ -502,8 +505,8 @@ public void testPreWarmWithTimeout() throws Exception { spyClient.mockAppId).getYarnApplicationState()) .thenReturn(YarnApplicationState.RUNNING); when( - spyClient.sessionAmProxy.getAMStatus((RpcController) any(), - (GetAMStatusRequestProto) any())) + spyClient.sessionAmProxy.getAMStatus(any(), + any())) .thenReturn( GetAMStatusResponseProto.newBuilder().setStatus( TezAppMasterStatusProto.INITIALIZING).build()); @@ -518,15 +521,15 @@ public void testPreWarmWithTimeout() throws Exception { endTime = Time.monotonicNow(); assertTrue("Time taken is not as expected", (endTime - startTime) > timeout); - verify(spyClient, times(0)).submitDAG(any(DAG.class)); + verify(spyClient, times(0)).submitDAG(any()); Assert.assertTrue("Unexpected Exception message", te.getMessage().contains("Tez AM not ready")); } when( - spyClient.sessionAmProxy.getAMStatus((RpcController) any(), - (GetAMStatusRequestProto) any())) + spyClient.sessionAmProxy.getAMStatus(any(), + any())) .thenReturn( GetAMStatusResponseProto.newBuilder().setStatus( TezAppMasterStatusProto.READY).build()); @@ -536,7 +539,7 @@ public void testPreWarmWithTimeout() throws Exception { endTime = Time.monotonicNow(); assertTrue("Time taken is not as expected", (endTime - startTime) <= timeout); - verify(spyClient, times(1)).submitDAG(any(DAG.class)); + verify(spyClient, times(1)).submitDAG(any()); } catch (TezException te) { fail("PreWarm should have succeeded!"); } @@ -571,7 +574,7 @@ public void run() { endTime = Time.monotonicNow(); assertTrue("Time taken is not as expected", (endTime - startTime) <= timeout); - verify(spyClient, times(2)).submitDAG(any(DAG.class)); + verify(spyClient, times(2)).submitDAG(any()); setClientToReportStoppedDags(client); spyClient.stop(); client.stop(); @@ -926,8 +929,7 @@ public void testAMClientHeartbeat() throws Exception { Thread.sleep(1000); } client.stop(); - verify(client.sessionAmProxy, atLeast(3)).getAMStatus(any(RpcController.class), - any(GetAMStatusRequestProto.class)); + verify(client.sessionAmProxy, atLeast(3)).getAMStatus(any(), any()); conf.setInt(TezConfiguration.TEZ_AM_CLIENT_HEARTBEAT_TIMEOUT_SECS, -1); final TezClientForTest client2 = configureAndCreateTezClient(conf); @@ -940,10 +942,7 @@ public void testAMClientHeartbeat() throws Exception { Thread.sleep(1000); } client2.stop(); - verify(client2.sessionAmProxy, times(0)).getAMStatus(any(RpcController.class), - any(GetAMStatusRequestProto.class)); - - + verify(client2.sessionAmProxy, times(0)).getAMStatus(any(), any()); } @Test(timeout = 20000) @@ -987,8 +986,7 @@ public void testAMHeartbeatFailOnGetAMStatus() throws Exception { final TezClientForTest client = configureAndCreateTezClient(conf); client.start(); - when(client.sessionAmProxy.getAMStatus(any(RpcController.class), - any(GetAMStatusRequestProto.class))).thenThrow(new ServiceException("error")); + when(client.sessionAmProxy.getAMStatus(any(), any())).thenThrow(new ServiceException("error")); client.callRealGetSessionAMProxy = true; when(client.mockYarnClient.getApplicationReport(client.mockAppId).getYarnApplicationState()) .thenReturn(YarnApplicationState.FAILED); diff --git a/tez-api/src/test/java/org/apache/tez/common/security/TestTokenCache.java b/tez-api/src/test/java/org/apache/tez/common/security/TestTokenCache.java index fcb1e98aee..4c144174c7 100644 --- a/tez-api/src/test/java/org/apache/tez/common/security/TestTokenCache.java +++ b/tez-api/src/test/java/org/apache/tez/common/security/TestTokenCache.java @@ -18,7 +18,7 @@ package org.apache.tez.common.security; -import static org.mockito.Matchers.any; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -153,7 +153,7 @@ private MockFileSystem createFileSystemForServiceName(final String service) throws IOException { MockFileSystem mockFs = new MockFileSystem(); when(mockFs.getCanonicalServiceName()).thenReturn(service); - when(mockFs.getDelegationToken(any(String.class))).thenAnswer( + when(mockFs.getDelegationToken(any())).thenAnswer( new Answer>() { int unique = 0; @Override diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/TestEntityDescriptor.java b/tez-api/src/test/java/org/apache/tez/dag/api/TestEntityDescriptor.java index 606bf42717..82c22ad4ef 100644 --- a/tez-api/src/test/java/org/apache/tez/dag/api/TestEntityDescriptor.java +++ b/tez-api/src/test/java/org/apache/tez/dag/api/TestEntityDescriptor.java @@ -23,7 +23,6 @@ import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; -import java.nio.ByteBuffer; import org.apache.commons.lang.RandomStringUtils; import org.apache.hadoop.conf.Configuration; @@ -31,11 +30,11 @@ import org.apache.tez.common.TezUtils; import org.junit.Assert; import org.junit.Test; -import org.mockito.Mockito; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; public class TestEntityDescriptor { @@ -57,7 +56,7 @@ public void testSingularWrite(InputDescriptor entityDescriptor, InputDescriptor ByteArrayOutputStream bos = new ByteArrayOutputStream(out.getData().length); bos.write(out.getData()); - Mockito.verify(entityDescriptor).writeSingular(eq(out), any(ByteBuffer.class)); + verify(entityDescriptor).writeSingular(eq(out), any()); deserialized.readFields(new DataInputStream(new ByteArrayInputStream(bos.toByteArray()))); verifyResults(entityDescriptor, deserialized, payload, confVal); } @@ -69,12 +68,12 @@ public void testSegmentedWrite(InputDescriptor entityDescriptor, InputDescriptor entityDescriptor.write(out); out.close(); - Mockito.verify(entityDescriptor).writeSegmented(eq(out), any(ByteBuffer.class)); + verify(entityDescriptor).writeSegmented(eq(out), any()); deserialized.readFields(new DataInputStream(new ByteArrayInputStream(bos.toByteArray()))); verifyResults(entityDescriptor, deserialized, payload, confVal); } - @Test (timeout=1000) + @Test (timeout=3000) public void testEntityDescriptorHadoopSerialization() throws IOException { /* This tests the alternate serialization code path * if the DataOutput is not DataOutputBuffer diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java index f0cc20b062..0ea5d1a26d 100644 --- a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java +++ b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java @@ -20,8 +20,10 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.*; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.argThat; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.isNull; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -164,30 +166,22 @@ private void setUpData(){ .setVertexCounters(vertexCountersProto) .build(); } - - private static class DAGCounterRequestMatcher extends ArgumentMatcher{ + + private static class DAGCounterRequestMatcher implements ArgumentMatcher{ @Override - public boolean matches(Object argument) { - if (argument instanceof GetDAGStatusRequestProto){ - GetDAGStatusRequestProto requestProto = (GetDAGStatusRequestProto)argument; - return requestProto.getStatusOptionsCount() != 0 - && requestProto.getStatusOptionsList().get(0) == StatusGetOptsProto.GET_COUNTERS; - } - return false; + public boolean matches(GetDAGStatusRequestProto requestProto) { + return requestProto != null && requestProto.getStatusOptionsCount() != 0 + && requestProto.getStatusOptionsList().get(0) == StatusGetOptsProto.GET_COUNTERS; } } - - private static class VertexCounterRequestMatcher extends ArgumentMatcher{ + + private static class VertexCounterRequestMatcher implements ArgumentMatcher{ @Override - public boolean matches(Object argument) { - if (argument instanceof GetVertexStatusRequestProto){ - GetVertexStatusRequestProto requestProto = (GetVertexStatusRequestProto)argument; - return requestProto.getStatusOptionsCount() != 0 - && requestProto.getStatusOptionsList().get(0) == StatusGetOptsProto.GET_COUNTERS; - } - return false; + public boolean matches(GetVertexStatusRequestProto requestProto) { + return requestProto != null && requestProto.getStatusOptionsCount() != 0 + && requestProto.getStatusOptionsList().get(0) == StatusGetOptsProto.GET_COUNTERS; } } @@ -202,14 +196,14 @@ public void setUp() throws YarnException, IOException, TezException, ServiceExce dagIdStr = "dag_9999_0001_1"; mockProxy = mock(DAGClientAMProtocolBlockingPB.class); // return the response with Counters is the request match the CounterMatcher - when(mockProxy.getDAGStatus(isNull(RpcController.class), any(GetDAGStatusRequestProto.class))) + when(mockProxy.getDAGStatus(isNull(), any())) .thenReturn(GetDAGStatusResponseProto.newBuilder().setDagStatus(dagStatusProtoWithoutCounters).build()); - when(mockProxy.getDAGStatus(isNull(RpcController.class), argThat(new DAGCounterRequestMatcher()))) + when(mockProxy.getDAGStatus(isNull(), argThat(new DAGCounterRequestMatcher()))) .thenReturn(GetDAGStatusResponseProto.newBuilder().setDagStatus(dagStatusProtoWithCounters).build()); - when(mockProxy.getVertexStatus(isNull(RpcController.class), any(GetVertexStatusRequestProto.class))) + when(mockProxy.getVertexStatus(isNull(), any())) .thenReturn(GetVertexStatusResponseProto.newBuilder().setVertexStatus(vertexStatusProtoWithoutCounters).build()); - when(mockProxy.getVertexStatus(isNull(RpcController.class), argThat(new VertexCounterRequestMatcher()))) + when(mockProxy.getVertexStatus(isNull(), argThat(new VertexCounterRequestMatcher()))) .thenReturn(GetVertexStatusResponseProto.newBuilder().setVertexStatus(vertexStatusProtoWithCounters).build()); TezConfiguration tezConf = new TezConfiguration(); @@ -270,7 +264,7 @@ public void testTryKillDAG() throws Exception{ @Test(timeout = 5000) public void testWaitForCompletion() throws Exception{ // first time return DAG_RUNNING, second time return DAG_SUCCEEDED - when(mockProxy.getDAGStatus(isNull(RpcController.class), any(GetDAGStatusRequestProto.class))) + when(mockProxy.getDAGStatus(isNull(), any())) .thenReturn(GetDAGStatusResponseProto.newBuilder().setDagStatus(dagStatusProtoWithoutCounters) .build()) .thenReturn(GetDAGStatusResponseProto.newBuilder().setDagStatus @@ -291,7 +285,7 @@ public void testWaitForCompletion() throws Exception{ public void testWaitForCompletionWithStatusUpdates() throws Exception{ // first time and second time return DAG_RUNNING, third time return DAG_SUCCEEDED - when(mockProxy.getDAGStatus(isNull(RpcController.class), any(GetDAGStatusRequestProto.class))) + when(mockProxy.getDAGStatus(isNull(), any())) .thenReturn(GetDAGStatusResponseProto.newBuilder().setDagStatus( DAGStatusProto.newBuilder(dagStatusProtoWithCounters) .setState(DAGStatusStateProto.DAG_RUNNING).build()).build()) @@ -317,7 +311,7 @@ public void testWaitForCompletionWithStatusUpdates() throws Exception{ verify(mockProxy, times(4)) .getDAGStatus(rpcControllerArgumentCaptor.capture(), argumentCaptor.capture()); - when(mockProxy.getDAGStatus(isNull(RpcController.class), any(GetDAGStatusRequestProto.class))) + when(mockProxy.getDAGStatus(isNull(), any())) .thenReturn(GetDAGStatusResponseProto.newBuilder().setDagStatus( DAGStatusProto.newBuilder(dagStatusProtoWithCounters) .setState(DAGStatusStateProto.DAG_RUNNING).build()).build()) @@ -555,7 +549,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { return GetDAGStatusResponseProto.newBuilder().setDagStatus(constructDagStatusProto( stateProto)).build(); } - }).when(mock).getDAGStatus(isNull(RpcController.class), any(GetDAGStatusRequestProto.class)); + }).when(mock).getDAGStatus(isNull(), any()); return mock; } diff --git a/tez-common/pom.xml b/tez-common/pom.xml index 0fd4255abe..9ccf02a38c 100644 --- a/tez-common/pom.xml +++ b/tez-common/pom.xml @@ -63,7 +63,7 @@ org.mockito - mockito-all + mockito-core test diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml index 723082c54e..356ed2c2e9 100644 --- a/tez-dag/pom.xml +++ b/tez-dag/pom.xml @@ -87,6 +87,10 @@ org.apache.hadoop hadoop-yarn-server-web-proxy + + org.roaringbitmap + RoaringBitmap + org.slf4j slf4j-api @@ -123,7 +127,7 @@ org.mockito - mockito-all + mockito-core test diff --git a/tez-dag/src/test/java/org/apache/tez/dag/api/client/TestDAGClientHandler.java b/tez-dag/src/test/java/org/apache/tez/dag/api/client/TestDAGClientHandler.java index bf07838a45..c7daebf02d 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/api/client/TestDAGClientHandler.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/api/client/TestDAGClientHandler.java @@ -51,11 +51,11 @@ public void testDAGClientHandler() throws TezException { DAG mockDAG = mock(DAG.class); when(mockDAG.getID()).thenReturn(mockTezDAGId); DAGStatusBuilder mockDagStatusBuilder = mock(DAGStatusBuilder.class); - when(mockDAG.getDAGStatus(anySetOf(StatusGetOpts.class))).thenReturn( + when(mockDAG.getDAGStatus(anySet())).thenReturn( mockDagStatusBuilder); VertexStatusBuilder mockVertexStatusBuilder = mock(VertexStatusBuilder.class); - when(mockDAG.getVertexStatus(anyString(), anySetOf(StatusGetOpts.class))) + when(mockDAG.getVertexStatus(anyString(), anySet())) .thenReturn(mockVertexStatusBuilder); DAGAppMaster mockDagAM = mock(DAGAppMaster.class); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClientAMProtocolBlockingPBServerImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClientAMProtocolBlockingPBServerImpl.java index 040ca2fb7b..1aea8f793d 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClientAMProtocolBlockingPBServerImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClientAMProtocolBlockingPBServerImpl.java @@ -28,8 +28,11 @@ import static junit.framework.TestCase.assertEquals; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.FileSystem; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java index 1c09d5d9e8..57148ee7e9 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java @@ -111,7 +111,7 @@ public void setUp() throws IllegalArgumentException, IOException { mockAppMaster.dagIDs = new HashSet(); when(mockAppMaster.getConfig()).thenReturn(new Configuration()); mockDAGImpl = mock(DAGImpl.class); - when(mockAppMaster.createDAG(any(DAGPlan.class), any(TezDAGID.class))).thenReturn(mockDAGImpl); + when(mockAppMaster.createDAG(any(), any())).thenReturn(mockDAGImpl); parser = new RecoveryParser(mockAppMaster, localFS, recoveryPath, 3); } @@ -198,7 +198,7 @@ public void testSkipAllOtherEvents_1() throws IOException { assertTrue(dagData.reason.contains("DAG Commit was in progress, not recoverable,")); // DAGSubmittedEvent is handled but DAGInitializedEvent and DAGStartedEvent in the next attempt are both skipped // due to the dag is not recoerable. - verify(mockAppMaster).createDAG(any(DAGPlan.class),any(TezDAGID.class)); + verify(mockAppMaster).createDAG(any(), any()); assertNull(dagData.getDAGInitializedEvent()); assertNull(dagData.getDAGStartedEvent()); } @@ -245,7 +245,7 @@ public void testSkipAllOtherEvents_2() throws IOException { assertEquals(DAGState.FAILED, dagData.dagState); assertEquals(true, dagData.isCompleted); // DAGSubmittedEvent, DAGInitializedEvent and DAGFinishedEvent is handled - verify(mockAppMaster).createDAG(any(DAGPlan.class),any(TezDAGID.class)); + verify(mockAppMaster).createDAG(any(), any()); // DAGInitializedEvent may not been handled before DAGFinishedEvent, // because DAGFinishedEvent's writeToRecoveryImmediately is true assertNotNull(dagData.getDAGFinishedEvent()); @@ -294,7 +294,7 @@ public void testLastCorruptedRecoveryRecord() throws IOException { assertEquals(null, dagData.reason); assertEquals(false, dagData.nonRecoverable); // verify DAGSubmitedEvent & DAGInititlizedEvent is handled. - verify(mockAppMaster).createDAG(any(DAGPlan.class),any(TezDAGID.class)); + verify(mockAppMaster).createDAG(any(), any()); assertNotNull(dagData.getDAGInitializedEvent()); } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager.java index c7f97d3ea9..3935789860 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager.java @@ -18,10 +18,10 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; import static org.mockito.Mockito.RETURNS_DEEP_STUBS; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; @@ -187,7 +187,7 @@ public void testEventRouting() throws Exception { AppContext appContext = mock(AppContext.class, RETURNS_DEEP_STUBS); NodeId nodeId = NodeId.newInstance("host1", 3131); - when(appContext.getAllContainers().get(any(ContainerId.class)).getContainer().getNodeId()) + when(appContext.getAllContainers().get(any()).getContainer().getNodeId()) .thenReturn(nodeId); TaskHeartbeatHandler thh = mock(TaskHeartbeatHandler.class); ContainerHeartbeatHandler chh = mock(ContainerHeartbeatHandler.class); @@ -348,7 +348,7 @@ public void testTaskCommunicatorUserError() { assertTrue(event.getDiagnosticInfo().contains(expectedId)); - when(appContext.getAllContainers().get(any(ContainerId.class)).getContainer().getNodeId()) + when(appContext.getAllContainers().get(any()).getContainer().getNodeId()) .thenReturn(mock(NodeId.class)); taskCommunicatorManager.registerRunningContainer(mock(ContainerId.class), 0); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager1.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager1.java index 0f8afaafe6..d7e62ee530 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager1.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager1.java @@ -19,7 +19,7 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; @@ -144,7 +144,7 @@ public void setUp() throws TezException { AMContainer amContainer = mock(AMContainer.class); Container container = mock(Container.class); doReturn(nodeId).when(container).getNodeId(); - doReturn(amContainer).when(amContainerMap).get(any(ContainerId.class)); + doReturn(amContainer).when(amContainerMap).get(any()); doReturn(container).when(amContainer).getContainer(); Configuration conf = new TezConfiguration(); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager2.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager2.java index bb7e94b5c7..ac60001a46 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager2.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager2.java @@ -16,10 +16,10 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.reset; import static org.mockito.Mockito.times; @@ -256,7 +256,7 @@ private static class TaskCommunicatorManagerWrapperForTest { doReturn(dag).when(appContext).getCurrentDAG(); doReturn(vertex).when(dag).getVertex(eq(vertexId)); doReturn(new TaskAttemptEventInfo(0, new LinkedList(), 0)).when(vertex) - .getTaskAttemptTezEvents(any(TezTaskAttemptID.class), anyInt(), anyInt(), anyInt()); + .getTaskAttemptTezEvents(any(), anyInt(), anyInt(), anyInt()); doReturn(appAttemptId).when(appContext).getApplicationAttemptId(); doReturn(credentials).when(appContext).getAppCredentials(); doReturn(appAcls).when(appContext).getApplicationACLs(); @@ -267,7 +267,7 @@ private static class TaskCommunicatorManagerWrapperForTest { AMContainer amContainer = mock(AMContainer.class); Container container = mock(Container.class); doReturn(nodeId).when(container).getNodeId(); - doReturn(amContainer).when(amContainerMap).get(any(ContainerId.class)); + doReturn(amContainer).when(amContainerMap).get(any()); doReturn(container).when(amContainer).getContainer(); userPayload = TezUtils.createUserPayloadFromConf(conf); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java index ffa4309602..2c5aa337b7 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java @@ -15,8 +15,8 @@ package org.apache.tez.dag.app.dag; import static org.junit.Assert.assertEquals; -import static org.mockito.Matchers.any; import static org.mockito.Mockito.RETURNS_DEEP_STUBS; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; @@ -101,7 +101,7 @@ public void testEventBeforeSuccess() throws Exception { eventList.add(te1); initializerWrapper.handleInputInitializerEvents(eventList); - verify(initializer, never()).handleInputInitializerEvent(any(List.class)); + verify(initializer, never()).handleInputInitializerEvent(any()); eventList.clear(); // First attempt, Task success notification @@ -123,7 +123,7 @@ public void testEventBeforeSuccess() throws Exception { eventList.add(te2); initializerWrapper.handleInputInitializerEvents(eventList); - verify(initializer, never()).handleInputInitializerEvent(any(List.class)); + verify(initializer, never()).handleInputInitializerEvent(any()); eventList.clear(); reset(initializer); @@ -176,7 +176,7 @@ public void testSuccessBeforeEvent() throws Exception { eventList.add(te1); initializerWrapper.handleInputInitializerEvents(eventList); - verify(initializer, never()).handleInputInitializerEvent(any(List.class)); + verify(initializer, never()).handleInputInitializerEvent(any()); eventList.clear(); // First attempt, Task success notification @@ -192,7 +192,7 @@ public void testSuccessBeforeEvent() throws Exception { TezTaskAttemptID srcTaskAttemptId12 = TezTaskAttemptID.getInstance(srcTaskId1, 2); // 2nd attempt succeeded initializerWrapper.onTaskSucceeded(srcVertexName, srcTaskId1, srcTaskAttemptId12.getId()); - verify(initializer, never()).handleInputInitializerEvent(any(List.class)); + verify(initializer, never()).handleInputInitializerEvent(any()); // 2nd attempt send event EventMetaData sourceInfo12 = @@ -203,7 +203,7 @@ public void testSuccessBeforeEvent() throws Exception { eventList.add(te2); initializerWrapper.handleInputInitializerEvents(eventList); - verify(initializer, never()).handleInputInitializerEvent(any(List.class)); + verify(initializer, never()).handleInputInitializerEvent(any()); } @@ -269,4 +269,4 @@ static void awaitInitialize() throws InterruptedException { } } } -} \ No newline at end of file +} diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestStateChangeNotifier.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestStateChangeNotifier.java index d20903d348..b43782e165 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestStateChangeNotifier.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestStateChangeNotifier.java @@ -20,7 +20,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; @@ -139,14 +139,14 @@ public void testEventsOnRegistration() { verify(mockListener13, times(1)).onStateUpdated(argumentCaptor.capture()); assertEquals(VertexState.RUNNING, argumentCaptor.getValue().getVertexState()); - verify(mockListener14, never()).onStateUpdated(any(VertexStateUpdate.class)); + verify(mockListener14, never()).onStateUpdated(any()); // Vertex has not notified of state tracker.reset(); VertexStateUpdateListener mockListener2 = mock(VertexStateUpdateListener.class); tracker.registerForVertexUpdates(v2.getName(), null, mockListener2); Assert.assertEquals(0, tracker.totalCount.get()); // there should no be any event sent out - verify(mockListener2, never()).onStateUpdated(any(VertexStateUpdate.class)); + verify(mockListener2, never()).onStateUpdated(any()); // Vertex has notified about parallelism update only tracker.stateChanged(v3.getVertexId(), new VertexStateUpdateParallelismUpdated(v3.getName(), 23, -1)); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java index f2b16a9094..9a3292e8b8 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java @@ -18,11 +18,13 @@ package org.apache.tez.dag.app.dag.impl; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; import java.io.IOException; @@ -149,7 +151,6 @@ import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Test; -import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -856,7 +857,7 @@ public void setup() { when(appContext.getHadoopShim()).thenReturn(defaultShim); when(appContext.getApplicationID()).thenReturn(appAttemptId.getApplicationId()); - Mockito.doAnswer(new Answer() { + doAnswer(new Answer() { public ListenableFuture answer(InvocationOnMock invocation) { Object[] args = invocation.getArguments(); CallableEvent e = (CallableEvent) args[0]; @@ -1942,17 +1943,17 @@ public void testDAGHang() throws Exception { conf.setBoolean( TezConfiguration.TEZ_AM_COMMIT_ALL_OUTPUTS_ON_DAG_SUCCESS, false); - dag = Mockito.spy(new DAGImpl(dagId, conf, dagPlan, + dag = spy(new DAGImpl(dagId, conf, dagPlan, dispatcher.getEventHandler(), taskCommunicatorManagerInterface, fsTokens, clock, "user", thh, appContext)); StateMachineTez spyStateMachine = - Mockito.spy(new StateMachineTez( + spy(new StateMachineTez( dag.stateMachineFactory.make(dag), dag)); when(dag.getStateMachine()).thenReturn(spyStateMachine); dag.entityUpdateTracker = new StateChangeNotifierForTest(dag); doReturn(dag).when(appContext).getCurrentDAG(); - DAGImpl.OutputKey outputKey = Mockito.mock(DAGImpl.OutputKey.class); - ListenableFuture future = Mockito.mock(ListenableFuture.class); + DAGImpl.OutputKey outputKey = mock(DAGImpl.OutputKey.class); + ListenableFuture future = mock(ListenableFuture.class); dag.commitFutures.put(outputKey, future); initDAG(dag); startDAG(dag); @@ -1975,7 +1976,7 @@ public void testDAGHang() throws Exception { DAGEventCommitCompleted dagEvent = new DAGEventCommitCompleted( dagId, outputKey, false , new RuntimeException("test")); doThrow(new RuntimeException("test")).when( - dag).logJobHistoryUnsuccesfulEvent(any(DAGState.class), any(TezCounters.class)); + dag).logJobHistoryUnsuccesfulEvent(any(), any()); dag.handle(dagEvent); dispatcher.await(); Assert.assertTrue("DAG did not terminate!", dag.getInternalState() == DAGState.FAILED); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java index acbef67dc0..159dd9fb7f 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java @@ -17,11 +17,6 @@ */ package org.apache.tez.dag.app.dag.impl; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -132,13 +127,17 @@ import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; -import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import static org.junit.Assert.*; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; @@ -326,7 +325,7 @@ public void setup() { when(appContext.getApplicationID()).thenReturn(appAttemptId.getApplicationId()); when(appContext.getClock()).thenReturn(new SystemClock()); - Mockito.doAnswer(new Answer() { + doAnswer(new Answer() { public ListenableFuture answer(InvocationOnMock invocation) { Object[] args = invocation.getArguments(); CallableEvent e = (CallableEvent) args[0]; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGSchedulerNaturalOrderControlled.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGSchedulerNaturalOrderControlled.java index 3b1db4c6d1..f79f359a7b 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGSchedulerNaturalOrderControlled.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGSchedulerNaturalOrderControlled.java @@ -19,7 +19,7 @@ package org.apache.tez.dag.app.dag.impl; import static org.junit.Assert.assertEquals; -import static org.mockito.Matchers.any; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; @@ -65,35 +65,35 @@ public void testSimpleFlow() { for (int i = 0; i < vertices[0].getTotalTasks(); i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[0].getVertexId(), i, 0)); } - verify(eventHandler, times(vertices[0].getTotalTasks())).handle(any(Event.class)); + verify(eventHandler, times(vertices[0].getTotalTasks())).handle(any()); reset(eventHandler); // Schedule 3 tasks belonging to v2 for (int i = 0; i < 3; i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[2].getVertexId(), i, 0)); } - verify(eventHandler, times(3)).handle(any(Event.class)); + verify(eventHandler, times(3)).handle(any()); reset(eventHandler); // Schedule 3 tasks belonging to v3 for (int i = 0; i < 3; i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[3].getVertexId(), i, 0)); } - verify(eventHandler, times(3)).handle(any(Event.class)); + verify(eventHandler, times(3)).handle(any()); reset(eventHandler); // Schedule remaining tasks belonging to v2 for (int i = 3; i < vertices[2].getTotalTasks(); i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[2].getVertexId(), i, 0)); } - verify(eventHandler, times(vertices[2].getTotalTasks() - 3)).handle(any(Event.class)); + verify(eventHandler, times(vertices[2].getTotalTasks() - 3)).handle(any()); reset(eventHandler); // Schedule remaining tasks belonging to v3 for (int i = 3; i < vertices[3].getTotalTasks(); i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[3].getVertexId(), i, 0)); } - verify(eventHandler, times(vertices[3].getTotalTasks() - 3)).handle(any(Event.class)); + verify(eventHandler, times(vertices[3].getTotalTasks() - 3)).handle(any()); reset(eventHandler); @@ -101,7 +101,7 @@ public void testSimpleFlow() { for (int i = 0; i < vertices[4].getTotalTasks(); i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[4].getVertexId(), i, 0)); } - verify(eventHandler, times(vertices[4].getTotalTasks())).handle(any(Event.class)); + verify(eventHandler, times(vertices[4].getTotalTasks())).handle(any()); reset(eventHandler); } @@ -124,7 +124,7 @@ public void testSourceRequestDelayed() { for (int i = 0; i < vertices[0].getTotalTasks(); i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[0].getVertexId(), i, 0)); } - verify(eventHandler, times(vertices[0].getTotalTasks())).handle(any(Event.class)); + verify(eventHandler, times(vertices[0].getTotalTasks())).handle(any()); reset(eventHandler); // v2 behaving as if configured with slow-start. @@ -132,14 +132,14 @@ public void testSourceRequestDelayed() { for (int i = 0; i < vertices[3].getTotalTasks(); i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[3].getVertexId(), i, 0)); } - verify(eventHandler, times(vertices[3].getTotalTasks())).handle(any(Event.class)); + verify(eventHandler, times(vertices[3].getTotalTasks())).handle(any()); reset(eventHandler); // Scheduling all tasks belonging to v4. None should get scheduled. for (int i = 0; i < vertices[4].getTotalTasks(); i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[4].getVertexId(), i, 0)); } - verify(eventHandler, never()).handle(any(Event.class)); + verify(eventHandler, never()).handle(any()); reset(eventHandler); // v2 now starts scheduling ... @@ -147,7 +147,7 @@ public void testSourceRequestDelayed() { for (int i = 0; i < 3; i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[2].getVertexId(), i, 0)); } - verify(eventHandler, times(3)).handle(any(Event.class)); + verify(eventHandler, times(3)).handle(any()); reset(eventHandler); // Schedule remaining tasks belonging to v2 @@ -192,7 +192,7 @@ public void testParallelismUpdated() { for (int i = 0; i < vertices[0].getTotalTasks(); i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[0].getVertexId(), i, 0)); } - verify(eventHandler, times(vertices[0].getTotalTasks())).handle(any(Event.class)); + verify(eventHandler, times(vertices[0].getTotalTasks())).handle(any()); reset(eventHandler); assertEquals(10, vertices[2].getTotalTasks()); @@ -202,14 +202,14 @@ public void testParallelismUpdated() { for (int i = 0; i < vertices[3].getTotalTasks(); i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[3].getVertexId(), i, 0)); } - verify(eventHandler, times(vertices[3].getTotalTasks())).handle(any(Event.class)); + verify(eventHandler, times(vertices[3].getTotalTasks())).handle(any()); reset(eventHandler); // Schedule all tasks belonging to v4 for (int i = 0; i < vertices[4].getTotalTasks(); i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[4].getVertexId(), i, 0)); } - verify(eventHandler, never()).handle(any(Event.class)); + verify(eventHandler, never()).handle(any()); reset(eventHandler); // Reset the parallelism for v2. @@ -221,7 +221,7 @@ public void testParallelismUpdated() { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[2].getVertexId(), i, 0)); } verify(eventHandler, times(vertices[2].getTotalTasks() + vertices[4].getTotalTasks())) - .handle(any(Event.class)); + .handle(any()); reset(eventHandler); } @@ -243,7 +243,7 @@ public void testMultipleRequestsForSameTask() { for (int i = 0; i < vertices[0].getTotalTasks() - 1; i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[0].getVertexId(), i, 0)); } - verify(eventHandler, times(vertices[0].getTotalTasks() - 1)).handle(any(Event.class)); + verify(eventHandler, times(vertices[0].getTotalTasks() - 1)).handle(any()); reset(eventHandler); @@ -252,7 +252,7 @@ public void testMultipleRequestsForSameTask() { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[2].getVertexId(), i, 0)); } // Nothing should be scheduled - verify(eventHandler, never()).handle(any(Event.class)); + verify(eventHandler, never()).handle(any()); reset(eventHandler); // Schedule an extra attempt for all but 1 task belonging to v0 @@ -260,14 +260,14 @@ public void testMultipleRequestsForSameTask() { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[0].getVertexId(), i, 1)); } // Only v0 requests should have gone out - verify(eventHandler, times(vertices[0].getTotalTasks() - 1)).handle(any(Event.class)); + verify(eventHandler, times(vertices[0].getTotalTasks() - 1)).handle(any()); reset(eventHandler); // Schedule last task of v0, with attempt 1 dagScheduler.scheduleTaskEx( createScheduleRequest(vertices[0].getVertexId(), vertices[0].getTotalTasks() - 1, 1)); // One v0 request and all of v2 should have gone out - verify(eventHandler, times(1 + vertices[2].getTotalTasks())).handle(any(Event.class)); + verify(eventHandler, times(1 + vertices[2].getTotalTasks())).handle(any()); } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestEdge.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestEdge.java index c9c99f5630..c6578ffa31 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestEdge.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestEdge.java @@ -21,7 +21,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.reset; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestImmediateStartVertexManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestImmediateStartVertexManager.java index a17c7c5b45..67a9da52b1 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestImmediateStartVertexManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestImmediateStartVertexManager.java @@ -37,7 +37,7 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import static org.mockito.Matchers.anyList; +import static org.mockito.Mockito.anyList; import static org.mockito.Mockito.anySet; import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.doAnswer; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestRootInputVertexManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestRootInputVertexManager.java index 39a291ebbd..144d28d4a5 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestRootInputVertexManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestRootInputVertexManager.java @@ -23,11 +23,11 @@ import static org.apache.tez.dag.app.dag.impl.RootInputVertexManager.TEZ_ROOT_INPUT_VERTEX_MANAGER_MIN_SRC_FRACTION; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyListOf; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyList; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -161,7 +161,7 @@ public void testRootInputVertexManagerSlowStart() { VertexManagerPluginContext mockContext = mock(VertexManagerPluginContext.class); - when(mockContext.getVertexStatistics(any(String.class))) + when(mockContext.getVertexStatistics(any())) .thenReturn(mock(VertexStatistics.class)); when(mockContext.getInputVertexEdgeProperties()) .thenReturn(mockInputVertices); @@ -176,7 +176,7 @@ public void testRootInputVertexManagerSlowStart() { final List scheduledTasks = Lists.newLinkedList(); doAnswer(new ScheduledTasksAnswer(scheduledTasks)).when( - mockContext).scheduleTasks(anyListOf(VertexManagerPluginContext.ScheduleTaskRequest.class)); + mockContext).scheduleTasks(anyList()); // source vertices have 0 tasks. immediate start of all managed tasks when(mockContext.getVertexNumTasks(mockSrcVertexId1)).thenReturn(0); @@ -508,7 +508,7 @@ public void testTezDrainCompletionsOnVertexStart() throws IOException { VertexManagerPluginContext mockContext = mock(VertexManagerPluginContext.class); - when(mockContext.getVertexStatistics(any(String.class))) + when(mockContext.getVertexStatistics(any())) .thenReturn(mock(VertexStatistics.class)); when(mockContext.getInputVertexEdgeProperties()) .thenReturn(mockInputVertices); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java index dc59b4615c..d0088bfc54 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java @@ -22,7 +22,9 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.fail; -import static org.mockito.Matchers.anyInt; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -128,7 +130,6 @@ import org.junit.BeforeClass; import org.junit.Test; import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -181,11 +182,11 @@ private void createMockVertex(Configuration conf) { when(mockVertex.getServicePluginInfo()).thenReturn(servicePluginInfo); when(mockVertex.getVertexConfig()).thenReturn(new VertexImpl.VertexConfigImpl(conf)); AppContext appContext = mock(AppContext.class); - when(appContext.getTaskScheduerIdentifier(Mockito.anyString())).thenReturn(0); + when(appContext.getTaskScheduerIdentifier(anyString())).thenReturn(0); when(mockVertex.getAppContext()).thenReturn(appContext); AMNodeTracker nodeTracker = mock(AMNodeTracker.class); - when(nodeTracker.getNumNodes(Mockito.anyInt())).thenReturn(10); - when(nodeTracker.getNumActiveNodes(Mockito.anyInt())).thenReturn(8); + when(nodeTracker.getNumNodes(anyInt())).thenReturn(10); + when(nodeTracker.getNumActiveNodes(anyInt())).thenReturn(8); when(appContext.getNodeTracker()).thenReturn(nodeTracker); } @@ -1922,7 +1923,7 @@ taListener, taskConf, new SystemClock(), assertEquals(true, taImpl.inputFailedReported); int expectedEventsAfterFetchFailure = expectedEventsTillSucceeded + 2; - arg.getAllValues().clear(); + arg = ArgumentCaptor.forClass(Event.class); verify(eventHandler, times(expectedEventsAfterFetchFailure)).handle(arg.capture()); Event e = verifyEventType( arg.getAllValues().subList(expectedEventsTillSucceeded, @@ -2195,10 +2196,9 @@ private void testMapTaskFailingForFetchFailureType(boolean isLocalFetch, when(destTaskAttemptId.getTaskID()).thenReturn(mock(TezTaskID.class)); when(destTaskAttemptId.getVertexID()).thenReturn(mock(TezVertexID.class)); when(appCtx.getCurrentDAG()).thenReturn(mock(DAG.class)); - when(appCtx.getCurrentDAG().getVertex(Mockito.any(TezVertexID.class))) + when(appCtx.getCurrentDAG().getVertex(any(TezVertexID.class))) .thenReturn(mock(Vertex.class)); - when(appCtx.getCurrentDAG().getVertex(Mockito.any(TezVertexID.class)).getRunningTasks()) - .thenReturn(100); + when(mock(Vertex.class).getRunningTasks()).thenReturn(100); EventMetaData mockMeta = mock(EventMetaData.class); when(mockMeta.getTaskAttemptID()).thenReturn(destTaskAttemptId); @@ -2228,12 +2228,13 @@ public void testMapTaskIsBlamedByDownstreamAttemptsFromDifferentHosts() { InputReadErrorEvent inputReadErrorEvent2 = InputReadErrorEvent.create("", 1, 1, 1, false, false, "downstream_host_2"); - TezTaskAttemptID destTaskAttemptId = mock(TezTaskAttemptID.class); + TezTaskAttemptID destTaskAttemptId = mock(TezTaskAttemptID.class, RETURNS_DEEP_STUBS); when(destTaskAttemptId.getTaskID()).thenReturn(mock(TezTaskID.class)); when(destTaskAttemptId.getTaskID().getVertexID()).thenReturn(mock(TezVertexID.class)); when(appCtx.getCurrentDAG()).thenReturn(mock(DAG.class)); - when(appCtx.getCurrentDAG().getVertex(Mockito.any(TezVertexID.class))).thenReturn(mock(Vertex.class)); - when(appCtx.getCurrentDAG().getVertex(Mockito.any(TezVertexID.class)).getRunningTasks()).thenReturn(100); + when(appCtx.getCurrentDAG().getVertex(any(TezVertexID.class))) + .thenReturn(mock(Vertex.class)); + when(mock(Vertex.class).getRunningTasks()).thenReturn(100); EventMetaData mockMeta = mock(EventMetaData.class); when(mockMeta.getTaskAttemptID()).thenReturn(destTaskAttemptId); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java index f4e40f7333..3bf59977e5 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java @@ -769,7 +769,7 @@ public void testTaskSucceedAndRetroActiveFailure() { // The task should now have succeeded assertTaskSucceededState(); - verify(mockTask.stateChangeNotifier).taskSucceeded(any(String.class), eq(taskId), + verify(mockTask.stateChangeNotifier).taskSucceeded(any(), eq(taskId), eq(mockTask.getLastAttempt().getTaskAttemptID().getId())); ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(DAGHistoryEvent.class); @@ -818,7 +818,7 @@ public void testTaskSucceedAndRetroActiveKilled() { // The task should now have succeeded assertTaskSucceededState(); - verify(mockTask.stateChangeNotifier).taskSucceeded(any(String.class), eq(taskId), + verify(mockTask.stateChangeNotifier).taskSucceeded(any(), eq(taskId), eq(mockTask.getLastAttempt().getTaskAttemptID().getId())); eventHandler.events.clear(); @@ -982,7 +982,7 @@ public void testKilledAttemptUpdatesDAGScheduler() { // The task should now have succeeded and sent kill to other attempt assertTaskSucceededState(); - verify(mockTask.stateChangeNotifier).taskSucceeded(any(String.class), eq(taskId), + verify(mockTask.stateChangeNotifier).taskSucceeded(any(), eq(taskId), eq(firstAttempt.getTaskAttemptID().getId())); @SuppressWarnings("rawtypes") Event event = eventHandler.events.get(eventHandler.events.size()-1); @@ -1019,7 +1019,7 @@ public void testSpeculatedThenRetroactiveFailure() { // The task should now have succeeded and sent kill to other attempt assertTaskSucceededState(); - verify(mockTask.stateChangeNotifier).taskSucceeded(any(String.class), eq(taskId), + verify(mockTask.stateChangeNotifier).taskSucceeded(any(), eq(taskId), eq(firstAttempt.getTaskAttemptID().getId())); @SuppressWarnings("rawtypes") Event event = eventHandler.events.get(eventHandler.events.size()-1); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java index c118110948..10ecfe503c 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java @@ -32,9 +32,16 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.RETURNS_DEEP_STUBS; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doCallRealMethod; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.when; import java.io.ByteArrayOutputStream; import java.io.DataInput; @@ -85,6 +92,7 @@ import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads; import org.apache.tez.test.GraceShuffleVertexManagerForTest; +import org.mockito.Mockito; import org.roaringbitmap.RoaringBitmap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -230,7 +238,6 @@ import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Test; -import org.mockito.Mockito; import org.mockito.internal.util.collections.Sets; import com.google.common.base.Joiner; @@ -2739,7 +2746,7 @@ public void setupPreDagCreation() { dagId = TezDAGID.getInstance(appAttemptId.getApplicationId(), 1); taskSpecificLaunchCmdOption = mock(TaskSpecificLaunchCmdOption.class); doReturn(false).when(taskSpecificLaunchCmdOption).addTaskSpecificLaunchCmdOption( - any(String.class), + any(), anyInt()); } @@ -2816,7 +2823,7 @@ public void setupPostDagCreation(boolean cleanupShuffleDataAtVertexLevel) throws execService = mock(ListeningExecutorService.class); final ListenableFuture mockFuture = mock(ListenableFuture.class); - Mockito.doAnswer(new Answer() { + doAnswer(new Answer() { public ListenableFuture answer(InvocationOnMock invocation) { Object[] args = invocation.getArguments(); CallableEvent e = (CallableEvent) args[0]; @@ -3133,7 +3140,7 @@ public void testVertexConfigureEventWithReconfigure() throws Exception { Assert.assertEquals("vertex2", listener.events.get(0).getVertexName()); Assert.assertEquals(org.apache.tez.dag.api.event.VertexState.CONFIGURED, listener.events.get(0).getVertexState()); - updateTracker.unregisterForVertexUpdates("vertex2", listener); + updateTracker.unregisterForVertexUpdates("vertex2", listener); } @Test (timeout=5000) @@ -7512,8 +7519,7 @@ public void testVertexShuffleDelete() throws Exception { checkSpannedVertices(); runVertices(); Mockito.verify(appContext.getAppMaster().getContainerLauncherManager(), - times(3)).vertexComplete(any(TezVertexID.class), - any(JobTokenSecretManager.class), any(Set.class)); + times(3)).vertexComplete(any(), any(), any()); } private void checkSpannedVertices() { diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java index f7a60377c4..058dc76164 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java @@ -24,8 +24,9 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; import static org.mockito.Mockito.RETURNS_DEEP_STUBS; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; @@ -70,7 +71,6 @@ import org.junit.Before; import org.junit.Test; import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -92,7 +92,7 @@ public void setup() { mockAppContext = mock(AppContext.class, RETURNS_DEEP_STUBS); execService = mock(ListeningExecutorService.class); final ListenableFuture mockFuture = mock(ListenableFuture.class); - Mockito.doAnswer(new Answer() { + doAnswer(new Answer() { public ListenableFuture answer(InvocationOnMock invocation) { Object[] args = invocation.getArguments(); CallableEvent e = (CallableEvent) args[0]; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherManager.java index b3568eb20f..0bd41f4a60 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherManager.java @@ -19,7 +19,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; @@ -340,7 +340,7 @@ public void testContainerLauncherUserError() throws ServicePluginException { // launch container doThrow(new RuntimeException("testexception")).when(containerLauncher) - .launchContainer(any(ContainerLaunchRequest.class)); + .launchContainer(any()); ContainerLaunchContext clc1 = mock(ContainerLaunchContext.class); Container container1 = mock(Container.class); ContainerLauncherLaunchRequestEvent launchRequestEvent = @@ -365,7 +365,7 @@ public void testContainerLauncherUserError() throws ServicePluginException { // stop container doThrow(new RuntimeException("teststopexception")).when(containerLauncher) - .stopContainer(any(ContainerStopRequest.class)); + .stopContainer(any()); ContainerId containerId2 = mock(ContainerId.class); NodeId nodeId2 = mock(NodeId.class); ContainerLauncherStopRequestEvent stopRequestEvent = diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java index ef61cad94e..fe5a888b56 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java @@ -21,10 +21,10 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertEquals; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -203,9 +203,9 @@ public void testDelayedReuseContainerBecomesAvailable() TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); verify(taskSchedulerManager).taskAllocated( - eq(0), eq(ta11), any(Object.class), eq(containerHost1)); + eq(0), eq(ta11), any(), eq(containerHost1)); verify(taskSchedulerManager).taskAllocated( - eq(0), eq(ta21), any(Object.class), eq(containerHost2)); + eq(0), eq(ta21), any(), eq(containerHost2)); // Adding the event later so that task1 assigned to containerHost1 // is deterministic. @@ -217,9 +217,9 @@ public void testDelayedReuseContainerBecomesAvailable() drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta11, true, null, null); verify(taskSchedulerManager, times(1)).taskAllocated( - eq(0), eq(ta31), any(Object.class), eq(containerHost1)); + eq(0), eq(ta31), any(), eq(containerHost1)); verify(rmClient, times(0)).releaseAssignedContainer( - eq(containerHost1.getId())); + eq(containerHost1.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); eventHandler.reset(); @@ -325,8 +325,8 @@ public void testDelayedReuseContainerNotAvailable() taskScheduler.onContainersAllocated(Lists.newArrayList(containerHost1, containerHost2)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta11), any(Object.class), eq(containerHost1)); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta21), any(Object.class), + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta11), any(), eq(containerHost1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta21), any(), eq(containerHost2)); // Adding the event later so that task1 assigned to containerHost1 is deterministic. @@ -338,7 +338,7 @@ public void testDelayedReuseContainerNotAvailable() drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta21, true, null, null); verify(taskSchedulerManager, times(0)).taskAllocated( - eq(0), eq(ta31), any(Object.class), eq(containerHost2)); + eq(0), eq(ta31), any(), eq(containerHost2)); verify(rmClient, times(1)).releaseAssignedContainer( eq(containerHost2.getId())); eventHandler.verifyInvocation(AMContainerEventStopRequest.class); @@ -426,7 +426,7 @@ public void testSimpleReuse() throws IOException, InterruptedException, Executio taskScheduler.onContainersAllocated(Collections.singletonList(container1)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta11), any(Object.class), + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta11), any(), eq(container1)); // Task assigned to container completed successfully. Container should be re-used. @@ -435,7 +435,7 @@ public void testSimpleReuse() throws IOException, InterruptedException, Executio null, 0)); drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta11, true, null, null); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta12), any(Object.class), eq(container1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta12), any(), eq(container1)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); eventHandler.reset(); @@ -447,7 +447,7 @@ public void testSimpleReuse() throws IOException, InterruptedException, Executio null, 0)); drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta12, true, null, null); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta13), any(Object.class), + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta13), any(), eq(container1)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); @@ -458,7 +458,7 @@ public void testSimpleReuse() throws IOException, InterruptedException, Executio new AMSchedulerEventTAEnded(ta13, container1.getId(), TaskAttemptState.FAILED, null, "TIMEOUT", 0)); drainableAppCallback.drain(); - verify(taskSchedulerManager, times(0)).taskAllocated(eq(0), eq(ta14), any(Object.class), + verify(taskSchedulerManager, times(0)).taskAllocated(eq(0), eq(ta14), any(), eq(container1)); verifyDeAllocateTask(taskScheduler, ta13, false, null, "TIMEOUT"); verify(rmClient).releaseAssignedContainer(eq(container1.getId())); @@ -471,7 +471,7 @@ public void testSimpleReuse() throws IOException, InterruptedException, Executio taskScheduler.onContainersAllocated(Collections.singletonList(container2)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta14), any(Object.class), + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta14), any(), eq(container2)); // Task assigned to container completed successfully. No pending requests. Container should be released. @@ -571,7 +571,7 @@ public void testReuseWithTaskSpecificLaunchCmdOption() throws IOException, Inter taskScheduler.onContainersAllocated(Collections.singletonList(container1)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta11), any(Object.class), + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta11), any(), eq(container1)); // First task had profiling on. This container can not be reused further. @@ -580,7 +580,7 @@ public void testReuseWithTaskSpecificLaunchCmdOption() throws IOException, Inter null, 0)); drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta11, true, null, null); - verify(taskSchedulerManager, times(0)).taskAllocated(eq(0), eq(ta12), any(Object.class), + verify(taskSchedulerManager, times(0)).taskAllocated(eq(0), eq(ta12), any(), eq(container1)); verify(rmClient, times(1)).releaseAssignedContainer(eq(container1.getId())); eventHandler.verifyInvocation(AMContainerEventStopRequest.class); @@ -614,7 +614,7 @@ public void testReuseWithTaskSpecificLaunchCmdOption() throws IOException, Inter taskScheduler.onContainersAllocated(Collections.singletonList(container2)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta13), any(Object.class), eq(container2)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta13), any(), eq(container2)); // Verify that the container can not be reused when profiling option is turned on // Even for 2 tasks having same profiling option can have container reusability. @@ -623,7 +623,7 @@ public void testReuseWithTaskSpecificLaunchCmdOption() throws IOException, Inter null, 0)); drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta13, true, null, null); - verify(taskSchedulerManager, times(0)).taskAllocated(eq(0), eq(ta14), any(Object.class), + verify(taskSchedulerManager, times(0)).taskAllocated(eq(0), eq(ta14), any(), eq(container2)); verify(rmClient, times(1)).releaseAssignedContainer(eq(container2.getId())); eventHandler.verifyInvocation(AMContainerEventStopRequest.class); @@ -661,7 +661,7 @@ public void testReuseWithTaskSpecificLaunchCmdOption() throws IOException, Inter taskScheduler.onContainersAllocated(Collections.singletonList(container3)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta15), any(Object.class), + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta15), any(), eq(container3)); //Ensure task 6 (of vertex 1) is allocated to same container @@ -670,7 +670,7 @@ public void testReuseWithTaskSpecificLaunchCmdOption() throws IOException, Inter null, 0)); drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta15, true, null, null); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta16), any(Object.class), eq(container3)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta16), any(), eq(container3)); eventHandler.reset(); taskScheduler.shutdown(); @@ -759,7 +759,7 @@ public void testReuseNonLocalRequest() TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); verify(taskSchedulerManager).taskAllocated( - eq(0), eq(ta11), any(Object.class), eq(container1)); + eq(0), eq(ta11), any(), eq(container1)); // Send launch request for task2 (vertex2) taskSchedulerManager.handleEvent(lrEvent12); @@ -773,7 +773,7 @@ public void testReuseNonLocalRequest() drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta11, true, null, null); verify(taskSchedulerManager, times(0)).taskAllocated( - eq(0), eq(ta12), any(Object.class), eq(container1)); + eq(0), eq(ta12), any(), eq(container1)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); eventHandler.reset(); @@ -781,7 +781,7 @@ public void testReuseNonLocalRequest() TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); verify(taskSchedulerManager).taskAllocated( - eq(0), eq(ta12), any(Object.class), eq(container1)); + eq(0), eq(ta12), any(), eq(container1)); // TA12 completed. taskSchedulerManager.handleEvent( @@ -887,7 +887,7 @@ public void testReuseAcrossVertices() TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); verify(taskSchedulerManager).taskAllocated( - eq(0), eq(ta11), any(Object.class), eq(container1)); + eq(0), eq(ta11), any(), eq(container1)); // Send launch request for task2 (vertex2) taskSchedulerManager.handleEvent(lrEvent21); @@ -900,7 +900,7 @@ public void testReuseAcrossVertices() drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta11, true, null, null); verify(taskSchedulerManager).taskAllocated( - eq(0), eq(ta21), any(Object.class), eq(container1)); + eq(0), eq(ta21), any(), eq(container1)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId())); // Task 2 completes. @@ -999,7 +999,7 @@ public void testReuseLocalResourcesChanged() throws IOException, InterruptedExce taskScheduler.onContainersAllocated(Collections.singletonList(container1)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta111), any(Object.class), eq(container1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta111), any(), eq(container1)); assignEvent = (AMContainerEventAssignTA) eventHandler.verifyInvocation(AMContainerEventAssignTA.class); assertEquals(1, assignEvent.getRemoteTaskLocalResources().size()); @@ -1009,7 +1009,7 @@ public void testReuseLocalResourcesChanged() throws IOException, InterruptedExce null, 0)); drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta111, true, null, null); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta112), any(Object.class), eq(container1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta112), any(), eq(container1)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); assignEvent = (AMContainerEventAssignTA) eventHandler.verifyInvocation(AMContainerEventAssignTA.class); @@ -1052,7 +1052,7 @@ public void testReuseLocalResourcesChanged() throws IOException, InterruptedExce TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta211), any(Object.class), + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta211), any(), eq(container1)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); @@ -1065,7 +1065,7 @@ public void testReuseLocalResourcesChanged() throws IOException, InterruptedExce null, 0)); drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta211, true, null, null); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta212), any(Object.class), eq(container1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta212), any(), eq(container1)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); assignEvent = (AMContainerEventAssignTA) eventHandler.verifyInvocation(AMContainerEventAssignTA.class); @@ -1186,7 +1186,7 @@ public void testReuseConflictLocalResources() throws IOException, InterruptedExc taskScheduler.onContainersAllocated(Collections.singletonList(container1)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta111), any(Object.class), eq(container1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta111), any(), eq(container1)); assignEvent = (AMContainerEventAssignTA) eventHandler.verifyInvocation(AMContainerEventAssignTA.class); assertEquals(1, assignEvent.getRemoteTaskLocalResources().size()); @@ -1196,7 +1196,7 @@ public void testReuseConflictLocalResources() throws IOException, InterruptedExc null, 0)); drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta111, true, null, null); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta112), any(Object.class), eq(container1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta112), any(), eq(container1)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); assignEvent = (AMContainerEventAssignTA) eventHandler.verifyInvocation(AMContainerEventAssignTA.class); @@ -1217,7 +1217,7 @@ public void testReuseConflictLocalResources() throws IOException, InterruptedExc taskSchedulerManager.handleEvent(lrEvent13); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta113), any(Object.class), eq(container1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta113), any(), eq(container1)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); eventHandler.reset(); @@ -1234,7 +1234,7 @@ public void testReuseConflictLocalResources() throws IOException, InterruptedExc taskSchedulerManager.handleEvent(lrEvent14); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta114), any(Object.class), eq(container1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta114), any(), eq(container1)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); eventHandler.reset(); @@ -1276,7 +1276,7 @@ public void testReuseConflictLocalResources() throws IOException, InterruptedExc taskScheduler.onContainersAllocated(Collections.singletonList(container2)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta211), any(Object.class), eq(container2)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta211), any(), eq(container2)); eventHandler.reset(); taskSchedulerManager.handleEvent(new AMSchedulerEventTAEnded(ta211, container2.getId(), @@ -1305,7 +1305,7 @@ public void testReuseConflictLocalResources() throws IOException, InterruptedExc taskSchedulerManager.handleEvent(lrEvent31); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta311), any(Object.class), eq(container2)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta311), any(), eq(container2)); eventHandler.reset(); taskScheduler.shutdown(); @@ -1376,7 +1376,7 @@ public void testAssignmentOnShutdown() taskScheduler.onContainersAllocated(Collections.singletonList(container1)); drainableAppCallback.drain(); verify(taskSchedulerManager, times(0)).taskAllocated(eq(0), eq(ta11), - any(Object.class), eq(container1)); + any(), eq(container1)); taskScheduler.shutdown(); taskSchedulerManager.close(); } @@ -1467,20 +1467,20 @@ eventHandler, rmClient, new AlwaysMatchesContainerMatcher(), taskScheduler.onContainersAllocated(Collections.singletonList(container1)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta11), any(Object.class), eq(container1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta11), any(), eq(container1)); // Second container allocated, should start ta13 taskScheduler.onContainersAllocated(Collections.singletonList(container2)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta13), any(Object.class), eq(container2)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta13), any(), eq(container2)); // ta11 finished, should start ta12 taskSchedulerManager.handleEvent(new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED, null, null, 0)); drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta11, true, null, null); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta12), any(Object.class), eq(container1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta12), any(), eq(container1)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); eventHandler.reset(); @@ -1490,7 +1490,7 @@ eventHandler, rmClient, new AlwaysMatchesContainerMatcher(), TaskAttemptState.SUCCEEDED, null, null, 0)); drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta13, true, null, null); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta14), any(Object.class), eq(container2)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta14), any(), eq(container2)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container2.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); eventHandler.reset(); @@ -1609,14 +1609,14 @@ eventHandler, rmClient, new ContainerContextMatcher(), taskScheduler.onContainersAllocated(Collections.singletonList(container1)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta11), any(Object.class), eq(container1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta11), any(), eq(container1)); // finish ta11, should start ta13 taskSchedulerManager.handleEvent(new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED, null, null, 0)); drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta11, true, null, null); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta13), any(Object.class), eq(container1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta13), any(), eq(container1)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); eventHandler.reset(); @@ -1634,7 +1634,7 @@ eventHandler, rmClient, new ContainerContextMatcher(), taskScheduler.onContainersAllocated(Collections.singletonList(container2)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta12), any(Object.class), eq(container2)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta12), any(), eq(container2)); // ta12 finished, cannot reuse container, should release container2 taskSchedulerManager.handleEvent(new AMSchedulerEventTAEnded(ta12, container2.getId(), @@ -1649,7 +1649,7 @@ eventHandler, rmClient, new ContainerContextMatcher(), taskScheduler.onContainersAllocated(Collections.singletonList(container3)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta14), any(Object.class), eq(container3)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta14), any(), eq(container3)); // ta14 finished, should release container3 taskSchedulerManager.handleEvent(new AMSchedulerEventTAEnded(ta14, container3.getId(), diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java index 48dd93825b..1465bfaaf6 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java @@ -77,11 +77,11 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyListOf; -import static org.mockito.Matchers.eq; -import static org.mockito.Matchers.isNull; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyList; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.isNull; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; @@ -170,23 +170,23 @@ public void testNoReuse() throws Exception { racks, mockPriority, null, mockCookie1); drainableAppCallback.drain(); verify(mockRMClient, times(1)). - addContainerRequest(any(TaskRequest.class)); + addContainerRequest(any()); // returned from task requests before allocation happens assertFalse(scheduler.deallocateTask(mockTask1, true, null, null)); - verify(mockApp, times(0)).containerBeingReleased(any(ContainerId.class)); + verify(mockApp, times(0)).containerBeingReleased(any()); verify(mockRMClient, times(1)). - removeContainerRequest(any(TaskRequest.class)); + removeContainerRequest(any()); verify(mockRMClient, times(0)). - releaseAssignedContainer((ContainerId) any()); + releaseAssignedContainer(any()); // deallocating unknown task assertFalse(scheduler.deallocateTask(mockTask1, true, null, null)); - verify(mockApp, times(0)).containerBeingReleased(any(ContainerId.class)); + verify(mockApp, times(0)).containerBeingReleased(any()); verify(mockRMClient, times(1)). - removeContainerRequest(any(TaskRequest.class)); + removeContainerRequest(any()); verify(mockRMClient, times(0)). - releaseAssignedContainer((ContainerId) any()); + releaseAssignedContainer(any()); // allocate tasks Object mockTask2 = new MockTask("task2"); @@ -237,7 +237,7 @@ public void testNoReuse() throws Exception { verify(mockApp).taskAllocated(mockTask2, mockCookie2, mockContainer2); verify(mockApp).taskAllocated(mockTask3, mockCookie3, mockContainer3); // no other allocations returned - verify(mockApp, times(3)).taskAllocated(any(), any(), (Container) any()); + verify(mockApp, times(3)).taskAllocated(any(), any(), any()); verify(mockRMClient).removeContainerRequest(request1); verify(mockRMClient).removeContainerRequest(request2); verify(mockRMClient).removeContainerRequest(request3); @@ -253,7 +253,7 @@ public void testNoReuse() throws Exception { assertEquals(mockTask2, scheduler.deallocateContainer(mockCId2)); drainableAppCallback.drain(); verify(mockRMClient).releaseAssignedContainer(mockCId2); - verify(mockRMClient, times(3)).releaseAssignedContainer((ContainerId) any()); + verify(mockRMClient, times(3)).releaseAssignedContainer(any()); List statuses = new ArrayList<>(); ContainerStatus mockStatus1 = mock(ContainerStatus.class); @@ -277,16 +277,16 @@ public void testNoReuse() throws Exception { // currently allocated container status returned and not released verify(mockApp).containerCompleted(mockTask3, mockStatus3); // no other statuses returned - verify(mockApp, times(3)).containerCompleted(any(), (ContainerStatus) any()); - verify(mockRMClient, times(3)).releaseAssignedContainer((ContainerId) any()); + verify(mockApp, times(3)).containerCompleted(any(), any()); + verify(mockRMClient, times(3)).releaseAssignedContainer(any()); // verify blacklisting - verify(mockRMClient, times(0)).updateBlacklist(anyListOf(String.class), anyListOf(String.class)); + verify(mockRMClient, times(0)).updateBlacklist(anyList(), anyList()); String badHost = "host6"; NodeId badNodeId = NodeId.newInstance(badHost, 1); scheduler.blacklistNode(badNodeId); List badNodeList = Collections.singletonList(badHost); - verify(mockRMClient, times(1)).updateBlacklist(eq(badNodeList), isNull(List.class)); + verify(mockRMClient, times(1)).updateBlacklist(eq(badNodeList), isNull()); Object mockTask4 = new MockTask("task4"); Object mockCookie4 = new Object(); scheduler.allocateTask(mockTask4, mockCapability, null, @@ -300,10 +300,10 @@ public void testNoReuse() throws Exception { scheduler.onContainersAllocated(containers); drainableAppCallback.drain(); // no new allocation - verify(mockApp, times(3)).taskAllocated(any(), any(), (Container) any()); + verify(mockApp, times(3)).taskAllocated(any(), any(), any()); // verify blacklisted container released verify(mockRMClient).releaseAssignedContainer(mockCId5); - verify(mockRMClient, times(4)).releaseAssignedContainer((ContainerId) any()); + verify(mockRMClient, times(4)).releaseAssignedContainer(any()); // verify request added back verify(mockRMClient, times(6)).addContainerRequest(requestCaptor.capture()); NodeId host6 = NodeId.newInstance("host6", 6); @@ -314,17 +314,17 @@ public void testNoReuse() throws Exception { scheduler.onContainersAllocated(containers); drainableAppCallback.drain(); // new allocation - verify(mockApp, times(4)).taskAllocated(any(), any(), (Container) any()); + verify(mockApp, times(4)).taskAllocated(any(), any(), any()); verify(mockApp).taskAllocated(mockTask4, mockCookie4, mockContainer6); // deallocate allocated task assertTrue(scheduler.deallocateTask(mockTask4, true, null, null)); drainableAppCallback.drain(); verify(mockApp).containerBeingReleased(mockCId6); verify(mockRMClient).releaseAssignedContainer(mockCId6); - verify(mockRMClient, times(5)).releaseAssignedContainer((ContainerId) any()); + verify(mockRMClient, times(5)).releaseAssignedContainer(any()); // test unblacklist scheduler.unblacklistNode(badNodeId); - verify(mockRMClient, times(1)).updateBlacklist(isNull(List.class), eq(badNodeList)); + verify(mockRMClient, times(1)).updateBlacklist(isNull(), eq(badNodeList)); assertEquals(0, scheduler.getNumBlacklistedNodes()); float progress = 0.5f; @@ -335,16 +335,16 @@ public void testNoReuse() throws Exception { scheduler.allocateTask(mockTask1, mockCapability, hosts, racks, mockPriority, null, mockCookie1); drainableAppCallback.drain(); - verify(mockRMClient, times(7)).addContainerRequest(any(TaskRequest.class)); + verify(mockRMClient, times(7)).addContainerRequest(any()); verify(mockRMClient, times(6)). - removeContainerRequest(any(TaskRequest.class)); + removeContainerRequest(any()); scheduler.allocateTask(mockTask1, mockCapability, hosts, racks, mockPriority, null, mockCookie1); drainableAppCallback.drain(); // old request removed and new one added verify(mockRMClient, times(7)). - removeContainerRequest(any(TaskRequest.class)); - verify(mockRMClient, times(8)).addContainerRequest(any(TaskRequest.class)); + removeContainerRequest(any()); + verify(mockRMClient, times(8)).addContainerRequest(any()); assertFalse(scheduler.deallocateTask(mockTask1, true, null, null)); // test speculative node adjustment @@ -360,9 +360,9 @@ public void testNoReuse() throws Exception { mockPriority, null, mockCookie5); drainableAppCallback.drain(); // no new allocation - verify(mockApp, times(4)).taskAllocated(any(), any(), (Container) any()); + verify(mockApp, times(4)).taskAllocated(any(), any(), any()); // verify container released - verify(mockRMClient, times(5)).releaseAssignedContainer((ContainerId) any()); + verify(mockRMClient, times(5)).releaseAssignedContainer(any()); // verify request added back verify(mockRMClient, times(9)).addContainerRequest(requestCaptor.capture()); @@ -377,7 +377,7 @@ public void testNoReuse() throws Exception { drainableAppCallback.drain(); verify(mockApp) .reportError(eq(YarnTaskSchedulerServiceError.RESOURCEMANAGER_ERROR), argumentCaptor.capture(), - any(DagInfo.class)); + any()); assertTrue(argumentCaptor.getValue().contains("mockexception")); scheduler.onShutdownRequest(); @@ -468,8 +468,8 @@ public void testSimpleReuseLocalMatching() throws Exception { assertTrue(scheduler.deallocateTask(taskv0t0.task, true, null, null)); clock.incrementTime(10000); drainableAppCallback.drain(); - verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, never()).containerBeingReleased(any()); + verify(mockRMClient, never()).releaseAssignedContainer(any()); verify(mockApp).taskAllocated(taskv0t2.task, taskv0t2.cookie, container1); verify(mockRMClient).removeContainerRequest(reqv0t2); @@ -477,8 +477,8 @@ public void testSimpleReuseLocalMatching() throws Exception { assertTrue(scheduler.deallocateTask(taskv0t2.task, true, null, null)); clock.incrementTime(10000); drainableAppCallback.drain(); - verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, never()).containerBeingReleased(any()); + verify(mockRMClient, never()).releaseAssignedContainer(any()); verify(mockApp).taskAllocated(taskv1t0.task, taskv1t0.cookie, container1); verify(mockRMClient).removeContainerRequest(reqv1t0); @@ -575,8 +575,8 @@ public void testSimpleReuseRackMatching() throws Exception { assertTrue(scheduler.deallocateTask(taskv0t0.task, true, null, null)); clock.incrementTime(10000); drainableAppCallback.drain(); - verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, never()).containerBeingReleased(any()); + verify(mockRMClient, never()).releaseAssignedContainer(any()); verify(mockApp).taskAllocated(taskv1t0.task, taskv1t0.cookie, container1); verify(mockRMClient).removeContainerRequest(reqv1t0); @@ -584,8 +584,8 @@ public void testSimpleReuseRackMatching() throws Exception { assertTrue(scheduler.deallocateTask(taskv1t0.task, true, null, null)); clock.incrementTime(10000); drainableAppCallback.drain(); - verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, never()).containerBeingReleased(any()); + verify(mockRMClient, never()).releaseAssignedContainer(any()); verify(mockApp).taskAllocated(taskv0t1.task, taskv0t1.cookie, container1); verify(mockRMClient).removeContainerRequest(reqv0t1); @@ -682,8 +682,8 @@ public void testSimpleReuseAnyMatching() throws Exception { assertTrue(scheduler.deallocateTask(taskv0t0.task, true, null, null)); clock.incrementTime(10000); drainableAppCallback.drain(); - verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, never()).containerBeingReleased(any()); + verify(mockRMClient, never()).releaseAssignedContainer(any()); verify(mockApp).taskAllocated(taskv1t0.task, taskv1t0.cookie, container1); verify(mockRMClient).removeContainerRequest(reqv1t0); @@ -691,8 +691,8 @@ public void testSimpleReuseAnyMatching() throws Exception { assertTrue(scheduler.deallocateTask(taskv1t0.task, true, null, null)); clock.incrementTime(10000); drainableAppCallback.drain(); - verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, never()).containerBeingReleased(any()); + verify(mockRMClient, never()).releaseAssignedContainer(any()); verify(mockApp).taskAllocated(taskv0t1.task, taskv0t1.cookie, container1); verify(mockRMClient).removeContainerRequest(reqv0t1); @@ -700,8 +700,8 @@ public void testSimpleReuseAnyMatching() throws Exception { assertTrue(scheduler.deallocateTask(taskv0t1.task, true, null, null)); clock.incrementTime(10000); drainableAppCallback.drain(); - verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, never()).containerBeingReleased(any()); + verify(mockRMClient, never()).releaseAssignedContainer(any()); verify(mockApp).taskAllocated(taskv0t2.task, taskv0t2.cookie, container1); verify(mockRMClient).removeContainerRequest(reqv0t2); @@ -795,8 +795,8 @@ public void testReuseWithAffinity() throws Exception { assertTrue(scheduler.deallocateTask(taskv0t0.task, true, null, null)); clock.incrementTime(10000); drainableAppCallback.drain(); - verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, never()).containerBeingReleased(any()); + verify(mockRMClient, never()).releaseAssignedContainer(any()); verify(mockApp).taskAllocated(taskv1t0.task, taskv1t0.cookie, container1); verify(mockRMClient).removeContainerRequest(reqv1t0); @@ -895,8 +895,8 @@ public void testReuseVertexDescendants() throws Exception { assertTrue(scheduler.deallocateTask(taskv0t0.task, true, null, null)); clock.incrementTime(10000); drainableAppCallback.drain(); - verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, never()).containerBeingReleased(any()); + verify(mockRMClient, never()).releaseAssignedContainer(any()); verify(mockApp).taskAllocated(taskv2t0.task, taskv2t0.cookie, container1); verify(mockRMClient).removeContainerRequest(reqv2t0); @@ -905,8 +905,8 @@ public void testReuseVertexDescendants() throws Exception { assertTrue(scheduler.deallocateTask(taskv2t0.task, true, null, null)); clock.incrementTime(10000); drainableAppCallback.drain(); - verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, never()).containerBeingReleased(any()); + verify(mockRMClient, never()).releaseAssignedContainer(any()); verify(mockApp).taskAllocated(taskv0t1.task, taskv0t1.cookie, container1); verify(mockRMClient).removeContainerRequest(reqv0t1); @@ -914,8 +914,8 @@ public void testReuseVertexDescendants() throws Exception { assertTrue(scheduler.deallocateTask(taskv0t1.task, true, null, null)); clock.incrementTime(10000); drainableAppCallback.drain(); - verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, never()).containerBeingReleased(any()); + verify(mockRMClient, never()).releaseAssignedContainer(any()); verify(mockApp).taskAllocated(taskv1t0.task, taskv1t0.cookie, container1); verify(mockRMClient).removeContainerRequest(reqv1t0); @@ -1070,15 +1070,15 @@ public void testSessionContainers() throws Exception { assertTrue(scheduler.deallocateTask(taskv0t4.task, true, null, null)); assertTrue(scheduler.deallocateTask(taskv0t5.task, true, null, null)); assertTrue(scheduler.deallocateTask(taskv0t6.task, true, null, null)); - verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, never()).containerBeingReleased(any()); + verify(mockRMClient, never()).releaseAssignedContainer(any()); // verify only two of the containers were released after idle expiration // and the rest were spread across the nodes and racks clock.incrementTime(5000); drainableAppCallback.drain(); - verify(mockApp, times(2)).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, times(2)).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, times(2)).containerBeingReleased(any()); + verify(mockRMClient, times(2)).releaseAssignedContainer(any()); Set hosts = new HashSet<>(); Set racks = new HashSet<>(); for (HeldContainer hc : scheduler.getSessionContainers()) { @@ -1202,7 +1202,7 @@ public void testPreemptionNoHeadroom() throws Exception { scheduler.getProgress(); scheduler.getProgress(); drainableAppCallback.drain(); - verify(mockApp, times(1)).preemptContainer(any(ContainerId.class)); + verify(mockApp, times(1)).preemptContainer(any()); verify(mockApp).preemptContainer(cid2); assertEquals(taskv1t1.task, scheduler.deallocateContainer(cid2)); drainableAppCallback.drain(); @@ -1226,13 +1226,13 @@ public void testPreemptionNoHeadroom() throws Exception { scheduler.getProgress(); scheduler.getProgress(); drainableAppCallback.drain(); - verify(mockApp, times(1)).preemptContainer(any(ContainerId.class)); + verify(mockApp, times(1)).preemptContainer(any()); // adding request for v0 should trigger preemption on next heartbeat taskRequestCaptor.scheduleTask(taskv0t1); scheduler.getProgress(); drainableAppCallback.drain(); - verify(mockApp, times(2)).preemptContainer(any(ContainerId.class)); + verify(mockApp, times(2)).preemptContainer(any()); verify(mockApp).preemptContainer(cid1); assertEquals(taskv1t0.task, scheduler.deallocateContainer(cid1)); drainableAppCallback.drain(); @@ -1346,7 +1346,7 @@ public void testPreemptionWhenBlocked() throws Exception { scheduler.getProgress(); scheduler.getProgress(); drainableAppCallback.drain(); - verify(mockApp, times(1)).preemptContainer(any(ContainerId.class)); + verify(mockApp, times(1)).preemptContainer(any()); verify(mockApp).preemptContainer(cid1); String appMsg = "success"; AppFinalStatus finalStatus = diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java index 70e31f33e2..e193ee98f2 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java @@ -190,12 +190,12 @@ public void preemptDescendantsOnly() { Answer answer = new Answer() { @Override public Void answer(InvocationOnMock invocation) { - ContainerId containerId = invocation.getArgumentAt(0, ContainerId.class); + ContainerId containerId = invocation.getArgument(0, ContainerId.class); taskSchedulerService.deallocateContainer(containerId); return null; } }; - doAnswer(answer).when(mockContext).preemptContainer(any(ContainerId.class)); + doAnswer(answer).when(mockContext).preemptContainer(any()); taskSchedulerService.initialize(); taskSchedulerService.start(); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java index 965b8d76a4..ff34084d37 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java @@ -23,11 +23,12 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyString; -import static org.mockito.Matchers.eq; import static org.mockito.Mockito.RETURNS_DEEP_STUBS; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -70,7 +71,6 @@ import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.TaskSchedulerWithDrainableContext; import org.apache.tez.dag.app.rm.YarnTaskSchedulerService.CookieContainerRequest; import org.apache.tez.dag.app.rm.YarnTaskSchedulerService.HeldContainer; -import org.apache.tez.serviceplugins.api.DagInfo; import org.apache.tez.serviceplugins.api.TaskSchedulerContext; import org.apache.tez.serviceplugins.api.TaskSchedulerContext.AppFinalStatus; import org.junit.After; @@ -79,7 +79,6 @@ import org.junit.BeforeClass; import org.junit.Test; import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -174,7 +173,7 @@ public void testTaskSchedulerNoReuse() throws Exception { // returned from task requests before allocation happens assertFalse(scheduler.deallocateTask(mockTask1, true, null, null)); - verify(mockApp, times(0)).containerBeingReleased(any(ContainerId.class)); + verify(mockApp, times(0)).containerBeingReleased(any()); verify(mockRMClient, times(1)). removeContainerRequest((CookieContainerRequest) any()); verify(mockRMClient, times(0)). @@ -182,7 +181,7 @@ public void testTaskSchedulerNoReuse() throws Exception { // deallocating unknown task assertFalse(scheduler.deallocateTask(mockTask1, true, null, null)); - verify(mockApp, times(0)).containerBeingReleased(any(ContainerId.class)); + verify(mockApp, times(0)).containerBeingReleased(any()); verify(mockRMClient, times(1)). removeContainerRequest((CookieContainerRequest) any()); verify(mockRMClient, times(0)). @@ -378,7 +377,7 @@ public void testTaskSchedulerNoReuse() throws Exception { drainableAppCallback.drain(); verify(mockApp) .reportError(eq(YarnTaskSchedulerServiceError.RESOURCEMANAGER_ERROR), argumentCaptor.capture(), - any(DagInfo.class)); + any()); assertTrue(argumentCaptor.getValue().contains("mockexception")); scheduler.onShutdownRequest(); @@ -744,7 +743,7 @@ public void testTaskSchedulerWithReuse() throws Exception { scheduler.onError(mockException); drainableAppCallback.drain(); verify(mockApp).reportError(eq(YarnTaskSchedulerServiceError.RESOURCEMANAGER_ERROR), argumentCaptor.capture(), - any(DagInfo.class)); + any()); assertTrue(argumentCaptor.getValue().contains("mockexception")); scheduler.onShutdownRequest(); @@ -791,43 +790,43 @@ public void testTaskSchedulerDetermineMinHeldContainers() throws Exception { Resource r = Resource.newInstance(0, 0); ContainerId mockCId1 = ContainerId.newInstance(appId, 0); Container c1 = Container.newInstance(mockCId1, emptyHost, null, r, null, null); - HeldContainer hc1 = Mockito.spy(new HeldContainer(c1, 0, 0, null, containerSignatureMatcher)); + HeldContainer hc1 = spy(new HeldContainer(c1, 0, 0, null, containerSignatureMatcher)); when(hc1.getNode()).thenReturn(node1Rack1); when(hc1.getRack()).thenReturn(rack1); when(hc1.getContainer()).thenReturn(c1); ContainerId mockCId2 = ContainerId.newInstance(appId, 1); Container c2 = Container.newInstance(mockCId2, emptyHost, null, r, null, null); - HeldContainer hc2 = Mockito.spy(new HeldContainer(c2, 0, 0, null, containerSignatureMatcher)); + HeldContainer hc2 = spy(new HeldContainer(c2, 0, 0, null, containerSignatureMatcher)); when(hc2.getNode()).thenReturn(node2Rack1); when(hc2.getRack()).thenReturn(rack1); when(hc2.getContainer()).thenReturn(c2); ContainerId mockCId3 = ContainerId.newInstance(appId, 2); Container c3 = Container.newInstance(mockCId3, emptyHost, null, r, null, null); - HeldContainer hc3 = Mockito.spy(new HeldContainer(c3, 0, 0, null, containerSignatureMatcher)); + HeldContainer hc3 = spy(new HeldContainer(c3, 0, 0, null, containerSignatureMatcher)); when(hc3.getNode()).thenReturn(node1Rack1); when(hc3.getRack()).thenReturn(rack1); when(hc3.getContainer()).thenReturn(c3); ContainerId mockCId4 = ContainerId.newInstance(appId, 3); Container c4 = Container.newInstance(mockCId4, emptyHost, null, r, null, null); - HeldContainer hc4 = Mockito.spy(new HeldContainer(c4, 0, 0, null, containerSignatureMatcher)); + HeldContainer hc4 = spy(new HeldContainer(c4, 0, 0, null, containerSignatureMatcher)); when(hc4.getNode()).thenReturn(node2Rack1); when(hc4.getRack()).thenReturn(rack1); when(hc4.getContainer()).thenReturn(c4); ContainerId mockCId5 = ContainerId.newInstance(appId, 4); Container c5 = Container.newInstance(mockCId5, emptyHost, null, r, null, null); - HeldContainer hc5 = Mockito.spy(new HeldContainer(c5, 0, 0, null, containerSignatureMatcher)); + HeldContainer hc5 = spy(new HeldContainer(c5, 0, 0, null, containerSignatureMatcher)); when(hc5.getNode()).thenReturn(node1Rack2); when(hc5.getRack()).thenReturn(rack2); when(hc5.getContainer()).thenReturn(c5); ContainerId mockCId6 = ContainerId.newInstance(appId, 5); Container c6 = Container.newInstance(mockCId6, emptyHost, null, r, null, null); - HeldContainer hc6 = Mockito.spy(new HeldContainer(c6, 0, 0, null, containerSignatureMatcher)); + HeldContainer hc6 = spy(new HeldContainer(c6, 0, 0, null, containerSignatureMatcher)); when(hc6.getNode()).thenReturn(node2Rack2); when(hc6.getRack()).thenReturn(rack2); when(hc6.getContainer()).thenReturn(c6); ContainerId mockCId7 = ContainerId.newInstance(appId, 6); Container c7 = Container.newInstance(mockCId7, emptyHost, null, r, null, null); - HeldContainer hc7 = Mockito.spy(new HeldContainer(c7, 0, 0, null, containerSignatureMatcher)); + HeldContainer hc7 = spy(new HeldContainer(c7, 0, 0, null, containerSignatureMatcher)); when(hc7.getNode()).thenReturn(node1Rack3); when(hc7.getRack()).thenReturn(rack3); when(hc7.getContainer()).thenReturn(c7); @@ -1143,7 +1142,7 @@ public List> answer( }); - Mockito.doAnswer(new Answer() { + doAnswer(new Answer() { public Object answer(InvocationOnMock invocation) { Object[] args = invocation.getArguments(); ContainerId cId = (ContainerId) args[0]; @@ -1342,7 +1341,7 @@ public void testTaskSchedulerPreemption2() throws Exception { List containers = new ArrayList(); containers.add(mockContainer1); - Mockito.doAnswer(new Answer() { + doAnswer(new Answer() { public Object answer(InvocationOnMock invocation) { Object[] args = invocation.getArguments(); ContainerId cId = (ContainerId) args[0]; @@ -1603,7 +1602,7 @@ public void testContainerExpired() throws Exception { statuses.add(mockStatus2); scheduler.onContainersCompleted(statuses); - verify(mockApp, times(0)).taskAllocated(any(), any(), any(Container.class)); + verify(mockApp, times(0)).taskAllocated(any(), any(), any()); verify(mockRMClient, times(3)).addContainerRequest(requestCaptor.capture()); CookieContainerRequest resubmitRequest = requestCaptor.getValue(); assertEquals(request2.getCookie().getTask(), resubmitRequest.getCookie().getTask()); @@ -1616,7 +1615,7 @@ public void testContainerExpired() throws Exception { assertFalse(scheduler.deallocateTask(mockTask2, true, null, null)); scheduler.onContainersAllocated(containers); scheduler.onContainersCompleted(statuses); - verify(mockApp, times(0)).taskAllocated(any(), any(), any(Container.class)); + verify(mockApp, times(0)).taskAllocated(any(), any(), any()); verify(mockRMClient, times(3)).addContainerRequest(requestCaptor.capture()); } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java index 3ddd1861d4..095bd4e287 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java @@ -23,10 +23,10 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; import static org.mockito.Mockito.RETURNS_DEEP_STUBS; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; @@ -661,7 +661,7 @@ mockTaskAttempt1, mock(TaskLocationHint.class), 1, mock(ContainerContext.class), tseh.handle(launchRequest1); verify(tseh.getTestTaskScheduler(0)).allocateTask(eq(mockTaskAttempt1), eq(resource), - any(String[].class), any(String[].class), any(Priority.class), any(Object.class), + any(String[].class), any(String[].class), any(), any(), eq(launchRequest1)); AMSchedulerEventTALaunchRequest launchRequest2 = @@ -670,7 +670,7 @@ mockTaskAttempt2, mock(TaskLocationHint.class), 1, mock(ContainerContext.class), 0); tseh.handle(launchRequest2); verify(tseh.getTestTaskScheduler(1)).allocateTask(eq(mockTaskAttempt2), eq(resource), - any(String[].class), any(String[].class), any(Priority.class), any(Object.class), + any(String[].class), any(String[].class), any(), any(), eq(launchRequest2)); } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java index d3614d9ff2..3061ceaa88 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java @@ -23,10 +23,10 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.reset; @@ -1311,7 +1311,7 @@ protected void mockDAGID() { */ @SuppressWarnings("unchecked") public void verifyNoOutgoingEvents() { - verify(eventHandler, never()).handle(any(Event.class)); + verify(eventHandler, never()).handle(any()); } /** diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java b/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java index 94c8df6233..ced9ca0abe 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java @@ -18,11 +18,11 @@ package org.apache.tez.dag.app.web; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyString; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/recovery/TestRecoveryService.java b/tez-dag/src/test/java/org/apache/tez/dag/history/recovery/TestRecoveryService.java index 790e2d8059..5d1fff9121 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/history/recovery/TestRecoveryService.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/history/recovery/TestRecoveryService.java @@ -20,8 +20,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml index 8c2d84ee7c..609b896fd8 100644 --- a/tez-ext-service-tests/pom.xml +++ b/tez-ext-service-tests/pom.xml @@ -57,9 +57,13 @@ org.mockito - mockito-all + mockito-core test + + org.apache.tez + hadoop-shim + org.apache.tez tez-runtime-internals diff --git a/tez-mapreduce/pom.xml b/tez-mapreduce/pom.xml index ee7c318ff0..a22870fc38 100644 --- a/tez-mapreduce/pom.xml +++ b/tez-mapreduce/pom.xml @@ -39,6 +39,10 @@ test-jar test + + org.apache.tez + hadoop-shim + org.apache.tez tez-runtime-library @@ -91,7 +95,7 @@ org.mockito - mockito-all + mockito-core test diff --git a/tez-mapreduce/src/test/java/org/apache/hadoop/mapred/split/TestGroupedSplits.java b/tez-mapreduce/src/test/java/org/apache/hadoop/mapred/split/TestGroupedSplits.java index 3dce417459..9d700652a7 100644 --- a/tez-mapreduce/src/test/java/org/apache/hadoop/mapred/split/TestGroupedSplits.java +++ b/tez-mapreduce/src/test/java/org/apache/hadoop/mapred/split/TestGroupedSplits.java @@ -325,7 +325,7 @@ public void testGroupedSplitSize() throws IOException { for (int i=0; i org.mockito - mockito-all + mockito-core test diff --git a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java index b91e0eb203..40b16857ad 100644 --- a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java @@ -26,6 +26,9 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; import static org.junit.Assume.assumeTrue; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -97,7 +100,6 @@ import org.junit.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -1480,7 +1482,7 @@ public void testSendMapCount() throws Exception { final ChannelHandlerContext mockCtx = mock(ChannelHandlerContext.class); final Channel mockCh = mock(AbstractChannel.class); - final ChannelPipeline mockPipeline = Mockito.mock(ChannelPipeline.class); + final ChannelPipeline mockPipeline = mock(ChannelPipeline.class); // Mock HttpRequest and ChannelFuture final FullHttpRequest httpRequest = createHttpRequest(); @@ -1490,11 +1492,11 @@ public void testSendMapCount() throws Exception { new ShuffleHandler.TimeoutHandler(); // Mock Netty Channel Context and Channel behavior - Mockito.doReturn(mockCh).when(mockCtx).channel(); - Mockito.when(mockCh.pipeline()).thenReturn(mockPipeline); - Mockito.when(mockPipeline.get(Mockito.any(String.class))).thenReturn(timerHandler); + doReturn(mockCh).when(mockCtx).channel(); + when(mockCh.pipeline()).thenReturn(mockPipeline); + when(mockPipeline.get(any(String.class))).thenReturn(timerHandler); when(mockCtx.channel()).thenReturn(mockCh); - Mockito.doReturn(mockFuture).when(mockCh).writeAndFlush(Mockito.any(Object.class)); + doReturn(mockFuture).when(mockCh).writeAndFlush(any()); when(mockCh.writeAndFlush(Object.class)).thenReturn(mockFuture); final ShuffleHandler sh = new MockShuffleHandler(); @@ -1570,8 +1572,8 @@ public ChannelFuture createMockChannelFuture(Channel mockCh, final List listenerList) { final ChannelFuture mockFuture = mock(ChannelFuture.class); when(mockFuture.channel()).thenReturn(mockCh); - Mockito.doReturn(true).when(mockFuture).isSuccess(); - Mockito.doAnswer(new Answer() { + doReturn(true).when(mockFuture).isSuccess(); + doAnswer(new Answer() { @Override public Object answer(InvocationOnMock invocation) throws Throwable { //Add ReduceMapFileCount listener to a list @@ -1581,7 +1583,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { invocation.getArguments()[0]); return null; } - }).when(mockFuture).addListener(Mockito.any( + }).when(mockFuture).addListener(any( ShuffleHandler.ReduceMapFileCount.class)); return mockFuture; } diff --git a/tez-plugins/tez-history-parser/pom.xml b/tez-plugins/tez-history-parser/pom.xml index 5240bba1d8..f0758dcd15 100644 --- a/tez-plugins/tez-history-parser/pom.xml +++ b/tez-plugins/tez-history-parser/pom.xml @@ -132,7 +132,7 @@ org.mockito - mockito-all + mockito-core test diff --git a/tez-plugins/tez-protobuf-history-plugin/pom.xml b/tez-plugins/tez-protobuf-history-plugin/pom.xml index 9d82ed3965..607a31f95b 100644 --- a/tez-plugins/tez-protobuf-history-plugin/pom.xml +++ b/tez-plugins/tez-protobuf-history-plugin/pom.xml @@ -47,7 +47,7 @@ org.mockito - mockito-all + mockito-core test diff --git a/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml b/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml index 50ed0bd77d..0b61b7d958 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml @@ -126,7 +126,7 @@ org.mockito - mockito-all + mockito-core test diff --git a/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml b/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml index 4ba760f1aa..75df73f3a6 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml @@ -36,6 +36,10 @@ org.apache.tez tez-dag + + org.apache.tez + hadoop-shim + org.apache.tez tez-yarn-timeline-history @@ -130,7 +134,7 @@ org.mockito - mockito-all + mockito-core test @@ -155,4 +159,3 @@ - diff --git a/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSV15HistoryLoggingService.java b/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSV15HistoryLoggingService.java index 65ce91aab2..7ba14fffef 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSV15HistoryLoggingService.java +++ b/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSV15HistoryLoggingService.java @@ -24,10 +24,8 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.anyObject; -import static org.mockito.Matchers.anyString; -import static org.mockito.Matchers.anyVararg; import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; @@ -42,7 +40,6 @@ import java.util.Map; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.security.token.Token; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; @@ -53,7 +50,6 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse; import org.apache.hadoop.yarn.client.api.TimelineClient; import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier; import org.apache.tez.common.security.DAGAccessControls; import org.apache.tez.common.security.HistoryACLPolicyManager; import org.apache.tez.dag.api.TezConfiguration; @@ -71,7 +67,6 @@ import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.hadoop.shim.HadoopShim; import org.junit.Test; -import org.mockito.Matchers; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -448,17 +443,17 @@ private ATSV15HistoryLoggingService createService(int numDagsPerGroup) throws IO entityLog = new HashMap<>(); //timelineClient.init(conf); when(timelineClient.getDelegationToken(anyString())).thenReturn(null); - when(timelineClient.renewDelegationToken(Matchers.>any())).thenReturn(0L); - when(timelineClient.putEntities(Matchers.anyVararg())).thenAnswer(new Answer() { + when(timelineClient.renewDelegationToken(any())).thenReturn(0L); + when(timelineClient.putEntities(any())).thenAnswer(new Answer() { @Override public TimelinePutResponse answer(InvocationOnMock invocation) throws Throwable { return putEntityHelper(DEFAULT_GROUP_ID, invocation.getArguments(), 0); } }); - when(timelineClient.putEntities(any(ApplicationAttemptId.class), any(TimelineEntityGroupId.class), Matchers.anyVararg())).thenAnswer(new Answer() { + when(timelineClient.putEntities(any(), any(), any())).thenAnswer(new Answer() { @Override public TimelinePutResponse answer(InvocationOnMock invocation) throws Throwable { - return putEntityHelper(invocation.getArgumentAt(1, TimelineEntityGroupId.class), invocation.getArguments(), 2); + return putEntityHelper(invocation.getArgument(1, TimelineEntityGroupId.class), invocation.getArguments(), 2); } }); service.timelineClient = timelineClient; diff --git a/tez-plugins/tez-yarn-timeline-history/pom.xml b/tez-plugins/tez-yarn-timeline-history/pom.xml index 78b83dd877..3f17e97327 100644 --- a/tez-plugins/tez-yarn-timeline-history/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history/pom.xml @@ -120,7 +120,7 @@ org.mockito - mockito-all + mockito-core test diff --git a/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSHistoryLoggingService.java b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSHistoryLoggingService.java index 6603f4f668..0536568ab4 100644 --- a/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSHistoryLoggingService.java +++ b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSHistoryLoggingService.java @@ -18,6 +18,8 @@ package org.apache.tez.dag.history.logging.ats; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -25,11 +27,9 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity; import org.apache.hadoop.yarn.client.api.TimelineClient; import org.apache.hadoop.yarn.util.SystemClock; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.tez.common.security.DAGAccessControls; import org.apache.tez.common.security.HistoryACLPolicyManager; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.records.DAGProtos.DAGPlan; @@ -49,13 +49,9 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.mockito.Matchers; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; import static org.mockito.Mockito.any; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -102,7 +98,7 @@ public void setup() throws Exception { when(appContext.getCurrentDAGID()).thenReturn(null); when(appContext.getApplicationID()).thenReturn(appId); when(atsHistoryLoggingService.timelineClient.putEntities( - Matchers.anyVararg())).thenAnswer( + any())).thenAnswer( new Answer() { @Override public Object answer(InvocationOnMock invocation) throws Throwable { @@ -184,8 +180,7 @@ public void testTimelineServiceDisable() throws Exception { atsHistoryLoggingService1.setAppContext(appContext); atsHistoryLoggingService1.timelineClient = mock(TimelineClient.class); - when(atsHistoryLoggingService1.timelineClient.putEntities( - Matchers.anyVararg())).thenAnswer( + when(atsHistoryLoggingService1.timelineClient.putEntities(any())).thenAnswer( new Answer() { @Override public Object answer(InvocationOnMock invocation) throws Throwable { @@ -227,12 +222,11 @@ public Object answer(InvocationOnMock invocation) throws Throwable { @Test(timeout=10000) public void testNonSessionDomains() throws Exception { - when(historyACLPolicyManager.setupSessionACLs((Configuration)any(), (ApplicationId)any())) - .thenReturn( - Collections.singletonMap(TezConfiguration.YARN_ATS_ACL_SESSION_DOMAIN_ID, "session-id")); + when(historyACLPolicyManager.setupSessionACLs(any(), any())) + .thenReturn( + Collections.singletonMap(TezConfiguration.YARN_ATS_ACL_SESSION_DOMAIN_ID, "session-id")); atsHistoryLoggingService.start(); - verify(historyACLPolicyManager, times(1)).setupSessionACLs( - (Configuration)any(), (ApplicationId)any()); + verify(historyACLPolicyManager, times(1)).setupSessionACLs(any(), any()); // Send the event and wait for completion. TezDAGID dagId1 = TezDAGID.getInstance(appId, 0); @@ -245,7 +239,7 @@ public void testNonSessionDomains() throws Exception { } // No dag domain were created. verify(historyACLPolicyManager, times(0)) - .setupSessionDAGACLs((Configuration)any(), eq(appId), eq("0"), (DAGAccessControls)any()); + .setupSessionDAGACLs(any(), eq(appId), eq("0"), any()); // All calls made with session domain id. verify(historyACLPolicyManager, times(6)).updateTimelineEntityDomain(any(), eq("session-id")); @@ -253,11 +247,10 @@ public void testNonSessionDomains() throws Exception { @Test(timeout=10000) public void testNonSessionDomainsFailed() throws Exception { - when(historyACLPolicyManager.setupSessionACLs((Configuration)any(), (ApplicationId)any())) - .thenThrow(new IOException()); + when(historyACLPolicyManager.setupSessionACLs(any(), any())) + .thenThrow(new IOException()); atsHistoryLoggingService.start(); - verify(historyACLPolicyManager, times(1)).setupSessionACLs( - (Configuration)any(), (ApplicationId)any()); + verify(historyACLPolicyManager, times(1)).setupSessionACLs(any(), any()); // Send the event and wait for completion. TezDAGID dagId1 = TezDAGID.getInstance(appId, 0); @@ -269,7 +262,7 @@ public void testNonSessionDomainsFailed() throws Exception { } // No dag domain were created. verify(historyACLPolicyManager, times(0)) - .setupSessionDAGACLs((Configuration)any(), eq(appId), eq("0"), (DAGAccessControls)any()); + .setupSessionDAGACLs(any(), eq(appId), eq("0"), any()); // All calls made with session domain id. verify(historyACLPolicyManager, times(0)).updateTimelineEntityDomain(any(), eq("session-id")); @@ -278,11 +271,10 @@ public void testNonSessionDomainsFailed() throws Exception { @Test(timeout=10000) public void testNonSessionDomainsAclNull() throws Exception { - when(historyACLPolicyManager.setupSessionACLs((Configuration)any(), (ApplicationId)any())) - .thenReturn(null); + when(historyACLPolicyManager.setupSessionACLs(any(), any())) + .thenReturn(null); atsHistoryLoggingService.start(); - verify(historyACLPolicyManager, times(1)).setupSessionACLs( - (Configuration)any(), (ApplicationId)any()); + verify(historyACLPolicyManager, times(1)).setupSessionACLs(any(), any()); // Send the event and wait for completion. TezDAGID dagId1 = TezDAGID.getInstance(appId, 0); @@ -295,7 +287,7 @@ public void testNonSessionDomainsAclNull() throws Exception { } // No dag domain were created. verify(historyACLPolicyManager, times(0)) - .setupSessionDAGACLs((Configuration)any(), eq(appId), eq("0"), (DAGAccessControls)any()); + .setupSessionDAGACLs(any(), eq(appId), eq("0"), any()); // All calls made with session domain id. verify(historyACLPolicyManager, times(0)).updateTimelineEntityDomain(any(), eq("session-id")); @@ -304,19 +296,15 @@ public void testNonSessionDomainsAclNull() throws Exception { @Test(timeout=10000) public void testSessionDomains() throws Exception { - when(historyACLPolicyManager.setupSessionACLs((Configuration)any(), (ApplicationId)any())) - .thenReturn( - Collections.singletonMap(TezConfiguration.YARN_ATS_ACL_SESSION_DOMAIN_ID, "test-domain")); + when(historyACLPolicyManager.setupSessionACLs(any(), any())) + .thenReturn(Collections.singletonMap(TezConfiguration.YARN_ATS_ACL_SESSION_DOMAIN_ID, "test-domain")); - when(historyACLPolicyManager.setupSessionDAGACLs( - (Configuration)any(), (ApplicationId)any(), eq("0"), (DAGAccessControls)any())) - .thenReturn( - Collections.singletonMap(TezConfiguration.YARN_ATS_ACL_DAG_DOMAIN_ID, "dag-domain")); + when(historyACLPolicyManager.setupSessionDAGACLs(any(), any(), eq("0"), any())) + .thenReturn(Collections.singletonMap(TezConfiguration.YARN_ATS_ACL_DAG_DOMAIN_ID, "dag-domain")); when(appContext.isSession()).thenReturn(true); atsHistoryLoggingService.start(); - verify(historyACLPolicyManager, times(1)).setupSessionACLs((Configuration)any(), - (ApplicationId)any()); + verify(historyACLPolicyManager, times(1)).setupSessionACLs(any(), any()); // Send the event and wait for completion. TezDAGID dagId1 = TezDAGID.getInstance(appId, 0); @@ -329,7 +317,7 @@ public void testSessionDomains() throws Exception { } // No dag domain were created. verify(historyACLPolicyManager, times(1)) - .setupSessionDAGACLs((Configuration)any(), eq(appId), eq("0"), (DAGAccessControls)any()); + .setupSessionDAGACLs(any(), eq(appId), eq("0"), any()); // All calls made with session domain id. verify(historyACLPolicyManager, times(1)).updateTimelineEntityDomain(any(), eq("test-domain")); @@ -338,18 +326,15 @@ public void testSessionDomains() throws Exception { @Test(timeout=10000) public void testSessionDomainsFailed() throws Exception { - when(historyACLPolicyManager.setupSessionACLs((Configuration)any(), (ApplicationId)any())) - .thenThrow(new IOException()); + when(historyACLPolicyManager.setupSessionACLs(any(), any())) + .thenThrow(new IOException()); - when(historyACLPolicyManager.setupSessionDAGACLs( - (Configuration)any(), (ApplicationId)any(), eq("0"), (DAGAccessControls)any())) - .thenReturn( - Collections.singletonMap(TezConfiguration.YARN_ATS_ACL_DAG_DOMAIN_ID, "dag-domain")); + when(historyACLPolicyManager.setupSessionDAGACLs(any(), any(), eq("0"), any())) + .thenReturn(Collections.singletonMap(TezConfiguration.YARN_ATS_ACL_DAG_DOMAIN_ID, "dag-domain")); when(appContext.isSession()).thenReturn(true); atsHistoryLoggingService.start(); - verify(historyACLPolicyManager, times(1)).setupSessionACLs((Configuration)any(), - (ApplicationId)any()); + verify(historyACLPolicyManager, times(1)).setupSessionACLs(any(), any()); // Send the event and wait for completion. TezDAGID dagId1 = TezDAGID.getInstance(appId, 0); @@ -361,27 +346,24 @@ public void testSessionDomainsFailed() throws Exception { } // No dag domain were created. verify(historyACLPolicyManager, times(0)) - .setupSessionDAGACLs((Configuration)any(), eq(appId), eq("0"), (DAGAccessControls)any()); + .setupSessionDAGACLs(any(), eq(appId), eq("0"), any()); // No calls were made for domains. - verify(historyACLPolicyManager, times(0)).updateTimelineEntityDomain(any(), (String)any()); + verify(historyACLPolicyManager, times(0)).updateTimelineEntityDomain(any(), any()); Assert.assertEquals(0, atsEntitiesCounter); } @Test(timeout=10000) public void testSessionDomainsDagFailed() throws Exception { - when(historyACLPolicyManager.setupSessionACLs((Configuration)any(), (ApplicationId)any())) - .thenReturn(Collections.singletonMap( - TezConfiguration.YARN_ATS_ACL_SESSION_DOMAIN_ID, "session-domain")); + when(historyACLPolicyManager.setupSessionACLs(any(), any())) + .thenReturn(Collections.singletonMap(TezConfiguration.YARN_ATS_ACL_SESSION_DOMAIN_ID, "session-domain")); - when(historyACLPolicyManager.setupSessionDAGACLs( - (Configuration)any(), (ApplicationId)any(), eq("0"), (DAGAccessControls)any())) - .thenThrow(new IOException()); + when(historyACLPolicyManager.setupSessionDAGACLs(any(), any(), eq("0"), any())) + .thenThrow(new IOException()); when(appContext.isSession()).thenReturn(true); atsHistoryLoggingService.start(); - verify(historyACLPolicyManager, times(1)).setupSessionACLs((Configuration)any(), - (ApplicationId)any()); + verify(historyACLPolicyManager, times(1)).setupSessionACLs(any(), any()); // Send the event and wait for completion. TezDAGID dagId1 = TezDAGID.getInstance(appId, 0); @@ -394,29 +376,27 @@ public void testSessionDomainsDagFailed() throws Exception { } // DAG domain was called once. verify(historyACLPolicyManager, times(1)) - .setupSessionDAGACLs((Configuration)any(), eq(appId), eq("0"), (DAGAccessControls)any()); + .setupSessionDAGACLs(any(), eq(appId), eq("0"), any()); // All calls made with session domain id. verify(historyACLPolicyManager, times(1)) .updateTimelineEntityDomain(any(), eq("session-domain")); verify(historyACLPolicyManager, times(1)) - .updateTimelineEntityDomain(any(), (String)any()); + .updateTimelineEntityDomain(any(), any()); Assert.assertEquals(1, atsEntitiesCounter); } @Test(timeout=10000) public void testSessionDomainsAclNull() throws Exception { - when(historyACLPolicyManager.setupSessionACLs((Configuration)any(), (ApplicationId)any())) - .thenReturn(null); + when(historyACLPolicyManager.setupSessionACLs(any(), any())) + .thenReturn(null); - when(historyACLPolicyManager.setupSessionDAGACLs( - (Configuration)any(), (ApplicationId)any(), eq("0"), (DAGAccessControls)any())) - .thenReturn(null); + when(historyACLPolicyManager.setupSessionDAGACLs(any(), any(), eq("0"), any())) + .thenReturn(null); when(appContext.isSession()).thenReturn(true); atsHistoryLoggingService.start(); - verify(historyACLPolicyManager, times(1)).setupSessionACLs((Configuration)any(), - (ApplicationId)any()); + verify(historyACLPolicyManager, times(1)).setupSessionACLs(any(), any()); // Send the event and wait for completion. TezDAGID dagId1 = TezDAGID.getInstance(appId, 0); @@ -429,10 +409,10 @@ public void testSessionDomainsAclNull() throws Exception { } // No dag domain were created. verify(historyACLPolicyManager, times(1)) - .setupSessionDAGACLs((Configuration)any(), eq(appId), eq("0"), (DAGAccessControls)any()); + .setupSessionDAGACLs(any(), eq(appId), eq("0"), any()); // All calls made with session domain id. - verify(historyACLPolicyManager, times(0)).updateTimelineEntityDomain(any(), (String)any()); + verify(historyACLPolicyManager, times(0)).updateTimelineEntityDomain(any(), any()); Assert.assertEquals(6, atsEntitiesCounter); } diff --git a/tez-runtime-internals/pom.xml b/tez-runtime-internals/pom.xml index 65841cf666..0619ca1e16 100644 --- a/tez-runtime-internals/pom.xml +++ b/tez-runtime-internals/pom.xml @@ -25,6 +25,10 @@ tez-runtime-internals + + commons-lang + commons-lang + org.apache.tez tez-api @@ -71,7 +75,7 @@ org.mockito - mockito-all + mockito-core test diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/TestLogicalIOProcessorRuntimeTask.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/TestLogicalIOProcessorRuntimeTask.java index ba9a66d0f1..1524cac357 100644 --- a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/TestLogicalIOProcessorRuntimeTask.java +++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/TestLogicalIOProcessorRuntimeTask.java @@ -22,7 +22,11 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.mockito.Mockito.anyList; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.only; +import static org.mockito.Mockito.verify; import java.io.IOException; import java.nio.ByteBuffer; @@ -75,7 +79,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Multimap; -import org.mockito.Mockito; public class TestLogicalIOProcessorRuntimeTask { @@ -190,7 +193,7 @@ public void testEventsCantBeSentInCleanup() throws Exception { runner.call(); // We verify that no events were sent - Mockito.verify(umbilical, Mockito.only()).addEvents(Collections. emptyList()); + verify(umbilical, only()).addEvents(Collections. emptyList()); } /** @@ -229,7 +232,7 @@ public void testExceptionHappensInClose() throws Exception { fail("RuntimeException should have been thrown"); } catch (RuntimeException e) { // No events should be sent thorught the umbilical protocol - Mockito.verify(umbilical, Mockito.never()).addEvents(Mockito.anyList()); + verify(umbilical, never()).addEvents(anyList()); } } finally { sharedExecutor.shutdownNow(); diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskReporter.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskReporter.java index 9f514f405b..147d17655b 100644 --- a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskReporter.java +++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskReporter.java @@ -18,7 +18,7 @@ package org.apache.tez.runtime.task; -import static org.mockito.Matchers.any; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.isA; @@ -84,7 +84,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { throw new TezUncheckedException("Invalid request id for test: " + request.getRequestId()); } } - }).when(mockUmbilical).heartbeat(any(TezHeartbeatRequest.class)); + }).when(mockUmbilical).heartbeat(any()); TezTaskAttemptID mockTaskAttemptId = mock(TezTaskAttemptID.class); LogicalIOProcessorRuntimeTask mockTask = mock(LogicalIOProcessorRuntimeTask.class); @@ -105,10 +105,10 @@ public Object answer(InvocationOnMock invocation) throws Throwable { lock.wait(); } } - verify(mockUmbilical, times(3)).heartbeat(any(TezHeartbeatRequest.class)); + verify(mockUmbilical, times(3)).heartbeat(any()); Thread.sleep(200l); // Sleep for less than the callable sleep time. No more invocations. - verify(mockUmbilical, times(3)).heartbeat(any(TezHeartbeatRequest.class)); + verify(mockUmbilical, times(3)).heartbeat(any()); } finally { executor.shutdownNow(); } diff --git a/tez-runtime-library/pom.xml b/tez-runtime-library/pom.xml index e43f90f415..c913e87d43 100644 --- a/tez-runtime-library/pom.xml +++ b/tez-runtime-library/pom.xml @@ -102,7 +102,7 @@ org.mockito - mockito-all + mockito-core test diff --git a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestFairShuffleVertexManager.java b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestFairShuffleVertexManager.java index de857bc06f..5108b8f9e6 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestFairShuffleVertexManager.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestFairShuffleVertexManager.java @@ -28,7 +28,6 @@ import org.apache.tez.dag.api.InputDescriptor; import org.apache.tez.dag.api.OutputDescriptor; import org.apache.tez.dag.api.TezUncheckedException; -import org.apache.tez.dag.api.VertexLocationHint; import org.apache.tez.dag.api.VertexManagerPluginContext; import org.apache.tez.dag.api.event.VertexState; import org.apache.tez.dag.api.event.VertexStateUpdate; @@ -311,7 +310,7 @@ private void testSchedulingWithPartitionStats( doAnswer(new reconfigVertexAnswer(mockContext, mockManagedVertexId, newEdgeManagers)).when(mockContext).reconfigureVertex( - anyInt(), any(VertexLocationHint.class), anyMap()); + anyInt(), any(), anyMap()); // check initialization manager = createFairShuffleVertexManager(conf, mockContext, diff --git a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestInputReadyVertexManager.java b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestInputReadyVertexManager.java index d59439e735..2eaaba4871 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestInputReadyVertexManager.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestInputReadyVertexManager.java @@ -30,7 +30,6 @@ import org.apache.tez.dag.api.InputDescriptor; import org.apache.tez.dag.api.OutputDescriptor; import org.apache.tez.dag.api.TezUncheckedException; -import org.apache.tez.dag.api.VertexLocationHint; import org.apache.tez.dag.api.VertexManagerPluginContext; import org.apache.tez.dag.api.EdgeProperty.SchedulingType; import org.apache.tez.dag.api.VertexManagerPluginContext.ScheduleTaskRequest; @@ -275,8 +274,7 @@ public void testComplex() throws Exception { } catch (TezUncheckedException e) { e.getMessage().contains("1-1 source vertices must have identical concurrency"); } - verify(mockContext, times(1)).reconfigureVertex(anyInt(), (VertexLocationHint) any(), - anyMap()); // not invoked + verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(), any()); // not invoked when(mockContext.getVertexNumTasks(mockSrcVertexId3)).thenReturn(3); @@ -288,8 +286,7 @@ public void testComplex() throws Exception { manager.onVertexStateUpdated(new VertexStateUpdate(mockSrcVertexId1, VertexState.CONFIGURED)); manager.onVertexStateUpdated(new VertexStateUpdate(mockSrcVertexId2, VertexState.CONFIGURED)); manager.onVertexStateUpdated(new VertexStateUpdate(mockSrcVertexId3, VertexState.CONFIGURED)); - verify(mockContext, times(1)).reconfigureVertex(anyInt(), (VertexLocationHint) any(), - anyMap()); // not invoked + verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(), any()); // not invoked verify(mockContext, times(2)).doneReconfiguringVertex(); manager.onVertexStarted(initialCompletions); // all 1-1 0's done but not scheduled because v1 is not done diff --git a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManager.java b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManager.java index b824d0b908..fcbcb42ac4 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManager.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManager.java @@ -36,12 +36,12 @@ import java.util.List; import java.util.Map; -import static org.mockito.Matchers.eq; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyInt; import static org.mockito.Mockito.anyList; import static org.mockito.Mockito.anyMap; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -142,8 +142,8 @@ public void testLargeDataSize() throws IOException { manager.onVertexManagerEventReceived(vmEvent); manager.onSourceTaskCompleted(createTaskAttemptIdentifier(mockSrcVertexId2, 1)); // Auto-reduce is triggered - verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(VertexLocationHint.class), anyMap()); - verify(mockContext, times(1)).reconfigureVertex(eq(2), any(VertexLocationHint.class), anyMap()); + verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(), anyMap()); + verify(mockContext, times(1)).reconfigureVertex(eq(2), any(), anyMap()); Assert.assertEquals(2, newEdgeManagers.size()); Assert.assertEquals(0, manager.pendingTasks.size()); // all tasks scheduled Assert.assertEquals(2, scheduledTasks.size()); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerBase.java b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerBase.java index 9c3a5b34d8..75866248c3 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerBase.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerBase.java @@ -27,7 +27,6 @@ import org.apache.tez.dag.api.InputDescriptor; import org.apache.tez.dag.api.OutputDescriptor; import org.apache.tez.dag.api.TezUncheckedException; -import org.apache.tez.dag.api.VertexLocationHint; import org.apache.tez.dag.api.VertexManagerPluginContext; import org.apache.tez.dag.api.event.VertexState; import org.apache.tez.dag.api.event.VertexStateUpdate; @@ -114,8 +113,7 @@ public void testZeroSourceTasksWithVertexStartedFirst() { manager.onVertexStateUpdated(new VertexStateUpdate(mockSrcVertexId2, VertexState.CONFIGURED)); manager.onVertexStateUpdated(new VertexStateUpdate(mockSrcVertexId3, VertexState.CONFIGURED)); Assert.assertTrue(manager.pendingTasks.isEmpty()); - verify(mockContext, times(1)).reconfigureVertex(eq(1), any - (VertexLocationHint.class), anyMap()); + verify(mockContext, times(1)).reconfigureVertex(eq(1), any(), anyMap()); verify(mockContext, times(1)).doneReconfiguringVertex(); // reconfig done Assert.assertTrue(scheduledTasks.size() == 1); // all tasks scheduled and parallelism changed scheduledTasks.clear(); @@ -153,8 +151,7 @@ public void testZeroSourceTasksWithVertexStateUpdatedFirst() { // trigger start and processing of pending notification events manager.onVertexStarted(emptyCompletions); Assert.assertTrue(manager.bipartiteSources == 2); - verify(mockContext, times(1)).reconfigureVertex(eq(1), any - (VertexLocationHint.class), anyMap()); + verify(mockContext, times(1)).reconfigureVertex(eq(1), any(), anyMap()); verify(mockContext, times(1)).doneReconfiguringVertex(); // reconfig done Assert.assertTrue(manager.pendingTasks.isEmpty()); Assert.assertTrue(scheduledTasks.size() == 1); // all tasks scheduled and parallelism changed @@ -322,8 +319,8 @@ public void testTez978() throws IOException { vmEvent = getVertexManagerEvent(null, 160 * MB, mockSrcVertexId2); manager.onVertexManagerEventReceived(vmEvent); Assert.assertTrue(manager.determineParallelismAndApply(0.25f)); //ensure parallelism is determined - verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(VertexLocationHint.class), anyMap()); - verify(mockContext, times(1)).reconfigureVertex(eq(2), any(VertexLocationHint.class), anyMap()); + verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(), anyMap()); + verify(mockContext, times(1)).reconfigureVertex(eq(2), any(), anyMap()); manager.onSourceTaskCompleted(createTaskAttemptIdentifier(mockSrcVertexId2, 0)); Assert.assertEquals(0, manager.pendingTasks.size()); Assert.assertEquals(2, scheduledTasks.size()); @@ -340,7 +337,7 @@ public void testTez978() throws IOException { //min/max fraction of 0.0/0.2 manager = createManager(conf, mockContext, 0.0f, 0.2f); // initial invocation count == 3 - verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(VertexLocationHint.class), anyMap()); + verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(), anyMap()); manager.onVertexStarted(emptyCompletions); manager.onVertexStateUpdated(new VertexStateUpdate(mockSrcVertexId1, VertexState.CONFIGURED)); manager.onVertexStateUpdated(new VertexStateUpdate(mockSrcVertexId2, VertexState.CONFIGURED)); @@ -354,18 +351,18 @@ public void testTez978() throws IOException { manager.onVertexManagerEventReceived(getVertexManagerEvent(null, 10 * MB, mockSrcVertexId1)); manager.onSourceTaskCompleted(createTaskAttemptIdentifier(mockSrcVertexId1, i)); //should not change parallelism - verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(VertexLocationHint.class), anyMap()); + verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(), anyMap()); } for(int i=0;i<3;i++) { manager.onSourceTaskCompleted(createTaskAttemptIdentifier(mockSrcVertexId2, i)); - verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(VertexLocationHint.class), anyMap()); + verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(), anyMap()); } //Since max threshold (40 * 0.2 = 8) is met, vertex manager should determine parallelism manager.onSourceTaskCompleted(createTaskAttemptIdentifier(mockSrcVertexId2, 8)); // parallelism updated - verify(mockContext, times(2)).reconfigureVertex(anyInt(), any(VertexLocationHint.class), anyMap()); + verify(mockContext, times(2)).reconfigureVertex(anyInt(), any(), anyMap()); // check exact update value - 8 events with 100 each => 20 -> 2000 => 2 tasks (with 1000 per task) - verify(mockContext, times(2)).reconfigureVertex(eq(2), any(VertexLocationHint.class), anyMap()); + verify(mockContext, times(2)).reconfigureVertex(eq(2), any(), anyMap()); } @Test(timeout = 5000) @@ -418,8 +415,8 @@ public void testAutoParallelism() throws Exception { manager.onSourceTaskCompleted(createTaskAttemptIdentifier(mockSrcVertexId2, 1)); // managedVertex tasks reduced - verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(VertexLocationHint.class), anyMap()); - verify(mockContext, times(1)).reconfigureVertex(eq(2), any(VertexLocationHint.class), anyMap()); + verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(), anyMap()); + verify(mockContext, times(1)).reconfigureVertex(eq(2), any(), anyMap()); Assert.assertEquals(2, newEdgeManagers.size()); // TODO improve tests for parallelism Assert.assertEquals(0, manager.pendingTasks.size()); // all tasks scheduled @@ -432,7 +429,7 @@ public void testAutoParallelism() throws Exception { // more completions dont cause recalculation of parallelism manager.onSourceTaskCompleted(createTaskAttemptIdentifier(mockSrcVertexId2, 0)); - verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(VertexLocationHint.class), anyMap()); + verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(), anyMap()); Assert.assertEquals(2, newEdgeManagers.size()); EdgeManagerPluginOnDemand edgeManager = @@ -493,7 +490,7 @@ public void testShuffleVertexManagerSlowStart() { String mockManagedVertexId = "Vertex4"; VertexManagerPluginContext mockContext = mock(VertexManagerPluginContext.class); - when(mockContext.getVertexStatistics(any(String.class))).thenReturn(mock(VertexStatistics.class)); + when(mockContext.getVertexStatistics(any())).thenReturn(mock(VertexStatistics.class)); when(mockContext.getInputVertexEdgeProperties()).thenReturn(mockInputVertices); when(mockContext.getVertexName()).thenReturn(mockManagedVertexId); when(mockContext.getVertexNumTasks(mockManagedVertexId)).thenReturn(3); @@ -877,8 +874,7 @@ public void test_Tez1649_with_scatter_gather_edges() throws IOException { Assert.assertTrue(manager.totalNumBipartiteSourceTasks == 6); //Ensure that setVertexParallelism is not called for R2. - verify(mockContext_R2, times(0)).reconfigureVertex(anyInt(), any(VertexLocationHint.class), - anyMap()); + verify(mockContext_R2, times(0)).reconfigureVertex(anyInt(), any(), anyMap()); //ShuffleVertexManager's updatePendingTasks relies on getVertexNumTasks. Setting this for test when(mockContext_R2.getVertexNumTasks(mockManagedVertexId_R2)).thenReturn(1); @@ -886,9 +882,8 @@ public void test_Tez1649_with_scatter_gather_edges() throws IOException { // complete configuration of r1 triggers the scheduling manager.onVertexStateUpdated(new VertexStateUpdate(r1, VertexState.CONFIGURED)); Assert.assertTrue(manager.totalNumBipartiteSourceTasks == 9); - verify(mockContext_R2, times(1)).reconfigureVertex(eq(1), any(VertexLocationHint.class), - anyMap()); - + verify(mockContext_R2, times(1)).reconfigureVertex(eq(1), any(), anyMap()); + Assert.assertTrue(manager.pendingTasks.size() == 0); // all tasks scheduled Assert.assertTrue(scheduledTasks.size() == 1); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerUtils.java b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerUtils.java index 92812224cd..44adc462bc 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerUtils.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerUtils.java @@ -32,7 +32,6 @@ import org.apache.tez.dag.api.InputDescriptor; import org.apache.tez.dag.api.OutputDescriptor; import org.apache.tez.dag.api.UserPayload; -import org.apache.tez.dag.api.VertexLocationHint; import org.apache.tez.dag.api.VertexManagerPluginContext; import org.apache.tez.dag.library.vertexmanager.FairShuffleVertexManager.FairRoutingType; import org.apache.tez.dag.library.vertexmanager.FairShuffleVertexManager.FairShuffleVertexManagerConfigBuilder; @@ -56,10 +55,10 @@ import java.util.List; import java.util.Map; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyList; -import static org.mockito.Matchers.anyMap; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyList; +import static org.mockito.Mockito.anyMap; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -113,7 +112,7 @@ VertexManagerPluginContext createVertexManagerContext( mockContext).scheduleTasks(anyList()); doAnswer(new reconfigVertexAnswer(mockContext, mockManagedVertexId, newEdgeManagers)).when(mockContext).reconfigureVertex( - anyInt(), any(VertexLocationHint.class), anyMap()); + anyInt(), any(), anyMap()); return mockContext; } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestCartesianProductVertexManagerPartitioned.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestCartesianProductVertexManagerPartitioned.java index 1012a36fb4..a77e2c712e 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestCartesianProductVertexManagerPartitioned.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestCartesianProductVertexManagerPartitioned.java @@ -21,7 +21,6 @@ import org.apache.tez.dag.api.EdgeProperty; import org.apache.tez.dag.api.TezReflectionException; import org.apache.tez.dag.api.UserPayload; -import org.apache.tez.dag.api.VertexLocationHint; import org.apache.tez.dag.api.VertexManagerPluginContext; import org.apache.tez.dag.api.VertexManagerPluginContext.ScheduleTaskRequest; import org.apache.tez.dag.api.event.VertexState; @@ -37,7 +36,6 @@ import org.junit.Test; import org.mockito.ArgumentCaptor; import org.mockito.Captor; -import org.mockito.Matchers; import org.mockito.MockitoAnnotations; import java.util.ArrayList; @@ -48,8 +46,9 @@ import static org.apache.tez.dag.api.EdgeProperty.DataMovementType.BROADCAST; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; -import static org.mockito.Matchers.eq; -import static org.mockito.Matchers.isNull; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.isNull; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; @@ -75,7 +74,7 @@ public void setup() throws TezReflectionException { private void setupWithConfig(CartesianProductConfigProto config) throws TezReflectionException { - MockitoAnnotations.initMocks(this); + MockitoAnnotations.openMocks(this); context = mock(VertexManagerPluginContext.class); when(context.getVertexName()).thenReturn("cp"); when(context.getVertexNumTasks("cp")).thenReturn(-1); @@ -110,7 +109,7 @@ private void testReconfigureVertexHelper(CartesianProductConfigProto config, vertexManager.onVertexStateUpdated(new VertexStateUpdate("v0", VertexState.CONFIGURED)); verify(context, times(1)).reconfigureVertex(parallelismCaptor.capture(), - isNull(VertexLocationHint.class), edgePropertiesCaptor.capture()); + isNull(), edgePropertiesCaptor.capture()); assertEquals((int)parallelismCaptor.getValue(), parallelism); assertNull(edgePropertiesCaptor.getValue()); } @@ -134,12 +133,12 @@ public void testScheduling() throws Exception { vertexManager.onSourceTaskCompleted(allCompletions.get(0)); vertexManager.onSourceTaskCompleted(allCompletions.get(1)); - verify(context, never()).scheduleTasks(Matchers.>any()); + verify(context, never()).scheduleTasks(any()); List scheduleTaskRequests; vertexManager.onSourceTaskCompleted(allCompletions.get(2)); // shouldn't start schedule because broadcast src is not in RUNNING state - verify(context, never()).scheduleTasks(Matchers.>any()); + verify(context, never()).scheduleTasks(any()); vertexManager.onVertexStateUpdated(new VertexStateUpdate("v2", VertexState.RUNNING)); verify(context, times(1)).scheduleTasks(scheduleTaskRequestCaptor.capture()); @@ -161,7 +160,7 @@ public void testScheduling() throws Exception { for (int i = 6; i < 8; i++) { vertexManager.onSourceTaskCompleted(allCompletions.get(i)); - verify(context, times(4)).scheduleTasks(Matchers.>any()); + verify(context, times(4)).scheduleTasks(any()); } } @@ -191,7 +190,7 @@ private void testOnVertexStartHelper(boolean broadcastRunning) throws Exception vertexManager.onVertexStarted(completions); if (!broadcastRunning) { - verify(context, never()).scheduleTasks(Matchers.>any()); + verify(context, never()).scheduleTasks(any()); vertexManager.onVertexStateUpdated(new VertexStateUpdate("v2", VertexState.RUNNING)); } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestFairCartesianProductVertexManager.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestFairCartesianProductVertexManager.java index 6219a158d0..54d771ad08 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestFairCartesianProductVertexManager.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestFairCartesianProductVertexManager.java @@ -22,7 +22,6 @@ import com.google.protobuf.InvalidProtocolBufferException; import org.apache.tez.dag.api.EdgeManagerPluginDescriptor; import org.apache.tez.dag.api.EdgeProperty; -import org.apache.tez.dag.api.VertexLocationHint; import org.apache.tez.dag.api.VertexManagerPluginContext; import org.apache.tez.dag.api.VertexManagerPluginContext.ScheduleTaskRequest; import org.apache.tez.dag.api.event.VertexState; @@ -52,10 +51,10 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyMapOf; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyMap; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; @@ -72,7 +71,7 @@ public class TestFairCartesianProductVertexManager { @Before public void setup() { - MockitoAnnotations.initMocks(this); + MockitoAnnotations.openMocks(this); ctx = mock(VertexManagerPluginContext.class); vertexManager = new FairCartesianProductVertexManager(ctx); } @@ -224,11 +223,11 @@ public void testDAGVertexOnlyGroupByMaxParallelism() throws Exception { vertexManager.onVertexManagerEventReceived(getVMEvent(250, "v0", 0)); verify(ctx, never()).reconfigureVertex( - anyInt(), any(VertexLocationHint.class), anyMapOf(String.class, EdgeProperty.class)); + anyInt(), any(), anyMap()); vertexManager.onVertexManagerEventReceived(getVMEvent(200, "v1", 0)); verify(ctx, times(1)).reconfigureVertex( - eq(30), any(VertexLocationHint.class), edgePropertiesCaptor.capture()); + eq(30), any(), edgePropertiesCaptor.capture()); Map edgeProperties = edgePropertiesCaptor.getValue(); verifyEdgeProperties(edgeProperties.get("v0"), new String[]{"v0", "v1"}, new int[]{5, 6}, 30); verifyVertexGroupInfo(edgeProperties.get("v0"), 0); @@ -259,7 +258,7 @@ public void testDAGVertexOnlyGroupByMinOpsPerWorker() throws Exception { } verify(ctx, times(1)).reconfigureVertex( - eq(12), any(VertexLocationHint.class), edgePropertiesCaptor.capture()); + eq(12), any(), edgePropertiesCaptor.capture()); Map edgeProperties = edgePropertiesCaptor.getValue(); verifyEdgeProperties(edgeProperties.get("v0"), new String[]{"v0", "v1"}, new int[]{4, 3}, 100); verifyEdgeProperties(edgeProperties.get("v1"), new String[]{"v0", "v1"}, new int[]{4, 3}, 100); @@ -289,7 +288,7 @@ public void testDAGVertexGroup() throws Exception { vertexManager.onVertexManagerEventReceived(getVMEvent(5, "v2", 0)); vertexManager.onVertexManagerEventReceived(getVMEvent(5, "v2", 1)); verify(ctx, times(1)).reconfigureVertex( - eq(100), any(VertexLocationHint.class), edgePropertiesCaptor.capture()); + eq(100), any(), edgePropertiesCaptor.capture()); Map edgeProperties = edgePropertiesCaptor.getValue(); for (int i = 0; i < 3; i++) { verifyEdgeProperties(edgeProperties.get("v" + i), new String[]{"v0", "g0"}, @@ -323,7 +322,7 @@ public void testDAGVertexGroupOnly() throws Exception { vertexManager.onVertexManagerEventReceived(getVMEvent(16, "v3", 0)); verify(ctx, times(1)).reconfigureVertex( - eq(100), any(VertexLocationHint.class), edgePropertiesCaptor.capture()); + eq(100), any(), edgePropertiesCaptor.capture()); Map edgeProperties = edgePropertiesCaptor.getValue(); for (int i = 0; i < 4; i++) { verifyEdgeProperties(edgeProperties.get("v" + i), new String[]{"g0", "g1"}, @@ -352,7 +351,7 @@ public void testSchedulingVertexOnlyWithBroadcast() throws Exception { vertexManager.onVertexManagerEventReceived(getVMEvent(250, "v0", 0)); vertexManager.onVertexManagerEventReceived(getVMEvent(200, "v1", 0)); verify(ctx, times(1)).reconfigureVertex( - eq(30), any(VertexLocationHint.class), edgePropertiesCaptor.capture()); + eq(30), any(), edgePropertiesCaptor.capture()); assertFalse(edgePropertiesCaptor.getValue().containsKey("v2")); vertexManager.onVertexStarted(null); @@ -400,7 +399,7 @@ public void testZeroSrcTask() throws Exception { vertexManager.initialize(config); vertexManager.onVertexStateUpdated(new VertexStateUpdate("v0", VertexState.CONFIGURED)); vertexManager.onVertexStateUpdated(new VertexStateUpdate("v1", VertexState.CONFIGURED)); - vertexManager.onVertexStarted(new ArrayList()); + vertexManager.onVertexStarted(new ArrayList<>()); vertexManager.onSourceTaskCompleted(getTaId("v0", 0)); vertexManager.onSourceTaskCompleted(getTaId("v0", 1)); } @@ -430,11 +429,11 @@ public void testGroupingFraction() throws Exception { vertexManager.onSourceTaskCompleted(getTaId("v1", i)); } verify(ctx, never()).reconfigureVertex( - anyInt(), any(VertexLocationHint.class), anyMapOf(String.class, EdgeProperty.class)); + anyInt(), any(), anyMap()); vertexManager.onSourceTaskCompleted(getTaId("v1", 14)); verify(ctx, times(1)).reconfigureVertex( - eq(24), any(VertexLocationHint.class), edgePropertiesCaptor.capture()); + eq(24), any(), edgePropertiesCaptor.capture()); } @Test(timeout = 5000) @@ -448,7 +447,7 @@ public void testGroupFractionWithZeroStats() throws Exception { vertexManager.onSourceTaskCompleted(getTaId("v1", i)); } verify(ctx, never()).reconfigureVertex( - anyInt(), any(VertexLocationHint.class), anyMapOf(String.class, EdgeProperty.class)); + anyInt(), any(), anyMap()); } @Test(timeout = 5000) @@ -462,7 +461,7 @@ public void testGroupingFractionWithZeroOutput() throws Exception { vertexManager.onSourceTaskCompleted(getTaId("v1", i)); } verify(ctx, times(1)).reconfigureVertex( - eq(0), any(VertexLocationHint.class), edgePropertiesCaptor.capture()); + eq(0), any(), edgePropertiesCaptor.capture()); } @Test(timeout = 5000) @@ -476,7 +475,7 @@ public void testZeroSrcOutput() throws Exception { vertexManager.onVertexManagerEventReceived(getVMEvent(0, "v1", 1)); vertexManager.onVertexManagerEventReceived(getVMEvent(0, "v1", 2)); verify(ctx, times(1)).reconfigureVertex( - eq(0), any(VertexLocationHint.class), edgePropertiesCaptor.capture()); + eq(0), any(), edgePropertiesCaptor.capture()); } @Test(timeout = 5000) @@ -495,7 +494,7 @@ public void testDisableGrouping() throws Exception { vertexManager.onVertexManagerEventReceived(getVMEvent(250, "v0", 0)); vertexManager.onVertexManagerEventReceived(getVMEvent(200, "v1", 0)); verify(ctx, times(1)).reconfigureVertex( - eq(6), any(VertexLocationHint.class), edgePropertiesCaptor.capture()); + eq(6), any(), edgePropertiesCaptor.capture()); } @Test(timeout = 5000) @@ -511,7 +510,7 @@ public void testParallelismTwoSkewedSource() throws Exception { } verify(ctx, times(1)).reconfigureVertex( - eq(99), any(VertexLocationHint.class), edgePropertiesCaptor.capture()); + eq(99), any(), edgePropertiesCaptor.capture()); Map edgeProperties = edgePropertiesCaptor.getValue(); verifyEdgeProperties(edgeProperties.get("v0"), new String[]{"v0", "v1"}, new int[]{99, 1}, 100); @@ -539,9 +538,9 @@ public void testParallelismThreeSkewedSource() throws Exception { } verify(ctx, times(1)).reconfigureVertex( - eq(93), any(VertexLocationHint.class), edgePropertiesCaptor.capture()); + eq(93), any(), edgePropertiesCaptor.capture()); Map edgeProperties = edgePropertiesCaptor.getValue(); verifyEdgeProperties(edgeProperties.get("v0"), new String[]{"v0", "v1", "v2"}, new int[]{31, 3, 1}, 100); } -} \ No newline at end of file +} diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java index 338f640f91..efba8ea6a7 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java @@ -20,14 +20,15 @@ import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.tez.runtime.library.common.CompositeInputAttemptIdentifier; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyLong; -import static org.mockito.Matchers.anyString; -import static org.mockito.Matchers.eq; + import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyLong; +import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; @@ -193,7 +194,7 @@ public Path answer(InvocationOnMock invocation) throws Throwable { Object[] args = invocation.getArguments(); return new Path(SHUFFLE_INPUT_FILE_PREFIX + args[0]); } - }).when(fetcher).getShuffleInputFileName(anyString(), anyString()); + }).when(fetcher).getShuffleInputFileName(anyString(), any()); doAnswer(new Answer() { @Override diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java index 446801ac2f..3363cb6627 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java @@ -36,7 +36,7 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.mockito.Matchers; +import org.mockito.ArgumentMatchers; import org.slf4j.Logger; import java.io.ByteArrayInputStream; @@ -257,28 +257,28 @@ public void testGenerateOnSpillEvent_With_All_EmptyPartitions() throws Exception outputContext, spillId, new TezSpillRecord(indexFile, conf), physicalOutputs, true, pathComponent, null, false, auxiliaryService, TezCommonUtils.newBestCompressionDeflater()); - Assert.assertTrue(events.size() == 2); //one for VM + Assert.assertEquals(2, events.size()); //one for VM Assert.assertTrue(events.get(0) instanceof VertexManagerEvent); Assert.assertTrue(events.get(1) instanceof CompositeDataMovementEvent); CompositeDataMovementEvent cdme = (CompositeDataMovementEvent) events.get(1); - Assert.assertTrue(cdme.getCount() == physicalOutputs); - Assert.assertTrue(cdme.getSourceIndexStart() == 0); + Assert.assertEquals(cdme.getCount(), physicalOutputs); + Assert.assertEquals(0, cdme.getSourceIndexStart()); ShuffleUserPayloads.DataMovementEventPayloadProto dmeProto = ShuffleUserPayloads.DataMovementEventPayloadProto.parseFrom(ByteString.copyFrom( cdme.getUserPayload())); //spill details should be present - Assert.assertTrue(dmeProto.getSpillId() == 0); + Assert.assertEquals(0, dmeProto.getSpillId()); Assert.assertTrue(dmeProto.hasLastEvent() && dmeProto.getLastEvent()); - Assert.assertTrue(dmeProto.getPathComponent().equals("")); + Assert.assertEquals("", dmeProto.getPathComponent()); byte[] emptyPartitions = TezCommonUtils.decompressByteStringToByteArray(dmeProto .getEmptyPartitions()); BitSet emptyPartitionsBitSet = TezUtilsInternal.fromByteArray(emptyPartitions); - Assert.assertTrue("emptyPartitionBitSet cardinality (expecting 10) = " + emptyPartitionsBitSet - .cardinality(), emptyPartitionsBitSet.cardinality() == 10); + Assert.assertEquals("emptyPartitionBitSet cardinality (expecting 10) = " + emptyPartitionsBitSet + .cardinality(), 10, emptyPartitionsBitSet.cardinality()); } @@ -289,10 +289,10 @@ public void testInternalErrorTranslation() throws Exception { when(mockCodecStream.read(any(byte[].class), anyInt(), anyInt())) .thenThrow(new InternalError(codecErrorMsg)); Decompressor mockDecoder = mock(Decompressor.class); - CompressionCodec mockCodec = mock(ConfigurableCodecForTest.class); - when(((ConfigurableCodecForTest) mockCodec).getConf()).thenReturn(mock(Configuration.class)); + ConfigurableCodecForTest mockCodec = mock(ConfigurableCodecForTest.class); + when(mockCodec.getConf()).thenReturn(mock(Configuration.class)); when(mockCodec.createDecompressor()).thenReturn(mockDecoder); - when(mockCodec.createInputStream(any(InputStream.class), any(Decompressor.class))) + when(mockCodec.createInputStream(any(), any())) .thenReturn(mockCodecStream); byte[] header = new byte[] { (byte) 'T', (byte) 'I', (byte) 'F', (byte) 1}; try { @@ -312,10 +312,10 @@ public void testExceptionTranslation() throws Exception { when(mockCodecStream.read(any(byte[].class), anyInt(), anyInt())) .thenThrow(new IllegalArgumentException(codecErrorMsg)); Decompressor mockDecoder = mock(Decompressor.class); - CompressionCodec mockCodec = mock(ConfigurableCodecForTest.class); - when(((ConfigurableCodecForTest) mockCodec).getConf()).thenReturn(mock(Configuration.class)); + ConfigurableCodecForTest mockCodec = mock(ConfigurableCodecForTest.class); + when(mockCodec.getConf()).thenReturn(mock(Configuration.class)); when(mockCodec.createDecompressor()).thenReturn(mockDecoder); - when(mockCodec.createInputStream(any(InputStream.class), any(Decompressor.class))) + when(mockCodec.createInputStream(any(), any())) .thenReturn(mockCodecStream); byte[] header = new byte[] { (byte) 'T', (byte) 'I', (byte) 'F', (byte) 1}; try { @@ -329,10 +329,10 @@ public void testExceptionTranslation() throws Exception { CompressionInputStream mockCodecStream1 = mock(CompressionInputStream.class); when(mockCodecStream1.read(any(byte[].class), anyInt(), anyInt())) .thenThrow(new SocketTimeoutException(codecErrorMsg)); - CompressionCodec mockCodec1 = mock(ConfigurableCodecForTest.class); - when(((ConfigurableCodecForTest) mockCodec1).getConf()).thenReturn(mock(Configuration.class)); + ConfigurableCodecForTest mockCodec1 = mock(ConfigurableCodecForTest.class); + when(mockCodec1.getConf()).thenReturn(mock(Configuration.class)); when(mockCodec1.createDecompressor()).thenReturn(mockDecoder); - when(mockCodec1.createInputStream(any(InputStream.class), any(Decompressor.class))) + when(mockCodec1.createInputStream(any(), any())) .thenReturn(mockCodecStream1); try { ShuffleUtils.shuffleToMemory(new byte[1024], new ByteArrayInputStream(header), @@ -345,10 +345,10 @@ public void testExceptionTranslation() throws Exception { CompressionInputStream mockCodecStream2 = mock(CompressionInputStream.class); when(mockCodecStream2.read(any(byte[].class), anyInt(), anyInt())) .thenThrow(new InternalError(codecErrorMsg)); - CompressionCodec mockCodec2 = mock(ConfigurableCodecForTest.class); - when(((ConfigurableCodecForTest) mockCodec2).getConf()).thenReturn(mock(Configuration.class)); + ConfigurableCodecForTest mockCodec2 = mock(ConfigurableCodecForTest.class); + when(mockCodec2.getConf()).thenReturn(mock(Configuration.class)); when(mockCodec2.createDecompressor()).thenReturn(mockDecoder); - when(mockCodec2.createInputStream(any(InputStream.class), any(Decompressor.class))) + when(mockCodec2.createInputStream(any(), any())) .thenReturn(mockCodecStream2); try { ShuffleUtils.shuffleToMemory(new byte[1024], new ByteArrayInputStream(header), @@ -394,14 +394,14 @@ public void testFetchStatsLogger() throws Exception { logger.logIndividualFetchComplete(10, 100, 1000, "testType", ident); } verify(activeLogger, times(0)).info(anyString()); - verify(aggregateLogger, times(1)).info(anyString(), Matchers.anyVararg()); + verify(aggregateLogger, times(1)).info(anyString(), ArgumentMatchers.any()); when(activeLogger.isInfoEnabled()).thenReturn(true); for (int i = 0; i < 1000; i++) { logger.logIndividualFetchComplete(10, 100, 1000, "testType", ident); } verify(activeLogger, times(1000)).info(anyString()); - verify(aggregateLogger, times(1)).info(anyString(), Matchers.anyVararg()); + verify(aggregateLogger, times(1)).info(anyString(), ArgumentMatchers.any()); } /** diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java index 8a774dc1a5..9f6b4a3a43 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java @@ -18,11 +18,11 @@ package org.apache.tez.runtime.library.common.shuffle.impl; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyString; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -199,8 +199,8 @@ private InputContext createInputContext() throws IOException { @Override public ExecutorService answer(InvocationOnMock invocation) throws Throwable { return sharedExecutor.createExecutorService( - invocation.getArgumentAt(0, Integer.class), - invocation.getArgumentAt(1, String.class)); + invocation.getArgument(0, Integer.class), + invocation.getArgument(1, String.class)); } }); return inputContext; @@ -265,7 +265,7 @@ public void testPipelinedShuffleEvents() throws IOException { //0--> 1 with spill id 1 (attemptNum 1). This should report exception dme = createDataMovementEvent(true, 0, 1, 1, false, new BitSet(), 4, 1); handler.handleEvents(Collections.singletonList(dme)); - verify(inputContext).killSelf(any(Throwable.class), anyString()); + verify(inputContext).killSelf(any(), anyString()); } /** @@ -297,7 +297,7 @@ public void testPipelinedShuffleEvents_WithOutOfOrderAttempts() throws IOExcepti //Now send attemptNum 0. This should throw exception, because attempt #1 is already added dme = createDataMovementEvent(true, 0, 1, 0, false, new BitSet(), 4, 0); handler.handleEvents(Collections.singletonList(dme)); - verify(inputContext).killSelf(any(Throwable.class), anyString()); + verify(inputContext).killSelf(any(), anyString()); } /** @@ -338,7 +338,7 @@ public void testPipelinedShuffleEvents_WithEmptyPartitions() throws IOException //Now send attemptNum 1. This should throw exception, because attempt #1 is already added dme = createDataMovementEvent(true, 0, 1, 0, false, new BitSet(), 4, 1); handler.handleEvents(Collections.singletonList(dme)); - verify(inputContext).killSelf(any(Throwable.class), anyString()); + verify(inputContext).killSelf(any(), anyString()); } private Event createDataMovementEvent(boolean addSpillDetails, int srcIdx, int targetIdx, diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java index fda2c896d1..ba854b9c14 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java @@ -20,9 +20,9 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; @@ -179,8 +179,8 @@ private InputContext createInputContext() throws IOException { @Override public ExecutorService answer(InvocationOnMock invocation) throws Throwable { return sharedExecutor.createExecutorService( - invocation.getArgumentAt(0, Integer.class), - invocation.getArgumentAt(1, String.class)); + invocation.getArgument(0, Integer.class), + invocation.getArgument(1, String.class)); } }); return inputContext; diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java index f283780c12..068c8f7b02 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java @@ -19,15 +19,15 @@ package org.apache.tez.runtime.library.common.shuffle.orderedgrouped; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyBoolean; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyLong; -import static org.mockito.Matchers.anyString; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyBoolean; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyLong; +import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; @@ -163,7 +163,7 @@ public void testLocalFetchModeSetting1() throws Exception { spyFetcher.fetchNext(); verify(spyFetcher, times(1)).setupLocalDiskFetch(mapHost); - verify(spyFetcher, never()).copyFromHost(any(MapHost.class)); + verify(spyFetcher, never()).copyFromHost(any()); // if hostname does not match use http mapHost = new MapHost(HOST + "_OTHER", PORT, 0, 1); @@ -177,7 +177,7 @@ public void testLocalFetchModeSetting1() throws Exception { spyFetcher.fetchNext(); - verify(spyFetcher, never()).setupLocalDiskFetch(any(MapHost.class)); + verify(spyFetcher, never()).setupLocalDiskFetch(any()); verify(spyFetcher, times(1)).copyFromHost(mapHost); // if port does not match use http @@ -192,7 +192,7 @@ public void testLocalFetchModeSetting1() throws Exception { spyFetcher.fetchNext(); - verify(spyFetcher, never()).setupLocalDiskFetch(any(MapHost.class)); + verify(spyFetcher, never()).setupLocalDiskFetch(any()); verify(spyFetcher, times(1)).copyFromHost(mapHost); //if local fetch is not enabled @@ -266,8 +266,8 @@ public MapOutput answer(InvocationOnMock invocation) throws Throwable { return mapOutput; } }).when(spyFetcher) - .getMapOutputForDirectDiskFetch(any(InputAttemptIdentifier.class), any(Path.class), - any(TezIndexRecord.class)); + .getMapOutputForDirectDiskFetch(any(), any(), + any()); doAnswer(new Answer() { @Override @@ -275,7 +275,7 @@ public Path answer(InvocationOnMock invocation) throws Throwable { Object[] args = invocation.getArguments(); return new Path(SHUFFLE_INPUT_FILE_PREFIX + args[0]); } - }).when(spyFetcher).getShuffleInputFileName(anyString(), anyString()); + }).when(spyFetcher).getShuffleInputFileName(any(), any()); for (int i = 0; i < host.getPartitionCount(); i++) { doAnswer(new Answer() { @@ -291,7 +291,7 @@ public TezIndexRecord answer(InvocationOnMock invocation) throws Throwable { // match with params for copySucceeded below. return new TezIndexRecord(p * 10, (p+1) * 1000, (p+2) * 100); } - }).when(spyFetcher).getIndexRecord(anyString(), eq(host.getPartitionId() + i)); + }).when(spyFetcher).getIndexRecord(any(), eq(host.getPartitionId() + i)); } doNothing().when(scheduler).copySucceeded(any(InputAttemptIdentifier.class), any(MapHost.class), @@ -369,7 +369,7 @@ public Path answer(InvocationOnMock invocation) throws Throwable { Object[] args = invocation.getArguments(); return new Path(SHUFFLE_INPUT_FILE_PREFIX + args[0]); } - }).when(spyFetcher).getShuffleInputFileName(anyString(), anyString()); + }).when(spyFetcher).getShuffleInputFileName(any(), anyString()); for (int i = 0; i < host.getPartitionCount(); i++) { doAnswer(new Answer() { @@ -459,7 +459,7 @@ public Path answer(InvocationOnMock invocation) throws Throwable { Object[] args = invocation.getArguments(); return new Path(SHUFFLE_INPUT_FILE_PREFIX + args[0]); } - }).when(spyFetcher).getShuffleInputFileName(anyString(), anyString()); + }).when(spyFetcher).getShuffleInputFileName(any(), any()); for (int i = 0; i < host.getPartitionCount(); i++) { doAnswer(new Answer() { @@ -477,7 +477,7 @@ public TezIndexRecord answer(InvocationOnMock invocation) throws Throwable { // match with params for copySucceeded below. return new TezIndexRecord(p * 10, (p + 1) * 1000, (p + 2) * 100); } - }).when(spyFetcher).getIndexRecord(anyString(), eq(host.getPartitionId() + i)); + }).when(spyFetcher).getIndexRecord(any(), eq(host.getPartitionId() + i)); } doNothing().when(scheduler).copySucceeded(any(InputAttemptIdentifier.class), any(MapHost.class), @@ -591,7 +591,7 @@ public void testWithRetry() throws Exception { new InputAttemptIdentifier(3, 4, InputAttemptIdentifier.PATH_PREFIX + "pathComponent_3") ); doReturn(srcAttempts).when(scheduler).getMapsForHost(host); - doReturn(true).when(fetcher).setupConnection(any(MapHost.class), any(Collection.class)); + doReturn(true).when(fetcher).setupConnection(any(), any()); URL url = ShuffleUtils.constructInputURL("http://" + HOST + ":" + PORT + "/mapOutput?job=job_123&&reduce=1&map=", srcAttempts, false); fetcher.httpConnection = new FakeHttpConnection(url, null, "", null); @@ -605,7 +605,7 @@ public MapOutput answer(InvocationOnMock invocation) throws Throwable { doReturn(args[0]).when(mapOutput).getAttemptIdentifier(); return mapOutput; } - }).when(merger).reserve(any(InputAttemptIdentifier.class), anyInt(), anyInt(), anyInt()); + }).when(merger).reserve(any(), anyInt(), anyInt(), anyInt()); //Create read timeout when reading data doAnswer(new Answer() { @@ -614,7 +614,7 @@ public MapOutput answer(InvocationOnMock invocation) throws Throwable { // Simulate read timeout by throwing proper exception throw new FetcherReadTimeoutException("creating fetcher socket read timeout exception"); } - }).when(fetcher).copyMapOutput(any(MapHost.class), any(DataInputStream.class), any(InputAttemptIdentifier.class)); + }).when(fetcher).copyMapOutput(any(), any(), any()); try { fetcher.copyFromHost(host); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffle.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffle.java index 0a9c37e9ea..590affc078 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffle.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffle.java @@ -16,10 +16,10 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyString; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -160,8 +160,8 @@ private InputContext createTezInputContext() throws IOException { @Override public ExecutorService answer(InvocationOnMock invocation) throws Throwable { return sharedExecutor.createExecutorService( - invocation.getArgumentAt(0, Integer.class), - invocation.getArgumentAt(1, String.class)); + invocation.getArgument(0, Integer.class), + invocation.getArgument(1, String.class)); } }); return inputContext; diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleInputEventHandlerOrderedGrouped.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleInputEventHandlerOrderedGrouped.java index cf4ff6a826..8da4adcf48 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleInputEventHandlerOrderedGrouped.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleInputEventHandlerOrderedGrouped.java @@ -37,11 +37,11 @@ import java.util.concurrent.ExecutorService; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyString; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -95,8 +95,8 @@ private InputContext createTezInputContext() throws IOException { @Override public ExecutorService answer(InvocationOnMock invocation) throws Throwable { return sharedExecutor.createExecutorService( - invocation.getArgumentAt(0, Integer.class), - invocation.getArgumentAt(1, String.class)); + invocation.getArgument(0, Integer.class), + invocation.getArgument(1, String.class)); } }); return inputContext; @@ -272,7 +272,7 @@ public void testPiplinedShuffleEvents_WithOutofOrderAttempts() throws IOExceptio handler.handleEvents(Collections.singletonList(dme2)); // task should issue kill request - verify(scheduler, times(1)).killSelf(any(IOException.class), any(String.class)); + verify(scheduler, times(1)).killSelf(any(), any()); } @Test (timeout = 5000) @@ -307,7 +307,7 @@ public void testPipelinedShuffle_WithObsoleteEvents() throws IOException, Interr handler.handleEvents(events); // task should issue kill request, as inputs are scheduled for download already. - verify(scheduler, times(1)).killSelf(any(IOException.class), any(String.class)); + verify(scheduler, times(1)).killSelf(any(), any()); } @Test(timeout = 5000) @@ -348,8 +348,8 @@ public void testAllPartitionsEmpty() throws IOException { events.add(dme); handler.handleEvents(events); InputAttemptIdentifier expectedIdentifier = new InputAttemptIdentifier(targetIdx, 0); - verify(scheduler).copySucceeded(eq(expectedIdentifier), any(MapHost.class), eq(0l), - eq(0l), eq(0l), any(MapOutput.class), eq(true)); + verify(scheduler).copySucceeded(eq(expectedIdentifier), any(), eq(0L), + eq(0L), eq(0L), any(), eq(true)); } @Test(timeout = 5000) @@ -362,8 +362,8 @@ public void testCurrentPartitionEmpty() throws IOException { events.add(dme); handler.handleEvents(events); InputAttemptIdentifier expectedIdentifier = new InputAttemptIdentifier(targetIdx, 0); - verify(scheduler).copySucceeded(eq(expectedIdentifier), any(MapHost.class), eq(0l), - eq(0l), eq(0l), any(MapOutput.class), eq(true)); + verify(scheduler).copySucceeded(eq(expectedIdentifier), any(), eq(0L), + eq(0L), eq(0L), any(), eq(true)); } @Test(timeout = 5000) diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java index b89ffb0ce9..9df9aaf69b 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java @@ -16,9 +16,9 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; @@ -267,10 +267,10 @@ public void _testReducerHealth_1(Configuration conf) throws IOException { if (minFailurePerHost <= 4) { //As per test threshold. Should fail & retrigger shuffle - verify(shuffle, atLeast(0)).reportException(any(Throwable.class)); + verify(shuffle, atLeast(0)).reportException(any()); } else if (minFailurePerHost > 100) { //host failure is so high that this would not retrigger shuffle re-execution - verify(shuffle, atLeast(1)).reportException(any(Throwable.class)); + verify(shuffle, atLeast(1)).reportException(any()); } } @@ -335,7 +335,7 @@ public void testReducerHealth_2() throws IOException, InterruptedException { } //Shuffle has not stalled. so no issues. - verify(scheduler.reporter, times(0)).reportException(any(Throwable.class)); + verify(scheduler.reporter, times(0)).reportException(any()); //stall shuffle scheduler.lastProgressTime = System.currentTimeMillis() - 250000; @@ -346,7 +346,7 @@ public void testReducerHealth_2() throws IOException, InterruptedException { new MapHost("host" + (190 % totalProducerNodes), 10000, 190, 1), false, true); //Even when it is stalled, need (320 - 300 = 20) * 3 = 60 failures - verify(scheduler.reporter, times(0)).reportException(any(Throwable.class)); + verify(scheduler.reporter, times(0)).reportException(any()); assertEquals(11, scheduler.failedShufflesSinceLastCompletion); @@ -370,7 +370,7 @@ public void testReducerHealth_2() throws IOException, InterruptedException { assertEquals(61, scheduler.failedShufflesSinceLastCompletion); assertEquals(10, scheduler.remainingMaps.get()); - verify(shuffle, atLeast(0)).reportException(any(Throwable.class)); + verify(shuffle, atLeast(0)).reportException(any()); //fail another 30 for (int i = 110; i < 120; i++) { @@ -388,7 +388,7 @@ public void testReducerHealth_2() throws IOException, InterruptedException { // Should fail now due to fetcherHealthy. (stall has already happened and // these are the only pending tasks) - verify(shuffle, atLeast(1)).reportException(any(Throwable.class)); + verify(shuffle, atLeast(1)).reportException(any()); } @@ -452,7 +452,7 @@ public void testReducerHealth_3() throws IOException { false, true); // failedShufflesSinceLastCompletion has crossed the limits. Throw error - verify(shuffle, times(0)).reportException(any(Throwable.class)); + verify(shuffle, times(0)).reportException(any()); } @Test(timeout = 60000) @@ -538,13 +538,13 @@ public void testReducerHealth_4() throws IOException { // failedShufflesSinceLastCompletion has crossed the limits. 20% of other nodes had failures as // well. However, it has failed only in one host. So this should proceed // until AM decides to restart the producer. - verify(shuffle, times(0)).reportException(any(Throwable.class)); + verify(shuffle, times(0)).reportException(any()); //stall the shuffle (but within limits) scheduler.lastProgressTime = System.currentTimeMillis() - 300000; scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier), new MapHost("host" + (319 % totalProducerNodes), 10000, 319, 1), false, true); - verify(shuffle, times(1)).reportException(any(Throwable.class)); + verify(shuffle, times(1)).reportException(any()); } @@ -607,7 +607,7 @@ public void testReducerHealth_5() throws IOException { false, true); //Shuffle has not received the events completely. So do not bail out yet. - verify(shuffle, times(0)).reportException(any(Throwable.class)); + verify(shuffle, times(0)).reportException(any()); } @@ -675,7 +675,7 @@ public void _testReducerHealth_6(Configuration conf) throws IOException { assertEquals(scheduler.remainingMaps.get(), 310); //Do not bail out (number of failures is just 5) - verify(scheduler.reporter, times(0)).reportException(any(Throwable.class)); + verify(scheduler.reporter, times(0)).reportException(any()); //5 fetches fail repeatedly for (int i = 10; i < 15; i++) { @@ -696,10 +696,10 @@ public void _testReducerHealth_6(Configuration conf) throws IOException { // Now bail out, as Shuffle has crossed the // failedShufflesSinceLastCompletion limits. (even // though reducerHeathly is - verify(shuffle, atLeast(1)).reportException(any(Throwable.class)); + verify(shuffle, atLeast(1)).reportException(any()); } else { //Do not bail out yet. - verify(shuffle, atLeast(0)).reportException(any(Throwable.class)); + verify(shuffle, atLeast(0)).reportException(any()); } } @@ -755,7 +755,7 @@ public void testReducerHealth_7() throws IOException { false, true); } - verify(shuffle, atLeast(1)).reportException(any(Throwable.class)); + verify(shuffle, atLeast(1)).reportException(any()); } private ShuffleSchedulerForTest createScheduler(long startTime, int @@ -1016,8 +1016,8 @@ private InputContext createTezInputContext() throws IOException { @Override public ExecutorService answer(InvocationOnMock invocation) throws Throwable { return sharedExecutor.createExecutorService( - invocation.getArgumentAt(0, Integer.class), - invocation.getArgumentAt(1, String.class)); + invocation.getArgument(0, Integer.class), + invocation.getArgument(1, String.class)); } }); return inputContext; diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java index bb0f22fabd..84ec143808 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java @@ -66,7 +66,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java index 754fbfa497..a56536dfe8 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java @@ -26,8 +26,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyLong; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyLong; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; @@ -63,7 +63,6 @@ import org.apache.tez.dag.api.UserPayload; import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.ExecutionContext; -import org.apache.tez.runtime.api.MemoryUpdateCallback; import org.apache.tez.runtime.api.OutputContext; import org.apache.tez.runtime.api.OutputStatisticsReporter; import org.apache.tez.runtime.api.events.CompositeDataMovementEvent; @@ -703,7 +702,7 @@ private OutputContext createTezOutputContext() throws IOException { callback.memoryAssigned(requestedSize); return null; } - }).when(context).requestInitialMemory(anyLong(), any(MemoryUpdateCallback.class)); + }).when(context).requestInitialMemory(anyLong(), any()); return context; } -} \ No newline at end of file +} diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java index c39bf3f707..dc7357c9cc 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java @@ -21,13 +21,13 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyListOf; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyList; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.atMost; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; @@ -55,13 +55,13 @@ import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.permission.FsAction; import org.apache.tez.dag.api.TezConfiguration; -import org.apache.tez.runtime.api.TaskFailureType; import org.apache.tez.runtime.api.events.VertexManagerEvent; import org.apache.tez.runtime.library.common.Constants; import org.apache.tez.runtime.library.common.writers.UnorderedPartitionedKVWriter.SpillInfo; import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads; import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.VertexManagerEventPayloadProto; import org.apache.tez.runtime.library.utils.DATA_RANGE_IN_MB; +import org.mockito.invocation.InvocationOnMock; import org.roaringbitmap.RoaringBitmap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -107,7 +107,6 @@ import com.google.common.collect.Multimap; import com.google.common.collect.Sets; import org.mockito.ArgumentCaptor; -import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @RunWith(value = Parameterized.class) @@ -418,7 +417,7 @@ public void textTest(int numRegularRecords, int numPartitions, long availableMem numRecordsWritten++; } if (pipeliningEnabled) { - verify(outputContext, times(numLargeKeys)).sendEvents(anyListOf(Event.class)); + verify(outputContext, times(numLargeKeys)).sendEvents(anyList()); } // Write Large val records @@ -434,7 +433,7 @@ public void textTest(int numRegularRecords, int numPartitions, long availableMem numRecordsWritten++; } if (pipeliningEnabled) { - verify(outputContext, times(numLargevalues + numLargeKeys)).sendEvents(anyListOf(Event.class)); + verify(outputContext, times(numLargevalues + numLargeKeys)).sendEvents(anyList()); } // Write records where key + val are large (but both can fit in the buffer individually) @@ -451,11 +450,11 @@ public void textTest(int numRegularRecords, int numPartitions, long availableMem } if (pipeliningEnabled) { verify(outputContext, times(numLargevalues + numLargeKeys + numLargeKvPairs)) - .sendEvents(anyListOf(Event.class)); + .sendEvents(anyList()); } List events = kvWriter.close(); - verify(outputContext, never()).reportFailure(any(TaskFailureType.class), any(Throwable.class), any(String.class)); + verify(outputContext, never()).reportFailure(any(), any(), any()); if (!pipeliningEnabled) { VertexManagerEvent vmEvent = null; @@ -732,8 +731,8 @@ private void baseTestWithPipelinedTransfer(int numRecords, int numPartitions, Se } verifyPartitionStats(VMEvent, partitionsWithData); - verify(outputContext, never()).reportFailure(any(TaskFailureType.class), - any(Throwable.class), any(String.class)); + verify(outputContext, never()).reportFailure(any(), + any(), any()); assertNull(kvWriter.currentBuffer); assertEquals(0, kvWriter.availableBuffers.size()); @@ -976,8 +975,8 @@ private void baseTestWithFinalMergeDisabled(int numRecords, int numPartitions, } } - verify(outputContext, never()).reportFailure(any(TaskFailureType.class), - any(Throwable.class), any(String.class)); + verify(outputContext, never()).reportFailure(any(), + any(), any()); assertNull(kvWriter.currentBuffer); assertEquals(0, kvWriter.availableBuffers.size()); @@ -1187,7 +1186,7 @@ private void baseTest(int numRecords, int numPartitions, Set skippedPar int recordsPerBuffer = sizePerBuffer / sizePerRecordWithOverhead; int numExpectedSpills = numRecordsWritten / recordsPerBuffer / kvWriter.spillLimit; - verify(outputContext, never()).reportFailure(any(TaskFailureType.class), any(Throwable.class), any(String.class)); + verify(outputContext, never()).reportFailure(any(), any(), any()); assertNull(kvWriter.currentBuffer); assertEquals(0, kvWriter.availableBuffers.size()); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/input/TestOrderedGroupedKVInput.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/input/TestOrderedGroupedKVInput.java index 56b6805a63..c1cdf7cfa0 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/input/TestOrderedGroupedKVInput.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/input/TestOrderedGroupedKVInput.java @@ -15,7 +15,7 @@ package org.apache.tez.runtime.library.input; import static org.junit.Assert.assertEquals; -import static org.mockito.Matchers.any; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java index a7c7ca28cd..573b3e9a18 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java @@ -14,8 +14,8 @@ package org.apache.tez.runtime.library.output; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyLong; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyLong; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -28,7 +28,6 @@ import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.UserPayload; -import org.apache.tez.runtime.api.MemoryUpdateCallback; import org.apache.tez.runtime.api.OutputContext; import org.apache.tez.runtime.api.OutputStatisticsReporter; import org.apache.tez.runtime.api.impl.ExecutionContextImpl; @@ -66,7 +65,7 @@ static OutputContext createOutputContext(Configuration conf, Configuration userP callback.memoryAssigned(requestedSize); return null; } - }).when(ctx).requestInitialMemory(anyLong(), any(MemoryUpdateCallback.class)); + }).when(ctx).requestInitialMemory(anyLong(), any()); doReturn(conf).when(ctx).getContainerConfiguration(); doReturn(TezUtils.createUserPayloadFromConf(userPayloadConf)).when(ctx).getUserPayload(); doReturn("taskVertex").when(ctx).getTaskVertexName(); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileSortedOutput.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileSortedOutput.java index 7999d45fcd..47f841c3d6 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileSortedOutput.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileSortedOutput.java @@ -32,7 +32,6 @@ import org.apache.tez.dag.api.UserPayload; import org.apache.tez.runtime.api.ExecutionContext; import org.apache.tez.runtime.api.Event; -import org.apache.tez.runtime.api.MemoryUpdateCallback; import org.apache.tez.runtime.api.OutputContext; import org.apache.tez.runtime.api.OutputStatisticsReporter; import org.apache.tez.runtime.api.events.CompositeDataMovementEvent; @@ -68,8 +67,8 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyLong; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyLong; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -420,7 +419,7 @@ private OutputContext createTezOutputContext() throws IOException { callback.memoryAssigned(requestedSize); return null; } - }).when(context).requestInitialMemory(anyLong(), any(MemoryUpdateCallback.class)); + }).when(context).requestInitialMemory(anyLong(), any()); ExecutionContext ExecutionContext = mock(ExecutionContext.class); doReturn(HOST).when(ExecutionContext).getHostName(); doReturn(reporter).when(context).getStatisticsReporter(); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileUnorderedKVOutput.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileUnorderedKVOutput.java index 963300cd40..cdb246d069 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileUnorderedKVOutput.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileUnorderedKVOutput.java @@ -18,8 +18,8 @@ package org.apache.tez.runtime.library.output; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyLong; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyLong; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; @@ -61,7 +61,6 @@ import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.runtime.LogicalIOProcessorRuntimeTask; import org.apache.tez.runtime.api.Event; -import org.apache.tez.runtime.api.MemoryUpdateCallback; import org.apache.tez.runtime.api.OutputContext; import org.apache.tez.runtime.api.events.CompositeDataMovementEvent; import org.apache.tez.runtime.api.impl.ExecutionContextImpl; @@ -282,7 +281,7 @@ null, auxEnv, new MemoryDistributor(1, 1, payloadConf), outputDescriptor, null, callback.memoryAssigned(requestedSize); return null; } - }).when(outputContext).requestInitialMemory(anyLong(), any(MemoryUpdateCallback.class)); + }).when(outputContext).requestInitialMemory(anyLong(), any()); return outputContext; } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/utils/TestCodecUtils.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/utils/TestCodecUtils.java index afa645974f..5141598200 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/utils/TestCodecUtils.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/utils/TestCodecUtils.java @@ -49,7 +49,8 @@ import org.apache.tez.runtime.library.common.sort.impl.IFileInputStream; import org.junit.Assert; import org.junit.Test; -import org.mockito.Mockito; + +import static org.mockito.Mockito.mock; public class TestCodecUtils { @@ -80,7 +81,7 @@ private void testConcurrentDecompressorCreationWithModifiedBuffersizeOnCodec( Decompressor decompressor = CodecUtils.getDecompressor(codec); DecompressorStream stream = (DecompressorStream) CodecUtils.getDecompressedInputStreamWithBufferSize(codec, - Mockito.mock(IFileInputStream.class), decompressor, modifiedBufferSize); + mock(IFileInputStream.class), decompressor, modifiedBufferSize); Assert.assertEquals("stream buffer size is incorrect", modifiedBufferSize, getBufferSize(stream)); @@ -128,7 +129,7 @@ private void testConcurrentCompressorDecompressorCreationOnCodec(CompressionCode Decompressor decompressor = CodecUtils.getDecompressor(codec); CompressionInputStream stream = (CompressionInputStream) CodecUtils.getDecompressedInputStreamWithBufferSize(codec, - Mockito.mock(IFileInputStream.class), decompressor, modifiedBufferSize); + mock(IFileInputStream.class), decompressor, modifiedBufferSize); Assert.assertEquals("stream buffer size is incorrect", modifiedBufferSize, getBufferSize(stream)); @@ -146,7 +147,7 @@ private void testConcurrentCompressorDecompressorCreationOnCodec(CompressionCode Compressor compressor = CodecUtils.getCompressor(codec); CompressionOutputStream stream = - CodecUtils.createOutputStream(codec, Mockito.mock(OutputStream.class), compressor); + CodecUtils.createOutputStream(codec, mock(OutputStream.class), compressor); Assert.assertEquals("stream buffer size is incorrect", CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT, getBufferSize(stream)); @@ -164,7 +165,7 @@ private void testConcurrentCompressorDecompressorCreationOnCodec(CompressionCode Decompressor decompressor = CodecUtils.getDecompressor(codec); CompressionInputStream stream = - CodecUtils.createInputStream(codec, Mockito.mock(InputStream.class), decompressor); + CodecUtils.createInputStream(codec, mock(InputStream.class), decompressor); Assert.assertEquals("stream buffer size is incorrect", CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT, getBufferSize(stream)); diff --git a/tez-tools/analyzers/job-analyzer/pom.xml b/tez-tools/analyzers/job-analyzer/pom.xml index 89ae9088ff..0cc85834cb 100644 --- a/tez-tools/analyzers/job-analyzer/pom.xml +++ b/tez-tools/analyzers/job-analyzer/pom.xml @@ -46,6 +46,14 @@ org.apache.tez tez-dag + + org.apache.tez + hadoop-shim + + + org.apache.tez + tez-runtime-library + org.apache.tez tez-tests @@ -136,7 +144,7 @@ org.mockito - mockito-all + mockito-core test From 86dc4fd83f64494f5584c154e6812a9cee3cf4fb Mon Sep 17 00:00:00 2001 From: johnnyCake1 <92083002+johnnyCake1@users.noreply.github.com> Date: Fri, 10 Jun 2022 22:39:17 +0200 Subject: [PATCH 378/512] TEZ-4038: Add a /prof profiler endpoint like HiveServer2 has (#212) (Zheenbek Akimzhanov reviewed by Laszlo Bodor) --- .../apache/tez/common/TezUtilsInternal.java | 20 + .../tez/common/web/ProfileOutputServlet.java | 66 ++++ .../apache/tez/common/web/ProfileServlet.java | 362 ++++++++++++++++++ .../web/ServletToControllerAdapters.java | 13 + .../apache/tez/dag/app/web/WebUIService.java | 20 + .../test/java/org/apache/tez/test/TestAM.java | 26 +- 6 files changed, 504 insertions(+), 3 deletions(-) create mode 100644 tez-common/src/main/java/org/apache/tez/common/web/ProfileOutputServlet.java create mode 100644 tez-common/src/main/java/org/apache/tez/common/web/ProfileServlet.java diff --git a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java index adcae8a964..c2efb29cb6 100644 --- a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java +++ b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java @@ -21,6 +21,7 @@ import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.IOException; +import java.lang.management.ManagementFactory; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.nio.charset.Charset; @@ -322,6 +323,25 @@ public static > Set getEnums(Configuration conf, String con return enums; } + public static Integer getPid() { + String pidStr = null; + String name = ManagementFactory.getRuntimeMXBean().getName(); + if (name != null) { + int idx = name.indexOf("@"); + if (idx != -1) { + pidStr = name.substring(0, name.indexOf("@")); + } + } + try { + if (pidStr != null) { + return Integer.valueOf(pidStr); + } + } catch (NumberFormatException nfe) { + LOG.info("Couldn't parse \"{}\" into integer pid", pidStr); + } + return null; + } + @Private public static void setHadoopCallerContext(HadoopShim hadoopShim, TezTaskAttemptID attemptID) { hadoopShim.setHadoopCallerContext("tez_ta:" + attemptID.toString()); diff --git a/tez-common/src/main/java/org/apache/tez/common/web/ProfileOutputServlet.java b/tez-common/src/main/java/org/apache/tez/common/web/ProfileOutputServlet.java new file mode 100644 index 0000000000..2fac77cdc8 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/common/web/ProfileOutputServlet.java @@ -0,0 +1,66 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.common.web; + +import org.apache.hadoop.yarn.webapp.MimeType; +import org.eclipse.jetty.servlet.DefaultServlet; + +import javax.servlet.ServletException; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import java.io.File; +import java.io.IOException; +import java.io.PrintWriter; +import java.nio.file.Files; +import java.nio.file.Paths; + +/** + * Servlet to serve files generated by {@link ProfileServlet}. + */ +public class ProfileOutputServlet extends DefaultServlet { + public static final String FILE_QUERY_PARAM = "file"; + + public void doGet(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException { + String queriedFile = request.getParameter(FILE_QUERY_PARAM); + if (queriedFile == null) { + writeMessage(response, "Run the profiler to be able to receive its output"); + return; + } + File outputFile = new File(ProfileServlet.OUTPUT_DIR, queriedFile); + if (!outputFile.exists()) { + writeMessage(response, "Requested file does not exist: " + queriedFile); + return; + } + if (outputFile.length() < 100) { + response.setIntHeader("Refresh", 2); + writeMessage(response, "This page auto-refreshes every 2 seconds until output file is ready..."); + return; + } + response.setContentType(MimeType.HTML); + response.getOutputStream().write(Files.readAllBytes(Paths.get(outputFile.getPath()))); + response.getOutputStream().flush(); + response.getOutputStream().close(); + } + + private void writeMessage(HttpServletResponse response, String message) throws IOException { + response.setContentType(MimeType.TEXT); + PrintWriter out = response.getWriter(); + out.println(message); + out.close(); + } +} diff --git a/tez-common/src/main/java/org/apache/tez/common/web/ProfileServlet.java b/tez-common/src/main/java/org/apache/tez/common/web/ProfileServlet.java new file mode 100644 index 0000000000..1cdddfbf9c --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/common/web/ProfileServlet.java @@ -0,0 +1,362 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.common.web; + +import com.google.common.base.Joiner; + +import org.apache.hadoop.http.HttpServer2; +import org.apache.tez.common.TezUtilsInternal; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.servlet.ServletException; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import java.io.File; +import java.io.IOException; +import java.io.PrintStream; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +/** + *
+ * Servlet that runs async-profiler as web-endpoint.
+ * Following options from async-profiler can be specified as query paramater.
+ * //  -e event          profiling event: cpu|alloc|lock|cache-misses etc.
+ * //  -d duration       run profiling for{@literal } seconds (integer)
+ * //  -i interval       sampling interval in nanoseconds (long)
+ * //  -j jstackdepth    maximum Java stack depth (integer)
+ * //  -b bufsize        frame buffer size (long)
+ * //  -t                profile different threads separately
+ * //  -s                simple class names instead of FQN
+ * //  -o fmt[,fmt...]   output format: summary|traces|flat|collapsed|svg|tree|jfr
+ * //  --width px        SVG width pixels (integer)
+ * //  --height px       SVG frame height pixels (integer)
+ * //  --minwidth px     skip frames smaller than px (double)
+ * //  --reverse         generate stack-reversed FlameGraph / Call tree
+ * Example:
+ * - To collect 30 second CPU profile of current process (returns FlameGraph svg)
+ * {@literal curl "http://localhost:10002/prof"}
+ * - To collect 1 minute CPU profile of current process and output in tree format (html)
+ * {@literal curl  "http://localhost:10002/prof?output=tree&duration=60"}
+ * - To collect 30 second heap allocation profile of current process (returns FlameGraph svg)
+ * {@literal curl "http://localhost:10002/prof?event=alloc"}
+ * - To collect lock contention profile of current process (returns FlameGraph svg)
+ * {@literal curl "http://localhost:10002/prof?event=lock"}
+ * Following event types are supported (default is 'cpu') (NOTE: not all OS'es support all events)
+ * // Perf events:
+ * //    cpu
+ * //    page-faults
+ * //    context-switches
+ * //    cycles
+ * //    instructions
+ * //    cache-references
+ * //    cache-misses
+ * //    branches
+ * //    branch-misses
+ * //    bus-cycles
+ * //    L1-dcache-load-misses
+ * //    LLC-load-misses
+ * //    dTLB-load-misses
+ * //    mem:breakpoint
+ * //    trace:tracepoint
+ * // Java events:
+ * //    alloc
+ * //    lock
+ * 
+ */ +public class ProfileServlet extends HttpServlet { + private static final long serialVersionUID = 1L; + private static final Logger LOG = LoggerFactory.getLogger(ProfileServlet.class); + private static final String ACCESS_CONTROL_ALLOW_METHODS = "Access-Control-Allow-Methods"; + private static final String ALLOWED_METHODS = "GET"; + private static final String ACCESS_CONTROL_ALLOW_ORIGIN = "Access-Control-Allow-Origin"; + private static final String CONTENT_TYPE_TEXT = "text/plain; charset=utf-8"; + private static final String ASYNC_PROFILER_HOME_ENV = "ASYNC_PROFILER_HOME"; + private static final String ASYNC_PROFILER_HOME_SYSTEM_PROPERTY = "async.profiler.home"; + private static final String PROFILER_SCRIPT = "/profiler.sh"; + private static final int DEFAULT_DURATION_SECONDS = 10; + private static final AtomicInteger ID_GEN = new AtomicInteger(0); + public static final String OUTPUT_DIR = System.getProperty("java.io.tmpdir") + "/prof-output"; + + enum Event { + CPU("cpu"), + ALLOC("alloc"), + LOCK("lock"), + PAGE_FAULTS("page-faults"), + CONTEXT_SWITCHES("context-switches"), + CYCLES("cycles"), + INSTRUCTIONS("instructions"), + CACHE_REFERENCES("cache-references"), + CACHE_MISSES("cache-misses"), + BRANCHES("branches"), + BRANCH_MISSES("branch-misses"), + BUS_CYCLES("bus-cycles"), + L1_DCACHE_LOAD_MISSES("L1-dcache-load-misses"), + LLC_LOAD_MISSES("LLC-load-misses"), + DTLB_LOAD_MISSES("dTLB-load-misses"), + MEM_BREAKPOINT("mem:breakpoint"), + TRACE_TRACEPOINT("trace:tracepoint"); + + private final String internalName; + + Event(final String internalName) { + this.internalName = internalName; + } + + public String getInternalName() { + return internalName; + } + + public static Event fromInternalName(final String name) { + for (Event event : values()) { + if (event.getInternalName().equalsIgnoreCase(name)) { + return event; + } + } + return null; + } + } + + enum Output { + SUMMARY, TRACES, FLAT, COLLAPSED, SVG, TREE, JFR + } + + private final Lock profilerLock = new ReentrantLock(); + private Integer pid; + private String asyncProfilerHome; + private transient Process process; + + public ProfileServlet() { + this.asyncProfilerHome = getAsyncProfilerHome(); + this.pid = TezUtilsInternal.getPid(); + LOG.info("Servlet process PID: {} asyncProfilerHome: {}", pid, asyncProfilerHome); + } + + public void doGet(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException { + response.setContentType("text/plain; charset=UTF-8"); + PrintStream out = new PrintStream(response.getOutputStream(), false, "UTF-8"); + if (!HttpServer2.isInstrumentationAccessAllowed(this.getServletContext(), request, response)) { + response.setStatus(HttpServletResponse.SC_UNAUTHORIZED); + setResponseHeader(response); + out.println("Unauthorized: Instrumentation access is not allowed!"); + out.close(); + return; + } + + // make sure async profiler home is set + if (asyncProfilerHome == null || asyncProfilerHome.trim().isEmpty()) { + response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + setResponseHeader(response); + out.println("ASYNC_PROFILER_HOME env is not set"); + out.close(); + return; + } + + // if pid is explicitly specified, use it else default to current process + pid = getInteger(request, "pid", pid); + // if pid is not specified in query param and if current process pid cannot be determined + if (pid == null) { + response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + setResponseHeader(response); + out.println("'pid' query parameter unspecified or unable to determine PID of current process."); + out.close(); + return; + } + + final int duration = getInteger(request, "duration", DEFAULT_DURATION_SECONDS); + final Output output = getOutput(request); + final Event event = getEvent(request); + final Long interval = getLong(request, "interval"); + final Integer jstackDepth = getInteger(request, "jstackdepth", null); + final Long bufsize = getLong(request, "bufsize"); + final boolean thread = request.getParameterMap().containsKey("thread"); + final boolean simple = request.getParameterMap().containsKey("simple"); + final Integer width = getInteger(request, "width", null); + final Integer height = getInteger(request, "height", null); + final Double minwidth = getMinWidth(request); + final boolean reverse = request.getParameterMap().containsKey("reverse"); + if (process == null || !process.isAlive()) { + try { + int lockTimeoutSecs = 3; + if (profilerLock.tryLock(lockTimeoutSecs, TimeUnit.SECONDS)) { + try { + File outputFile = new File(OUTPUT_DIR, + "async-prof-pid-" + pid + "-" + event.name().toLowerCase() + "-" + ID_GEN.incrementAndGet() + "." + + output.name().toLowerCase()); + List cmd = new ArrayList<>(); + cmd.add(asyncProfilerHome + PROFILER_SCRIPT); + cmd.add("-e"); + cmd.add(event.getInternalName()); + cmd.add("-d"); + cmd.add("" + duration); + cmd.add("-o"); + cmd.add(output.name().toLowerCase()); + cmd.add("-f"); + cmd.add(outputFile.getAbsolutePath()); + if (interval != null) { + cmd.add("-i"); + cmd.add(interval.toString()); + } + if (jstackDepth != null) { + cmd.add("-j"); + cmd.add(jstackDepth.toString()); + } + if (bufsize != null) { + cmd.add("-b"); + cmd.add(bufsize.toString()); + } + if (thread) { + cmd.add("-t"); + } + if (simple) { + cmd.add("-s"); + } + if (width != null) { + cmd.add("--width"); + cmd.add(width.toString()); + } + if (height != null) { + cmd.add("--height"); + cmd.add(height.toString()); + } + if (minwidth != null) { + cmd.add("--minwidth"); + cmd.add(minwidth.toString()); + } + if (reverse) { + cmd.add("--reverse"); + } + cmd.add(pid.toString()); + process = new ProcessBuilder(cmd).start(); + + // set response and set refresh header to output location + setResponseHeader(response); + response.setStatus(HttpServletResponse.SC_ACCEPTED); + String relativeUrl = "/prof-output"; + // to avoid auto-refresh by ProfileOutputServlet, refreshDelay can be specified via url param + int refreshDelay = getInteger(request, "refreshDelay", 0); + // instead of sending redirect, set auto-refresh so that browsers will refresh with redirected url + response.setHeader("Refresh", (duration + refreshDelay) + "; URL=" + relativeUrl + '?' + + ProfileOutputServlet.FILE_QUERY_PARAM + '=' + outputFile.getName()); + + out.println("Profiled PID: " + pid); + out.println("Started [" + event.getInternalName() + + "] profiling. This page will automatically redirect to " + + relativeUrl + " after " + duration + " seconds.\n\ncommand:\n" + Joiner.on(" ").join(cmd)); + out.flush(); + } finally { + profilerLock.unlock(); + } + } else { + setResponseHeader(response); + response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + out.println("Unable to acquire lock. Another instance of profiler might be running."); + LOG.warn("Unable to acquire lock in {} seconds. Another instance of profiler might be running.", + lockTimeoutSecs); + } + } catch (InterruptedException e) { + LOG.warn("Interrupted while acquiring profile lock.", e); + response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + } + } else { + setResponseHeader(response); + response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + out.println("Another instance of profiler is already running."); + } + out.close(); + } + + private Integer getInteger(final HttpServletRequest req, final String param, final Integer defaultValue) { + final String value = req.getParameter(param); + if (value != null) { + try { + return Integer.valueOf(value); + } catch (NumberFormatException e) { + return defaultValue; + } + } + return defaultValue; + } + + private Long getLong(final HttpServletRequest req, final String param) { + final String value = req.getParameter(param); + if (value != null) { + try { + return Long.valueOf(value); + } catch (NumberFormatException e) { + return null; + } + } + return null; + } + + private Double getMinWidth(final HttpServletRequest req) { + final String value = req.getParameter("minwidth"); + if (value != null) { + try { + return Double.valueOf(value); + } catch (NumberFormatException e) { + return null; + } + } + return null; + } + + private Event getEvent(final HttpServletRequest req) { + final String eventArg = req.getParameter("event"); + if (eventArg != null) { + Event event = Event.fromInternalName(eventArg); + return event == null ? Event.CPU : event; + } + return Event.CPU; + } + + private Output getOutput(final HttpServletRequest req) { + final String outputArg = req.getParameter("output"); + if (outputArg != null) { + try { + return Output.valueOf(outputArg.trim().toUpperCase()); + } catch (IllegalArgumentException e) { + LOG.warn("Output format value is invalid, returning with default SVG"); + return Output.SVG; + } + } + return Output.SVG; + } + + private void setResponseHeader(final HttpServletResponse response) { + response.setHeader(ACCESS_CONTROL_ALLOW_METHODS, ALLOWED_METHODS); + response.setHeader(ACCESS_CONTROL_ALLOW_ORIGIN, "*"); + response.setContentType(CONTENT_TYPE_TEXT); + } + + public static String getAsyncProfilerHome() { + String asyncProfilerHome = System.getenv(ASYNC_PROFILER_HOME_ENV); + // if ENV is not set, see if -Dasync.profiler.home=/path/to/async/profiler/home is set + if (asyncProfilerHome == null || asyncProfilerHome.trim().isEmpty()) { + asyncProfilerHome = System.getProperty(ASYNC_PROFILER_HOME_SYSTEM_PROPERTY); + } + return asyncProfilerHome; + } +} diff --git a/tez-common/src/main/java/org/apache/tez/common/web/ServletToControllerAdapters.java b/tez-common/src/main/java/org/apache/tez/common/web/ServletToControllerAdapters.java index 35ca1b6408..304e9a9118 100644 --- a/tez-common/src/main/java/org/apache/tez/common/web/ServletToControllerAdapters.java +++ b/tez-common/src/main/java/org/apache/tez/common/web/ServletToControllerAdapters.java @@ -42,4 +42,17 @@ public StackServletController() throws ServletException { this.servlet = new StackServlet(); } } + + public static class ProfileServletController extends AbstractServletToControllerAdapter { + public ProfileServletController() throws ServletException { + this.servlet = new ProfileServlet(); + } + } + + public static class ProfileOutputServletController extends AbstractServletToControllerAdapter { + public ProfileOutputServletController() throws ServletException { + this.servlet = new ProfileOutputServlet(); + } + } + } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java index bf94a73002..da7e66d037 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java @@ -20,12 +20,19 @@ import static org.apache.hadoop.yarn.util.StringHelper.pajoin; +import java.io.IOException; import java.net.InetSocketAddress; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; import org.apache.tez.common.Preconditions; +import org.apache.tez.common.web.ProfileServlet; import org.apache.tez.common.web.ServletToControllerAdapters.ConfServletController; import org.apache.tez.common.web.ServletToControllerAdapters.JMXJsonServletController; import org.apache.tez.common.web.ServletToControllerAdapters.StackServletController; +import org.apache.tez.common.web.ServletToControllerAdapters.ProfileServletController; +import org.apache.tez.common.web.ServletToControllerAdapters.ProfileOutputServletController; import com.google.inject.name.Names; import org.slf4j.Logger; @@ -235,6 +242,19 @@ public void setup() { route("/jmx", JMXJsonServletController.class); route("/conf", ConfServletController.class); route("/stacks", StackServletController.class); + final String asyncProfilerHome = ProfileServlet.getAsyncProfilerHome(); + if (asyncProfilerHome != null && !asyncProfilerHome.trim().isEmpty()) { + Path tmpDir = Paths.get(ProfileServlet.OUTPUT_DIR); + try { + Files.createDirectories(tmpDir); + route("/prof", ProfileServletController.class); + route("/prof-output", ProfileOutputServletController.class); + } catch (IOException e) { + LOG.info("Could not create directory for profiler output: {} Disabling /prof endpoint... ", tmpDir); + } + } else { + LOG.info("ASYNC_PROFILER_HOME env or -Dasync.profiler.home not specified. Disabling /prof endpoint.."); + } } } } diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestAM.java b/tez-tests/src/test/java/org/apache/tez/test/TestAM.java index 3e8a58cf7a..93fb0b4739 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestAM.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestAM.java @@ -19,7 +19,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; - +import java.io.File; import java.io.IOException; import java.net.HttpURLConnection; import java.net.URL; @@ -74,8 +74,13 @@ public static void setup() throws IOException { Configuration tezClusterConf = new Configuration(); tezClusterConf.set("fs.defaultFS", remoteFs.getUri().toString()); // use HDFS tezClusterConf.setInt("yarn.nodemanager.delete.debug-delay-sec", 20000); - tezClusterConf.setLong(TezConfiguration.TEZ_AM_SLEEP_TIME_BEFORE_EXIT_MILLIS, 1000); + tezClusterConf.setLong(TezConfiguration.TEZ_AM_SLEEP_TIME_BEFORE_EXIT_MILLIS, 2000); tezClusterConf.set(YarnConfiguration.PROXY_ADDRESS, "localhost"); + //provide temporary profiler script to test /prof endpoint + File profiler = getProfiler(); + profiler.createNewFile(); + profiler.setExecutable(true, false); + tezClusterConf.set(TezConfiguration.TEZ_AM_LAUNCH_CMD_OPTS, ("-Dasync.profiler.home=" + getProfilerHomePath())); tezCluster.init(tezClusterConf); tezCluster.start(); } @@ -91,6 +96,7 @@ public static void tearDown() { dfsCluster.shutdown(); dfsCluster = null; } + getProfiler().delete(); } @Test(timeout = 60000) @@ -122,6 +128,8 @@ public void testAMWebUIService() throws TezException, IOException, InterruptedEx checkAddress(webUIAddress + "/jmx"); checkAddress(webUIAddress + "/conf"); checkAddress(webUIAddress + "/stacks"); + checkAddress(webUIAddress + "/prof", 202); + checkAddress(webUIAddress + "/prof-output"); URL url = new URL(webUIAddress); IntegerRanges portRange = conf.getRange(TezConfiguration.TEZ_AM_WEBSERVICE_PORT_RANGE, @@ -133,14 +141,26 @@ public void testAMWebUIService() throws TezException, IOException, InterruptedEx } private void checkAddress(String url) { + checkAddress(url, 200); + } + + private void checkAddress(String url, int expectedCode) { boolean success = false; try { HttpURLConnection connection = (HttpURLConnection) new URL(url).openConnection(); connection.connect(); - success = (connection.getResponseCode() == 200); + success = (connection.getResponseCode() == expectedCode); } catch (Exception e) { LOG.error("Error while checking url: " + url, e); } assertTrue(url + " should be available", success); } + + private static File getProfiler() { + return new File(getProfilerHomePath(), "profiler.sh"); + } + + private static String getProfilerHomePath() { + return System.getProperty("java.io.tmpdir"); + } } From 3da8438e68263f71462e99eb0b728e121702f5bf Mon Sep 17 00:00:00 2001 From: Aman Raj <104416558+amanraj2520@users.noreply.github.com> Date: Wed, 15 Jun 2022 14:55:38 +0530 Subject: [PATCH 379/512] [TEZ-4422] [CVE-2021-43138] Upgrade async from 2.3.0 to 2.6.4 to fix the vulnerability. Also an upgrade of yarn version to 1.6.0 and frontend maven plugin to 1.8.0 was done. (#217) (Aman Raj reviewed by Laszlo Bodor) --- pom.xml | 2 +- tez-ui/pom.xml | 2 +- tez-ui/src/main/webapp/package.json | 3 +++ tez-ui/src/main/webapp/yarn.lock | 18 +++++++++--------- 4 files changed, 14 insertions(+), 11 deletions(-) diff --git a/pom.xml b/pom.xml index c496e0ce87..9fd644bc9a 100644 --- a/pom.xml +++ b/pom.xml @@ -67,7 +67,7 @@ 0.7.45 ${env.PROTOC_PATH} scm:git:https://gitbox.apache.org/repos/asf/tez.git - 1.4 + 1.8.0 3.0.5 3.1.1 8.35 diff --git a/tez-ui/pom.xml b/tez-ui/pom.xml index 80cb05944b..ea4b520222 100644 --- a/tez-ui/pom.xml +++ b/tez-ui/pom.xml @@ -374,7 +374,7 @@ ${nodeVersion} - v0.21.3 + v1.6.0 diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json index ad3aa74c5d..613660211b 100644 --- a/tez-ui/src/main/webapp/package.json +++ b/tez-ui/src/main/webapp/package.json @@ -62,5 +62,8 @@ }, "dependencies": { "em-tgraph": "0.0.14" + }, + "resolutions": { + "**/form-data/async": "2.6.4" } } diff --git a/tez-ui/src/main/webapp/yarn.lock b/tez-ui/src/main/webapp/yarn.lock index 660ac80d87..a8c96491d5 100644 --- a/tez-ui/src/main/webapp/yarn.lock +++ b/tez-ui/src/main/webapp/yarn.lock @@ -212,16 +212,16 @@ async@0.9.0: version "0.9.0" resolved "https://registry.yarnpkg.com/async/-/async-0.9.0.tgz#ac3613b1da9bed1b47510bb4651b8931e47146c7" +async@2.6.4, async@^2.0.1: + version "2.6.4" + resolved "https://registry.yarnpkg.com/async/-/async-2.6.4.tgz#706b7ff6084664cd7eae713f6f965433b5504221" + dependencies: + lodash "^4.17.14" + async@^1.0.0: version "1.5.2" resolved "https://registry.yarnpkg.com/async/-/async-1.5.2.tgz#ec6a61ae56480c0c3cb241c95618e20892f9672a" -async@^2.0.1: - version "2.3.0" - resolved "https://registry.yarnpkg.com/async/-/async-2.3.0.tgz#1013d1051047dd320fe24e494d5c66ecaf6147d9" - dependencies: - lodash "^4.14.0" - async@~0.2.6, async@~0.2.9: version "0.2.10" resolved "https://registry.yarnpkg.com/async/-/async-0.2.10.tgz#b6bbe0b0674b9d719708ca38de8c237cb526c3d1" @@ -3115,9 +3115,9 @@ lodash@^3.10.0, lodash@^3.6.0, lodash@^3.9.3: version "3.10.1" resolved "https://registry.yarnpkg.com/lodash/-/lodash-3.10.1.tgz#5bf45e8e49ba4189e17d482789dfd15bd140b7b6" -lodash@^4.14.0: - version "4.17.4" - resolved "https://registry.yarnpkg.com/lodash/-/lodash-4.17.4.tgz#78203a4d1c328ae1d86dca6460e369b57f4055ae" +lodash@^4.17.14: + version "4.17.21" + resolved "https://registry.yarnpkg.com/lodash/-/lodash-4.17.21.tgz#679591c564c3bffaae8454cf0b3df370c3d6911c" lodash@~2.3.0: version "2.3.0" From c6eb047aa2df2b60a673c2f852594295b5cb416a Mon Sep 17 00:00:00 2001 From: Aman Raj <104416558+amanraj2520@users.noreply.github.com> Date: Wed, 15 Jun 2022 18:52:43 +0530 Subject: [PATCH 380/512] [TEZ-4423][CVE-2021-44906] Upgrade minimist version from 0.0.8 to 1.2.6 to fix the vulnerability (#220) --- tez-ui/src/main/webapp/package.json | 4 +++- tez-ui/src/main/webapp/yarn.lock | 6 +++--- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json index 613660211b..47ac36c110 100644 --- a/tez-ui/src/main/webapp/package.json +++ b/tez-ui/src/main/webapp/package.json @@ -64,6 +64,8 @@ "em-tgraph": "0.0.14" }, "resolutions": { - "**/form-data/async": "2.6.4" + "**/form-data/async": "2.6.4", + "**/mkdirp/minimist": "1.2.6", + "**/optimist/minimist": "1.2.6" } } diff --git a/tez-ui/src/main/webapp/yarn.lock b/tez-ui/src/main/webapp/yarn.lock index a8c96491d5..32030ce87a 100644 --- a/tez-ui/src/main/webapp/yarn.lock +++ b/tez-ui/src/main/webapp/yarn.lock @@ -3290,9 +3290,9 @@ minimatch@~0.2.9: lru-cache "2" sigmund "~1.0.0" -minimist@0.0.8, minimist@~0.0.1: - version "0.0.8" - resolved "https://registry.yarnpkg.com/minimist/-/minimist-0.0.8.tgz#857fcabfc3397d2625b8228262e86aa7a011b05d" +minimist@0.0.8, minimist@1.2.6, minimist@~0.0.1: + version "1.2.6" + resolved "https://registry.yarnpkg.com/minimist/-/minimist-1.2.6.tgz#8637a5b759ea0d6e98702cfb3a9283323c93af44" minimist@^1.1.0, minimist@^1.1.1: version "1.2.0" From 8ffa32df467c115ef6358ceef9c1703147d10a6a Mon Sep 17 00:00:00 2001 From: Aman Raj <104416558+amanraj2520@users.noreply.github.com> Date: Mon, 20 Jun 2022 12:15:25 +0530 Subject: [PATCH 381/512] [TEZ-4424][CVE-2021-3918] Upgrade json-schema from 0.2.3 to 0.4.0 to fix the vulnerability (#222) (Aman Raj reviewed by Laszlo Bodor) --- tez-ui/src/main/webapp/package.json | 3 ++- tez-ui/src/main/webapp/yarn.lock | 6 +++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json index 47ac36c110..c5e8e8d6f0 100644 --- a/tez-ui/src/main/webapp/package.json +++ b/tez-ui/src/main/webapp/package.json @@ -66,6 +66,7 @@ "resolutions": { "**/form-data/async": "2.6.4", "**/mkdirp/minimist": "1.2.6", - "**/optimist/minimist": "1.2.6" + "**/optimist/minimist": "1.2.6", + "**/jsprim/json-schema": "0.4.0" } } diff --git a/tez-ui/src/main/webapp/yarn.lock b/tez-ui/src/main/webapp/yarn.lock index 32030ce87a..b9f1220508 100644 --- a/tez-ui/src/main/webapp/yarn.lock +++ b/tez-ui/src/main/webapp/yarn.lock @@ -2789,9 +2789,9 @@ json-parse-helpfulerror@^1.0.2: dependencies: jju "^1.1.0" -json-schema@0.2.3: - version "0.2.3" - resolved "https://registry.yarnpkg.com/json-schema/-/json-schema-0.2.3.tgz#b480c892e59a2f05954ce727bd3f2a4e882f9e13" +json-schema@0.2.3, json-schema@0.4.0: + version "0.4.0" + resolved "https://registry.yarnpkg.com/json-schema/-/json-schema-0.4.0.tgz#f7de4cf6efab838ebaeb3236474cbba5a1930ab5" json-stable-stringify@^1.0.0, json-stable-stringify@^1.0.1: version "1.0.1" From 1ab7fc97c91cd9db5c8150e46ee7f98375bcced9 Mon Sep 17 00:00:00 2001 From: Aman Raj <104416558+amanraj2520@users.noreply.github.com> Date: Tue, 21 Jun 2022 00:41:30 +0530 Subject: [PATCH 382/512] [TEZ-4425][WS-2020-0345] Upgrade jsonpointer version from 4.0.1 to 4.1.0 (#226) (Aman Raj reviewed by Laszlo Bodor) --- tez-ui/src/main/webapp/package.json | 3 ++- tez-ui/src/main/webapp/yarn.lock | 6 +++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json index c5e8e8d6f0..4e4aff67eb 100644 --- a/tez-ui/src/main/webapp/package.json +++ b/tez-ui/src/main/webapp/package.json @@ -67,6 +67,7 @@ "**/form-data/async": "2.6.4", "**/mkdirp/minimist": "1.2.6", "**/optimist/minimist": "1.2.6", - "**/jsprim/json-schema": "0.4.0" + "**/jsprim/json-schema": "0.4.0", + "jsonpointer": "4.1.0" } } diff --git a/tez-ui/src/main/webapp/yarn.lock b/tez-ui/src/main/webapp/yarn.lock index b9f1220508..a3734301f2 100644 --- a/tez-ui/src/main/webapp/yarn.lock +++ b/tez-ui/src/main/webapp/yarn.lock @@ -2821,9 +2821,9 @@ jsonify@~0.0.0: version "0.0.0" resolved "https://registry.yarnpkg.com/jsonify/-/jsonify-0.0.0.tgz#2c74b6ee41d93ca51b7b5aaee8f503631d252a73" -jsonpointer@^4.0.0: - version "4.0.1" - resolved "https://registry.yarnpkg.com/jsonpointer/-/jsonpointer-4.0.1.tgz#4fd92cb34e0e9db3c89c8622ecf51f9b978c6cb9" +jsonpointer@4.1.0, jsonpointer@^4.0.0: + version "4.1.0" + resolved "https://registry.yarnpkg.com/jsonpointer/-/jsonpointer-4.1.0.tgz#501fb89986a2389765ba09e6053299ceb4f2c2cc" jsprim@^1.2.2: version "1.4.0" From 5f181ea32a630f583c2830101dd4969e6ab8d77d Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Tue, 21 Jun 2022 00:43:55 +0530 Subject: [PATCH 383/512] TEZ-4429: Upgrade guava to 31.1 to fix CVE-2020-8908. (#223) (Ayush Saxena reviewed by Laszlo Bodor) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 9fd644bc9a..ebc1198d6b 100644 --- a/pom.xml +++ b/pom.xml @@ -57,7 +57,7 @@ https://repository.apache.org/service/local/staging/deploy/maven2 ${user.home}/clover.license - 27.0-jre + 31.1-jre 3.3.1 4.1.72.Final 0.13.0 From f25831ee40904813f31b65f794228b58a4800b8e Mon Sep 17 00:00:00 2001 From: Mark Bathori <57758037+mbathori-cloudera@users.noreply.github.com> Date: Tue, 21 Jun 2022 09:34:59 +0200 Subject: [PATCH 384/512] TEZ-4428: Use protoc-jar-maven-plugin to generate protobuf classes (#218) (Mark Bathori reviewed by Martin Tzvetanov Grigorov, Sylwester Lachiewicz, Laszlo Bodor) --- .travis.yml | 3 - BUILDING.txt | 9 +- build-tools/docker/Dockerfile | 2 - build-tools/install-protobuf.sh | 88 ------------------- build-tools/protocw | 32 ------- pom.xml | 22 ----- tez-api/pom.xml | 53 +++-------- tez-dag/pom.xml | 35 ++++---- tez-ext-service-tests/pom.xml | 31 +++---- tez-mapreduce/pom.xml | 29 +++--- .../tez-protobuf-history-plugin/pom.xml | 44 +++------- tez-runtime-internals/pom.xml | 29 +++--- tez-runtime-library/pom.xml | 31 +++---- 13 files changed, 109 insertions(+), 299 deletions(-) delete mode 100755 build-tools/install-protobuf.sh delete mode 100755 build-tools/protocw diff --git a/.travis.yml b/.travis.yml index 3637a0defe..65eaf77783 100644 --- a/.travis.yml +++ b/.travis.yml @@ -22,9 +22,6 @@ sudo: required env: MAVEN_OPTS="-Xmx2G -XX:MaxPermSize=512M" -before_install: - - ./build-tools/install-protobuf.sh - script: - jdk_switcher use oraclejdk8 - mvn -B clean install package -DskipTests=true -Dmaven.javadoc.skip=true diff --git a/BUILDING.txt b/BUILDING.txt index ae81d9c6c8..8b89a559de 100644 --- a/BUILDING.txt +++ b/BUILDING.txt @@ -122,10 +122,11 @@ The version of Protocol Buffer compiler, protoc, can be defined on-the-fly as: The default version is defined in the root pom.xml. -If you have multiple versions of protoc in your system, you can set in your -build shell the PROTOC_PATH environment variable to point to the one you -want to use for the Tez build. If you don't define this environment variable, -protoc is looked up in the PATH. +If you have multiple versions of protoc in your system, you can set in your +build shell the PROTOC_PATH environment variable to point to the one you +want to use for the Tez build. If you don't define this environment variable then the +embedded protoc compiler will be used with the version defined in ${protobuf.version}. +It detects the platform and executes the corresponding protoc binary at build time. You can also specify the path to protoc while building using -Dprotoc.path diff --git a/build-tools/docker/Dockerfile b/build-tools/docker/Dockerfile index 3e792604ec..f51cc3bd2b 100644 --- a/build-tools/docker/Dockerfile +++ b/build-tools/docker/Dockerfile @@ -55,8 +55,6 @@ RUN apt-get -q update && apt-get -q install --no-install-recommends -y \ rsync \ software-properties-common \ ssh-client \ - sudo \ - wget \ xz-utils \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* diff --git a/build-tools/install-protobuf.sh b/build-tools/install-protobuf.sh deleted file mode 100755 index c28729a8a7..0000000000 --- a/build-tools/install-protobuf.sh +++ /dev/null @@ -1,88 +0,0 @@ -#!/bin/bash - -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -# This script attempts to install an arbitrary version of protobuf if needed. -# The desired version should be the first parameter: $1. -# Typical usage: the script is automatically called from tez-api (by maven) during the build process. - -# This script runs from build-tools folder. The user can remove -# the dynamically installed protobuf anytime like: -# rm -rf ./build-tools/protobuf/ #from root folder - -set -x -PROTOBUF_VERSION=${1:-2.5.0} -PROTOBUF_MAJOR_VERSION=$(echo "$PROTOBUF_VERSION" | cut -d. -f1) -if [ -n "$ZSH_VERSION" ]; then - SCRIPT_DIR="${0:a:h}" -else - SCRIPT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" -fi - -function install_protobuf { - # before protobuf 3, there is no pre-compiled executables are host on github, let's try to build and make it - if (( PROTOBUF_MAJOR_VERSION < 3 )); then - wget "https://github.com/google/protobuf/releases/download/v$PROTOBUF_VERSION/protobuf-$PROTOBUF_VERSION.tar.gz" - tar -xzvf "protobuf-$PROTOBUF_VERSION.tar.gz" - rm "protobuf-$PROTOBUF_VERSION.tar.gz" - cd "protobuf-$PROTOBUF_VERSION" && ./configure --prefix=/usr && make && sudo make install - # since protobuf 3, there are precompiled protoc executables on github, let's quickly download and use it - else - ARCH=`uname -m` - case "$(uname -s)" in - Darwin) - FILE_NAME="protoc-$PROTOBUF_VERSION-osx-$ARCH" - ;; - Linux) - if test $ARCH = "aarch64"; then - ARCH="aarch_64" - fi - FILE_NAME="protoc-$PROTOBUF_VERSION-linux-$ARCH" - ;; - *) - echo "Unsupported OS returned by uname -s, you'll have to install protobuf 3.x manually" - exit 1 - ;; - esac - rm -f "$FILE_NAME.zip" #cleanup unfinished file if any - wget "https://github.com/google/protobuf/releases/download/v$PROTOBUF_VERSION/$FILE_NAME.zip" - mkdir "$SCRIPT_DIR/protobuf" - unzip -o "$FILE_NAME.zip" -d "$SCRIPT_DIR/protobuf" - rm "$FILE_NAME.zip" - fi -} - -if test -f "$SCRIPT_DIR/protobuf/bin/protoc"; then - PROTOBUF_INSTALLED_VERSION=$("$SCRIPT_DIR/protobuf/bin/protoc" --version) -else - PROTOBUF_INSTALLED_VERSION=$(protoc --version) -fi - -PROTOC_EXIT_CODE=$? - -if [ $PROTOC_EXIT_CODE -eq 0 ]; then - PROTOBUF_INSTALLED_VERSION=$(echo "$PROTOBUF_INSTALLED_VERSION" | tr -s ' ' | cut -d ' ' -f 2) - if [ "$PROTOBUF_INSTALLED_VERSION" == "$PROTOBUF_VERSION" ]; then - echo "Current protobuf version is equal to the requested ($PROTOBUF_INSTALLED_VERSION), exiting..." - else - echo "Current protobuf version ($PROTOBUF_INSTALLED_VERSION) is not equal to the requested ($PROTOBUF_VERSION), installing $PROTOBUF_VERSION" - install_protobuf - fi -else - echo "protoc --version command had non-zero return value, need to install probuf" - install_protobuf -fi diff --git a/build-tools/protocw b/build-tools/protocw deleted file mode 100755 index 6196071815..0000000000 --- a/build-tools/protocw +++ /dev/null @@ -1,32 +0,0 @@ -#!/bin/bash - -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -### This is a protoc wrapper for tez, which can dinamically call protoc from a downloaded protobuf. - -if [ -n "$ZSH_VERSION" ]; then - SCRIPT_DIR="${0:a:h}" -else - SCRIPT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" -fi - -if test -f "$SCRIPT_DIR/protobuf/bin/protoc"; then - "$SCRIPT_DIR/protobuf/bin/protoc" "$@" -else - protoc "$@" -fi -exit $? diff --git a/pom.xml b/pom.xml index ebc1198d6b..5f381cdd27 100644 --- a/pom.xml +++ b/pom.xml @@ -1310,28 +1310,6 @@
- - - protoc-path-env-variable-not-defined - - - !env.PROTOC_PATH - - - - ${basedir}/../build-tools/protocw - - diff --git a/tez-api/pom.xml b/tez-api/pom.xml index cd5abdb03a..1d10bc1f85 100644 --- a/tez-api/pom.xml +++ b/tez-api/pom.xml @@ -145,53 +145,28 @@ org.apache.rat apache-rat-plugin - - exec-maven-plugin - org.codehaus.mojo - 1.6.0 + com.github.os72 + protoc-jar-maven-plugin + 3.11.4 - Install protobuf - initialize - - exec - - - ${basedir}/../build-tools/install-protobuf.sh - ${protobuf.version} - - - - - - org.apache.hadoop - hadoop-maven-plugins - - - compile-protoc generate-sources - protoc + run - ${protobuf.version} + com.google.protobuf:protoc:${protobuf.version} ${protoc.path} - - ${basedir}/src/main/proto - - - ${basedir}/src/main/proto - - DAGApiRecords.proto - DAGClientAMProtocol.proto - Events.proto - - - ${project.build.directory}/generated-sources/java + none + + ${basedir}/src/main/proto + + + + ${project.build.directory}/generated-sources/java + + diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml index 356ed2c2e9..fda9fb2a54 100644 --- a/tez-dag/pom.xml +++ b/tez-dag/pom.xml @@ -211,30 +211,31 @@ - org.apache.hadoop - hadoop-maven-plugins + com.github.os72 + protoc-jar-maven-plugin + 3.11.4 - compile-protoc generate-sources - protoc + run - ${protobuf.version} + com.google.protobuf:protoc:${protobuf.version} ${protoc.path} - - ${basedir}/src/main/proto - ${basedir}/../tez-api/src/main/proto - ${basedir}/../tez-runtime-internals/src/main/proto - - - ${basedir}/src/main/proto - - HistoryEvents.proto - - - ${project.build.directory}/generated-sources/java + none + + ${basedir}/../tez-api/src/main/proto + ${basedir}/../tez-runtime-internals/src/main/proto + + + ${basedir}/src/main/proto + + + + ${project.build.directory}/generated-sources/java + + diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml index 609b896fd8..a0bcbe32c0 100644 --- a/tez-ext-service-tests/pom.xml +++ b/tez-ext-service-tests/pom.xml @@ -165,29 +165,30 @@ apache-rat-plugin - org.apache.hadoop - hadoop-maven-plugins + com.github.os72 + protoc-jar-maven-plugin + 3.11.4 - compile-protoc generate-sources - protoc + run - ${protobuf.version} + com.google.protobuf:protoc:${protobuf.version} ${protoc.path} - - ${basedir}/src/test/proto + none + ${basedir}/../tez-api/src/main/proto - - - ${basedir}/src/test/proto - - TezDaemonProtocol.proto - - - ${project.build.directory}/generated-test-sources/java + + + ${basedir}/src/test/proto + + + + ${project.build.directory}/generated-test-sources/java + + diff --git a/tez-mapreduce/pom.xml b/tez-mapreduce/pom.xml index a22870fc38..1e35d2ea11 100644 --- a/tez-mapreduce/pom.xml +++ b/tez-mapreduce/pom.xml @@ -137,28 +137,27 @@ apache-rat-plugin - org.apache.hadoop - hadoop-maven-plugins + com.github.os72 + protoc-jar-maven-plugin + 3.11.4 - compile-protoc generate-sources - protoc + run - ${protobuf.version} + com.google.protobuf:protoc:${protobuf.version} ${protoc.path} - - ${basedir}/src/main/proto - - - ${basedir}/src/main/proto - - MRRuntimeProtos.proto - - - ${project.build.directory}/generated-sources/java + none + + ${basedir}/src/main/proto + + + + ${project.build.directory}/generated-sources/java + + diff --git a/tez-plugins/tez-protobuf-history-plugin/pom.xml b/tez-plugins/tez-protobuf-history-plugin/pom.xml index 607a31f95b..0fa526413e 100644 --- a/tez-plugins/tez-protobuf-history-plugin/pom.xml +++ b/tez-plugins/tez-protobuf-history-plugin/pom.xml @@ -59,47 +59,31 @@ apache-rat-plugin - org.apache.hadoop - hadoop-maven-plugins + com.github.os72 + protoc-jar-maven-plugin + 3.11.4 - compile-protoc generate-sources - protoc + run - ${protobuf.version} + com.google.protobuf:protoc:${protobuf.version} ${protoc.path} - - ${basedir}/src/main/proto - - - ${basedir}/src/main/proto - - HistoryLogger.proto - - - ${project.build.directory}/generated-sources/java + none + + ${basedir}/src/main/proto + + + + ${project.build.directory}/generated-sources/java + + - - - - - protoc-path-env-variable-not-defined - - - !env.PROTOC_PATH - - - - ${basedir}/../../build-tools/protocw - - - diff --git a/tez-runtime-internals/pom.xml b/tez-runtime-internals/pom.xml index 0619ca1e16..efe01e058e 100644 --- a/tez-runtime-internals/pom.xml +++ b/tez-runtime-internals/pom.xml @@ -91,28 +91,27 @@ apache-rat-plugin - org.apache.hadoop - hadoop-maven-plugins + com.github.os72 + protoc-jar-maven-plugin + 3.11.4 - compile-protoc generate-sources - protoc + run - ${protobuf.version} + com.google.protobuf:protoc:${protobuf.version} ${protoc.path} - - ${basedir}/src/main/proto - - - ${basedir}/src/main/proto - - RuntimeEvents.proto - - - ${project.build.directory}/generated-sources/java + none + + ${basedir}/src/main/proto + + + + ${project.build.directory}/generated-sources/java + + diff --git a/tez-runtime-library/pom.xml b/tez-runtime-library/pom.xml index c913e87d43..e6db47e535 100644 --- a/tez-runtime-library/pom.xml +++ b/tez-runtime-library/pom.xml @@ -114,30 +114,27 @@ apache-rat-plugin - org.apache.hadoop - hadoop-maven-plugins + com.github.os72 + protoc-jar-maven-plugin + 3.11.4 - compile-protoc generate-sources - protoc + run - ${protobuf.version} + com.google.protobuf:protoc:${protobuf.version} ${protoc.path} - - ${basedir}/src/main/proto - - - ${basedir}/src/main/proto - - ShufflePayloads.proto - CartesianProductPayload.proto - FairShufflePayloads.proto - - - ${project.build.directory}/generated-sources/java + none + + ${basedir}/src/main/proto + + + + ${project.build.directory}/generated-sources/java + + From adcc3138d3f79b056131bb2c7864a2b9a39848ea Mon Sep 17 00:00:00 2001 From: Aman Raj <104416558+amanraj2520@users.noreply.github.com> Date: Tue, 21 Jun 2022 13:11:57 +0530 Subject: [PATCH 385/512] TEZ-4426: [CVE-2018-1000620] Upgrade cryptiles from 2.0.5 to 4.1.2 (#227) (Aman Raj reviewed by Nikhil Gupta, Laszlo Bodor) --- tez-ui/pom.xml | 2 +- tez-ui/src/main/webapp/package.json | 3 ++- tez-ui/src/main/webapp/yarn.lock | 18 ++++++++++++++---- 3 files changed, 17 insertions(+), 6 deletions(-) diff --git a/tez-ui/pom.xml b/tez-ui/pom.xml index ea4b520222..59a02a2ab0 100644 --- a/tez-ui/pom.xml +++ b/tez-ui/pom.xml @@ -29,7 +29,7 @@ src/main/webapp - v5.12.0 + v8.9.0 ${basedir}/src/main/webapp/node/node node/yarn/dist/bin/yarn.js diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json index 4e4aff67eb..f79e99e589 100644 --- a/tez-ui/src/main/webapp/package.json +++ b/tez-ui/src/main/webapp/package.json @@ -68,6 +68,7 @@ "**/mkdirp/minimist": "1.2.6", "**/optimist/minimist": "1.2.6", "**/jsprim/json-schema": "0.4.0", - "jsonpointer": "4.1.0" + "jsonpointer": "4.1.0", + "cryptiles": "4.1.2" } } diff --git a/tez-ui/src/main/webapp/yarn.lock b/tez-ui/src/main/webapp/yarn.lock index a3734301f2..3a1513e314 100644 --- a/tez-ui/src/main/webapp/yarn.lock +++ b/tez-ui/src/main/webapp/yarn.lock @@ -473,6 +473,12 @@ boom@2.x.x: dependencies: hoek "2.x.x" +boom@7.x.x: + version "7.3.0" + resolved "https://registry.yarnpkg.com/boom/-/boom-7.3.0.tgz#733a6d956d33b0b1999da3fe6c12996950d017b9" + dependencies: + hoek "6.x.x" + bower-config@0.6.1: version "0.6.1" resolved "https://registry.yarnpkg.com/bower-config/-/bower-config-0.6.1.tgz#7093155688bef44079bf4cb32d189312c87ded60" @@ -1180,11 +1186,11 @@ cross-spawn-async@^2.0.0: lru-cache "^4.0.0" which "^1.2.8" -cryptiles@2.x.x: - version "2.0.5" - resolved "https://registry.yarnpkg.com/cryptiles/-/cryptiles-2.0.5.tgz#3bdfecdc608147c1c67202fa291e7dca59eaa3b8" +cryptiles@2.x.x, cryptiles@4.1.2: + version "4.1.2" + resolved "https://registry.yarnpkg.com/cryptiles/-/cryptiles-4.1.2.tgz#363c9ab5c859da9d2d6fb901b64d980966181184" dependencies: - boom "2.x.x" + boom "7.x.x" ctype@0.5.3: version "0.5.3" @@ -2472,6 +2478,10 @@ hoek@2.x.x: version "2.16.3" resolved "https://registry.yarnpkg.com/hoek/-/hoek-2.16.3.tgz#20bb7403d3cea398e91dc4710a8ff1b8274a25ed" +hoek@6.x.x: + version "6.1.3" + resolved "https://registry.yarnpkg.com/hoek/-/hoek-6.1.3.tgz#73b7d33952e01fe27a38b0457294b79dd8da242c" + home-or-tmp@^1.0.0: version "1.0.0" resolved "https://registry.yarnpkg.com/home-or-tmp/-/home-or-tmp-1.0.0.tgz#4b9f1e40800c3e50c6c27f781676afcce71f3985" From be40104104796e265947b6c9666a5e1cd3ca848e Mon Sep 17 00:00:00 2001 From: Aman Raj <104416558+amanraj2520@users.noreply.github.com> Date: Thu, 30 Jun 2022 15:47:03 +0530 Subject: [PATCH 386/512] [TEZ-4427][CVE-2019-10744] Upgrade lodash.merge version to 4.6.2 to fix vulnerability (#229) (Aman Raj reviewed by Nikhil Gupta, Laszlo Bodor) --- tez-ui/src/main/webapp/package.json | 3 +- tez-ui/src/main/webapp/yarn.lock | 56 ++--------------------------- 2 files changed, 5 insertions(+), 54 deletions(-) diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json index f79e99e589..3500b8aaff 100644 --- a/tez-ui/src/main/webapp/package.json +++ b/tez-ui/src/main/webapp/package.json @@ -69,6 +69,7 @@ "**/optimist/minimist": "1.2.6", "**/jsprim/json-schema": "0.4.0", "jsonpointer": "4.1.0", - "cryptiles": "4.1.2" + "cryptiles": "4.1.2", + "lodash.merge": "4.6.2" } } diff --git a/tez-ui/src/main/webapp/yarn.lock b/tez-ui/src/main/webapp/yarn.lock index 3a1513e314..2df3036ac9 100644 --- a/tez-ui/src/main/webapp/yarn.lock +++ b/tez-ui/src/main/webapp/yarn.lock @@ -2926,14 +2926,6 @@ lodash-node@^3.4.0: version "3.10.2" resolved "https://registry.yarnpkg.com/lodash-node/-/lodash-node-3.10.2.tgz#2598d5b1b54e6a68b4cb544e5c730953cbf632f7" -lodash._arraycopy@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/lodash._arraycopy/-/lodash._arraycopy-3.0.0.tgz#76e7b7c1f1fb92547374878a562ed06a3e50f6e1" - -lodash._arrayeach@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/lodash._arrayeach/-/lodash._arrayeach-3.0.0.tgz#bab156b2a90d3f1bbd5c653403349e5e5933ef9e" - lodash._baseassign@^3.0.0: version "3.2.0" resolved "https://registry.yarnpkg.com/lodash._baseassign/-/lodash._baseassign-3.2.0.tgz#8c38a099500f215ad09e59f1722fd0c52bfe0a4e" @@ -2954,10 +2946,6 @@ lodash._basecopy@^3.0.0: version "3.0.1" resolved "https://registry.yarnpkg.com/lodash._basecopy/-/lodash._basecopy-3.0.1.tgz#8da0e6a876cf344c0ad8a54882111dd3c5c7ca36" -lodash._basefor@^3.0.0: - version "3.0.3" - resolved "https://registry.yarnpkg.com/lodash._basefor/-/lodash._basefor-3.0.3.tgz#7550b4e9218ef09fad24343b612021c79b4c20c2" - lodash._baseindexof@^3.0.0: version "3.1.0" resolved "https://registry.yarnpkg.com/lodash._baseindexof/-/lodash._baseindexof-3.1.0.tgz#fe52b53a1c6761e42618d654e4a25789ed61822c" @@ -3031,14 +3019,6 @@ lodash.isarray@^3.0.0: version "3.0.4" resolved "https://registry.yarnpkg.com/lodash.isarray/-/lodash.isarray-3.0.4.tgz#79e4eb88c36a8122af86f844aa9bcd851b5fbb55" -lodash.isplainobject@^3.0.0: - version "3.2.0" - resolved "https://registry.yarnpkg.com/lodash.isplainobject/-/lodash.isplainobject-3.2.0.tgz#9a8238ae16b200432960cd7346512d0123fbf4c5" - dependencies: - lodash._basefor "^3.0.0" - lodash.isarguments "^3.0.0" - lodash.keysin "^3.0.0" - lodash.istypedarray@^3.0.0: version "3.0.6" resolved "https://registry.yarnpkg.com/lodash.istypedarray/-/lodash.istypedarray-3.0.6.tgz#c9a477498607501d8e8494d283b87c39281cef62" @@ -3051,32 +3031,9 @@ lodash.keys@^3.0.0: lodash.isarguments "^3.0.0" lodash.isarray "^3.0.0" -lodash.keysin@^3.0.0: - version "3.0.8" - resolved "https://registry.yarnpkg.com/lodash.keysin/-/lodash.keysin-3.0.8.tgz#22c4493ebbedb1427962a54b445b2c8a767fb47f" - dependencies: - lodash.isarguments "^3.0.0" - lodash.isarray "^3.0.0" - -lodash.merge@^3.0.2, lodash.merge@^3.3.2: - version "3.3.2" - resolved "https://registry.yarnpkg.com/lodash.merge/-/lodash.merge-3.3.2.tgz#0d90d93ed637b1878437bb3e21601260d7afe994" - dependencies: - lodash._arraycopy "^3.0.0" - lodash._arrayeach "^3.0.0" - lodash._createassigner "^3.0.0" - lodash._getnative "^3.0.0" - lodash.isarguments "^3.0.0" - lodash.isarray "^3.0.0" - lodash.isplainobject "^3.0.0" - lodash.istypedarray "^3.0.0" - lodash.keys "^3.0.0" - lodash.keysin "^3.0.0" - lodash.toplainobject "^3.0.0" - -lodash.merge@^4.5.1: - version "4.6.0" - resolved "https://registry.yarnpkg.com/lodash.merge/-/lodash.merge-4.6.0.tgz#69884ba144ac33fe699737a6086deffadd0f89c5" +lodash.merge@4.6.2, lodash.merge@^3.0.2, lodash.merge@^3.3.2, lodash.merge@^4.5.1: + version "4.6.2" + resolved "https://registry.yarnpkg.com/lodash.merge/-/lodash.merge-4.6.2.tgz#558aa53b43b661e1925a0afdfa36a9a1085fe57a" lodash.pad@^4.1.0: version "4.5.1" @@ -3100,13 +3057,6 @@ lodash.restparam@^3.0.0: version "3.6.1" resolved "https://registry.yarnpkg.com/lodash.restparam/-/lodash.restparam-3.6.1.tgz#936a4e309ef330a7645ed4145986c85ae5b20805" -lodash.toplainobject@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/lodash.toplainobject/-/lodash.toplainobject-3.0.0.tgz#28790ad942d293d78aa663a07ecf7f52ca04198d" - dependencies: - lodash._basecopy "^3.0.0" - lodash.keysin "^3.0.0" - lodash.uniq@^3.2.2: version "3.2.2" resolved "https://registry.yarnpkg.com/lodash.uniq/-/lodash.uniq-3.2.2.tgz#146c36f25e75d19501ba402e88ba14937f63cd8b" From 06fff5c9cec13bc656c9c2e1678ae8a8e2271208 Mon Sep 17 00:00:00 2001 From: Ganesha Shreedhara Date: Mon, 4 Jul 2022 14:08:37 +0530 Subject: [PATCH 387/512] TEZ-4430: Fix tez.task.launch.cmd-opts property not working (#228) (Ganesha Shreedhara reviewed by Laszlo Bodor) --- .../src/main/java/org/apache/tez/client/TezClientUtils.java | 3 ++- .../test/java/org/apache/tez/client/TestTezClientUtils.java | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index d0fc374a81..d78ccf3e89 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -29,6 +29,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -767,7 +768,7 @@ static void maybeAddDefaultLoggingJavaOpts(String logLevel, List vargs) public static String maybeAddDefaultLoggingJavaOpts(String logLevel, String javaOpts) { List vargs = new ArrayList(5); if (javaOpts != null) { - vargs.add(javaOpts); + Collections.addAll(vargs, javaOpts.split(" ")); } else { vargs.add(""); } diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java index 29a6769716..0341f27f66 100644 --- a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java +++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java @@ -684,12 +684,13 @@ public void testDefaultLoggingJavaOpts() { @Test public void testDefaultLoggingJavaOptsWithRootLogger() { - String origJavaOpts = "-D" + TezConstants.TEZ_ROOT_LOGGER_NAME + "=INFO"; + String origJavaOpts = "-D" + TezConstants.TEZ_ROOT_LOGGER_NAME + "=INFO -DtestProperty=value"; String javaOpts = TezClientUtils.maybeAddDefaultLoggingJavaOpts("FOOBAR", origJavaOpts); Assert.assertNotNull(javaOpts); Assert.assertTrue(javaOpts.contains("-D" + TezConstants.TEZ_ROOT_LOGGER_NAME + "=FOOBAR")); Assert.assertTrue(javaOpts.contains(TezConstants.TEZ_CONTAINER_LOG4J_PROPERTIES_FILE) && javaOpts.contains("-Dlog4j.configuratorClass=org.apache.tez.common.TezLog4jConfigurator")); + Assert.assertTrue(javaOpts.contains("-DtestProperty=value")); } @Test (timeout = 5000) From a192ec4443032c33269f5326755a292c87a98292 Mon Sep 17 00:00:00 2001 From: Laszlo Bodor Date: Mon, 4 Jul 2022 11:04:34 +0200 Subject: [PATCH 388/512] Revert "TEZ-4397 Open Tez Input splits asynchronously" This reverts commit f724c546069885e29e6446813805bb63bf0d5d9d. --- .../split/TezGroupedSplitsInputFormat.java | 72 ++----------------- .../mapreduce/grouper/TezSplitGrouper.java | 11 --- 2 files changed, 4 insertions(+), 79 deletions(-) diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java index 6266ec1bcf..61ba560300 100644 --- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java +++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java @@ -19,16 +19,8 @@ package org.apache.hadoop.mapred.split; import java.io.IOException; -import java.util.LinkedList; import java.util.Objects; -import java.util.Queue; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicInteger; -import com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.apache.tez.mapreduce.grouper.TezSplitGrouper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience.Public; @@ -137,58 +129,14 @@ public class TezGroupedSplitsRecordReader implements RecordReader { int idx = 0; long progress; RecordReader curReader; - final AtomicInteger initIndex; - final int numReaders; - final ExecutorService initReaderExecService; - final Queue>> initedReaders; - + public TezGroupedSplitsRecordReader(TezGroupedSplit split, JobConf job, Reporter reporter) throws IOException { this.groupedSplit = split; this.job = job; this.reporter = reporter; - this.initIndex = new AtomicInteger(0); - int numThreads = conf.getInt(TezSplitGrouper.TEZ_GROUPING_SPLIT_INIT_THREADS, - TezSplitGrouper.TEZ_GROUPING_SPLIT_INIT_THREADS_DEFAULT); - this.numReaders = conf.getInt(TezSplitGrouper.TEZ_GROUPING_SPLIT_INIT_NUM_RECORDREADERS, - TezSplitGrouper.TEZ_GROUPING_SPLIT_INIT_NUM_RECORDREADERS_DEFAULT); - this.initReaderExecService = Executors.newFixedThreadPool(numThreads, - new ThreadFactoryBuilder() - .setDaemon(true) - .setPriority(Thread.MAX_PRIORITY) - .setNameFormat("TEZ-Split-Init-Thread-%d") - .build()); - this.initedReaders = new LinkedList<>(); - preInitReaders(); initNextRecordReader(); } - - private void preInitReaders() { - if (initReaderExecService == null) { - LOG.info("Init record reader threadpool is not initialized"); - return; - } - for (int i = 0; i < numReaders; i++) { - initedReaders.offer(this.initReaderExecService.submit(() -> { - try { - int index = initIndex.getAndIncrement(); - if (index >= groupedSplit.wrappedSplits.size()) { - return null; - } - InputSplit s = groupedSplit.wrappedSplits.get(index); - RecordReader reader = wrappedInputFormat.getRecordReader(s, job, reporter); - LOG.debug("Init Thread processed reader number {} initialization", index); - return reader; - } catch (Exception e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - cancelsFutures(); - throw new RuntimeException(e); - } - })); - } - } @Override public boolean next(K key, V value) throws IOException { @@ -235,8 +183,6 @@ protected boolean initNextRecordReader() throws IOException { // if all chunks have been processed, nothing more to do. if (idx == groupedSplit.wrappedSplits.size()) { - LOG.info("Shutting down the init record reader threadpool"); - initReaderExecService.shutdownNow(); return false; } @@ -247,25 +193,15 @@ protected boolean initNextRecordReader() throws IOException { // get a record reader for the idx-th chunk try { - curReader = initedReaders.poll().get(); - preInitReaders(); + curReader = wrappedInputFormat.getRecordReader( + groupedSplit.wrappedSplits.get(idx), job, reporter); } catch (Exception e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - cancelsFutures(); - throw new RuntimeException(e); + throw new RuntimeException (e); } idx++; return true; } - private void cancelsFutures() { - for (Future> f : initedReaders) { - f.cancel(true); - } - } - @Override public long getPos() throws IOException { long subprogress = 0; // bytes processed in current split diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java index 3b2f17d1ff..a1d6b6c806 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java @@ -102,17 +102,6 @@ public abstract class TezSplitGrouper { public static final String TEZ_GROUPING_NODE_LOCAL_ONLY = "tez.grouping.node.local.only"; public static final boolean TEZ_GROUPING_NODE_LOCAL_ONLY_DEFAULT = false; - /** - * Number of threads used to initialize the grouped splits, to asynchronously open the readers. - */ - public static final String TEZ_GROUPING_SPLIT_INIT_THREADS = "tez.grouping.split.init-threads"; - public static final int TEZ_GROUPING_SPLIT_INIT_THREADS_DEFAULT = 4; - - /** - * Number of record readers to asynchronously and proactively init. - */ - public static final String TEZ_GROUPING_SPLIT_INIT_NUM_RECORDREADERS = "tez.grouping.split.init.num-recordreaders"; - public static final int TEZ_GROUPING_SPLIT_INIT_NUM_RECORDREADERS_DEFAULT = 10; static class LocationHolder { List splits; From c386865e74d54a3a1b87a3e57b4cd14ccd332167 Mon Sep 17 00:00:00 2001 From: Mark Bathori <57758037+mark-bathori@users.noreply.github.com> Date: Fri, 22 Jul 2022 10:25:23 +0200 Subject: [PATCH 389/512] TEZ-4363: Bump protobuf dependency to 3.x (#192) (Mark Bathori reviewed by Laszlo Bodor, Aman Raj) --- BUILDING.txt | 2 +- build-tools/docker/Dockerfile | 8 ++++---- pom.xml | 2 +- .../apache/tez/dag/history/events/AMLaunchedEvent.java | 3 ++- .../org/apache/tez/dag/history/events/AMStartedEvent.java | 3 ++- .../tez/dag/history/events/ContainerLaunchedEvent.java | 4 +++- .../tez/dag/history/events/ContainerStoppedEvent.java | 4 +++- .../tez/dag/history/events/DAGCommitStartedEvent.java | 4 +++- .../apache/tez/dag/history/events/DAGFinishedEvent.java | 3 ++- .../tez/dag/history/events/DAGInitializedEvent.java | 4 +++- .../tez/dag/history/events/DAGKillRequestEvent.java | 4 +++- .../apache/tez/dag/history/events/DAGStartedEvent.java | 3 ++- .../apache/tez/dag/history/events/DAGSubmittedEvent.java | 3 ++- .../tez/dag/history/events/TaskAttemptFinishedEvent.java | 4 +++- .../tez/dag/history/events/TaskAttemptStartedEvent.java | 4 +++- .../apache/tez/dag/history/events/TaskFinishedEvent.java | 3 ++- .../apache/tez/dag/history/events/TaskStartedEvent.java | 3 ++- .../tez/dag/history/events/VertexCommitStartedEvent.java | 4 +++- .../dag/history/events/VertexConfigurationDoneEvent.java | 4 +++- .../tez/dag/history/events/VertexFinishedEvent.java | 4 +++- .../history/events/VertexGroupCommitFinishedEvent.java | 4 +++- .../dag/history/events/VertexGroupCommitStartedEvent.java | 4 +++- .../tez/dag/history/events/VertexInitializedEvent.java | 4 +++- .../apache/tez/dag/history/events/VertexStartedEvent.java | 3 ++- .../dag/history/logging/proto/ProtoMessageWritable.java | 3 ++- 25 files changed, 63 insertions(+), 28 deletions(-) diff --git a/BUILDING.txt b/BUILDING.txt index 8b89a559de..a917bfa1a2 100644 --- a/BUILDING.txt +++ b/BUILDING.txt @@ -9,7 +9,7 @@ Requirements: * JDK 1.8+ * Maven 3.1 or later * Findbugs 2.0.2 or later (if running findbugs) -* ProtocolBuffer 2.5.0 +* ProtocolBuffer 3.21.1 * Internet connection for first build (to fetch all dependencies) * Hadoop version should be 2.7.0 or higher. diff --git a/build-tools/docker/Dockerfile b/build-tools/docker/Dockerfile index f51cc3bd2b..a5f3e537aa 100644 --- a/build-tools/docker/Dockerfile +++ b/build-tools/docker/Dockerfile @@ -95,13 +95,13 @@ RUN add-apt-repository -y ppa:longsleep/golang-backports \ ############# ###### -# Install Google Protobuf 2.5.0 +# Install Google Protobuf 3.21.1 ###### -FROM tezbase AS protobuf250 +FROM tezbase AS protobuf SHELL ["/bin/bash", "-o", "pipefail", "-c"] RUN mkdir -p /opt/protobuf-src \ && curl -L -s -S \ - https://github.com/google/protobuf/releases/download/v2.5.0/protobuf-2.5.0.tar.gz \ + https://github.com/protocolbuffers/protobuf/releases/download/v21.1/protobuf-java-3.21.1.tar.gz \ -o /opt/protobuf.tar.gz \ && tar xzf /opt/protobuf.tar.gz --strip-components 1 -C /opt/protobuf-src WORKDIR /opt/protobuf-src @@ -261,7 +261,7 @@ RUN curl -sSL https://deb.nodesource.com/setup_14.x | bash - \ COPY --from=shellcheck /bin/shellcheck /bin/shellcheck COPY --from=hadolint /bin/hadolint /bin/hadolint COPY --from=buf /usr/local/bin/buf /usr/local/bin/buf -COPY --from=protobuf250 /opt/protobuf /opt/protobuf +COPY --from=protobuf /opt/protobuf /opt/protobuf ENV PROTOBUF_HOME /opt/protobuf ENV PROTOC_PATH /opt/protobuf/bin/protoc diff --git a/pom.xml b/pom.xml index 5f381cdd27..2e2cd8c128 100644 --- a/pom.xml +++ b/pom.xml @@ -63,7 +63,7 @@ 0.13.0 1.19 1.7.36 - 2.5.0 + 3.21.1 0.7.45 ${env.PROTOC_PATH} scm:git:https://gitbox.apache.org/repos/asf/tez.git diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMLaunchedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMLaunchedEvent.java index 001cbf0104..bb87fdb1fc 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMLaunchedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMLaunchedEvent.java @@ -22,6 +22,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.tez.dag.history.HistoryEvent; @@ -90,7 +91,7 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - AMLaunchedProto proto = inputStream.readMessage(AMLaunchedProto.PARSER, null); + AMLaunchedProto proto = inputStream.readMessage(AMLaunchedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMStartedEvent.java index 87daba68b8..6be528af96 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMStartedEvent.java @@ -22,6 +22,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.tez.dag.history.HistoryEvent; @@ -85,7 +86,7 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - AMStartedProto proto = inputStream.readMessage(AMStartedProto.PARSER, null); + AMStartedProto proto = inputStream.readMessage(AMStartedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerLaunchedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerLaunchedEvent.java index 11528e2f8c..9ec05926bb 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerLaunchedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerLaunchedEvent.java @@ -22,6 +22,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.util.ConverterUtils; @@ -83,7 +84,8 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - ContainerLaunchedProto proto = inputStream.readMessage(ContainerLaunchedProto.PARSER, null); + ContainerLaunchedProto proto = + inputStream.readMessage(ContainerLaunchedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerStoppedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerStoppedEvent.java index 528f6293b6..9360e49643 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerStoppedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerStoppedEvent.java @@ -22,6 +22,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.util.ConverterUtils; @@ -88,7 +89,8 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - ContainerStoppedProto proto = inputStream.readMessage(ContainerStoppedProto.PARSER, null); + ContainerStoppedProto proto = + inputStream.readMessage(ContainerStoppedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGCommitStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGCommitStartedEvent.java index 241dada2d6..694d9474ff 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGCommitStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGCommitStartedEvent.java @@ -23,6 +23,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; @@ -76,7 +77,8 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - DAGCommitStartedProto proto = inputStream.readMessage(DAGCommitStartedProto.PARSER, null); + DAGCommitStartedProto proto = + inputStream.readMessage(DAGCommitStartedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java index 6d1d6cd3ff..3364e1d2b2 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java @@ -24,6 +24,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.DagTypeConverters; @@ -129,7 +130,7 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - DAGFinishedProto proto = inputStream.readMessage(DAGFinishedProto.PARSER, null); + DAGFinishedProto proto = inputStream.readMessage(DAGFinishedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java index 4df116ad37..31a53f714a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java @@ -23,6 +23,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.records.DAGIDAware; @@ -91,7 +92,8 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - DAGInitializedProto proto = inputStream.readMessage(DAGInitializedProto.PARSER, null); + DAGInitializedProto proto = + inputStream.readMessage(DAGInitializedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGKillRequestEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGKillRequestEvent.java index c87f5ce7e7..b9e3da845c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGKillRequestEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGKillRequestEvent.java @@ -22,6 +22,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; @@ -76,7 +77,8 @@ public DAGKillRequestProto toProto() { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - DAGKillRequestProto proto = inputStream.readMessage(DAGKillRequestProto.PARSER, null); + DAGKillRequestProto proto = + inputStream.readMessage(DAGKillRequestProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java index 9e1f6c4a50..d49ad3706f 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java @@ -22,6 +22,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.app.dag.DAGState; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; @@ -81,7 +82,7 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - DAGStartedProto proto = inputStream.readMessage(DAGStartedProto.PARSER, null); + DAGStartedProto proto = inputStream.readMessage(DAGStartedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java index bed0f6f25a..ac9b923303 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java @@ -24,6 +24,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.records.DAGIDAware; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -134,7 +135,7 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - DAGSubmittedProto proto = inputStream.readMessage(DAGSubmittedProto.PARSER, null); + DAGSubmittedProto proto = inputStream.readMessage(DAGSubmittedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java index ce2eb729f6..41b0ed6643 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java @@ -24,6 +24,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.common.TezConverterUtils; import org.apache.tez.common.counters.CounterGroup; import org.apache.tez.common.counters.TezCounter; @@ -235,7 +236,8 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - TaskAttemptFinishedProto proto = inputStream.readMessage(TaskAttemptFinishedProto.PARSER, null); + TaskAttemptFinishedProto proto = + inputStream.readMessage(TaskAttemptFinishedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java index 10ba439e1c..1442ff4b43 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java @@ -22,6 +22,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.util.ConverterUtils; @@ -98,7 +99,8 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - TaskAttemptStartedProto proto = inputStream.readMessage(TaskAttemptStartedProto.PARSER, null); + TaskAttemptStartedProto proto = + inputStream.readMessage(TaskAttemptStartedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java index 215968274c..fd0a5fc6aa 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java @@ -22,6 +22,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.common.counters.CounterGroup; import org.apache.tez.common.counters.TezCounter; import org.apache.tez.dag.records.TaskIDAware; @@ -116,7 +117,7 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - TaskFinishedProto proto = inputStream.readMessage(TaskFinishedProto.PARSER, null); + TaskFinishedProto proto = inputStream.readMessage(TaskFinishedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java index 577aef963c..1379e04e19 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java @@ -22,6 +22,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.api.oldrecords.TaskState; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; @@ -90,7 +91,7 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - TaskStartedProto proto = inputStream.readMessage(TaskStartedProto.PARSER, null); + TaskStartedProto proto = inputStream.readMessage(TaskStartedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java index ca2cafc834..cb05bdd60b 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java @@ -24,6 +24,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; @@ -80,7 +81,8 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - VertexCommitStartedProto proto = inputStream.readMessage(VertexCommitStartedProto.PARSER, null); + VertexCommitStartedProto proto = + inputStream.readMessage(VertexCommitStartedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java index 2f4fac079f..d5a1ce5b2d 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java @@ -24,6 +24,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.api.DagTypeConverters; import org.apache.tez.dag.api.EdgeProperty; import org.apache.tez.dag.api.VertexLocationHint; @@ -162,7 +163,8 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - VertexConfigurationDoneProto proto = inputStream.readMessage(VertexConfigurationDoneProto.PARSER, null); + VertexConfigurationDoneProto proto = + inputStream.readMessage(VertexConfigurationDoneProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java index 24612a6a2e..4288a2f81a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java @@ -24,6 +24,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.app.dag.impl.ServicePluginInfo; import org.apache.tez.dag.records.VertexIDAware; import org.slf4j.Logger; @@ -131,7 +132,8 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - VertexFinishedProto proto = inputStream.readMessage(VertexFinishedProto.PARSER, null); + VertexFinishedProto proto = + inputStream.readMessage(VertexFinishedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java index c9d5aae770..7b3d0d4463 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java @@ -24,6 +24,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; @@ -101,7 +102,8 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - VertexGroupCommitFinishedProto proto = inputStream.readMessage(VertexGroupCommitFinishedProto.PARSER, null); + VertexGroupCommitFinishedProto proto = + inputStream.readMessage(VertexGroupCommitFinishedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java index cdd11bcc4c..d615debeb3 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java @@ -24,6 +24,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; @@ -101,7 +102,8 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - VertexGroupCommitStartedProto proto = inputStream.readMessage(VertexGroupCommitStartedProto.PARSER, null); + VertexGroupCommitStartedProto proto = + inputStream.readMessage(VertexGroupCommitStartedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java index a019fe3784..285c52001e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java @@ -25,6 +25,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.api.DagTypeConverters; import org.apache.tez.dag.api.InputDescriptor; import org.apache.tez.dag.api.InputInitializerDescriptor; @@ -158,7 +159,8 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - VertexInitializedProto proto = inputStream.readMessage(VertexInitializedProto.PARSER, null); + VertexInitializedProto proto = + inputStream.readMessage(VertexInitializedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java index cdd73b4483..53f00977a7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java @@ -22,6 +22,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.app.dag.VertexState; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; @@ -81,7 +82,7 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - VertexStartedProto proto = inputStream.readMessage(VertexStartedProto.PARSER, null); + VertexStartedProto proto = inputStream.readMessage(VertexStartedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWritable.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWritable.java index 34e47014e1..df5743660c 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWritable.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWritable.java @@ -25,6 +25,7 @@ import java.io.InputStream; import java.io.OutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.hadoop.io.Writable; import com.google.protobuf.CodedInputStream; @@ -96,6 +97,6 @@ public void readFields(DataInput in) throws IOException { cin.setSizeLimit(Integer.MAX_VALUE); } din.in = in; - message = cin.readMessage(parser, null); + message = cin.readMessage(parser, ExtensionRegistry.getEmptyRegistry()); } } From 5e31e4d970647aecba2b58e836ff65fedb144306 Mon Sep 17 00:00:00 2001 From: Aman Raj <104416558+amanraj2520@users.noreply.github.com> Date: Tue, 26 Jul 2022 12:04:04 +0530 Subject: [PATCH 390/512] [TEZ-4439] Update the protobuf documentation in TEZ codebase (#233) (Aman Raj reviewed by Laszlo Bodor) --- docs/src/site/markdown/install.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/src/site/markdown/install.md b/docs/src/site/markdown/install.md index b1cdcf9583..44ca862a82 100644 --- a/docs/src/site/markdown/install.md +++ b/docs/src/site/markdown/install.md @@ -34,11 +34,11 @@ or higher. 2. Build tez using `mvn clean package -DskipTests=true -Dmaven.javadoc.skip=true` - This assumes that you have already installed JDK8 or later and Maven 3 or later. - - Tez also requires Protocol Buffers 2.5.0, including the protoc-compiler. + - Tez also requires Protocol Buffers 3.19.4, including the protoc-compiler. * This can be downloaded from https://github.com/google/protobuf/tags/. * On Mac OS X with the homebrew package manager `brew install protobuf250` - * For rpm-based linux systems, the yum repos may not have the 2.5.0 version. - `rpm.pbone.net` has the protobuf-2.5.0 and protobuf-compiler-2.5.0 packages. + * For rpm-based linux systems, the yum repos may not have the 3.19.4 version. + `rpm.pbone.net` has the protobuf-3.19.4 and protobuf-compiler-3.19.4 packages. - If you prefer to run the unit tests, remove skipTests from the command above. - If you use Eclipse IDE, you can import the projects using From 621a83152c305ccb654f69f57fe711bc52d271be Mon Sep 17 00:00:00 2001 From: PJ Fanning Date: Tue, 2 Aug 2022 09:20:14 +0100 Subject: [PATCH 391/512] TEZ-4435: use jackson v2 - jackson v1 is EOL and full of security issues (#231) (PJ Fanning reviewed by Laszlo Bodor, Ayush Saxena) --- pom.xml | 29 ++++++++++++++++++- .../logging/proto/DagManifesFileScanner.java | 2 +- .../proto/HistoryEventProtoConverter.java | 2 +- 3 files changed, 30 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 2e2cd8c128..3ec471b9c0 100644 --- a/pom.xml +++ b/pom.xml @@ -70,8 +70,10 @@ 1.8.0 3.0.5 3.1.1 + 3.0.0 8.35 1.3.6 + 2.0.0 ${project.build.directory}/tmp @@ -1001,7 +1003,14 @@ org.apache.maven.plugins maven-enforcer-plugin - false + ${maven-enforcer-plugin.version} + + + de.skuzzle.enforcer + restrict-imports-enforcer-rule + ${restrict-imports.enforcer.version} + + clean @@ -1019,6 +1028,24 @@ + + banned-illegal-imports + process-sources + + enforce + + + + + true + Use Fasterxml Jackson 2 dependency in place of org.codehaus Jackson 1 + + org.codehaus.jackson.** + + + + + diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DagManifesFileScanner.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DagManifesFileScanner.java index 697083c30c..addb148530 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DagManifesFileScanner.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DagManifesFileScanner.java @@ -28,12 +28,12 @@ import java.util.List; import java.util.Map; +import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.security.UserGroupInformation; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.history.logging.proto.HistoryLoggerProtos.ManifestEntryProto; -import org.codehaus.jackson.map.ObjectMapper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java index f5f436eb9d..904c165684 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java @@ -24,6 +24,7 @@ import java.util.Map.Entry; import java.util.TreeMap; +import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.tez.common.ATSConstants; @@ -58,7 +59,6 @@ import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.dag.records.TezTaskID; import org.apache.tez.dag.records.TezVertexID; -import org.codehaus.jackson.map.ObjectMapper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; From db915ebafb6ad9bb9de062ee9b17c67eab67c7ae Mon Sep 17 00:00:00 2001 From: zhengchenyu Date: Mon, 29 Aug 2022 19:50:20 +0800 Subject: [PATCH 392/512] TEZ-4440. When tez app run in yarn fed cluster, may throw NPE (#235) (zhengchenyu reviewed by Laszlo Bodor) --- .../org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java | 3 ++- .../org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java | 5 +++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java index 29f350f9d0..1b8e19176d 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java @@ -780,7 +780,8 @@ public void onError(Throwable e) { @Override public Resource getAvailableResources() { - return client.getAvailableResources(); + Resource resource = client.getAvailableResources(); + return resource == null ? Resource.newInstance(0, 0) : resource; } @Override diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java index 6f51566f75..ea0bdb92e8 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java @@ -254,7 +254,8 @@ public YarnTaskSchedulerService(TaskSchedulerContext taskSchedulerContext) { @Override public Resource getAvailableResources() { - return amRmClient.getAvailableResources(); + Resource resource = amRmClient.getAvailableResources(); + return resource == null ? Resource.newInstance(0, 0) : resource; } @Override @@ -1166,7 +1167,7 @@ boolean preemptIfNeeded() { ContainerId[] preemptedContainers = null; int numPendingRequestsToService = 0; synchronized (this) { - Resource freeResources = amRmClient.getAvailableResources(); + Resource freeResources = this.getAvailableResources(); if (LOG.isDebugEnabled()) { LOG.debug(constructPreemptionPeriodicLog(freeResources)); } else { From 55b6031fa2c38cd8af1d384414d29c4e19a875d7 Mon Sep 17 00:00:00 2001 From: zhengchenyu Date: Mon, 29 Aug 2022 19:52:29 +0800 Subject: [PATCH 393/512] TEZ-4441: TezAppMaster may stuck because of reportError skip send error event (#236) (zhengchenyu reviewed by Laszlo Bodor) --- .../org/apache/tez/dag/app/DAGAppMaster.java | 2 +- .../tez/dag/app/rm/TaskSchedulerManager.java | 8 +- .../dag/app/rm/TestTaskSchedulerHelpers.java | 17 ++++ .../dag/app/rm/TestTaskSchedulerManager.java | 89 +++++++++++++++++++ 4 files changed, 114 insertions(+), 2 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index ca50b2ac6b..c8519201fd 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -2623,7 +2623,7 @@ public String getWebUIAddress() { } @VisibleForTesting - static void parseAllPlugins( + public static void parseAllPlugins( List taskSchedulerDescriptors, BiMap taskSchedulerPluginMap, List containerLauncherDescriptors, BiMap containerLauncherPluginMap, List taskCommDescriptors, BiMap taskCommPluginMap, diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java index 8a24ad3426..dfd48e6751 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java @@ -910,7 +910,8 @@ public void reportError(int taskSchedulerIndex, ServicePluginError servicePlugin LOG.info("Error reported by scheduler {} - {}", Utils.getTaskSchedulerIdentifierString(taskSchedulerIndex, appContext) + ": " + diagnostics); - if (taskSchedulerDescriptors[taskSchedulerIndex].getClassName().equals(yarnSchedulerClassName)) { + if (taskSchedulerDescriptors[taskSchedulerIndex].getEntityName() + .equals(TezConstants.getTezYarnServicePluginName())) { LOG.warn( "Reporting a SchedulerServiceError to the DAGAppMaster since the error" + " was reported by the YARN task scheduler"); @@ -1078,4 +1079,9 @@ public String getTaskSchedulerClassName(int taskSchedulerIndex) { return taskSchedulers[taskSchedulerIndex].getTaskScheduler().getClass().getName(); } + @VisibleForTesting + public TaskScheduler getTaskScheduler(int taskSchedulerIndex) { + return taskSchedulers[taskSchedulerIndex].getTaskScheduler(); + } + } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java index b7acc6876c..490067a54c 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java @@ -161,6 +161,19 @@ public TaskSchedulerManagerForTest(AppContext appContext, this.defaultPayload = defaultPayload; } + TaskSchedulerManagerForTest(AppContext appContext, + EventHandler eventHandler, + TezAMRMClientAsync amrmClientAsync, + ContainerSignatureMatcher containerSignatureMatcher, + UserPayload defaultPayload, + List descriptors) { + super(appContext, null, eventHandler, containerSignatureMatcher, null, descriptors, + false, new HadoopShimsLoader(appContext.getAMConf()).getHadoopShim()); + this.amrmClientAsync = amrmClientAsync; + this.containerSignatureMatcher = containerSignatureMatcher; + this.defaultPayload = defaultPayload; + } + @SuppressWarnings("unchecked") @Override public void instantiateSchedulers(String host, int port, String trackingUrl, @@ -224,6 +237,10 @@ public Event verifyInvocation(Class eventClass) { fail("Expected Event: " + eventClass.getName() + " not sent"); return null; } + + public int getEventSize() { + return this.events.size(); + } } static class TaskSchedulerWithDrainableContext extends YarnTaskSchedulerService { diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java index 095bd4e287..9d423c047c 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java @@ -45,11 +45,17 @@ import java.util.List; import java.util.Set; import java.util.concurrent.Executors; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.google.common.collect.BiMap; +import com.google.common.collect.HashBiMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.util.Time; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerExitStatus; @@ -59,6 +65,8 @@ import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.client.api.AMRMClient; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.Event; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.tez.common.ContainerSignatureMatcher; @@ -72,10 +80,16 @@ import org.apache.tez.dag.api.client.DAGClientServer; import org.apache.tez.dag.api.oldrecords.TaskAttemptState; import org.apache.tez.dag.app.AppContext; +import org.apache.tez.dag.app.ClusterInfo; import org.apache.tez.dag.app.ContainerContext; +import org.apache.tez.dag.app.ContainerHeartbeatHandler; +import org.apache.tez.dag.app.DAGAppMaster; +import org.apache.tez.dag.app.DAGAppMasterState; import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService; +import org.apache.tez.dag.app.TaskCommunicatorManagerInterface; import org.apache.tez.dag.app.dag.DAG; import org.apache.tez.dag.app.dag.TaskAttempt; +import org.apache.tez.dag.app.dag.event.DAGAppMasterEventSchedulingServiceError; import org.apache.tez.dag.app.dag.event.DAGAppMasterEventType; import org.apache.tez.dag.app.dag.event.DAGAppMasterEventUserServiceFatalError; import org.apache.tez.dag.app.dag.event.DAGEventTerminateDag; @@ -89,6 +103,8 @@ import org.apache.tez.dag.app.rm.container.AMContainerEventType; import org.apache.tez.dag.app.rm.container.AMContainerMap; import org.apache.tez.dag.app.rm.container.AMContainerState; +import org.apache.tez.dag.app.rm.container.ContainerContextMatcher; +import org.apache.tez.dag.app.rm.node.AMNodeTracker; import org.apache.tez.dag.app.web.WebUIService; import org.apache.tez.dag.helpers.DagInfoImplForTest; import org.apache.tez.dag.records.TaskAttemptTerminationCause; @@ -839,6 +855,60 @@ protected void instantiateSchedulers(String host, int port, String trackingUrl, } } + @Test(timeout = 10000) + public void testHandleException() throws Exception { + Configuration tezConf = new Configuration(new YarnConfiguration()); + UserPayload defaultPayload = TezUtils.createUserPayloadFromConf(tezConf); + + // Parse plugins + List tsDescriptors = Lists.newLinkedList(); + BiMap tsMap = HashBiMap.create(); + DAGAppMaster.parseAllPlugins(tsDescriptors, tsMap, Lists.newLinkedList(), HashBiMap.create(), Lists.newLinkedList(), + HashBiMap.create(), null, false, defaultPayload); + + // Only TezYarn found. + Assert.assertEquals(1, tsDescriptors.size()); + Assert.assertEquals(TezConstants.getTezYarnServicePluginName(), tsDescriptors.get(0).getEntityName()); + + // Construct eventHandler + TestTaskSchedulerHelpers.CapturingEventHandler eventHandler = new TestTaskSchedulerHelpers.CapturingEventHandler(); + TezDAGID dagID = TezDAGID.getInstance("0", 0, 0); + + // Construct AMRMClient + AMRMClient rmClientCore = + new TestTaskSchedulerHelpers.AMRMClientForTest(); + TezAMRMClientAsync rmClient = + spy(new TestTaskSchedulerHelpers.AMRMClientAsyncForTest(rmClientCore, 100)); + + // Construct appContext + AppContext appContext = mock(AppContext.class); + doReturn(new Configuration(false)).when(appContext).getAMConf(); + AMContainerMap amContainerMap = new AMContainerMap(mock(ContainerHeartbeatHandler.class), + mock(TaskCommunicatorManagerInterface.class), new ContainerContextMatcher(), appContext); + AMNodeTracker amNodeTracker = new AMNodeTracker(eventHandler, appContext); + doReturn(amContainerMap).when(appContext).getAllContainers(); + doReturn(amNodeTracker).when(appContext).getNodeTracker(); + doReturn(DAGAppMasterState.RUNNING).when(appContext).getAMState(); + doReturn(dagID).when(appContext).getCurrentDAGID(); + doReturn(mock(ClusterInfo.class)).when(appContext).getClusterInfo(); + + // Construct TaskSchedulerManager + TaskSchedulerManager taskSchedulerManagerReal = + new TestTaskSchedulerHelpers.TaskSchedulerManagerForTest(appContext, eventHandler, rmClient, + new TestTaskSchedulerHelpers.AlwaysMatchesContainerMatcher(), defaultPayload, tsDescriptors); + TaskSchedulerManager taskSchedulerManager = spy(taskSchedulerManagerReal); + taskSchedulerManager.init(tezConf); + taskSchedulerManager.start(); + + // Send error to schedule, then expect DAGAppMasterEventSchedulingServiceError event. + YarnTaskSchedulerService scheduler = ((YarnTaskSchedulerService) taskSchedulerManager.getTaskScheduler(0)); + scheduler.onError(new Exception("Trigger by unit test")); + waitFor(() -> { + return eventHandler.getEventSize() > 0; + }, 1000, 5000); + eventHandler.verifyInvocation(DAGAppMasterEventSchedulingServiceError.class); + } + private static class ExceptionAnswer implements Answer { @Override public Object answer(InvocationOnMock invocation) throws Throwable { @@ -1107,4 +1177,23 @@ public boolean hasUnregistered() throws ServicePluginException { public void dagComplete() throws ServicePluginException { } } + + public static void waitFor(Supplier check, int checkEveryMillis, + int waitForMillis) throws TimeoutException, InterruptedException { + Preconditions.checkNotNull(check, "Input supplier interface should be initailized"); + Preconditions.checkArgument(waitForMillis >= checkEveryMillis, + "Total wait time should be greater than check interval time"); + + long st = Time.monotonicNow(); + boolean result = check.get(); + + while (!result && (Time.monotonicNow() - st < waitForMillis)) { + Thread.sleep(checkEveryMillis); + result = check.get(); + } + + if (!result) { + throw new TimeoutException("Timed out waiting for condition."); + } + } } From de88f2a2c5e8857360996c502f8827202d3e78ee Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Wed, 14 Sep 2022 12:44:25 +0200 Subject: [PATCH 394/512] TEZ-4447: Collect VertexStatus properly when DAGClientServer is not used (local mode without network) (#239) (Laszlo Bodor reviewed by Rajesh Balamohan) --- .../apache/tez/client/FrameworkClient.java | 7 +++- .../java/org/apache/tez/client/TezClient.java | 11 +----- .../tez/dag/api/client/DAGClientImpl.java | 8 +++- .../dag/api/client/DAGClientImplLocal.java | 11 +++++- .../org/apache/tez/client/LocalClient.java | 37 +++++++++++++------ .../org/apache/tez/client/MRTezClient.java | 6 --- .../tez/mapreduce/client/YARNRunner.java | 5 +-- .../library/processor/SleepProcessor.java | 1 + .../org/apache/tez/test/TestLocalMode.java | 23 ++++++++---- 9 files changed, 66 insertions(+), 43 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java b/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java index 2ec6d2864d..26c11fd084 100644 --- a/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java @@ -145,7 +145,7 @@ public DAGClient submitDag(DAG dag, SubmitDAGRequestProto request, String client + ", applicationId=" + sessionAppId + ", dagId=" + dagId + ", dagName=" + dag.getName()); - return new DAGClientImpl(sessionAppId, dagId, tezConf, this, ugi); + return getDAGClient(sessionAppId, dagId, tezConf, ugi); } protected DAGClientAMProtocolBlockingPB waitForProxy(long clientTimeout, Configuration conf, @@ -186,4 +186,9 @@ protected DAGClientAMProtocolBlockingPB getProxy(Configuration conf, Application UserGroupInformation ugi) throws TezException, IOException { return TezClientUtils.getAMProxy(this, conf, sessionAppId, ugi); } + + public DAGClient getDAGClient(ApplicationId appId, String dagId, TezConfiguration tezConf, + UserGroupInformation ugi) { + return new DAGClientImpl(appId, dagId, tezConf, this, ugi); + } } diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java index c37f0c181d..93807fdf46 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java @@ -76,7 +76,6 @@ import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolBlockingPB; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetAMStatusRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto; -import org.apache.tez.dag.api.client.DAGClientImpl; import org.apache.tez.dag.api.records.DAGProtos.DAGPlan; import com.google.common.annotations.VisibleForTesting; @@ -1117,15 +1116,7 @@ private synchronized Map getTezJarResources(Credentials c @Private static DAGClient getDAGClient(ApplicationId appId, TezConfiguration tezConf, FrameworkClient frameworkClient, UserGroupInformation ugi) throws IOException, TezException { - return new DAGClientImpl(appId, getDefaultTezDAGID(appId), tezConf, frameworkClient, ugi); - } - - @Private // Used only for MapReduce compatibility code - static DAGClient getDAGClient(ApplicationId appId, TezConfiguration tezConf, - FrameworkClient frameworkClient) throws IOException, TezException { - UserGroupInformation ugi = - UserGroupInformation.createRemoteUser(UserGroupInformation.getCurrentUser().getUserName()); - return getDAGClient(appId, tezConf, frameworkClient, ugi); + return frameworkClient.getDAGClient(appId, getDefaultTezDAGID(appId), tezConf, ugi); } // DO NOT CHANGE THIS. This code is replicated from TezDAGID.java diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java index bfea96b998..95dd85f388 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java @@ -296,9 +296,13 @@ public DAGStatus getDAGStatus(@Nullable Set statusOptions) throws } @Override - public VertexStatus getVertexStatus(String vertexName, Set statusOptions) throws - IOException, TezException { + public VertexStatus getVertexStatus(String vertexName, Set statusOptions) + throws IOException, TezException { + return getVertexStatusInternal(statusOptions, vertexName); + } + protected VertexStatus getVertexStatusInternal(Set statusOptions, String vertexName) + throws IOException, TezException { if (!dagCompleted) { VertexStatus vertexStatus = getVertexStatusViaAM(vertexName, statusOptions); diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImplLocal.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImplLocal.java index a0509cdc5f..851bb687a1 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImplLocal.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImplLocal.java @@ -36,12 +36,15 @@ public class DAGClientImplLocal extends DAGClientImpl { private BiFunction, Long, DAGStatus> dagStatusFunction; + private BiFunction, String, VertexStatus> vertexStatusFunction; public DAGClientImplLocal(ApplicationId appId, String dagId, TezConfiguration conf, FrameworkClient frameworkClient, UserGroupInformation ugi, - BiFunction, Long, DAGStatus> dagStatusFunction) { + BiFunction, Long, DAGStatus> dagStatusFunction, + BiFunction, String, VertexStatus> vertexStatusFunction) { super(appId, dagId, conf, frameworkClient, ugi); this.dagStatusFunction = dagStatusFunction; + this.vertexStatusFunction = vertexStatusFunction; } @Override @@ -50,4 +53,10 @@ protected DAGStatus getDAGStatusInternal(@Nullable Set statusOpti return dagStatusFunction.apply(statusOptions == null ? new HashSet<>() : statusOptions, timeout); } + + @Override + protected VertexStatus getVertexStatusInternal(@Nullable Set statusOptions, String vertexName) + throws TezException, IOException { + return vertexStatusFunction.apply(statusOptions == null ? new HashSet<>() : statusOptions, vertexName); + } } diff --git a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java index d0580bbae9..c9b3d7314b 100644 --- a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java +++ b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java @@ -60,6 +60,7 @@ import org.apache.tez.dag.api.client.DAGClientImplLocal; import org.apache.tez.dag.api.client.DAGStatus; import org.apache.tez.dag.api.client.StatusGetOpts; +import org.apache.tez.dag.api.client.VertexStatus; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto; import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto; import org.apache.tez.dag.app.AppContext; @@ -426,20 +427,32 @@ public DAGClient submitDag(org.apache.tez.dag.api.DAG dag, SubmitDAGRequestProto } String dagId = dagAppMaster.submitDAGToAppMaster(request.getDAGPlan(), additionalResources); + return getDAGClient(sessionAppId, dagId, tezConf, ugi); + } + @Override + public DAGClient getDAGClient(ApplicationId appId, String dagId, TezConfiguration tezConf, + UserGroupInformation ugi) { return isLocalWithoutNetwork - ? new DAGClientImplLocal(sessionAppId, dagId, tezConf, this, - ugi, new BiFunction, Long, DAGStatus>() { - @Override - public DAGStatus apply(Set statusOpts, Long timeout) { - try { - return clientHandler.getDAGStatus(dagId, statusOpts, timeout); - } catch (TezException e) { - throw new RuntimeException(e); - } - } - }) - : new DAGClientImpl(sessionAppId, dagId, tezConf, this, ugi); + ? new DAGClientImplLocal(appId, dagId, tezConf, this, ugi, new BiFunction, Long, DAGStatus>() { + @Override + public DAGStatus apply(Set statusOpts, Long timeout) { + try { + return clientHandler.getDAGStatus(dagId, statusOpts, timeout); + } catch (TezException e) { + throw new RuntimeException(e); + } + } + }, new BiFunction, String, VertexStatus>() { + @Override + public VertexStatus apply(Set statusOpts, String vertexName) { + try { + return clientHandler.getVertexStatus(dagId, vertexName, statusOpts); + } catch (TezException e) { + throw new RuntimeException(e); + } + } + }) : new DAGClientImpl(appId, dagId, tezConf, this, ugi); } @Override diff --git a/tez-mapreduce/src/main/java/org/apache/tez/client/MRTezClient.java b/tez-mapreduce/src/main/java/org/apache/tez/client/MRTezClient.java index 86089e9a67..1057932e1d 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/client/MRTezClient.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/client/MRTezClient.java @@ -31,7 +31,6 @@ import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.api.client.DAGClient; -import org.apache.tez.dag.api.client.MRDAGClient; @InterfaceAudience.Private public class MRTezClient extends TezClient { @@ -46,9 +45,4 @@ public DAGClient submitDAGApplication(ApplicationId appId, org.apache.tez.dag.ap throws TezException, IOException { return super.submitDAGApplication(appId, dag); } - - public static MRDAGClient getDAGClient(ApplicationId appId, TezConfiguration tezConf, FrameworkClient frameworkClient) - throws IOException, TezException { - return new MRDAGClient(TezClient.getDAGClient(appId, tezConf, frameworkClient)); - } } diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java index 9dba357951..7aed4a04a8 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java @@ -639,7 +639,7 @@ public JobStatus submitJob(JobID jobId, String jobSubmitDir, Credentials ts) tezClient = new MRTezClient("MapReduce", dagAMConf, false, jobLocalResources, ts); tezClient.start(); - tezClient.submitDAGApplication(appId, dag); + dagClient = new MRDAGClient(tezClient.submitDAGApplication(appId, dag)); tezClient.stop(); } catch (TezException e) { throw new IOException(e); @@ -702,9 +702,6 @@ public JobStatus getJobStatus(JobID jobID) throws IOException, String jobFile = MRApps.getJobFile(conf, user, jobID); DAGStatus dagStatus; try { - if(dagClient == null) { - dagClient = MRTezClient.getDAGClient(TypeConverter.toYarn(jobID).getAppId(), tezConf, null); - } dagStatus = dagClient.getDAGStatus(null); return new DAGJobStatus(dagClient.getApplicationReport(), dagStatus, jobFile); } catch (TezException e) { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/processor/SleepProcessor.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/processor/SleepProcessor.java index 3efcd21c92..7750a13a63 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/processor/SleepProcessor.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/processor/SleepProcessor.java @@ -46,6 +46,7 @@ public class SleepProcessor extends AbstractLogicalIOProcessor { private static final Logger LOG = LoggerFactory.getLogger(SleepProcessor.class); + public static final String SLEEP_VERTEX_NAME = "Sleep"; private int timeToSleepMS; protected Map inputs; diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java b/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java index bdb71ad177..00125fd547 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java @@ -39,6 +39,7 @@ import org.apache.tez.dag.api.Vertex; import org.apache.tez.dag.api.client.DAGClient; import org.apache.tez.dag.api.client.DAGStatus; +import org.apache.tez.dag.api.client.VertexStatus; import org.apache.tez.examples.OrderedWordCount; import org.apache.tez.runtime.api.AbstractLogicalIOProcessor; import org.apache.tez.runtime.api.Event; @@ -131,6 +132,8 @@ public void testMultipleClientsWithSession() throws TezException, InterruptedExc DAGClient dagClient1 = tezClient1.submitDAG(dag1); dagClient1.waitForCompletion(); assertEquals(DAGStatus.State.SUCCEEDED, dagClient1.getDAGStatus(null).getState()); + assertEquals(VertexStatus.State.SUCCEEDED, + dagClient1.getVertexStatus(SleepProcessor.SLEEP_VERTEX_NAME, null).getState()); dagClient1.close(); tezClient1.stop(); @@ -142,6 +145,8 @@ public void testMultipleClientsWithSession() throws TezException, InterruptedExc DAGClient dagClient2 = tezClient2.submitDAG(dag2); dagClient2.waitForCompletion(); assertEquals(DAGStatus.State.SUCCEEDED, dagClient2.getDAGStatus(null).getState()); + assertEquals(VertexStatus.State.SUCCEEDED, + dagClient2.getVertexStatus(SleepProcessor.SLEEP_VERTEX_NAME, null).getState()); assertFalse(dagClient1.getExecutionContext().equals(dagClient2.getExecutionContext())); dagClient2.close(); tezClient2.stop(); @@ -159,7 +164,8 @@ public void testMultipleClientsWithoutSession() throws TezException, Interrupted DAGClient dagClient1 = tezClient1.submitDAG(dag1); dagClient1.waitForCompletion(); assertEquals(DAGStatus.State.SUCCEEDED, dagClient1.getDAGStatus(null).getState()); - + assertEquals(VertexStatus.State.SUCCEEDED, + dagClient1.getVertexStatus(SleepProcessor.SLEEP_VERTEX_NAME, null).getState()); dagClient1.close(); tezClient1.stop(); @@ -171,6 +177,8 @@ public void testMultipleClientsWithoutSession() throws TezException, Interrupted DAGClient dagClient2 = tezClient2.submitDAG(dag2); dagClient2.waitForCompletion(); assertEquals(DAGStatus.State.SUCCEEDED, dagClient2.getDAGStatus(null).getState()); + assertEquals(VertexStatus.State.SUCCEEDED, + dagClient2.getVertexStatus(SleepProcessor.SLEEP_VERTEX_NAME, null).getState()); assertFalse(dagClient1.getExecutionContext().equals(dagClient2.getExecutionContext())); dagClient2.close(); tezClient2.stop(); @@ -189,7 +197,8 @@ public void testNoSysExitOnSuccessfulDAG() throws TezException, InterruptedExcep DAGClient dagClient1 = tezClient1.submitDAG(dag1); dagClient1.waitForCompletion(); assertEquals(DAGStatus.State.SUCCEEDED, dagClient1.getDAGStatus(null).getState()); - + assertEquals(VertexStatus.State.SUCCEEDED, + dagClient1.getVertexStatus(SleepProcessor.SLEEP_VERTEX_NAME, null).getState()); // Sleep for more time than is required for the DAG to complete. Thread.sleep((long) (TezConstants.TEZ_DAG_SLEEP_TIME_BEFORE_EXIT * 1.5)); @@ -210,7 +219,8 @@ public void testNoSysExitOnFailinglDAG() throws TezException, InterruptedExcepti DAGClient dagClient1 = tezClient1.submitDAG(dag1); dagClient1.waitForCompletion(); assertEquals(DAGStatus.State.FAILED, dagClient1.getDAGStatus(null).getState()); - + assertEquals(VertexStatus.State.FAILED, + dagClient1.getVertexStatus(SleepProcessor.SLEEP_VERTEX_NAME, null).getState()); // Sleep for more time than is required for the DAG to complete. Thread.sleep((long) (TezConstants.TEZ_DAG_SLEEP_TIME_BEFORE_EXIT * 1.5)); @@ -245,12 +255,11 @@ public void run(Map inputs, Map out } private DAG createSimpleDAG(String dagName, String processorName) { - DAG dag = DAG.create(dagName).addVertex(Vertex.create("Sleep", ProcessorDescriptor.create( - processorName).setUserPayload( - new SleepProcessor.SleepProcessorConfig(1).toUserPayload()), 1)); + DAG dag = DAG.create(dagName).addVertex(Vertex.create(SleepProcessor.SLEEP_VERTEX_NAME, ProcessorDescriptor + .create(processorName).setUserPayload(new SleepProcessor.SleepProcessorConfig(1).toUserPayload()), 1)); return dag; - } + @Test(timeout=30000) public void testMultiDAGsOnSession() throws IOException, TezException, InterruptedException { int dags = 2;//two dags will be submitted to session From 921de531f1f0a5b275844d693cb1b528595020ab Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Mon, 10 Oct 2022 12:20:03 +0200 Subject: [PATCH 395/512] TEZ-4431: Apache Tez Release 0.10.2: update SNAPSHOT versions, DOAP file (#234) (Laszlo Bodor reviewed by Rajesh Balamohan, Ayush Saxena) --- Tez_DOAP.rdf | 7 +++++++ docs/pom.xml | 2 +- hadoop-shim-impls/hadoop-shim-2.7/pom.xml | 2 +- hadoop-shim-impls/hadoop-shim-2.8/pom.xml | 2 +- hadoop-shim-impls/pom.xml | 2 +- hadoop-shim/pom.xml | 2 +- pom.xml | 2 +- tez-api/pom.xml | 2 +- tez-build-tools/pom.xml | 2 +- tez-common/pom.xml | 2 +- tez-dag/pom.xml | 2 +- tez-dist/pom.xml | 2 +- tez-examples/pom.xml | 2 +- tez-ext-service-tests/pom.xml | 2 +- tez-mapreduce/pom.xml | 2 +- tez-plugins/pom.xml | 2 +- tez-plugins/tez-aux-services/pom.xml | 2 +- tez-plugins/tez-history-parser/pom.xml | 2 +- tez-plugins/tez-protobuf-history-plugin/pom.xml | 2 +- tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml | 2 +- tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml | 2 +- tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml | 2 +- tez-plugins/tez-yarn-timeline-history/pom.xml | 2 +- tez-runtime-internals/pom.xml | 2 +- tez-runtime-library/pom.xml | 2 +- tez-tests/pom.xml | 2 +- tez-tools/analyzers/job-analyzer/pom.xml | 2 +- tez-tools/analyzers/pom.xml | 2 +- tez-tools/pom.xml | 2 +- tez-tools/tez-javadoc-tools/pom.xml | 2 +- tez-tools/tez-tfile-parser/pom.xml | 2 +- tez-ui/pom.xml | 2 +- 32 files changed, 38 insertions(+), 31 deletions(-) diff --git a/Tez_DOAP.rdf b/Tez_DOAP.rdf index d4c473d493..94cae144d8 100644 --- a/Tez_DOAP.rdf +++ b/Tez_DOAP.rdf @@ -34,6 +34,13 @@ Java + + + Version 0.10.2 + 2022-07-30 + 0.10.2 + + Version 0.10.1 diff --git a/docs/pom.xml b/docs/pom.xml index 0d55177641..d82f003c41 100644 --- a/docs/pom.xml +++ b/docs/pom.xml @@ -27,7 +27,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-docs pom diff --git a/hadoop-shim-impls/hadoop-shim-2.7/pom.xml b/hadoop-shim-impls/hadoop-shim-2.7/pom.xml index 2557e036c3..caccbdca5e 100644 --- a/hadoop-shim-impls/hadoop-shim-2.7/pom.xml +++ b/hadoop-shim-impls/hadoop-shim-2.7/pom.xml @@ -19,7 +19,7 @@ hadoop-shim-impls org.apache.tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT 4.0.0 hadoop-shim-2.7 diff --git a/hadoop-shim-impls/hadoop-shim-2.8/pom.xml b/hadoop-shim-impls/hadoop-shim-2.8/pom.xml index 0a149cca04..4fdf57ff85 100644 --- a/hadoop-shim-impls/hadoop-shim-2.8/pom.xml +++ b/hadoop-shim-impls/hadoop-shim-2.8/pom.xml @@ -19,7 +19,7 @@ hadoop-shim-impls org.apache.tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT 4.0.0 hadoop-shim-2.8 diff --git a/hadoop-shim-impls/pom.xml b/hadoop-shim-impls/pom.xml index 25cf3fd25a..83becc56f7 100644 --- a/hadoop-shim-impls/pom.xml +++ b/hadoop-shim-impls/pom.xml @@ -20,7 +20,7 @@ tez org.apache.tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT hadoop-shim-impls pom diff --git a/hadoop-shim/pom.xml b/hadoop-shim/pom.xml index 18ddaca3df..15c86e52ed 100644 --- a/hadoop-shim/pom.xml +++ b/hadoop-shim/pom.xml @@ -20,7 +20,7 @@ tez org.apache.tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT hadoop-shim diff --git a/pom.xml b/pom.xml index 3ec471b9c0..899dd0c103 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ org.apache.tez tez pom - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez diff --git a/tez-api/pom.xml b/tez-api/pom.xml index 1d10bc1f85..7cabdf1160 100644 --- a/tez-api/pom.xml +++ b/tez-api/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-api diff --git a/tez-build-tools/pom.xml b/tez-build-tools/pom.xml index 232a20be89..ae05507e63 100644 --- a/tez-build-tools/pom.xml +++ b/tez-build-tools/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-build-tools diff --git a/tez-common/pom.xml b/tez-common/pom.xml index 9ccf02a38c..ecd4f7b4b7 100644 --- a/tez-common/pom.xml +++ b/tez-common/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-common diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml index fda9fb2a54..c2baf0c0d2 100644 --- a/tez-dag/pom.xml +++ b/tez-dag/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-dag diff --git a/tez-dist/pom.xml b/tez-dist/pom.xml index 741ed9d1e9..ccc6c7589c 100644 --- a/tez-dist/pom.xml +++ b/tez-dist/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-dist diff --git a/tez-examples/pom.xml b/tez-examples/pom.xml index f6ab04c020..e9040bec5a 100644 --- a/tez-examples/pom.xml +++ b/tez-examples/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-examples diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml index a0bcbe32c0..ddf5392fa4 100644 --- a/tez-ext-service-tests/pom.xml +++ b/tez-ext-service-tests/pom.xml @@ -20,7 +20,7 @@ tez org.apache.tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-ext-service-tests diff --git a/tez-mapreduce/pom.xml b/tez-mapreduce/pom.xml index 1e35d2ea11..bcebf81d03 100644 --- a/tez-mapreduce/pom.xml +++ b/tez-mapreduce/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-mapreduce diff --git a/tez-plugins/pom.xml b/tez-plugins/pom.xml index 27f19289ce..8bec3bbdb6 100644 --- a/tez-plugins/pom.xml +++ b/tez-plugins/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-plugins pom diff --git a/tez-plugins/tez-aux-services/pom.xml b/tez-plugins/tez-aux-services/pom.xml index 8b4ab74102..32a0f2de9c 100644 --- a/tez-plugins/tez-aux-services/pom.xml +++ b/tez-plugins/tez-aux-services/pom.xml @@ -20,7 +20,7 @@ tez-plugins org.apache.tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-aux-services diff --git a/tez-plugins/tez-history-parser/pom.xml b/tez-plugins/tez-history-parser/pom.xml index f0758dcd15..605daca345 100644 --- a/tez-plugins/tez-history-parser/pom.xml +++ b/tez-plugins/tez-history-parser/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez-plugins - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-history-parser diff --git a/tez-plugins/tez-protobuf-history-plugin/pom.xml b/tez-plugins/tez-protobuf-history-plugin/pom.xml index 0fa526413e..98678fd53e 100644 --- a/tez-plugins/tez-protobuf-history-plugin/pom.xml +++ b/tez-plugins/tez-protobuf-history-plugin/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-protobuf-history-plugin diff --git a/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml b/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml index 942d43658a..a362e6bd4a 100644 --- a/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml +++ b/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-yarn-timeline-cache-plugin diff --git a/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml b/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml index 0b61b7d958..5aedd28373 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-yarn-timeline-history-with-acls diff --git a/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml b/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml index 75df73f3a6..8811e06011 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-yarn-timeline-history-with-fs diff --git a/tez-plugins/tez-yarn-timeline-history/pom.xml b/tez-plugins/tez-yarn-timeline-history/pom.xml index 3f17e97327..059b52b494 100644 --- a/tez-plugins/tez-yarn-timeline-history/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-yarn-timeline-history diff --git a/tez-runtime-internals/pom.xml b/tez-runtime-internals/pom.xml index efe01e058e..66090c5764 100644 --- a/tez-runtime-internals/pom.xml +++ b/tez-runtime-internals/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-runtime-internals diff --git a/tez-runtime-library/pom.xml b/tez-runtime-library/pom.xml index e6db47e535..f54b26b76a 100644 --- a/tez-runtime-library/pom.xml +++ b/tez-runtime-library/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-runtime-library diff --git a/tez-tests/pom.xml b/tez-tests/pom.xml index 72f95d56ce..abf8674286 100644 --- a/tez-tests/pom.xml +++ b/tez-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-tests diff --git a/tez-tools/analyzers/job-analyzer/pom.xml b/tez-tools/analyzers/job-analyzer/pom.xml index 0cc85834cb..077de5cfb1 100644 --- a/tez-tools/analyzers/job-analyzer/pom.xml +++ b/tez-tools/analyzers/job-analyzer/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-perf-analyzer - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-job-analyzer diff --git a/tez-tools/analyzers/pom.xml b/tez-tools/analyzers/pom.xml index 3c989e68ec..72b5aba3ae 100644 --- a/tez-tools/analyzers/pom.xml +++ b/tez-tools/analyzers/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-tools - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-perf-analyzer pom diff --git a/tez-tools/pom.xml b/tez-tools/pom.xml index bffe9abc5d..f9987b3f97 100644 --- a/tez-tools/pom.xml +++ b/tez-tools/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-tools pom diff --git a/tez-tools/tez-javadoc-tools/pom.xml b/tez-tools/tez-javadoc-tools/pom.xml index 6b0546674b..8106872c34 100644 --- a/tez-tools/tez-javadoc-tools/pom.xml +++ b/tez-tools/tez-javadoc-tools/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-tools - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-javadoc-tools diff --git a/tez-tools/tez-tfile-parser/pom.xml b/tez-tools/tez-tfile-parser/pom.xml index a258a5e92b..281cbe9990 100644 --- a/tez-tools/tez-tfile-parser/pom.xml +++ b/tez-tools/tez-tfile-parser/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-tools - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-tfile-parser diff --git a/tez-ui/pom.xml b/tez-ui/pom.xml index 59a02a2ab0..5b90d6d0fb 100644 --- a/tez-ui/pom.xml +++ b/tez-ui/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-ui war From 40e9e0f6ab4201011ce8beb27eb80c8634c28f0b Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Sun, 23 Oct 2022 15:48:34 +0800 Subject: [PATCH 396/512] TEZ-4449: Upgrade jettison to 1.5.1 to fix CVE-2022-40149. (#242) (fanshilun reviewed by Laszlo Bodor) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 899dd0c103..5e03d2af85 100644 --- a/pom.xml +++ b/pom.xml @@ -738,7 +738,7 @@ org.codehaus.jettison jettison - 1.3.4 + 1.5.1 com.google.code.findbugs From ddb53cb014b68cb5935daedba06938a676063b7c Mon Sep 17 00:00:00 2001 From: Karel Kolman Date: Sun, 23 Oct 2022 10:41:27 +0200 Subject: [PATCH 397/512] TEZ-4448: Cannot submit Tez job when dag size is exceeds `ipc.maximum.data.length` and S3A is the filesystem (#241) (Karel Kolman reviewed by Laszlo Bodor) --- tez-api/src/main/java/org/apache/tez/client/TezClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java index 93807fdf46..d54484927e 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java @@ -695,8 +695,8 @@ private DAGClient submitDAGSession(DAG dag) throws TezException, IOException { + ", max IPC message size= " + amConfig.getTezConfiguration().getInt( CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH, CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH_DEFAULT)); request.writeTo(fsDataOutputStream); - request = requestBuilder.clear().setSerializedRequestPath(fs.resolvePath(dagPlanPath).toString()).build(); } + request = requestBuilder.clear().setSerializedRequestPath(fs.resolvePath(dagPlanPath).toString()).build(); } return frameworkClient.submitDag(dag, request, clientName, sessionAppId, clientTimeout, From fc5b6e9df08cc860037eeacba8c57c909b570e64 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Mon, 24 Oct 2022 17:12:00 +0800 Subject: [PATCH 398/512] TEZ-4453: Fix Typo of LocalClient. (#245) (slfan1989 reviewed by Laszlo Bodor) --- tez-dag/src/main/java/org/apache/tez/client/LocalClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java index c9b3d7314b..f6a626e64c 100644 --- a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java +++ b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java @@ -266,7 +266,7 @@ protected void startDAGAppMaster(final ApplicationSubmissionContext appContext) if (dagAMState.equals(DAGAppMasterState.NEW)) { LOG.info("DAGAppMaster is not started wait for 100ms..."); } else if (dagAMState.equals(DAGAppMasterState.INITED)) { - LOG.info("DAGAppMaster is not startetd wait for 100ms..."); + LOG.info("DAGAppMaster is not started wait for 100ms..."); } else if (dagAMState.equals(DAGAppMasterState.ERROR)) { throw new TezException("DAGAppMaster got an error during initialization"); } else if (dagAMState.equals(DAGAppMasterState.KILLED)) { From dbbd406aed3434e8866d1e1a64fce4ad9b3bf5a9 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Wed, 26 Oct 2022 06:27:53 +0200 Subject: [PATCH 399/512] TEZ-4039: Tez should inject dag id, query id into MDC (#98) (Laszlo Bodor reviewed by Rajesh Balamohan, Prasanth Jayachandran) --- .../apache/tez/dag/api/TezConfiguration.java | 47 ++++++ .../apache/tez/common/TezUtilsInternal.java | 19 ++- .../org/apache/tez/util/LoggingUtils.java | 151 ++++++++++++++++++ .../org/apache/tez/dag/app/DAGAppMaster.java | 23 ++- .../org/apache/tez/runtime/task/TezChild.java | 25 ++- .../tez/runtime/task/TezTaskRunner2.java | 17 +- .../tez/runtime/task/TestTezTaskRunner2.java | 14 +- .../java/org/apache/tez/test/TestTezJobs.java | 57 ++++++- 8 files changed, 326 insertions(+), 27 deletions(-) create mode 100644 tez-common/src/main/java/org/apache/tez/util/LoggingUtils.java diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 1ffd70a3ff..cd6d02249e 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -260,6 +260,53 @@ public TezConfiguration(boolean loadDefaults) { public static final String TEZ_TASK_LOG_LEVEL = TEZ_TASK_PREFIX + "log.level"; public static final String TEZ_TASK_LOG_LEVEL_DEFAULT = "INFO"; + /** + * By this option, user can easily override the logging pattern which is applied in + * TezContainerLogAppender in AM, regardless of the environmental settings. + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty + public static final String TEZ_LOG_PATTERN_LAYOUT_AM = TEZ_AM_PREFIX + "log.pattern.layout"; + + /** + * By this option, user can easily override the logging pattern which is applied in + * TezContainerLogAppender in tasks, regardless of the environmental settings. + */ + @ConfigurationScope(Scope.VERTEX) + @ConfigurationProperty + public static final String TEZ_LOG_PATTERN_LAYOUT_TASK = TEZ_TASK_PREFIX + "log.pattern.layout"; + + /** + * Set pattern to empty string to turn the custom log pattern feature off. + */ + public static final String TEZ_LOG_PATTERN_LAYOUT_DEFAULT = ""; + + /** + * Comma separated list of keys, which can used for defining keys in MDC. The corresponding values + * will be read from Configuration, see tez.mdc.custom.keys.conf.props for further details. + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty + public static final String TEZ_MDC_CUSTOM_KEYS = TEZ_PREFIX + "mdc.custom.keys"; + public static final String TEZ_MDC_CUSTOM_KEYS_DEFAULT = ""; + + /** + * Comma separated list of Configuration keys. Tez will try to fill MDC with key value pairs in a + * way that a key will be the nth item in tez.mdc.custom.keys and the value will be the value from + * a Configuration object pointed by the nth key of tez.mdc.custom.keys.conf.props like below: + * + * tez.mdc.custom.keys=queryId,otherKey + * tez.mdc.custom.keys.conf.props=awesome.sql.app.query.id,awesome.sql.app.other.key + * + * So MDC will contain key -{@literal >} value pairs as: + * queryId -{@literal >} conf.get("awesome.sql.app.query.id") + * otherKey -{@literal >} conf.get("awesome.sql.app.other.key") + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty + public static final String TEZ_MDC_CUSTOM_KEYS_CONF_PROPS = TEZ_MDC_CUSTOM_KEYS + ".conf.props"; + public static final String TEZ_MDC_CUSTOM_KEYS_CONF_PROPS_DEFAULT = ""; + /** * double value. Represents ratio of unique failed outputs / number of consumer * tasks. When this condition or value mentioned in {@link diff --git a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java index c2efb29cb6..7ee5bb457e 100644 --- a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java +++ b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java @@ -47,6 +47,7 @@ import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.log4j.Appender; +import org.apache.log4j.PatternLayout; import org.apache.tez.common.io.NonSyncByteArrayOutputStream; import org.apache.tez.dag.api.DagTypeConverters; import org.apache.tez.dag.records.TezDAGID; @@ -157,17 +158,25 @@ private static String sanitizeString(String srcString) { return res; // Number starts allowed rightnow } - public static void updateLoggers(String addend) throws FileNotFoundException { + public static void updateLoggers(Configuration configuration, String addend, String patternString) + throws FileNotFoundException { LOG.info("Redirecting log file based on addend: " + addend); - Appender appender = org.apache.log4j.Logger.getRootLogger().getAppender( - TezConstants.TEZ_CONTAINER_LOGGER_NAME); + Appender appender = + org.apache.log4j.Logger.getRootLogger().getAppender(TezConstants.TEZ_CONTAINER_LOGGER_NAME); if (appender != null) { if (appender instanceof TezContainerLogAppender) { TezContainerLogAppender claAppender = (TezContainerLogAppender) appender; - claAppender.setLogFileName(constructLogFileName( - TezConstants.TEZ_CONTAINER_LOG_FILE_NAME, addend)); + claAppender + .setLogFileName(constructLogFileName(TezConstants.TEZ_CONTAINER_LOG_FILE_NAME, addend)); + + // there was a configured pattern + if (patternString != null) { + PatternLayout layout = (PatternLayout) claAppender.getLayout(); + layout.setConversionPattern(patternString); + } + claAppender.activateOptions(); } else { LOG.warn("Appender is a " + appender.getClass() + "; require an instance of " diff --git a/tez-common/src/main/java/org/apache/tez/util/LoggingUtils.java b/tez-common/src/main/java/org/apache/tez/util/LoggingUtils.java new file mode 100644 index 0000000000..e09b6b0964 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/util/LoggingUtils.java @@ -0,0 +1,151 @@ +/** +* Licensed to the Apache Software Foundation (ASF) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. The ASF licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.tez.util; + +import java.lang.reflect.Constructor; +import java.lang.reflect.Field; +import java.lang.reflect.Modifier; +import java.util.Arrays; +import java.util.Hashtable; + +import org.apache.hadoop.conf.Configuration; +import org.apache.log4j.helpers.ThreadLocalMap; +import org.apache.tez.dag.api.TezConfiguration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public final class LoggingUtils { + private static final Logger LOG = LoggerFactory.getLogger(LoggingUtils.class); + + private LoggingUtils() {} + + @SuppressWarnings("unchecked") + public static void initLoggingContext(ThreadLocalMap threadLocalMap, Configuration conf, + String dagId, String taskAttemptId) { + Hashtable data = (Hashtable) threadLocalMap.get(); + if (data == null) { + data = new NonClonableHashtable(); + threadLocalMap.set(data); + } + data.put("dagId", dagId == null ? "" : dagId); + data.put("taskAttemptId", taskAttemptId == null ? "" : taskAttemptId); + + String[] mdcKeys = conf.getStrings(TezConfiguration.TEZ_MDC_CUSTOM_KEYS, + TezConfiguration.TEZ_MDC_CUSTOM_KEYS_DEFAULT); + + if (mdcKeys == null || mdcKeys.length == 0) { + return; + } + + String[] mdcKeysValuesFrom = conf.getStrings(TezConfiguration.TEZ_MDC_CUSTOM_KEYS_CONF_PROPS, + TezConfiguration.TEZ_MDC_CUSTOM_KEYS_CONF_PROPS_DEFAULT); + LOG.info("MDC_LOGGING: setting up MDC keys: keys: {} / conf: {}", Arrays.asList(mdcKeys), + Arrays.asList(mdcKeysValuesFrom)); + + int i = 0; + for (String mdcKey : mdcKeys) { + // don't want to fail on incorrect mdc key settings, but warn in app logs + if (mdcKey.isEmpty() || mdcKeysValuesFrom.length < i + 1) { + LOG.warn("cannot set mdc key: {}", mdcKey); + break; + } + + String mdcValue = mdcKeysValuesFrom[i] == null ? "" : conf.get(mdcKeysValuesFrom[i]); + // MDC is backed by a Hashtable, let's prevent NPE because of null values + if (mdcValue != null) { + data.put(mdcKey, mdcValue); + } else { + LOG.warn("MDC_LOGGING: mdc value is null for key: {}, config key: {}", mdcKey, + mdcKeysValuesFrom[i]); + } + + i++; + } + } + + public static String getPatternForAM(Configuration conf) { + String pattern = + conf.get(TezConfiguration.TEZ_LOG_PATTERN_LAYOUT_AM, TezConfiguration.TEZ_LOG_PATTERN_LAYOUT_DEFAULT); + return pattern.isEmpty() ? null : pattern; + } + + public static String getPatternForTask(Configuration conf) { + String pattern = + conf.get(TezConfiguration.TEZ_LOG_PATTERN_LAYOUT_TASK, TezConfiguration.TEZ_LOG_PATTERN_LAYOUT_DEFAULT); + return pattern.isEmpty() ? null : pattern; + } + + /** + * This method is for setting a NonClonableHashtable into log4j's mdc. Reflection hacks are + * needed, because MDC.mdc is well protected (final static MDC mdc = new MDC();). The logic below + * is supposed to be called once per JVM, so it's not a subject to performance bottlenecks. For + * further details of this solution, please check NonClonableHashtable class, which is set into + * the ThreadLocalMap. A wrong outcome of this method (any kind of runtime/reflection problems) + * should not affect the DAGAppMaster/TezChild. In case of an exception a ThreadLocalMap is + * returned, but it won't affect the content of the MDC. + */ + @SuppressWarnings("unchecked") + public static ThreadLocalMap setupLog4j() { + ThreadLocalMap mdcContext = new ThreadLocalMap(); + mdcContext.set(new NonClonableHashtable()); + + try { + final Constructor[] constructors = org.apache.log4j.MDC.class.getDeclaredConstructors(); + for (Constructor c : constructors) { + c.setAccessible(true); + } + + org.apache.log4j.MDC mdc = (org.apache.log4j.MDC) constructors[0].newInstance(); + Field tlmField = org.apache.log4j.MDC.class.getDeclaredField("tlm"); + tlmField.setAccessible(true); + tlmField.set(mdc, mdcContext); + + Field mdcField = org.apache.log4j.MDC.class.getDeclaredField("mdc"); + mdcField.setAccessible(true); + + Field modifiers = Field.class.getDeclaredField("modifiers"); + modifiers.setAccessible(true); + modifiers.setInt(mdcField, mdcField.getModifiers() & ~Modifier.FINAL); + + mdcField.set(null, mdc); + + } catch (Exception e) { + LOG.warn("Cannot set log4j global MDC, mdcContext won't be applied to log4j's MDC class", e); + } + + return mdcContext; + } + + /** + * NonClonableHashtable is a special class for hacking the log4j MDC context. By design, log4j's + * MDC uses a ThreadLocalMap, which clones parent thread's context before propagating it to child + * thread (see: @see {@link org.apache.log4j.helpers.ThreadLocalMap#childValue()}). In our + * usecase, this is not suitable, as we want to maintain only one context globally (and set e.g. + * dagId, taskAttemptId), then update it as easy as possible when dag/taskattempt changes, without + * having to propagate the update parameters to all the threads in the JVM. + */ + private static class NonClonableHashtable extends Hashtable { + private static final long serialVersionUID = 1L; + + @Override + public synchronized Object clone() { + return this; + } + } +} diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index c8519201fd..a5d7b7db9f 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -112,6 +112,7 @@ import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree; import org.apache.hadoop.yarn.util.SystemClock; +import org.apache.log4j.helpers.ThreadLocalMap; import org.apache.tez.common.AsyncDispatcher; import org.apache.tez.common.AsyncDispatcherConcurrent; import org.apache.tez.common.GcTimeUpdater; @@ -184,6 +185,7 @@ import org.apache.tez.dag.utils.Simple2LevelVersionComparator; import org.apache.tez.hadoop.shim.HadoopShim; import org.apache.tez.hadoop.shim.HadoopShimsLoader; +import org.apache.tez.util.LoggingUtils; import org.apache.tez.util.TezMxBeanResourceCalculator; import org.codehaus.jettison.json.JSONException; import org.slf4j.Logger; @@ -336,6 +338,7 @@ public class DAGAppMaster extends AbstractService { // must be LinkedHashMap to preserve order of service addition Map services = new LinkedHashMap(); + private ThreadLocalMap mdcContext; public DAGAppMaster(ApplicationAttemptId applicationAttemptId, ContainerId containerId, String nmHost, int nmPort, int nmHttpPort, @@ -343,6 +346,7 @@ public DAGAppMaster(ApplicationAttemptId applicationAttemptId, String [] localDirs, String[] logDirs, String clientVersion, Credentials credentials, String jobUserName, AMPluginDescriptorProto pluginDescriptorProto) { super(DAGAppMaster.class.getName()); + this.mdcContext = LoggingUtils.setupLog4j(); this.clock = clock; this.startTime = clock.getTime(); this.appSubmitTime = appSubmitTime; @@ -690,7 +694,7 @@ protected TaskSchedulerManager getTaskSchedulerManager() { private void handleInternalError(String errDiagnosticsPrefix, String errDiagDagEvent) { state = DAGAppMasterState.ERROR; if (currentDAG != null) { - _updateLoggers(currentDAG, "_post"); + updateLoggers(currentDAG, "_post"); LOG.info(errDiagnosticsPrefix + ". Aborting dag: " + currentDAG.getID()); // Inform the current DAG about the error sendEvent(new DAGEventInternalError(currentDAG.getID(), errDiagDagEvent)); @@ -760,7 +764,7 @@ protected synchronized void handle(DAGAppMasterEvent event) { if (!isSession) { LOG.info("Not a session, AM will unregister as DAG has completed"); this.taskSchedulerManager.setShouldUnregisterFlag(); - _updateLoggers(currentDAG, "_post"); + updateLoggers(currentDAG, "_post"); setStateOnDAGCompletion(); LOG.info("Shutting down on completion of dag:" + finishEvt.getDAGId()); shutdownHandler.shutdown(); @@ -768,7 +772,7 @@ protected synchronized void handle(DAGAppMasterEvent event) { LOG.info("DAG completed, dagId=" + finishEvt.getDAGId() + ", dagState=" + finishEvt.getDAGState()); lastDAGCompletionTime = clock.getTime(); - _updateLoggers(currentDAG, "_post"); + updateLoggers(currentDAG, "_post"); if (this.historyEventHandler.hasRecoveryFailed()) { String recoveryErrorMsg = "Recovery had a fatal error, shutting down session after" + " DAG completion"; @@ -879,9 +883,10 @@ protected synchronized void handle(DAGAppMasterEvent event) { } } - private void _updateLoggers(DAG dag, String appender) { + private void updateLoggers(DAG dag, String appender) { try { - TezUtilsInternal.updateLoggers(dag.getID().toString() + appender); + TezUtilsInternal.updateLoggers(dag.getConf(), dag.getID().toString() + appender, + LoggingUtils.getPatternForAM(dag.getConf())); } catch (FileNotFoundException e) { LOG.warn("Unable to update the logger. Continue with the old logger", e ); } @@ -2007,7 +2012,7 @@ public void serviceStart() throws Exception { + ", state=" + (recoveredDAGData.dagState == null ? "null" : recoveredDAGData.dagState) + ", failureReason=" + recoveredDAGData.reason); - _updateLoggers(recoveredDAGData.recoveredDAG, ""); + updateLoggers(recoveredDAGData.recoveredDAG, ""); if (recoveredDAGData.nonRecoverable) { addDiagnostic("DAG " + recoveredDAGData.recoveredDagID + " can not be recovered due to " + recoveredDAGData.reason); @@ -2042,7 +2047,7 @@ public void serviceStart() throws Exception { } } else { LOG.info("Found DAG to recover, dagId=" + recoveredDAGData.recoveredDAG.getID()); - _updateLoggers(recoveredDAGData.recoveredDAG, ""); + updateLoggers(recoveredDAGData.recoveredDAG, ""); DAGRecoveredEvent dagRecoveredEvent = new DAGRecoveredEvent(this.appAttemptID, recoveredDAGData.recoveredDAG.getID(), recoveredDAGData.recoveredDAG.getName(), recoveredDAGData.recoveredDAG.getUserName(), this.clock.getTime(), this.containerLogs); @@ -2467,7 +2472,9 @@ private void startDAG(DAGPlan dagPlan, Map additionalAMRe // /////////////////// Create the job itself. final DAG newDAG = createDAG(dagPlan); - _updateLoggers(newDAG, ""); + LoggingUtils.initLoggingContext(mdcContext, newDAG.getConf(), newDAG.getID().toString(), null); + + updateLoggers(newDAG, ""); if (LOG.isDebugEnabled()) { LOG.debug("Running a DAG with " + dagPlan.getVertexCount() + " vertices "); diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java index b89b12db2b..7ab532ad33 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java @@ -49,6 +49,7 @@ import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler; import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.api.ApplicationConstants.Environment; +import org.apache.log4j.helpers.ThreadLocalMap; import org.apache.tez.common.ContainerContext; import org.apache.tez.common.ContainerTask; import org.apache.tez.common.TezCommonUtils; @@ -63,6 +64,7 @@ import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.api.records.DAGProtos; +import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.dag.utils.RelocalizationUtils; import org.apache.tez.hadoop.shim.HadoopShim; @@ -71,6 +73,8 @@ import org.apache.tez.runtime.api.impl.ExecutionContextImpl; import org.apache.tez.runtime.common.objectregistry.ObjectRegistryImpl; import org.apache.tez.runtime.internals.api.TaskReporterInterface; +import org.apache.tez.util.LoggingUtils; + import org.apache.tez.util.TezRuntimeShutdownHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -125,6 +129,7 @@ public class TezChild { private TezVertexID lastVertexID; private final HadoopShim hadoopShim; private final TezExecutors sharedExecutor; + private ThreadLocalMap mdcContext; public TezChild(Configuration conf, String host, int port, String containerIdentifier, String tokenIdentifier, int appAttemptNumber, String workingDir, String[] localDirs, @@ -133,6 +138,7 @@ public TezChild(Configuration conf, String host, int port, String containerIdent ExecutionContext executionContext, Credentials credentials, long memAvailable, String user, TezTaskUmbilicalProtocol umbilical, boolean updateSysCounters, HadoopShim hadoopShim) throws IOException, InterruptedException { + this.mdcContext = LoggingUtils.setupLog4j(); this.defaultConf = conf; this.containerIdString = containerIdentifier; this.appAttemptNumber = appAttemptNumber; @@ -216,7 +222,7 @@ public ContainerExecutionResult run() throws IOException, InterruptedException, while (!executor.isTerminated() && !isShutdown.get()) { if (taskCount > 0) { - TezUtilsInternal.updateLoggers(""); + TezUtilsInternal.updateLoggers(defaultConf, "", LoggingUtils.getPatternForTask(defaultConf)); } ListenableFuture getTaskFuture = executor.submit(containerReporter); boolean error = false; @@ -240,6 +246,19 @@ public ContainerExecutionResult run() throws IOException, InterruptedException, shutdown(); } } + + TezTaskAttemptID attemptId = containerTask.getTaskSpec().getTaskAttemptID(); + if (containerTask.getTaskSpec().getTaskConf() != null) { + Configuration copy = new Configuration(defaultConf); + TezTaskRunner2.mergeTaskSpecConfToConf(containerTask.getTaskSpec(), copy); + + LoggingUtils.initLoggingContext(mdcContext, copy, + attemptId.getTaskID().getVertexID().getDAGID().toString(), attemptId.toString()); + } else { + LoggingUtils.initLoggingContext(mdcContext, defaultConf, + attemptId.getTaskID().getVertexID().getDAGID().toString(), attemptId.toString()); + } + TezCommonUtils.logCredentials(LOG, containerTask.getCredentials(), "containerTask"); if (containerTask.shouldDie()) { LOG.info("ContainerTask returned shouldDie=true for container {}, Exiting", containerIdString); @@ -256,7 +275,8 @@ public ContainerExecutionResult run() throws IOException, InterruptedException, containerTask.getTaskSpec().getTaskAttemptID().toString()); TezUtilsInternal.setHadoopCallerContext(hadoopShim, containerTask.getTaskSpec().getTaskAttemptID()); - TezUtilsInternal.updateLoggers(loggerAddend); + TezUtilsInternal.updateLoggers(defaultConf, loggerAddend, LoggingUtils.getPatternForTask(defaultConf)); + FileSystem.clearStatistics(); childUGI = handleNewTaskCredentials(containerTask, childUGI); @@ -270,6 +290,7 @@ public ContainerExecutionResult run() throws IOException, InterruptedException, serviceConsumerMetadata, serviceProviderEnvMap, startedInputsMap, taskReporter, executor, objectRegistry, pid, executionContext, memAvailable, updateSysCounters, hadoopShim, sharedExecutor); + boolean shouldDie; try { TaskRunner2Result result = taskRunner.run(); diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java index bbf037b8bf..ce379b5d0a 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java @@ -29,6 +29,7 @@ import java.util.concurrent.locks.ReentrantLock; import com.google.common.annotations.VisibleForTesting; + import org.apache.tez.common.Preconditions; import com.google.common.collect.Multimap; import org.apache.commons.lang.exception.ExceptionUtils; @@ -140,18 +141,22 @@ public TezTaskRunner2(Configuration tezConf, UserGroupInformation ugi, String[] this.umbilicalAndErrorHandler = new UmbilicalAndErrorHandler(); this.hadoopShim = hadoopShim; this.taskConf = new Configuration(tezConf); + mergeTaskSpecConfToConf(taskSpec, taskConf); + localExecutor = sharedExecutor == null ? new TezSharedExecutor(tezConf) : null; + this.task = new LogicalIOProcessorRuntimeTask(taskSpec, appAttemptNumber, taskConf, localDirs, + umbilicalAndErrorHandler, serviceConsumerMetadata, serviceProviderEnvMap, startedInputsMap, + objectRegistry, pid, executionContext, memAvailable, updateSysCounters, hadoopShim, + sharedExecutor == null ? localExecutor : sharedExecutor); + } + + static void mergeTaskSpecConfToConf(TaskSpec taskSpec, Configuration conf) { if (taskSpec.getTaskConf() != null) { Iterator> iter = taskSpec.getTaskConf().iterator(); while (iter.hasNext()) { Entry entry = iter.next(); - taskConf.set(entry.getKey(), entry.getValue()); + conf.set(entry.getKey(), entry.getValue()); } } - localExecutor = sharedExecutor == null ? new TezSharedExecutor(tezConf) : null; - this.task = new LogicalIOProcessorRuntimeTask(taskSpec, appAttemptNumber, taskConf, localDirs, - umbilicalAndErrorHandler, serviceConsumerMetadata, serviceProviderEnvMap, startedInputsMap, - objectRegistry, pid, executionContext, memAvailable, updateSysCounters, hadoopShim, - sharedExecutor == null ? localExecutor : sharedExecutor); } /** diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTezTaskRunner2.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTezTaskRunner2.java index 6876df93ec..a6d05beb5f 100644 --- a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTezTaskRunner2.java +++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTezTaskRunner2.java @@ -25,9 +25,14 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.tez.common.TezExecutors; import org.apache.tez.common.TezSharedExecutor; import org.apache.tez.dag.api.ProcessorDescriptor; +import org.apache.tez.dag.records.TezDAGID; +import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.dag.records.TezTaskID; +import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.hadoop.shim.DefaultHadoopShim; import org.apache.tez.runtime.api.impl.InputSpec; import org.apache.tez.runtime.api.impl.OutputSpec; @@ -50,8 +55,13 @@ public void testTaskConfUsage() throws Exception { List inputSpecList = new ArrayList<>(); List outputSpecList = new ArrayList<>(); - TaskSpec taskSpec = new TaskSpec("dagName", "vertexName", 1, mock(ProcessorDescriptor.class), - inputSpecList, outputSpecList, null, taskConf); + TaskSpec taskSpec = + new TaskSpec( + TezTaskAttemptID.getInstance( + TezTaskID.getInstance(TezVertexID + .getInstance(TezDAGID.getInstance(ApplicationId.fromString("application_1_1"), 0), 0), 0), 0), + "dagName", "vertexName", 1, mock(ProcessorDescriptor.class), inputSpecList, + outputSpecList, null, taskConf); TezExecutors sharedExecutor = new TezSharedExecutor(conf); TezTaskRunner2 taskRunner2 = new TezTaskRunner2(conf, mock(UserGroupInformation.class), localDirs, taskSpec, 1, null, null, null, mock(TaskReporter.class), null, null, "pid", diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java b/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java index 6de272358b..fd8f08b42c 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java @@ -164,10 +164,59 @@ public static void tearDown() { public void testHashJoinExample() throws Exception { HashJoinExample hashJoinExample = new HashJoinExample(); hashJoinExample.setConf(new Configuration(mrrTezCluster.getConfig())); - Path stagingDirPath = new Path("/tmp/tez-staging-dir"); - Path inPath1 = new Path("/tmp/hashJoin/inPath1"); - Path inPath2 = new Path("/tmp/hashJoin/inPath2"); - Path outPath = new Path("/tmp/hashJoin/outPath"); + runHashJoinExample(hashJoinExample); + } + + @Test(timeout = 60000) + public void testHashJoinExampleWithLogPattern() throws Exception { + HashJoinExample hashJoinExample = new HashJoinExample(); + + Configuration patternConfig = new Configuration(mrrTezCluster.getConfig()); + + patternConfig.set(TezConfiguration.TEZ_AM_LOG_LEVEL, "debug"); + patternConfig.set(TezConfiguration.TEZ_TASK_LOG_LEVEL, "debug"); + patternConfig.set(TezConfiguration.TEZ_LOG_PATTERN_LAYOUT_AM, + "%d{ISO8601} [%p] [%t (queryId=%X{queryId} dag=%X{dagId})] |%c{2}|: %m%n"); + patternConfig.set(TezConfiguration.TEZ_LOG_PATTERN_LAYOUT_TASK, + "%d{ISO8601} [%p] [%t (queryId=%X{queryId} dag=%X{dagId} task=%X{taskAttemptId})] |%c{2}|: %m%n"); + patternConfig.set(TezConfiguration.TEZ_MDC_CUSTOM_KEYS, "queryId"); + patternConfig.set(TezConfiguration.TEZ_MDC_CUSTOM_KEYS_CONF_PROPS, "hive.query.id"); + patternConfig.set("hive.query.id", "hello-upstream-application-12345"); + + //1. feature is on + //[main (queryId=hello-upstream-application-12345 dag=dag_1666683231618_0001_1)] |app.DAGAppMaster| + hashJoinExample.setConf(patternConfig); + runHashJoinExample(hashJoinExample); + + //2. feature is on, but custom keys are empty: expecting empty queryId with the same format + //[main (queryId= dag=dag_1666683231618_0002_1)] |app.DAGAppMaster| + patternConfig.set(TezConfiguration.TEZ_MDC_CUSTOM_KEYS, ""); + hashJoinExample.setConf(patternConfig); + runHashJoinExample(hashJoinExample); + + //3. feature is on, custom keys are defined but corresponding value is null in config: + //expecting empty queryId with the same format + //[main (queryId= dag=dag_1666683231618_0003_1)] |app.DAGAppMaster| + patternConfig.set(TezConfiguration.TEZ_MDC_CUSTOM_KEYS, "queryId"); + patternConfig.set(TezConfiguration.TEZ_MDC_CUSTOM_KEYS_CONF_PROPS, "hive.query.id.null"); + hashJoinExample.setConf(patternConfig); + runHashJoinExample(hashJoinExample); + + //4. feature is off: expecting to have properly formatted log lines with original log4j config (not empty string) + //[main] |app.DAGAppMaster| + patternConfig.set(TezConfiguration.TEZ_LOG_PATTERN_LAYOUT_AM, TezConfiguration.TEZ_LOG_PATTERN_LAYOUT_DEFAULT); + patternConfig.set(TezConfiguration.TEZ_LOG_PATTERN_LAYOUT_TASK, TezConfiguration.TEZ_LOG_PATTERN_LAYOUT_DEFAULT); + + hashJoinExample.setConf(patternConfig); + runHashJoinExample(hashJoinExample); + } + + private void runHashJoinExample(HashJoinExample hashJoinExample) throws Exception { + int random = new Random(System.currentTimeMillis()).nextInt(10000); + Path stagingDirPath = new Path(String.format("/tmp/tez-staging-dir%d", random)); + Path inPath1 = new Path(String.format("/tmp/hashJoin%d/inPath1", random)); + Path inPath2 = new Path(String.format("/tmp/hashJoin%d/inPath2", random)); + Path outPath = new Path(String.format("/tmp/hashJoin%d/outPath", random)); remoteFs.mkdirs(inPath1); remoteFs.mkdirs(inPath2); remoteFs.mkdirs(stagingDirPath); From 8ebc4b00f1d66ee88475b1e96691f658dab967be Mon Sep 17 00:00:00 2001 From: Ganesha Shreedhara Date: Wed, 26 Oct 2022 13:40:12 +0530 Subject: [PATCH 400/512] TEZ-4450: Shuffle data fetch fails when shuffle data is transferred via CompositeRoutedDataMovementEvent (#243) (Ganesha Shreedhara reviewed by Laszlo Bodor) --- .../impl/ShuffleInputEventHandlerImpl.java | 37 +++++----- .../TestShuffleInputEventHandlerImpl.java | 72 +++++++++++++++++++ .../org.mockito.plugins.MockMaker | 13 ++++ 3 files changed, 105 insertions(+), 17 deletions(-) create mode 100644 tez-runtime-library/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java index ca1259f30a..4f42f57a1e 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java @@ -171,8 +171,6 @@ public void logProgress(boolean updateOnClose) { private void processDataMovementEvent(DataMovementEvent dme, DataMovementEventPayloadProto shufflePayload, BitSet emptyPartitionsBitSet) throws IOException { int srcIndex = dme.getSourceIndex(); - String hostIdentifier = shufflePayload.getHost() + ":" + shufflePayload.getPort(); - if (LOG.isDebugEnabled()) { LOG.debug("DME srcIdx: " + srcIndex + ", targetIndex: " + dme.getTargetIndex() + ", attemptNum: " + dme.getVersion() + ", payload: " + ShuffleUtils @@ -198,20 +196,7 @@ private void processDataMovementEvent(DataMovementEvent dme, DataMovementEventPa CompositeInputAttemptIdentifier srcAttemptIdentifier = constructInputAttemptIdentifier(dme.getTargetIndex(), 1, dme.getVersion(), shufflePayload, (useSharedInputs && srcIndex == 0)); - if (shufflePayload.hasData()) { - DataProto dataProto = shufflePayload.getData(); - - FetchedInput fetchedInput = - inputAllocator.allocate(dataProto.getRawLength(), - dataProto.getCompressedLength(), srcAttemptIdentifier); - moveDataToFetchedInput(dataProto, fetchedInput, hostIdentifier); - shuffleManager.addCompletedInputWithData(srcAttemptIdentifier, fetchedInput); - - LOG.debug("Payload via DME : " + srcAttemptIdentifier); - } else { - shuffleManager.addKnownInput(shufflePayload.getHost(), shufflePayload.getPort(), - srcAttemptIdentifier, srcIndex); - } + processShufflePayload(shufflePayload, srcAttemptIdentifier, srcIndex); } private void moveDataToFetchedInput(DataProto dataProto, @@ -274,7 +259,25 @@ private void processCompositeRoutedDataMovementEvent(CompositeRoutedDataMovement CompositeInputAttemptIdentifier srcAttemptIdentifier = constructInputAttemptIdentifier(crdme.getTargetIndex(), crdme.getCount(), crdme.getVersion(), shufflePayload, (useSharedInputs && partitionId == 0)); - shuffleManager.addKnownInput(shufflePayload.getHost(), shufflePayload.getPort(), srcAttemptIdentifier, partitionId); + processShufflePayload(shufflePayload, srcAttemptIdentifier, partitionId); + } + + private void processShufflePayload(DataMovementEventPayloadProto shufflePayload, + CompositeInputAttemptIdentifier srcAttemptIdentifier, int srcIndex) throws IOException { + if (shufflePayload.hasData()) { + DataProto dataProto = shufflePayload.getData(); + String hostIdentifier = shufflePayload.getHost() + ":" + shufflePayload.getPort(); + FetchedInput fetchedInput = + inputAllocator.allocate(dataProto.getRawLength(), + dataProto.getCompressedLength(), srcAttemptIdentifier); + moveDataToFetchedInput(dataProto, fetchedInput, hostIdentifier); + shuffleManager.addCompletedInputWithData(srcAttemptIdentifier, fetchedInput); + + LOG.debug("Payload via DME : " + srcAttemptIdentifier); + } else { + shuffleManager.addKnownInput(shufflePayload.getHost(), shufflePayload.getPort(), + srcAttemptIdentifier, srcIndex); + } } private void processInputFailedEvent(InputFailedEvent ife) { diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java index 9f6b4a3a43..f4ddf590ef 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java @@ -19,17 +19,21 @@ package org.apache.tez.runtime.library.common.shuffle.impl; import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyBoolean; import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyLong; import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.io.IOException; +import java.io.InputStream; import java.nio.ByteBuffer; import java.util.BitSet; import java.util.Collections; @@ -40,6 +44,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.DataOutputBuffer; +import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.security.token.Token; import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.TezExecutors; @@ -54,15 +59,20 @@ import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.ExecutionContext; import org.apache.tez.runtime.api.InputContext; +import org.apache.tez.runtime.api.events.CompositeRoutedDataMovementEvent; import org.apache.tez.runtime.api.events.DataMovementEvent; import org.apache.tez.runtime.library.common.CompositeInputAttemptIdentifier; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; +import org.apache.tez.runtime.library.common.shuffle.FetchedInput; import org.apache.tez.runtime.library.common.shuffle.FetchedInputAllocator; +import org.apache.tez.runtime.library.common.shuffle.MemoryFetchedInput; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; +import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads; import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto; import org.junit.After; import org.junit.Before; import org.junit.Test; +import org.mockito.MockedStatic; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -341,6 +351,53 @@ public void testPipelinedShuffleEvents_WithEmptyPartitions() throws IOException verify(inputContext).killSelf(any(), anyString()); } + /** + * Verify that data movement events with shuffle data are processed properly. + * + * @throws IOException + */ + @Test(timeout = 5000) + public void testDataMovementEventsWithShuffleData() throws IOException { + InputContext inputContext = mock(InputContext.class); + ShuffleManager shuffleManager = mock(ShuffleManager.class); + ShuffleManager compositeFetchShuffleManager = mock(ShuffleManager.class); + FetchedInputAllocator inputAllocator = mock(FetchedInputAllocator.class); + MemoryFetchedInput memoryFetchedInput = mock(MemoryFetchedInput.class); + + when(memoryFetchedInput.getType()).thenReturn(FetchedInput.Type.MEMORY); + when(memoryFetchedInput.getBytes()).thenReturn("data".getBytes()); + when(inputAllocator.allocate(anyLong(), anyLong(), any(InputAttemptIdentifier.class))) + .thenReturn(memoryFetchedInput); + + ShuffleInputEventHandlerImpl eventHandler = new ShuffleInputEventHandlerImpl(inputContext, + shuffleManager, inputAllocator, null, true, 4, false); + + ShuffleInputEventHandlerImpl compositeFetchEventHandler = new ShuffleInputEventHandlerImpl(inputContext, + compositeFetchShuffleManager, inputAllocator, null, true, 4, true); + + DataMovementEvent dataMovementEvent = (DataMovementEvent) createDataMovementEventWithShuffleData(false); + CompositeRoutedDataMovementEvent compositeRoutedDataMovementEvent = + (CompositeRoutedDataMovementEvent) createDataMovementEventWithShuffleData(true); + + List eventListWithDme = new LinkedList<>(); + eventListWithDme.add(dataMovementEvent); + eventListWithDme.add(compositeRoutedDataMovementEvent); + + try (MockedStatic shuffleUtils = mockStatic(ShuffleUtils.class)) { + shuffleUtils.when(() -> ShuffleUtils + .shuffleToMemory(any(byte[].class), any(InputStream.class), anyInt(), anyInt(), any(CompressionCodec.class), + anyBoolean(), anyInt(), any(), any(InputAttemptIdentifier.class))) + .thenAnswer((Answer) invocation -> null); + eventHandler.handleEvents(eventListWithDme); + compositeFetchEventHandler.handleEvents(eventListWithDme); + + verify(shuffleManager, times(2)) + .addCompletedInputWithData(any(InputAttemptIdentifier.class), any(FetchedInput.class)); + verify(compositeFetchShuffleManager, times(2)) + .addCompletedInputWithData(any(InputAttemptIdentifier.class), any(FetchedInput.class)); + } + } + private Event createDataMovementEvent(boolean addSpillDetails, int srcIdx, int targetIdx, int spillId, boolean isLastSpill, BitSet emptyPartitions, int numPartitions, int attemptNum) throws IOException { @@ -396,4 +453,19 @@ private ByteString createEmptyPartitionByteString(int... emptyPartitions) throws return emptyPartitionsBytesString; } + private Event createDataMovementEventWithShuffleData(boolean isComposite) { + DataMovementEventPayloadProto.Builder builder = DataMovementEventPayloadProto.newBuilder(); + builder.setHost(HOST); + builder.setPort(PORT); + builder.setPathComponent(PATH_COMPONENT); + ShuffleUserPayloads.DataProto.Builder dataProtoBuilder = ShuffleUserPayloads.DataProto.newBuilder() + .setData(ByteString.copyFromUtf8("data")); + builder.setData(dataProtoBuilder); + + Event dme = isComposite? + CompositeRoutedDataMovementEvent.create(0, 1, 1, 0, builder.build().toByteString().asReadOnlyByteBuffer()): + DataMovementEvent.create(0, 1, 0, builder.build().toByteString().asReadOnlyByteBuffer()); + return dme; + } + } diff --git a/tez-runtime-library/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/tez-runtime-library/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker new file mode 100644 index 0000000000..a258d79ad3 --- /dev/null +++ b/tez-runtime-library/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker @@ -0,0 +1,13 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +mock-maker-inline From 8bb27b845ce103b8d02a9a9d7fe2a1af10097e9c Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Wed, 9 Nov 2022 22:50:46 +0800 Subject: [PATCH 401/512] TEZ-4454: remove extra commas. (#246) (slfan1989 reviewed by Laszlo Bodor) --- .../test/java/org/apache/tez/common/TestTezCommonUtils.java | 2 +- .../java/org/apache/tez/dag/app/dag/impl/VertexImpl.java | 6 +++--- .../library/common/shuffle/orderedgrouped/MergeManager.java | 2 +- .../tez/runtime/library/common/sort/impl/TezMerger.java | 2 +- tez-tests/src/test/java/org/apache/tez/test/TestInput.java | 2 +- tez-tests/src/test/java/org/apache/tez/test/TestOutput.java | 2 +- 6 files changed, 8 insertions(+), 8 deletions(-) diff --git a/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java b/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java index d5dc6fd6b5..e1ae6cd2cb 100644 --- a/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java +++ b/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java @@ -49,7 +49,7 @@ public class TestTezCommonUtils { private static final File LOCAL_STAGING_DIR = new File(System.getProperty("test.build.data"), TestTezCommonUtils.class.getSimpleName()).getAbsoluteFile(); private static String RESOLVED_STAGE_DIR; - private static Configuration conf = new Configuration();; + private static Configuration conf = new Configuration(); private static String TEST_ROOT_DIR = "target" + Path.SEPARATOR + TestTezCommonUtils.class.getName() + "-tmpDir"; private static MiniDFSCluster dfsCluster = null; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java index e55b10a659..f8f2750267 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java @@ -681,7 +681,7 @@ private void augmentStateMachine() { .registerStateEnteredCallback(VertexState.RUNNING, STATE_CHANGED_CALLBACK) .registerStateEnteredCallback(VertexState.INITIALIZING, - STATE_CHANGED_CALLBACK);; + STATE_CHANGED_CALLBACK); } private final StateMachineTez stateMachine; @@ -3886,7 +3886,7 @@ private void commitCompleted(VertexEventCommitCompleted commitCompletedEvent) { } else { String diag = "Commit failed for output:" + commitCompletedEvent.getOutputName() + ", vertexId=" + logIdentifier + ", " - + ExceptionUtils.getStackTrace(commitCompletedEvent.getException());; + + ExceptionUtils.getStackTrace(commitCompletedEvent.getException()); LOG.info(diag); addDiagnostic(diag); trySetTerminationCause(VertexTerminationCause.COMMIT_FAILURE); @@ -4398,7 +4398,7 @@ public void setOutputVertices(Map outVertices) { addIO(vertex.getName()); } } finally { - writeLock.unlock();; + writeLock.unlock(); } } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java index 8d3f37dbcd..9da1276b8a 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java @@ -322,7 +322,7 @@ void setupParentThread(Thread shuffleSchedulerThread) { if (this.memToMemMerger != null) { memToMemMerger.setParentThread(shuffleSchedulerThread); } - this.inMemoryMerger.setParentThread(shuffleSchedulerThread);; + this.inMemoryMerger.setParentThread(shuffleSchedulerThread); this.onDiskMerger.setParentThread(shuffleSchedulerThread); } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java index e83b34e114..12e5735199 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java @@ -699,7 +699,7 @@ protected boolean lessThan(Object a, Object b) { int s1 = key1.getPosition(); int l1 = key1.getLength(); int s2 = key2.getPosition(); - int l2 = key2.getLength();; + int l2 = key2.getLength(); return comparator.compare(key1.getData(), s1, l1, key2.getData(), s2, l2) < 0; } diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestInput.java b/tez-tests/src/test/java/org/apache/tez/test/TestInput.java index 36bb983d1d..7ad50b7e11 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestInput.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestInput.java @@ -393,7 +393,7 @@ public void handleEvents(List inputEvents) throws Exception { @Override public List close() throws Exception { - getContext().getCounters().findCounter(COUNTER_NAME, COUNTER_NAME).increment(1);; + getContext().getCounters().findCounter(COUNTER_NAME, COUNTER_NAME).increment(1); return null; } diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestOutput.java b/tez-tests/src/test/java/org/apache/tez/test/TestOutput.java index 8b292ab196..b595743f02 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestOutput.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestOutput.java @@ -80,7 +80,7 @@ public void handleEvents(List outputEvents) { @Override public List close() throws Exception { LOG.info("Sending data movement event with value: " + output); - getContext().getCounters().findCounter(COUNTER_NAME, COUNTER_NAME).increment(1);; + getContext().getCounters().findCounter(COUNTER_NAME, COUNTER_NAME).increment(1); ByteBuffer result = ByteBuffer.allocate(4).putInt(output); result.flip(); List events = Lists.newArrayListWithCapacity(getNumPhysicalOutputs()); From 65f9ee32a46518c89ad2347a8e39541d3e669e2d Mon Sep 17 00:00:00 2001 From: Dmitriy Fingerman Date: Wed, 23 Nov 2022 14:25:58 -0500 Subject: [PATCH 402/512] TEZ-4455: Add LoggingHandler in ShuffleHandler pipeline for better debuggability. (#249) (Dmitriy Fingerman reviewed by Laszlo Bodor) --- .../main/java/org/apache/tez/auxservices/ShuffleHandler.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java index 0fa1c03a96..3262c7c342 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java @@ -33,6 +33,8 @@ import static io.netty.handler.codec.http.HttpResponseStatus.OK; import static io.netty.handler.codec.http.HttpResponseStatus.UNAUTHORIZED; import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1; +import io.netty.handler.logging.LogLevel; +import io.netty.handler.logging.LoggingHandler; import java.io.File; import java.io.FileNotFoundException; @@ -603,6 +605,9 @@ public void initChannel(NioSocketChannel ch) throws Exception { if (sslFactory != null) { pipeline.addLast("ssl", new SslHandler(sslFactory.createSSLEngine())); } + if (LOG.isDebugEnabled()) { + pipeline.addLast("loggingHandler", new LoggingHandler(LogLevel.DEBUG)); + } pipeline.addLast("decoder", new HttpRequestDecoder()); pipeline.addLast("aggregator", new HttpObjectAggregator(1 << 16)); pipeline.addLast("encoder", new HttpResponseEncoder()); From 2fd7df4cf2b22c5ed1f34c5dc1c865fc75e3d832 Mon Sep 17 00:00:00 2001 From: ghanko <54805928+ghanko@users.noreply.github.com> Date: Sat, 26 Nov 2022 09:04:15 +0100 Subject: [PATCH 403/512] TEZ-4396 Ensure utility classes have only private (default) constructors + several code refactors (#197) (Gergely Hanko reviewed by Laszlo Bodor) --- .../org/apache/tez/client/TezClientUtils.java | 4 +- .../org/apache/tez/common/ATSConstants.java | 1 + .../java/org/apache/tez/common/RPCUtil.java | 25 +- .../apache/tez/common/ReflectionUtils.java | 6 +- .../org/apache/tez/common/TezCommonUtils.java | 21 +- .../java/org/apache/tez/common/TezUtils.java | 24 +- .../org/apache/tez/common/TezYARNUtils.java | 14 +- .../apache/tez/common/security/Master.java | 6 +- .../tez/common/security/TokenCache.java | 23 +- .../apache/tez/dag/api/DagTypeConverters.java | 4 +- .../dag/api/TezConfigurationConstants.java | 4 +- .../org/apache/tez/dag/api/TezConstants.java | 3 +- .../dag/api/client/TimelineReaderFactory.java | 38 ++- .../org/apache/tez/client/TestTezClient.java | 2 +- .../org/apache/tez/common/TezTestUtils.java | 6 +- .../apache/tez/common/TezUtilsInternal.java | 33 ++- .../tez/dag/utils/RelocalizationUtils.java | 7 +- .../src/main/java/org/apache/tez/Utils.java | 6 +- .../app/dag/impl/TaskAttemptImplHelpers.java | 4 +- .../app/rm/container/AMContainerHelpers.java | 53 ++--- .../impl/HistoryEventJsonConversion.java | 8 +- .../tez/dag/history/utils/DAGUtils.java | 4 +- .../tez/dag/history/utils/TezEventUtils.java | 4 +- .../org/apache/tez/dag/utils/ProtoUtils.java | 4 +- .../apache/tez/dag/utils/TezBuilderUtils.java | 4 +- .../tez/dag/utils/TezRuntimeChildJVM.java | 14 +- .../tez/dag/app/PluginWrapperTestHelpers.java | 21 +- .../dag/app/rm/TestTaskSchedulerHelpers.java | 27 +-- .../apache/tez/examples/ExampleDriver.java | 18 +- .../service/TezTestServiceConfConstants.java | 3 +- .../org/apache/tez/util/ProtoConverters.java | 4 +- .../split/SplitMetaInfoReaderTez.java | 7 +- .../apache/tez/common/MRFrameworkConfigs.java | 4 +- .../apache/tez/mapreduce/common/Utils.java | 6 +- .../tez/mapreduce/hadoop/DeprecatedKeys.java | 44 ++-- .../tez/mapreduce/hadoop/IDConverter.java | 4 +- .../tez/mapreduce/hadoop/MRHelpers.java | 13 +- .../tez/mapreduce/hadoop/MRInputHelpers.java | 98 +++----- .../MultiStageMRConfToTezTranslator.java | 12 +- .../hadoop/MultiStageMRConfigUtil.java | 13 +- .../mapreduce/hadoop/TezTypeConverters.java | 10 +- .../apache/tez/mapreduce/input/MRInput.java | 18 +- .../tez/mapreduce/lib/MRInputUtils.java | 19 +- .../apache/tez/mapreduce/TezTestUtils.java | 7 +- .../tez/mapreduce/processor/MapUtils.java | 27 ++- .../tez/history/parser/utils/Utils.java | 3 +- .../ats/HistoryEventTimelineConversion.java | 4 +- .../apache/tez/common/ProtoConverters.java | 3 +- .../apache/tez/common/TezConverterUtils.java | 5 +- .../task/TaskExecutionTestHelpers.java | 29 ++- .../runtime/task/TestContainerExecution.java | 2 +- .../common/TezRuntimeFrameworkConfigs.java | 4 +- .../library/api/TezRuntimeConfiguration.java | 223 +++++++++--------- .../runtime/library/common/ConfigUtils.java | 22 +- .../tez/runtime/library/common/Constants.java | 3 +- .../library/common/TezRuntimeUtils.java | 74 ++---- .../common/security/SecureShuffleUtils.java | 38 ++- .../library/common/shuffle/ShuffleUtils.java | 4 +- .../library/common/sort/impl/IFile.java | 80 +------ .../library/common/sort/impl/TezMerger.java | 68 +++--- .../tez/runtime/library/conf/Utils.java | 4 +- .../runtime/library/utils/BufferUtils.java | 5 +- .../library/utils/FastByteComparisons.java | 12 +- .../common/sort/impl/TestTezMerger.java | 117 ++++----- .../library/output/OutputTestHelpers.java | 11 +- .../runtime/library/testutils/KVDataGen.java | 12 +- .../testutils/RandomTextGenerator.java | 10 +- .../tez/mapreduce/examples/ExampleDriver.java | 10 +- .../org/apache/tez/test/SimpleTestDAG.java | 8 +- .../tez/test/SimpleTestDAG3Vertices.java | 4 +- .../java/org/apache/tez/test/TestDriver.java | 6 +- .../apache/tez/test/TestFaultTolerance.java | 26 +- .../tez/test/dag/FailingDagBuilder.java | 166 +++++++++++++ .../apache/tez/test/dag/MultiAttemptDAG.java | 48 ++-- .../tez/test/dag/SimpleReverseVTestDAG.java | 4 +- .../apache/tez/test/dag/SimpleVTestDAG.java | 4 +- .../tez/test/dag/SixLevelsFailingDAG.java | 95 -------- .../tez/test/dag/ThreeLevelsFailingDAG.java | 75 ------ .../tez/test/dag/TwoLevelsFailingDAG.java | 113 --------- .../tez/analyzer/plugins/AnalyzerDriver.java | 4 +- .../org/apache/tez/analyzer/utils/Utils.java | 4 +- .../javadoc/doclet/ConfigStandardDoclet.java | 12 +- 82 files changed, 865 insertions(+), 1124 deletions(-) create mode 100644 tez-tests/src/test/java/org/apache/tez/test/dag/FailingDagBuilder.java delete mode 100644 tez-tests/src/test/java/org/apache/tez/test/dag/SixLevelsFailingDAG.java delete mode 100644 tez-tests/src/test/java/org/apache/tez/test/dag/ThreeLevelsFailingDAG.java delete mode 100644 tez-tests/src/test/java/org/apache/tez/test/dag/TwoLevelsFailingDAG.java diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index d78ccf3e89..1225181fe2 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -116,11 +116,13 @@ import com.google.common.collect.Lists; @Private -public class TezClientUtils { +public final class TezClientUtils { private static Logger LOG = LoggerFactory.getLogger(TezClientUtils.class); private static final int UTF8_CHUNK_SIZE = 16 * 1024; + private TezClientUtils() {} + private static FileStatus[] getLRFileStatus(String fileName, Configuration conf) throws IOException { URI uri; diff --git a/tez-api/src/main/java/org/apache/tez/common/ATSConstants.java b/tez-api/src/main/java/org/apache/tez/common/ATSConstants.java index 47d536fdd9..e3c90d3154 100644 --- a/tez-api/src/main/java/org/apache/tez/common/ATSConstants.java +++ b/tez-api/src/main/java/org/apache/tez/common/ATSConstants.java @@ -138,4 +138,5 @@ public class ATSConstants { public static final String CALLER_TYPE = "callerType"; public static final String DESCRIPTION = "description"; + protected ATSConstants() {} } diff --git a/tez-api/src/main/java/org/apache/tez/common/RPCUtil.java b/tez-api/src/main/java/org/apache/tez/common/RPCUtil.java index caeb822bc7..232ad6a7cd 100644 --- a/tez-api/src/main/java/org/apache/tez/common/RPCUtil.java +++ b/tez-api/src/main/java/org/apache/tez/common/RPCUtil.java @@ -29,7 +29,9 @@ import com.google.protobuf.ServiceException; -public class RPCUtil { +public final class RPCUtil { + + private RPCUtil() {} /** * Returns an instance of {@link TezException} @@ -55,17 +57,8 @@ private static T instantiateException( return ex; // RemoteException contains useful information as against the // java.lang.reflect exceptions. - } catch (NoSuchMethodException e) { - throw re; - } catch (IllegalArgumentException e) { - throw re; - } catch (SecurityException e) { - throw re; - } catch (InstantiationException e) { - throw re; - } catch (IllegalAccessException e) { - throw re; - } catch (InvocationTargetException e) { + } catch (NoSuchMethodException | IllegalArgumentException | SecurityException | InstantiationException + | IllegalAccessException | InvocationTargetException e) { throw re; } } @@ -85,12 +78,6 @@ private static T instantiateRuntimeException( return instantiateException(cls, re); } - private static T instantiateSessionNotRunningException( - Class cls, RemoteException re) throws RemoteException { - return instantiateException(cls, re); - } - - /** * Utility method that unwraps and returns appropriate exceptions. * @@ -109,7 +96,7 @@ public static Void unwrapAndThrowException(ServiceException se) } else { if (cause instanceof RemoteException) { RemoteException re = (RemoteException) cause; - Class realClass = null; + Class realClass; try { realClass = Class.forName(re.getClassName()); } catch (ClassNotFoundException cnf) { diff --git a/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java b/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java index 73becdaa6a..5bfb41c586 100644 --- a/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java @@ -29,9 +29,11 @@ import org.apache.tez.dag.api.TezReflectionException; @Private -public class ReflectionUtils { +public final class ReflectionUtils { - private static final Map> CLAZZ_CACHE = new ConcurrentHashMap>(); + private static final Map> CLAZZ_CACHE = new ConcurrentHashMap<>(); + + private ReflectionUtils() {} @Private public static Class getClazz(String className) throws TezReflectionException { diff --git a/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java index 5c2876444c..ca5fcdf3e2 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java @@ -55,7 +55,7 @@ import com.google.protobuf.ByteString; @Private -public class TezCommonUtils { +public final class TezCommonUtils { public static final FsPermission TEZ_AM_DIR_PERMISSION = FsPermission .createImmutable((short) 0700); // rwx-------- public static final FsPermission TEZ_AM_FILE_PERMISSION = FsPermission @@ -64,6 +64,8 @@ public class TezCommonUtils { public static final String TEZ_SYSTEM_SUB_DIR = ".tez"; + private TezCommonUtils() {} + /** *

* This function returns the staging directory defined in the config with @@ -222,7 +224,6 @@ public static Path getTezTextPlanStagingPath(Path tezSysStagingPath, String strA * @param conf * Tez configuration * @return App recovery path - * @throws IOException */ @Private public static Path getRecoveryPath(Path tezSysStagingPath, Configuration conf) @@ -288,7 +289,6 @@ public static Path getSummaryRecoveryPath(Path attemptRecoverPath) { * Filesystem * @param dir * directory to be created - * @throws IOException */ public static void mkDirForAM(FileSystem fs, Path dir) throws IOException { FsPermission perm = new FsPermission(TEZ_AM_DIR_PERMISSION); @@ -312,7 +312,6 @@ public static void mkDirForAM(FileSystem fs, Path dir) throws IOException { * @param filePath * file path to create the file * @return FSDataOutputStream - * @throws IOException */ public static FSDataOutputStream createFileForAM(FileSystem fs, Path filePath) throws IOException { return FileSystem.create(fs, filePath, new FsPermission(TEZ_AM_FILE_PERMISSION)); @@ -417,7 +416,7 @@ public static String getCredentialsInfo(Credentials credentials, String identifi } StringBuilder sb = new StringBuilder(); - sb.append("Credentials: #" + identifier + "Tokens=").append(credentials.numberOfTokens()); + sb.append("Credentials: #").append(identifier).append("Tokens=").append(credentials.numberOfTokens()); if (credentials.numberOfTokens() > 0) { sb.append(", Services="); sb.append(credentials.getAllTokens().stream() @@ -435,16 +434,14 @@ public static ByteBuffer convertJobTokenToBytes( Token jobToken) throws IOException { DataOutputBuffer dob = new DataOutputBuffer(); jobToken.write(dob); - ByteBuffer bb = ByteBuffer.wrap(dob.getData(), 0, dob.getLength()); - return bb; + return ByteBuffer.wrap(dob.getData(), 0, dob.getLength()); } public static Credentials parseCredentialsBytes(byte[] credentialsBytes) throws IOException { Credentials credentials = new Credentials(); DataInputBuffer dib = new DataInputBuffer(); try { - byte[] tokenBytes = credentialsBytes; - dib.reset(tokenBytes, tokenBytes.length); + dib.reset(credentialsBytes, credentialsBytes.length); credentials.readTokenStorageStream(dib); return credentials; } finally { @@ -459,7 +456,7 @@ public static void logCredentials(Logger log, Credentials credentials, String id } public static Collection tokenizeString(String str, String delim) { - List values = new ArrayList(); + List values = new ArrayList<>(); if (str == null || str.isEmpty()) return values; StringTokenizer tokenizer = new StringTokenizer(str, delim); @@ -533,7 +530,7 @@ public static long getAMClientHeartBeatTimeoutMillis(Configuration conf) { if (val > 0 && val < TezConstants.TEZ_AM_CLIENT_HEARTBEAT_TIMEOUT_SECS_MINIMUM) { return TezConstants.TEZ_AM_CLIENT_HEARTBEAT_TIMEOUT_SECS_MINIMUM * 1000; } - return val * 1000; + return val * 1000L; } /** @@ -570,7 +567,7 @@ public static long getDAGSessionTimeout(Configuration conf) { if (timeoutSecs == 0) { timeoutSecs = 1; } - return 1000l * timeoutSecs; + return 1000L * timeoutSecs; } public static int getJavaVersion() { diff --git a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java index 1c0be98dcc..88920a4e53 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -49,10 +48,12 @@ * {@link org.apache.hadoop.conf.Configuration} to {@link org.apache.tez.dag.api.UserPayload} etc. */ @InterfaceAudience.Public -public class TezUtils { +public final class TezUtils { private static final Logger LOG = LoggerFactory.getLogger(TezUtils.class); + private TezUtils() {} + /** * Allows changing the log level for task / AM logging.

* @@ -73,18 +74,12 @@ public static void addLog4jSystemProperties(String logLevel, * @param conf * : Configuration to be converted * @return PB ByteString (compressed) - * @throws java.io.IOException */ public static ByteString createByteStringFromConf(Configuration conf) throws IOException { Objects.requireNonNull(conf, "Configuration must be specified"); ByteString.Output os = ByteString.newOutput(); - SnappyOutputStream compressOs = new SnappyOutputStream(os); - try { + try (SnappyOutputStream compressOs = new SnappyOutputStream(os)) { writeConfInPB(compressOs, conf); - } finally { - if (compressOs != null) { - compressOs.close(); - } } return os.toByteString(); } @@ -95,7 +90,6 @@ public static ByteString createByteStringFromConf(Configuration conf) throws IOE * * @param conf configuration to be converted * @return an instance of {@link org.apache.tez.dag.api.UserPayload} - * @throws java.io.IOException */ public static UserPayload createUserPayloadFromConf(Configuration conf) throws IOException { return UserPayload.create(ByteBuffer.wrap(createByteStringFromConf(conf).toByteArray())); @@ -113,11 +107,10 @@ private static DAGProtos.ConfigurationProto createConfProto(SnappyInputStream un * @param byteString byteString representation of the conf created using {@link * #createByteStringFromConf(org.apache.hadoop.conf.Configuration)} * @return Configuration - * @throws java.io.IOException */ public static Configuration createConfFromByteString(ByteString byteString) throws IOException { Objects.requireNonNull(byteString, "ByteString must be specified"); - try(SnappyInputStream uncompressIs = new SnappyInputStream(byteString.newInput());) { + try(SnappyInputStream uncompressIs = new SnappyInputStream(byteString.newInput())) { DAGProtos.ConfigurationProto confProto = createConfProto(uncompressIs); Configuration conf = new Configuration(false); readConfFromPB(confProto, conf); @@ -156,7 +149,6 @@ public static void addToConfFromByteString(Configuration configuration, ByteStri * @param payload {@link org.apache.tez.dag.api.UserPayload} created using {@link * #createUserPayloadFromConf(org.apache.hadoop.conf.Configuration)} * @return Configuration - * @throws java.io.IOException */ public static Configuration createConfFromUserPayload(UserPayload payload) throws IOException { return createConfFromByteString(ByteString.copyFrom(payload.getPayload())); @@ -186,12 +178,10 @@ public static String convertToHistoryText(String description, Configuration conf } if (conf != null) { JSONObject confJson = new JSONObject(); - Iterator> iter = conf.iterator(); - while (iter.hasNext()) { - Entry entry = iter.next(); + for (Entry entry : conf) { String key = entry.getKey(); String val = conf.get(entry.getKey()); - if(val != null) { + if (val != null) { confJson.put(key, val); } else { LOG.debug("null value in Configuration after replacement for key={}. Skipping.", key); diff --git a/tez-api/src/main/java/org/apache/tez/common/TezYARNUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezYARNUtils.java index bd6de11a59..1e5d4bdf4a 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezYARNUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezYARNUtils.java @@ -35,7 +35,7 @@ import org.apache.tez.dag.api.TezConstants; @Private -public class TezYARNUtils { +public final class TezYARNUtils { private static Logger LOG = LoggerFactory.getLogger(TezYARNUtils.class); public static final String ENV_NAME_REGEX = "[A-Za-z_][A-Za-z0-9_]*"; @@ -49,6 +49,8 @@ public class TezYARNUtils { + "([^,]*)" // val group ); + private TezYARNUtils() {} + public static String getFrameworkClasspath(Configuration conf, boolean usingArchive) { StringBuilder classpathBuilder = new StringBuilder(); boolean userClassesTakesPrecedence = @@ -126,9 +128,11 @@ private static void addUserSpecifiedClasspath(StringBuilder classpathBuilder, // Add PWD:PWD/* classpathBuilder.append(Environment.PWD.$()) - .append(File.pathSeparator) - .append(Environment.PWD.$() + File.separator + "*") - .append(File.pathSeparator); + .append(File.pathSeparator) + .append(Environment.PWD.$()) + .append(File.separator) + .append("*") + .append(File.pathSeparator); } public static void appendToEnvFromInputString(Map env, @@ -161,7 +165,7 @@ public static void appendToEnvFromInputString(Map env, public static void setEnvIfAbsentFromInputString(Map env, String envString) { if (envString != null && envString.length() > 0) { - String childEnvs[] = envString.split(","); + String[] childEnvs = envString.split(","); for (String cEnv : childEnvs) { String[] parts = cEnv.split("="); // split on '=' Matcher m = VAR_SUBBER .matcher(parts[1]); diff --git a/tez-api/src/main/java/org/apache/tez/common/security/Master.java b/tez-api/src/main/java/org/apache/tez/common/security/Master.java index de73d10092..d0b8d16c47 100644 --- a/tez-api/src/main/java/org/apache/tez/common/security/Master.java +++ b/tez-api/src/main/java/org/apache/tez/common/security/Master.java @@ -29,12 +29,14 @@ @Private @Unstable -public class Master { +public final class Master { public enum State { - INITIALIZING, RUNNING; + INITIALIZING, RUNNING } + private Master() {} + public static String getMasterUserName(Configuration conf) { return conf.get(YarnConfiguration.RM_PRINCIPAL); } diff --git a/tez-api/src/main/java/org/apache/tez/common/security/TokenCache.java b/tez-api/src/main/java/org/apache/tez/common/security/TokenCache.java index e56ef61270..21b1026d39 100644 --- a/tez-api/src/main/java/org/apache/tez/common/security/TokenCache.java +++ b/tez-api/src/main/java/org/apache/tez/common/security/TokenCache.java @@ -45,14 +45,16 @@ */ @InterfaceAudience.Private @InterfaceStability.Unstable -public class TokenCache { +public final class TokenCache { private static final Logger LOG = LoggerFactory.getLogger(TokenCache.class); + private TokenCache() {} + /** * auxiliary method to get user's secret keys.. - * @param alias + * * @return secret key from the storage */ public static byte[] getSecretKey(Credentials credentials, Text alias) { @@ -64,10 +66,9 @@ public static byte[] getSecretKey(Credentials credentials, Text alias) { /** * Convenience method to obtain delegation tokens from namenodes * corresponding to the paths passed. - * @param credentials + * @param credentials credentials * @param ps array of paths * @param conf configuration - * @throws IOException */ public static void obtainTokensForFileSystems(Credentials credentials, Path[] ps, Configuration conf) throws IOException { @@ -80,7 +81,7 @@ public static void obtainTokensForFileSystems(Credentials credentials, private static final int MAX_FS_OBJECTS = 10; static void obtainTokensForFileSystemsInternal(Credentials credentials, Path[] ps, Configuration conf) throws IOException { - Set fsSet = new HashSet(); + Set fsSet = new HashSet<>(); boolean limitExceeded = false; for(Path p: ps) { FileSystem fs = p.getFileSystem(conf); @@ -107,8 +108,8 @@ static boolean isTokenRenewalExcluded(FileSystem fs, Configuration conf) { conf.getStrings(TezConfiguration.TEZ_JOB_FS_SERVERS_TOKEN_RENEWAL_EXCLUDE); if (nns != null) { String host = fs.getUri().getHost(); - for(int i = 0; i < nns.length; i++) { - if (nns[i].equals(host)) { + for (String nn : nns) { + if (nn.equals(host)) { return true; } } @@ -118,11 +119,6 @@ static boolean isTokenRenewalExcluded(FileSystem fs, Configuration conf) { /** * get delegation token for a specific FS - * @param fs - * @param credentials - * @param p - * @param conf - * @throws IOException */ static void obtainTokensForFileSystemsInternal(FileSystem fs, Credentials credentials, Configuration conf) throws IOException { @@ -137,7 +133,7 @@ static void obtainTokensForFileSystemsInternal(FileSystem fs, } } - final Token tokens[] = fs.addDelegationTokens(delegTokenRenewer, + final Token[] tokens = fs.addDelegationTokens(delegTokenRenewer, credentials); if (tokens != null) { for (Token token : tokens) { @@ -150,7 +146,6 @@ static void obtainTokensForFileSystemsInternal(FileSystem fs, /** * store session specific token - * @param t */ @InterfaceAudience.Private public static void setSessionToken(Token t, diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java index c563f1fb47..a55e45a7fd 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java @@ -90,7 +90,9 @@ import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor; @Private -public class DagTypeConverters { +public final class DagTypeConverters { + + private DagTypeConverters() {} public static PlanLocalResourceVisibility convertToDAGPlan(LocalResourceVisibility visibility){ switch(visibility){ diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfigurationConstants.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfigurationConstants.java index 33abc77de4..4bd5e254fb 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfigurationConstants.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfigurationConstants.java @@ -27,12 +27,14 @@ */ @ConfigurationClass(templateFileName = "tez-conf-constants.xml") @Private -public class TezConfigurationConstants { +public final class TezConfigurationConstants { static { TezConfiguration.setupConfigurationScope(TezConfigurationConstants.class); } + private TezConfigurationConstants() {} + /** * String value. Set automatically by the client. The host name of the client the Tez application * was submitted from. diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java index cf5ab11ec7..379eb0cb1b 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java @@ -24,7 +24,7 @@ * Specifies all constant values in Tez */ @Private -public class TezConstants { +public final class TezConstants { public static final String TEZ_APPLICATION_MASTER_CLASS = @@ -129,4 +129,5 @@ public static String getTezUberServicePluginName() { public static final double TEZ_CONTAINER_MAX_JAVA_HEAP_FRACTION_SMALL_SLAB = 0.7; public static final double TEZ_CONTAINER_MAX_JAVA_HEAP_FRACTION_LARGE_SLAB = 0.8; + private TezConstants() {} } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/TimelineReaderFactory.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/TimelineReaderFactory.java index fc1595fa49..2d0476a721 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/TimelineReaderFactory.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/TimelineReaderFactory.java @@ -61,7 +61,7 @@ * */ @InterfaceAudience.Private -public class TimelineReaderFactory { +public final class TimelineReaderFactory { private static final Logger LOG = LoggerFactory.getLogger(TimelineReaderFactory.class); @@ -79,6 +79,8 @@ public class TimelineReaderFactory { private static Class delegationTokenAuthenticatorClazz = null; private static Method delegationTokenAuthenticateURLOpenConnectionMethod = null; + private TimelineReaderFactory() {} + public static TimelineReaderStrategy getTimelineReaderStrategy(Configuration conf, boolean useHttps, int connTimeout) throws TezException { @@ -140,17 +142,14 @@ public interface TimelineReaderStrategy { * auth strategy for secured and unsecured environment with delegation token (hadoop 2.6 and above) */ private static class TimelineReaderTokenAuthenticatedStrategy implements TimelineReaderStrategy { - private final Configuration conf; private final boolean useHttps; private final int connTimeout; - private ConnectionConfigurator connectionConfigurator; - private SSLFactory sslFactory; + private final SSLFactory sslFactory; public TimelineReaderTokenAuthenticatedStrategy(final Configuration conf, final boolean useHttps, final int connTimeout) { - this.conf = conf; this.useHttps = useHttps; this.connTimeout = connTimeout; this.sslFactory = useHttps ? new SSLFactory(CLIENT, conf) : null; @@ -161,11 +160,10 @@ public Client getHttpClient() throws IOException { Authenticator authenticator; UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); UserGroupInformation realUgi = ugi.getRealUser(); - UserGroupInformation authUgi; String doAsUser; ClientConfig clientConfig = new DefaultClientConfig(JSONRootElementProvider.App.class); - connectionConfigurator = getNewConnectionConf(conf, useHttps, - connTimeout, sslFactory); + ConnectionConfigurator connectionConfigurator = getNewConnectionConf(useHttps, + connTimeout, sslFactory); try { authenticator = getTokenAuthenticator(); @@ -175,17 +173,15 @@ public Client getHttpClient() throws IOException { } if (realUgi != null) { - authUgi = realUgi; doAsUser = ugi.getShortUserName(); } else { - authUgi = ugi; doAsUser = null; } HttpURLConnectionFactory connectionFactory; try { connectionFactory = new TokenAuthenticatedURLConnectionFactory(connectionConfigurator, authenticator, - authUgi, doAsUser); + doAsUser); } catch (TezException e) { throw new IOException("Fail to create TokenAuthenticatedURLConnectionFactory", e); } @@ -208,17 +204,14 @@ private static class TokenAuthenticatedURLConnectionFactory implements HttpURLCo private final Authenticator authenticator; private final ConnectionConfigurator connConfigurator; - private final UserGroupInformation authUgi; private final String doAsUser; private final AuthenticatedURL.Token token; public TokenAuthenticatedURLConnectionFactory(ConnectionConfigurator connConfigurator, Authenticator authenticator, - UserGroupInformation authUgi, String doAsUser) throws TezException { this.connConfigurator = connConfigurator; this.authenticator = authenticator; - this.authUgi = authUgi; this.doAsUser = doAsUser; this.token = ReflectionUtils.createClazzInstance( DELEGATION_TOKEN_AUTHENTICATED_URL_TOKEN_CLASS_NAME, null, null); @@ -264,15 +257,14 @@ public TimelineReaderPseudoAuthenticatedStrategy(final Configuration conf, final boolean useHttps, final int connTimeout) { sslFactory = useHttps ? new SSLFactory(CLIENT, conf) : null; - connectionConf = getNewConnectionConf(conf, useHttps, connTimeout, sslFactory); + connectionConf = getNewConnectionConf(useHttps, connTimeout, sslFactory); } @Override public Client getHttpClient() { ClientConfig config = new DefaultClientConfig(JSONRootElementProvider.App.class); HttpURLConnectionFactory urlFactory = new PseudoAuthenticatedURLConnectionFactory(connectionConf); - Client httpClient = new Client(new URLConnectionClientHandler(urlFactory), config); - return httpClient; + return new Client(new URLConnectionClientHandler(urlFactory), config); } @VisibleForTesting @@ -289,7 +281,7 @@ public HttpURLConnection getHttpURLConnection(URL url) throws IOException { URLEncoder.encode(UserGroupInformation.getCurrentUser().getShortUserName(), "UTF8"); HttpURLConnection httpURLConnection = - (HttpURLConnection) (new URL(url.toString() + tokenString)).openConnection(); + (HttpURLConnection) (new URL(url + tokenString)).openConnection(); this.connectionConf.configure(httpURLConnection); return httpURLConnection; @@ -304,14 +296,13 @@ public void close() { } } - private static ConnectionConfigurator getNewConnectionConf(final Configuration conf, - final boolean useHttps, + private static ConnectionConfigurator getNewConnectionConf(final boolean useHttps, final int connTimeout, final SSLFactory sslFactory) { ConnectionConfigurator connectionConf = null; if (useHttps) { try { - connectionConf = getNewSSLConnectionConf(conf, connTimeout, sslFactory); + connectionConf = getNewSSLConnectionConf(connTimeout, sslFactory); } catch (IOException e) { LOG.debug("Cannot load customized ssl related configuration." + " Falling back to system-generic settings.", e); @@ -321,7 +312,7 @@ private static ConnectionConfigurator getNewConnectionConf(final Configuration c if (connectionConf == null) { connectionConf = new ConnectionConfigurator() { @Override - public HttpURLConnection configure(HttpURLConnection httpURLConnection) throws IOException { + public HttpURLConnection configure(HttpURLConnection httpURLConnection) { setTimeouts(httpURLConnection, connTimeout); return httpURLConnection; } @@ -331,8 +322,7 @@ public HttpURLConnection configure(HttpURLConnection httpURLConnection) throws I return connectionConf; } - private static ConnectionConfigurator getNewSSLConnectionConf(final Configuration conf, - final int connTimeout, + private static ConnectionConfigurator getNewSSLConnectionConf(final int connTimeout, final SSLFactory sslFactory) throws IOException { final SSLSocketFactory sslSocketFactory; diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java index 4f33348f89..728076cbee 100644 --- a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java +++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java @@ -522,7 +522,7 @@ public void testPreWarmWithTimeout() throws Exception { assertTrue("Time taken is not as expected", (endTime - startTime) > timeout); verify(spyClient, times(0)).submitDAG(any()); - Assert.assertTrue("Unexpected Exception message", + Assert.assertTrue("Unexpected Exception message: " + te.getMessage(), te.getMessage().contains("Tez AM not ready")); } diff --git a/tez-common/src/main/java/org/apache/tez/common/TezTestUtils.java b/tez-common/src/main/java/org/apache/tez/common/TezTestUtils.java index 4b9d5c3218..1cbacbd138 100644 --- a/tez-common/src/main/java/org/apache/tez/common/TezTestUtils.java +++ b/tez-common/src/main/java/org/apache/tez/common/TezTestUtils.java @@ -21,12 +21,14 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.conf.YarnConfiguration; -public class TezTestUtils { +public final class TezTestUtils { + + private TezTestUtils() {} + /** * Ensures a reasonably high limit for yarn disk utilization. This is very important for tests, * as devs keep bumping into silent test hangs where yarn simply considers their machines as unhealthy, * as the default limit is 90%, even if a machine with 90% full disk is still able to function. - * @param conf */ public static void ensureHighDiskUtilizationLimit(Configuration conf) { if (conf.getFloat(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, diff --git a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java index 7ee5bb457e..bdfc6e02c1 100644 --- a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java +++ b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java @@ -24,7 +24,7 @@ import java.lang.management.ManagementFactory; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.BitSet; import java.util.HashSet; import java.util.List; @@ -67,10 +67,12 @@ import org.apache.tez.dag.records.TaskAttemptTerminationCause; @Private -public class TezUtilsInternal { +public final class TezUtilsInternal { private static final Logger LOG = LoggerFactory.getLogger(TezUtilsInternal.class); + private TezUtilsInternal() {} + public static ConfigurationProto readUserSpecifiedTezConfiguration(String baseDir) throws IOException { File confPBFile = new File(baseDir, TezConstants.TEZ_PB_BINARY_CONF_NAME); @@ -88,7 +90,7 @@ public static void addUserSpecifiedTezConfiguration(Configuration conf, } } - public static byte[] compressBytes(byte[] inBytes) throws IOException { + public static byte[] compressBytes(byte[] inBytes) { StopWatch sw = new StopWatch().start(); byte[] compressed = compressBytesInflateDeflate(inBytes); sw.stop(); @@ -120,8 +122,7 @@ private static byte[] compressBytesInflateDeflate(byte[] inBytes) { int count = deflater.deflate(buffer); bos.write(buffer, 0, count); } - byte[] output = bos.toByteArray(); - return output; + return bos.toByteArray(); } private static byte[] uncompressBytesInflateDeflate(byte[] inBytes) throws IOException { @@ -138,8 +139,7 @@ private static byte[] uncompressBytesInflateDeflate(byte[] inBytes) throws IOExc } bos.write(buffer, 0, count); } - byte[] output = bos.toByteArray(); - return output; + return bos.toByteArray(); } private static final Pattern pattern = Pattern.compile("\\W"); @@ -154,8 +154,7 @@ public static String cleanVertexName(String vertexName) { private static String sanitizeString(String srcString) { Matcher matcher = pattern.matcher(srcString); - String res = matcher.replaceAll("_"); - return res; // Number starts allowed rightnow + return matcher.replaceAll("_"); // Number starts allowed rightnow } public static void updateLoggers(Configuration configuration, String addend, String patternString) @@ -168,15 +167,14 @@ public static void updateLoggers(Configuration configuration, String addend, Str if (appender != null) { if (appender instanceof TezContainerLogAppender) { TezContainerLogAppender claAppender = (TezContainerLogAppender) appender; - claAppender - .setLogFileName(constructLogFileName(TezConstants.TEZ_CONTAINER_LOG_FILE_NAME, addend)); + claAppender.setLogFileName(constructLogFileName( + addend)); // there was a configured pattern if (patternString != null) { PatternLayout layout = (PatternLayout) claAppender.getLayout(); layout.setConversionPattern(patternString); } - claAppender.activateOptions(); } else { LOG.warn("Appender is a " + appender.getClass() + "; require an instance of " @@ -188,11 +186,11 @@ public static void updateLoggers(Configuration configuration, String addend, Str } } - private static String constructLogFileName(String base, String addend) { + private static String constructLogFileName(String addend) { if (addend == null || addend.isEmpty()) { - return base; + return TezConstants.TEZ_CONTAINER_LOG_FILE_NAME; } else { - return base + "_" + addend; + return TezConstants.TEZ_CONTAINER_LOG_FILE_NAME + "_" + addend; } } @@ -225,7 +223,6 @@ public static byte[] toByteArray(BitSet bits) { /** * Convert DAGPlan to text. Skip sensitive informations like credentials. * - * @param dagPlan * @return a string representation of the dag plan with sensitive information removed */ public static String convertDagPlanToString(DAGProtos.DAGPlan dagPlan) throws IOException { @@ -238,7 +235,7 @@ public static String convertDagPlanToString(DAGProtos.DAGPlan dagPlan) throws IO DagTypeConverters.convertByteStringToCredentials(dagPlan.getCredentialsBinary()); TextFormat.printField(entry.getKey(), ByteString.copyFrom(TezCommonUtils.getCredentialsInfo(credentials,"dag").getBytes( - Charset.forName("UTF-8"))), sb); + StandardCharsets.UTF_8)), sb); } } return sb.toString(); @@ -266,8 +263,6 @@ public static TaskAttemptTerminationCause fromTaskAttemptEndReason( return TaskAttemptTerminationCause.NODE_FAILED; case CONTAINER_EXITED: return TaskAttemptTerminationCause.CONTAINER_EXITED; - case OTHER: - return TaskAttemptTerminationCause.UNKNOWN_ERROR; default: return TaskAttemptTerminationCause.UNKNOWN_ERROR; } diff --git a/tez-common/src/main/java/org/apache/tez/dag/utils/RelocalizationUtils.java b/tez-common/src/main/java/org/apache/tez/dag/utils/RelocalizationUtils.java index 84a9474a39..9ccfc76be7 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/utils/RelocalizationUtils.java +++ b/tez-common/src/main/java/org/apache/tez/dag/utils/RelocalizationUtils.java @@ -31,15 +31,16 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.tez.common.ReflectionUtils; -import org.apache.tez.dag.api.TezException; import com.google.common.collect.Lists; @InterfaceAudience.Private -public class RelocalizationUtils { +public final class RelocalizationUtils { + + private RelocalizationUtils() {} public static List processAdditionalResources(Map additionalResources, - Configuration conf, String destDir) throws IOException, TezException { + Configuration conf, String destDir) throws IOException { if (additionalResources == null || additionalResources.isEmpty()) { return Collections.emptyList(); } diff --git a/tez-dag/src/main/java/org/apache/tez/Utils.java b/tez-dag/src/main/java/org/apache/tez/Utils.java index b352334f44..1f9fb6a652 100644 --- a/tez-dag/src/main/java/org/apache/tez/Utils.java +++ b/tez-dag/src/main/java/org/apache/tez/Utils.java @@ -37,10 +37,10 @@ import java.util.regex.Pattern; @InterfaceAudience.Private -/** +/* * Utility class within the tez-dag module */ -public class Utils { +public final class Utils { private static final Logger LOG = LoggerFactory.getLogger(Utils.class); @@ -49,6 +49,8 @@ public class Utils { */ private static Pattern sanitizeLabelPattern = Pattern.compile("[:\\-\\W]+"); + private Utils() {} + public static String getContainerLauncherIdentifierString(int launcherIndex, AppContext appContext) { String name; try { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImplHelpers.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImplHelpers.java index 9e4f2b4a6e..fba82613dd 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImplHelpers.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImplHelpers.java @@ -25,9 +25,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class TaskAttemptImplHelpers { +public final class TaskAttemptImplHelpers { private static final Logger LOG = LoggerFactory.getLogger(TaskAttemptImplHelpers.class); + + private TaskAttemptImplHelpers() {} static String[] resolveHosts(String[] src) { String[] result = new String[src.length]; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java index 19cf5b7e21..a0407c0a43 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java @@ -25,7 +25,6 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.TreeMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,7 +47,6 @@ import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.security.TokenCache; import org.apache.tez.dag.api.TezConfiguration; -import org.apache.tez.dag.api.TezConstants; import org.apache.tez.dag.api.TezUncheckedException; import org.apache.tez.dag.app.AppContext; import org.apache.tez.dag.records.TezDAGID; @@ -56,18 +54,20 @@ import com.google.common.annotations.VisibleForTesting; -public class AMContainerHelpers { +public final class AMContainerHelpers { private static final Logger LOG = LoggerFactory.getLogger(AMContainerHelpers.class); - private static Object commonContainerSpecLock = new Object(); + private static final Object COMMON_CONTAINER_SPEC_LOCK = new Object(); private static TezDAGID lastDAGID = null; - private static Map commonContainerSpecs = - new HashMap(); + private static final Map COMMON_CONTAINER_SPECS = + new HashMap<>(); + + private AMContainerHelpers() {} public static void dagComplete(TezDAGID dagId) { - synchronized (commonContainerSpecLock) { - commonContainerSpecs.remove(dagId); + synchronized (COMMON_CONTAINER_SPEC_LOCK) { + COMMON_CONTAINER_SPECS.remove(dagId); } } @@ -89,24 +89,21 @@ public static LocalResource createLocalResource(FileSystem fc, Path file, /** * Create the common {@link ContainerLaunchContext} for all attempts. - * - * @param applicationACLs - * @param auxiliaryService */ private static ContainerLaunchContext createCommonContainerLaunchContext( Map applicationACLs, Credentials credentials, String auxiliaryService) { // Application environment - Map environment = new HashMap(); + Map environment = new HashMap<>(); // Service data - Map serviceData = new HashMap(); + Map serviceData = new HashMap<>(); // Tokens // Setup up task credentials buffer - ByteBuffer containerCredentialsBuffer = ByteBuffer.wrap(new byte[] {}); + ByteBuffer containerCredentialsBuffer; try { Credentials containerCredentials = new Credentials(); @@ -135,10 +132,8 @@ private static ContainerLaunchContext createCommonContainerLaunchContext( // Construct the actual Container // The null fields are per-container and will be constructed for each // container separately. - ContainerLaunchContext container = - ContainerLaunchContext.newInstance(null, environment, null, - serviceData, containerCredentialsBuffer, applicationACLs); - return container; + return ContainerLaunchContext.newInstance(null, environment, null, + serviceData, containerCredentialsBuffer, applicationACLs); } @VisibleForTesting @@ -153,14 +148,14 @@ public static ContainerLaunchContext createContainerLaunchContext( AppContext appContext, Resource containerResource, Configuration conf, String auxiliaryService) { - ContainerLaunchContext commonContainerSpec = null; - synchronized (commonContainerSpecLock) { - if (!commonContainerSpecs.containsKey(tezDAGID)) { + ContainerLaunchContext commonContainerSpec; + synchronized (COMMON_CONTAINER_SPEC_LOCK) { + if (!COMMON_CONTAINER_SPECS.containsKey(tezDAGID)) { commonContainerSpec = createCommonContainerLaunchContext(acls, credentials, auxiliaryService); - commonContainerSpecs.put(tezDAGID, commonContainerSpec); + COMMON_CONTAINER_SPECS.put(tezDAGID, commonContainerSpec); } else { - commonContainerSpec = commonContainerSpecs.get(tezDAGID); + commonContainerSpec = COMMON_CONTAINER_SPECS.get(tezDAGID); } // Ensure that we remove container specs for previous AMs to reduce @@ -168,14 +163,14 @@ public static ContainerLaunchContext createContainerLaunchContext( if (lastDAGID == null) { lastDAGID = tezDAGID; } else if (!lastDAGID.equals(tezDAGID)) { - commonContainerSpecs.remove(lastDAGID); + COMMON_CONTAINER_SPECS.remove(lastDAGID); lastDAGID = tezDAGID; } } // Setup environment by cloning from common env. Map env = commonContainerSpec.getEnvironment(); - Map myEnv = new HashMap(env.size()); + Map myEnv = new HashMap<>(env.size()); myEnv.putAll(env); myEnv.putAll(vertexEnv); @@ -197,17 +192,15 @@ public static ContainerLaunchContext createContainerLaunchContext( appContext.getApplicationAttemptId().getAttemptId(), modifiedJavaOpts); // Duplicate the ByteBuffers for access by multiple containers. - Map myServiceData = new HashMap(); + Map myServiceData = new HashMap<>(); for (Entry entry : commonContainerSpec.getServiceData() .entrySet()) { myServiceData.put(entry.getKey(), entry.getValue().duplicate()); } // Construct the actual Container - ContainerLaunchContext container = - ContainerLaunchContext.newInstance(localResources, myEnv, commands, - myServiceData, commonContainerSpec.getTokens().duplicate(), acls); - return container; + return ContainerLaunchContext.newInstance(localResources, myEnv, commands, + myServiceData, commonContainerSpec.getTokens().duplicate(), acls); } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java b/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java index 8f4cd1fa6a..5cc940fc4e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java @@ -51,14 +51,16 @@ import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; -public class HistoryEventJsonConversion { +public final class HistoryEventJsonConversion { + + private HistoryEventJsonConversion() {} public static JSONObject convertToJson(HistoryEvent historyEvent) throws JSONException { if (!historyEvent.isHistoryEvent()) { throw new UnsupportedOperationException("Invalid Event, does not support history" + ", eventType=" + historyEvent.getEventType()); } - JSONObject jsonObject = null; + JSONObject jsonObject; switch (historyEvent.getEventType()) { case APP_LAUNCHED: jsonObject = convertAppLaunchedEvent((AppLaunchedEvent) historyEvent); @@ -402,7 +404,7 @@ private static JSONObject convertDAGInitializedEvent(DAGInitializedEvent event) JSONObject otherInfo = new JSONObject(); if (event.getVertexNameIDMap() != null) { - Map nameIdStrMap = new TreeMap(); + Map nameIdStrMap = new TreeMap<>(); for (Entry entry : event.getVertexNameIDMap().entrySet()) { nameIdStrMap.put(entry.getKey(), entry.getValue().toString()); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/utils/DAGUtils.java b/tez-dag/src/main/java/org/apache/tez/dag/history/utils/DAGUtils.java index f59e9ace7c..77e7179c29 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/utils/DAGUtils.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/utils/DAGUtils.java @@ -56,7 +56,7 @@ import org.apache.tez.common.Preconditions; -public class DAGUtils { +public final class DAGUtils { public static final String DAG_NAME_KEY = "dagName"; public static final String DAG_INFO_KEY = "dagInfo"; @@ -99,7 +99,7 @@ public class DAGUtils { public static final String VERTEX_GROUP_EDGE_MERGED_INPUTS_KEY = "edgeMergedInputs"; public static final String VERTEX_GROUP_DESTINATION_VERTEX_NAME_KEY = "destinationVertexName"; - + private DAGUtils() {} public static JSONObject generateSimpleJSONPlan(DAGPlan dagPlan) throws JSONException { JSONObject dagJson; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/utils/TezEventUtils.java b/tez-dag/src/main/java/org/apache/tez/dag/history/utils/TezEventUtils.java index cc89b9fb4d..16d59c42f5 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/utils/TezEventUtils.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/utils/TezEventUtils.java @@ -32,7 +32,9 @@ import org.apache.tez.runtime.api.impl.EventType; import org.apache.tez.runtime.api.impl.TezEvent; -public class TezEventUtils { +public final class TezEventUtils { + + private TezEventUtils() {} public static TezEventProto toProto(TezEvent event) throws IOException { TezEventProto.Builder evtBuilder = diff --git a/tez-dag/src/main/java/org/apache/tez/dag/utils/ProtoUtils.java b/tez-dag/src/main/java/org/apache/tez/dag/utils/ProtoUtils.java index 56e46a0556..e179b4fd9b 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/utils/ProtoUtils.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/utils/ProtoUtils.java @@ -24,7 +24,9 @@ import com.google.protobuf.ByteString; -public class ProtoUtils { +public final class ProtoUtils { + + private ProtoUtils() {} public static RecoveryProtos.SummaryEventProto toSummaryEventProto( TezDAGID dagID, long timestamp, HistoryEventType historyEventType, byte[] payload) { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/utils/TezBuilderUtils.java b/tez-dag/src/main/java/org/apache/tez/dag/utils/TezBuilderUtils.java index b7e6f724f7..25551a1ccb 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/utils/TezBuilderUtils.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/utils/TezBuilderUtils.java @@ -27,7 +27,9 @@ import org.apache.tez.dag.records.TezTaskID; import org.apache.tez.dag.records.TezVertexID; -public class TezBuilderUtils { +public final class TezBuilderUtils { + + private TezBuilderUtils() {} public static TezVertexID newVertexID(TezDAGID dagId, int vertexId) { return TezVertexID.getInstance(dagId, vertexId); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/utils/TezRuntimeChildJVM.java b/tez-dag/src/main/java/org/apache/tez/dag/utils/TezRuntimeChildJVM.java index 9458193dc1..b4871bd041 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/utils/TezRuntimeChildJVM.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/utils/TezRuntimeChildJVM.java @@ -28,10 +28,12 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.tez.runtime.task.TezChild; -public class TezRuntimeChildJVM { +public final class TezRuntimeChildJVM { + + private TezRuntimeChildJVM() {} // FIXME - public static enum LogName { + public enum LogName { /** Log on the stdout of the task. */ STDOUT ("stdout"), @@ -47,9 +49,9 @@ public static enum LogName { /** Log the debug script's stdout */ DEBUGOUT ("debugout"); - private String prefix; + private final String prefix; - private LogName(String prefix) { + LogName(String prefix) { this.prefix = prefix; } @@ -71,7 +73,7 @@ public static List getVMCommand( int applicationAttemptNumber, String javaOpts) { - Vector vargs = new Vector(9); + Vector vargs = new Vector<>(9); vargs.add(Environment.JAVA_HOME.$() + "/bin/java"); @@ -101,7 +103,7 @@ public static List getVMCommand( for (CharSequence str : vargs) { mergedCommand.append(str).append(" "); } - Vector vargsFinal = new Vector(1); + Vector vargsFinal = new Vector<>(1); vargsFinal.add(mergedCommand.toString()); return vargsFinal; } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/PluginWrapperTestHelpers.java b/tez-dag/src/test/java/org/apache/tez/dag/app/PluginWrapperTestHelpers.java index fb6faa1bb6..5009cefffd 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/PluginWrapperTestHelpers.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/PluginWrapperTestHelpers.java @@ -14,9 +14,7 @@ package org.apache.tez.dag.app; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.*; import static org.mockito.Mockito.mock; import java.lang.reflect.Constructor; @@ -30,15 +28,17 @@ import org.slf4j.LoggerFactory; -public class PluginWrapperTestHelpers { +public final class PluginWrapperTestHelpers { private static final Logger LOG = LoggerFactory.getLogger(PluginWrapperTestHelpers.class); + private PluginWrapperTestHelpers() {} + public static void testDelegation(Class delegateClass, Class rawClass, Set skipMethods) throws Exception { TrackingAnswer answer = new TrackingAnswer(); Object mock = mock(rawClass, answer); - Constructor ctor = delegateClass.getConstructor(rawClass); + Constructor ctor = delegateClass.getConstructor(rawClass); Object wrapper = ctor.newInstance(mock); // Run through all the methods on the wrapper, and invoke the methods. Constructs @@ -48,7 +48,7 @@ public static void testDelegation(Class delegateClass, Class rawClass, if (method.getDeclaringClass().equals(delegateClass) && !skipMethods.contains(method.getName())) { - assertTrue(method.getExceptionTypes().length == 1); + assertEquals(1, method.getExceptionTypes().length); assertEquals(Exception.class, method.getExceptionTypes()[0]); LOG.info("Checking method [{}] with parameterTypes [{}]", method.getName(), Arrays.toString(method.getParameterTypes())); @@ -65,8 +65,8 @@ public static void testDelegation(Class delegateClass, Class rawClass, if (answer.compareAsPrimitive) { assertEquals(answer.lastRetValue, result); } else { - assertTrue("Expected: " + System.identityHashCode(answer.lastRetValue) + ", actual=" + - System.identityHashCode(result), answer.lastRetValue == result); + assertSame("Expected: " + System.identityHashCode(answer.lastRetValue) + ", actual=" + + System.identityHashCode(result), answer.lastRetValue, result); } } } @@ -74,8 +74,7 @@ public static void testDelegation(Class delegateClass, Class rawClass, } - public static Object[] constructMethodArgs(Method method) throws IllegalAccessException, - InstantiationException { + public static Object[] constructMethodArgs(Method method) { Class[] paramTypes = method.getParameterTypes(); Object[] params = new Object[paramTypes.length]; for (int i = 0; i < paramTypes.length; i++) { @@ -112,7 +111,7 @@ private static Object getValueForPrimitiveOrString(Class clazz) { } else if (clazz.equals(int.class)) { return 224; } else if (clazz.equals(long.class)) { - return 445l; + return 445L; } else if (clazz.equals(float.class)) { return 2.24f; } else if (clazz.equals(double.class)) { diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java index 490067a54c..08941e762a 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java @@ -18,8 +18,7 @@ package org.apache.tez.dag.app.rm; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.fail; +import static org.junit.Assert.*; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -76,7 +75,9 @@ import org.apache.tez.serviceplugins.api.TaskScheduler; import org.apache.tez.serviceplugins.api.TaskSchedulerContext; -class TestTaskSchedulerHelpers { +final class TestTaskSchedulerHelpers { + + private TestTaskSchedulerHelpers() {} // Mocking AMRMClientImpl to make use of getMatchingRequest static class AMRMClientForTest extends AMRMClientImpl { @@ -143,9 +144,8 @@ RegisterApplicationMasterResponse getRegistrationResponse() { static class TaskSchedulerManagerForTest extends TaskSchedulerManager { - private TezAMRMClientAsync amrmClientAsync; - private ContainerSignatureMatcher containerSignatureMatcher; - private UserPayload defaultPayload; + private final TezAMRMClientAsync amrmClientAsync; + private final UserPayload defaultPayload; @SuppressWarnings("rawtypes") public TaskSchedulerManagerForTest(AppContext appContext, @@ -157,7 +157,6 @@ public TaskSchedulerManagerForTest(AppContext appContext, Lists.newArrayList(new NamedEntityDescriptor("FakeScheduler", null)), false, new HadoopShimsLoader(appContext.getAMConf()).getHadoopShim()); this.amrmClientAsync = amrmClientAsync; - this.containerSignatureMatcher = containerSignatureMatcher; this.defaultPayload = defaultPayload; } @@ -170,7 +169,6 @@ public TaskSchedulerManagerForTest(AppContext appContext, super(appContext, null, eventHandler, containerSignatureMatcher, null, descriptors, false, new HadoopShimsLoader(appContext.getAMConf()).getHadoopShim()); this.amrmClientAsync = amrmClientAsync; - this.containerSignatureMatcher = containerSignatureMatcher; this.defaultPayload = defaultPayload; } @@ -212,7 +210,7 @@ public void serviceStop() { @SuppressWarnings("rawtypes") static class CapturingEventHandler implements EventHandler { - private Queue events = new ConcurrentLinkedQueue(); + private final Queue events = new ConcurrentLinkedQueue(); public void handle(Event event) { events.add(event); @@ -224,7 +222,7 @@ public void reset() { public void verifyNoInvocations(Class eventClass) { for (Event e : events) { - assertFalse(e.getClass().getName().equals(eventClass.getName())); + assertNotEquals(e.getClass().getName(), eventClass.getName()); } } @@ -262,8 +260,8 @@ public TaskSchedulerContextDrainable getDrainableAppCallback() { static class TaskSchedulerContextDrainable implements TaskSchedulerContext { int completedEvents; int invocations; - private TaskSchedulerContext real; - private CountingExecutorService countingExecutorService; + private final TaskSchedulerContext real; + private final CountingExecutorService countingExecutorService; final AtomicInteger count = new AtomicInteger(0); public TaskSchedulerContextDrainable(TaskSchedulerContextImplWrapper real) { @@ -441,10 +439,7 @@ public boolean isSuperSet(Object cs1, Object cs2) { @Override public boolean isExactMatch(Object cs1, Object cs2) { - if (cs1 == cs2 && cs1 != null) { - return true; - } - return false; + return cs1 == cs2 && cs1 != null; } @Override diff --git a/tez-examples/src/main/java/org/apache/tez/examples/ExampleDriver.java b/tez-examples/src/main/java/org/apache/tez/examples/ExampleDriver.java index 074f77fa19..c143e21d77 100644 --- a/tez-examples/src/main/java/org/apache/tez/examples/ExampleDriver.java +++ b/tez-examples/src/main/java/org/apache/tez/examples/ExampleDriver.java @@ -18,29 +18,17 @@ package org.apache.tez.examples; -import java.io.IOException; -import java.text.DecimalFormat; -import java.util.EnumSet; -import java.util.Set; - import org.apache.hadoop.util.ProgramDriver; -import org.apache.tez.common.counters.TezCounters; -import org.apache.tez.dag.api.TezException; -import org.apache.tez.dag.api.client.DAGClient; -import org.apache.tez.dag.api.client.DAGStatus; -import org.apache.tez.dag.api.client.Progress; -import org.apache.tez.dag.api.client.StatusGetOpts; -import org.apache.tez.dag.api.client.VertexStatus; /** * A description of an example program based on its class and a * human-readable description. */ -public class ExampleDriver { +public final class ExampleDriver { - private static final DecimalFormat formatter = new DecimalFormat("###.##%"); + private ExampleDriver() {} - public static void main(String argv[]){ + public static void main(String[] argv){ int exitCode = -1; ProgramDriver pgd = new ProgramDriver(); try { diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceConfConstants.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceConfConstants.java index bf4a5bdae1..e088ef791e 100644 --- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceConfConstants.java +++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceConfConstants.java @@ -14,7 +14,7 @@ package org.apache.tez.service; -public class TezTestServiceConfConstants { +public final class TezTestServiceConfConstants { private static final String TEZ_TEST_SERVICE_PREFIX = "tez.test.service."; @@ -38,4 +38,5 @@ public class TezTestServiceConfConstants { public static final String TEZ_TEST_SERVICE_AM_COMMUNICATOR_NUM_THREADS = TEZ_TEST_SERVICE_PREFIX + "communicator.num.threads"; public static final int TEZ_TEST_SERVICE_AM_COMMUNICATOR_NUM_THREADS_DEFAULT = 2; + private TezTestServiceConfConstants() {} } diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/util/ProtoConverters.java b/tez-ext-service-tests/src/test/java/org/apache/tez/util/ProtoConverters.java index 25d61d0bf5..a595210df3 100644 --- a/tez-ext-service-tests/src/test/java/org/apache/tez/util/ProtoConverters.java +++ b/tez-ext-service-tests/src/test/java/org/apache/tez/util/ProtoConverters.java @@ -37,7 +37,9 @@ import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.TaskSpecProto; import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.TaskSpecProto.Builder; -public class ProtoConverters { +public final class ProtoConverters { + + private ProtoConverters() {} public static TaskSpec getTaskSpecfromProto(TaskSpecProto taskSpecProto) { TezTaskAttemptID taskAttemptID = diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/SplitMetaInfoReaderTez.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/SplitMetaInfoReaderTez.java index db156d2ce2..394c871ab9 100644 --- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/SplitMetaInfoReaderTez.java +++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/SplitMetaInfoReaderTez.java @@ -41,13 +41,15 @@ */ @InterfaceAudience.Private @InterfaceStability.Unstable -public class SplitMetaInfoReaderTez { +public final class SplitMetaInfoReaderTez { public static final Logger LOG = LoggerFactory.getLogger(SplitMetaInfoReaderTez.class); public static final int META_SPLIT_VERSION = JobSplit.META_SPLIT_VERSION; public static final byte[] META_SPLIT_FILE_HEADER = JobSplit.META_SPLIT_FILE_HEADER; + private SplitMetaInfoReaderTez() {} + private static FSDataInputStream getFSDataIS(Configuration conf, FileSystem fs) throws IOException { long maxMetaInfoSize = conf.getLong( @@ -69,7 +71,7 @@ private static FSDataInputStream getFSDataIS(Configuration conf, + FileSystem.getDefaultUri(conf)); } - FileStatus fStatus = null; + FileStatus fStatus; try { fStatus = fs.getFileStatus(metaSplitFile); if (maxMetaInfoSize > 0 && fStatus.getLen() > maxMetaInfoSize) { @@ -131,7 +133,6 @@ public static TaskSplitMetaInfo[] readSplitMetaInfo(Configuration conf, * @param fs FileSystem. * @param index the index of the task. * @return split meta info object of the task. - * @throws IOException */ public static TaskSplitMetaInfo getSplitMetaInfo(Configuration conf, FileSystem fs, int index) throws IOException { diff --git a/tez-mapreduce/src/main/java/org/apache/tez/common/MRFrameworkConfigs.java b/tez-mapreduce/src/main/java/org/apache/tez/common/MRFrameworkConfigs.java index 6831b52ec4..4cbb28530d 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/common/MRFrameworkConfigs.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/common/MRFrameworkConfigs.java @@ -23,7 +23,7 @@ import org.apache.hadoop.classification.InterfaceAudience; @InterfaceAudience.Private -public class MRFrameworkConfigs { +public final class MRFrameworkConfigs { /** * One local dir for the specific job. @@ -38,4 +38,6 @@ public class MRFrameworkConfigs { public static final String TASK_LOCAL_RESOURCE_DIR_DEFAULT = "/tmp"; public static final String JOB_LOCAL_DIR = MR_FRAMEWORK_PREFIX + "job.local.dir"; + + private MRFrameworkConfigs() {} } diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/common/Utils.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/common/Utils.java index 050ba7912b..670ee5db4e 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/common/Utils.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/common/Utils.java @@ -33,7 +33,9 @@ import org.apache.tez.mapreduce.hadoop.mapred.MRCounters; @Private -public class Utils { +public final class Utils { + + private Utils() {} /** * Gets a handle to the Statistics instance based on the scheme associated @@ -46,7 +48,7 @@ public class Utils { */ @Private public static List getFsStatistics(Path path, Configuration conf) throws IOException { - List matchedStats = new ArrayList(); + List matchedStats = new ArrayList<>(); path = path.getFileSystem(conf).makeQualified(path); String scheme = path.toUri().getScheme(); for (Statistics stats : FileSystem.getAllStatistics()) { diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java index 098057ba17..8712060546 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java @@ -27,7 +27,7 @@ import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; import org.apache.tez.runtime.library.common.Constants; -public class DeprecatedKeys { +public final class DeprecatedKeys { @@ -35,13 +35,13 @@ public class DeprecatedKeys { /** * Keys used by the DAG - mainly the AM. */ - private static Map mrParamToDAGParamMap = new HashMap(); + private static final Map MR_PARAM_TO_DAG_PARAM_MAP = new HashMap<>(); /** * Keys used by the Tez Runtime. */ - private static Map mrParamToTezRuntimeParamMap = - new HashMap(); + private static final Map MR_PARAM_TO_TEZ_RUNTIME_PARAM_MAP = + new HashMap<>(); @@ -51,20 +51,22 @@ public class DeprecatedKeys { addDeprecatedKeys(); } + private DeprecatedKeys() {} + private static void populateMRToDagParamMap() { // TODO Default value handling. - mrParamToDAGParamMap.put(MRJobConfig.MR_AM_TASK_LISTENER_THREAD_COUNT, + MR_PARAM_TO_DAG_PARAM_MAP.put(MRJobConfig.MR_AM_TASK_LISTENER_THREAD_COUNT, TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT); - mrParamToDAGParamMap.put(MRJobConfig.MAX_TASK_FAILURES_PER_TRACKER, + MR_PARAM_TO_DAG_PARAM_MAP.put(MRJobConfig.MAX_TASK_FAILURES_PER_TRACKER, TezConfiguration.TEZ_AM_MAX_TASK_FAILURES_PER_NODE); - mrParamToDAGParamMap.put(MRJobConfig.MR_AM_JOB_NODE_BLACKLISTING_ENABLE, + MR_PARAM_TO_DAG_PARAM_MAP.put(MRJobConfig.MR_AM_JOB_NODE_BLACKLISTING_ENABLE, TezConfiguration.TEZ_AM_NODE_BLACKLISTING_ENABLED); - mrParamToDAGParamMap.put( + MR_PARAM_TO_DAG_PARAM_MAP.put( MRJobConfig.MR_AM_IGNORE_BLACKLISTING_BLACKLISTED_NODE_PERECENT, TezConfiguration.TEZ_AM_NODE_BLACKLISTING_IGNORE_THRESHOLD); - mrParamToDAGParamMap.put(MRJobConfig.QUEUE_NAME, + MR_PARAM_TO_DAG_PARAM_MAP.put(MRJobConfig.QUEUE_NAME, TezConfiguration.TEZ_QUEUE_NAME); // Counter replacement will work in this manner, as long as TezCounters @@ -73,23 +75,23 @@ private static void populateMRToDagParamMap() { // may break. // Framework counters, like FILESYSTEM will likely be incompatible since // they enum key belongs to a different package. - mrParamToDAGParamMap.put(MRJobConfig.COUNTERS_MAX_KEY, + MR_PARAM_TO_DAG_PARAM_MAP.put(MRJobConfig.COUNTERS_MAX_KEY, TezConfiguration.TEZ_COUNTERS_MAX); - mrParamToDAGParamMap.put(MRJobConfig.COUNTER_GROUPS_MAX_KEY, + MR_PARAM_TO_DAG_PARAM_MAP.put(MRJobConfig.COUNTER_GROUPS_MAX_KEY, TezConfiguration.TEZ_COUNTERS_MAX_GROUPS); - mrParamToDAGParamMap.put(MRJobConfig.COUNTER_NAME_MAX_KEY, + MR_PARAM_TO_DAG_PARAM_MAP.put(MRJobConfig.COUNTER_NAME_MAX_KEY, TezConfiguration.TEZ_COUNTERS_COUNTER_NAME_MAX_LENGTH); - mrParamToDAGParamMap.put(MRJobConfig.COUNTER_GROUP_NAME_MAX_KEY, + MR_PARAM_TO_DAG_PARAM_MAP.put(MRJobConfig.COUNTER_GROUP_NAME_MAX_KEY, TezConfiguration.TEZ_COUNTERS_GROUP_NAME_MAX_LENGTH); - mrParamToDAGParamMap.put(MRJobConfig.TASK_TIMEOUT, + MR_PARAM_TO_DAG_PARAM_MAP.put(MRJobConfig.TASK_TIMEOUT, TezConfiguration.TASK_HEARTBEAT_TIMEOUT_MS); - mrParamToDAGParamMap.put(MRJobConfig.JOB_TAGS, + MR_PARAM_TO_DAG_PARAM_MAP.put(MRJobConfig.JOB_TAGS, TezConfiguration.TEZ_APPLICATION_TAGS); - mrParamToDAGParamMap.put(MRJobConfig.MAPREDUCE_JOB_USER_CLASSPATH_FIRST, + MR_PARAM_TO_DAG_PARAM_MAP.put(MRJobConfig.MAPREDUCE_JOB_USER_CLASSPATH_FIRST, TezConfiguration.TEZ_USER_CLASSPATH_FIRST); - mrParamToDAGParamMap.put(MRJobConfig.JOB_NAMENODES, + MR_PARAM_TO_DAG_PARAM_MAP.put(MRJobConfig.JOB_NAMENODES, TezConfiguration.TEZ_JOB_FS_SERVERS); - mrParamToDAGParamMap.put(MRJobConfig.JOB_NAMENODES_TOKEN_RENEWAL_EXCLUDE, + MR_PARAM_TO_DAG_PARAM_MAP.put(MRJobConfig.JOB_NAMENODES_TOKEN_RENEWAL_EXCLUDE, TezConfiguration.TEZ_JOB_FS_SERVERS_TOKEN_RENEWAL_EXCLUDE); } @@ -181,14 +183,14 @@ private static void addDeprecatedKeys() { private static void registerMRToRuntimeKeyTranslation(String mrKey, String tezKey) { - mrParamToTezRuntimeParamMap.put(mrKey, tezKey); + MR_PARAM_TO_TEZ_RUNTIME_PARAM_MAP.put(mrKey, tezKey); } public static Map getMRToDAGParamMap() { - return Collections.unmodifiableMap(mrParamToDAGParamMap); + return Collections.unmodifiableMap(MR_PARAM_TO_DAG_PARAM_MAP); } public static Map getMRToTezRuntimeParamMap() { - return Collections.unmodifiableMap(mrParamToTezRuntimeParamMap); + return Collections.unmodifiableMap(MR_PARAM_TO_TEZ_RUNTIME_PARAM_MAP); } } diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/IDConverter.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/IDConverter.java index 2fddd00522..8656147e2c 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/IDConverter.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/IDConverter.java @@ -28,7 +28,9 @@ import org.apache.tez.dag.records.TezTaskID; import org.apache.tez.dag.records.TezVertexID; -public class IDConverter { +public final class IDConverter { + + private IDConverter() {} // FIXME hardcoded assumption that one app is one dag public static JobID toMRJobId(TezDAGID dagId) { diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java index 1d06b7aeaf..dd13eb205a 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java @@ -45,10 +45,11 @@ */ @Public @Evolving -public class MRHelpers { +public final class MRHelpers { private static final Logger LOG = LoggerFactory.getLogger(MRHelpers.class); + private MRHelpers() {} /** * Translate MapReduce configuration keys to the equivalent Tez keys in the provided @@ -105,7 +106,7 @@ public static void configureMRApiUsage(Configuration conf) { } private static void convertVertexConfToTez(Configuration vertexConf, boolean preferTez) { - setStageKeysFromBaseConf(vertexConf, vertexConf, "unknown"); + setStageKeysFromBaseConf(vertexConf, vertexConf); processDirectConversion(vertexConf, preferTez); setupMRComponents(vertexConf); } @@ -136,7 +137,7 @@ private static void setupMRComponents(Configuration conf) { * require translation to tez keys. */ private static void setStageKeysFromBaseConf(Configuration conf, - Configuration baseConf, String stage) { + Configuration baseConf) { // Don't clobber explicit tez config. JobConf jobConf = null; if (conf.get(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS) == null) { @@ -151,7 +152,7 @@ private static void setStageKeysFromBaseConf(Configuration conf, if (LOG.isDebugEnabled()) { LOG.debug("Setting " + MRJobConfig.MAP_OUTPUT_KEY_CLASS - + " for stage: " + stage + + " for stage: unknown" + " based on job level configuration. Value: " + conf.get(MRJobConfig.MAP_OUTPUT_KEY_CLASS)); } @@ -168,7 +169,7 @@ private static void setStageKeysFromBaseConf(Configuration conf, .getMapOutputValueClass().getName()); if (LOG.isDebugEnabled()) { LOG.debug("Setting " + MRJobConfig.MAP_OUTPUT_VALUE_CLASS - + " for stage: " + stage + + " for stage: unknown" + " based on job level configuration. Value: " + conf.get(MRJobConfig.MAP_OUTPUT_VALUE_CLASS)); } @@ -223,7 +224,7 @@ private static void ensureNotSet(Configuration conf, String attr, String msg) private static String getLog4jCmdLineProperties(Configuration conf, boolean isMap) { - Vector logProps = new Vector(4); + Vector logProps = new Vector<>(4); TezUtils.addLog4jSystemProperties(getChildLogLevel(conf, isMap), logProps); StringBuilder sb = new StringBuilder(); for (String str : logProps) { diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java index a8e85a34e4..26bba4d002 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java @@ -82,6 +82,8 @@ public class MRInputHelpers { static final String JOB_SPLIT_RESOURCE_NAME = "job.split"; static final String JOB_SPLIT_METAINFO_RESOURCE_NAME = "job.splitmetainfo"; + protected MRInputHelpers() {} + /** * Setup split generation on the client, with splits being distributed via the traditional * MapReduce mechanism of distributing splits via the Distributed Cache. @@ -107,7 +109,7 @@ public class MRInputHelpers { public static DataSourceDescriptor configureMRInputWithLegacySplitGeneration(Configuration conf, Path splitsDir, boolean useLegacyInput) { - InputSplitInfo inputSplitInfo = null; + InputSplitInfo inputSplitInfo; try { inputSplitInfo = generateInputSplits(conf, splitsDir); @@ -117,17 +119,11 @@ public static DataSourceDescriptor configureMRInputWithLegacySplitGeneration(Con Map additionalLocalResources = new HashMap(); updateLocalResourcesForInputSplits(conf, inputSplitInfo, additionalLocalResources); - DataSourceDescriptor dsd = - DataSourceDescriptor.create(inputDescriptor, null, inputSplitInfo.getNumTasks(), - inputSplitInfo.getCredentials(), - VertexLocationHint.create(inputSplitInfo.getTaskLocationHints()), - additionalLocalResources); - return dsd; - } catch (IOException e) { - throw new TezUncheckedException("Failed to generate InputSplits", e); - } catch (InterruptedException e) { - throw new TezUncheckedException("Failed to generate InputSplits", e); - } catch (ClassNotFoundException e) { + return DataSourceDescriptor.create(inputDescriptor, null, inputSplitInfo.getNumTasks(), + inputSplitInfo.getCredentials(), + VertexLocationHint.create(inputSplitInfo.getTaskLocationHints()), + additionalLocalResources); + } catch (IOException | InterruptedException | ClassNotFoundException e) { throw new TezUncheckedException("Failed to generate InputSplits", e); } } @@ -139,7 +135,6 @@ public static DataSourceDescriptor configureMRInputWithLegacySplitGeneration(Con * @param payload the {@link org.apache.tez.dag.api.UserPayload} instance * @return an instance of {@link org.apache.tez.mapreduce.protos.MRRuntimeProtos.MRInputUserPayloadProto}, * which provides access to the underlying configuration bytes - * @throws IOException */ @InterfaceStability.Evolving @InterfaceAudience.LimitedPrivate({"hive, pig"}) @@ -156,7 +151,6 @@ public static MRRuntimeProtos.MRInputUserPayloadProto parseMRInputPayload(UserPa * instance representing the split * @param serializationFactory the serialization mechanism used to write out the split * @return an instance of the split - * @throws java.io.IOException */ @SuppressWarnings("unchecked") @InterfaceStability.Evolving @@ -192,7 +186,6 @@ public static InputSplit createOldFormatSplitFromUserPayload( * instance representing the split * @param serializationFactory the serialization mechanism used to write out the split * @return an instance of the split - * @throws IOException */ @InterfaceStability.Evolving @SuppressWarnings("unchecked") @@ -222,7 +215,7 @@ public static org.apache.hadoop.mapreduce.InputSplit createNewFormatSplitFromUse @InterfaceStability.Evolving public static MRRuntimeProtos.MRSplitProto createSplitProto( T newSplit, SerializationFactory serializationFactory) - throws IOException, InterruptedException { + throws IOException { MRRuntimeProtos.MRSplitProto.Builder builder = MRRuntimeProtos.MRSplitProto .newBuilder(); @@ -278,9 +271,6 @@ public static MRRuntimeProtos.MRSplitProto createSplitProto( * @param targetTasks the number of target tasks if grouping is enabled. Specify as 0 otherwise. * @return an instance of {@link InputSplitInfoMem} which supports a subset of * the APIs defined on {@link InputSplitInfo} - * @throws IOException - * @throws ClassNotFoundException - * @throws InterruptedException */ @InterfaceStability.Unstable @InterfaceAudience.LimitedPrivate({"hive, pig"}) @@ -310,16 +300,13 @@ public static InputSplitInfoMem generateInputSplitsToMem(Configuration conf, * @param targetTasks the number of target tasks if grouping is enabled. Specify as 0 otherwise. * @return an instance of {@link InputSplitInfoMem} which supports a subset of * the APIs defined on {@link InputSplitInfo} - * @throws IOException - * @throws ClassNotFoundException - * @throws InterruptedException */ @InterfaceStability.Unstable public static InputSplitInfoMem generateInputSplitsToMem(Configuration conf, boolean groupSplits, boolean sortSplits, int targetTasks) throws IOException, ClassNotFoundException, InterruptedException { - InputSplitInfoMem splitInfoMem = null; + InputSplitInfoMem splitInfoMem; JobConf jobConf = new JobConf(conf); if (jobConf.getUseNewMapper()) { LOG.debug("Generating mapreduce api input splits"); @@ -356,7 +343,7 @@ public TaskLocationHint apply( if (rack == null) { if (input.getLocations() != null) { return TaskLocationHint.createTaskLocationHint( - new HashSet(Arrays.asList(input.getLocations())), null); + new HashSet<>(Arrays.asList(input.getLocations())), null); } else { return TaskLocationHint.createTaskLocationHint(null, null); } @@ -366,7 +353,7 @@ public TaskLocationHint apply( } } else { return TaskLocationHint.createTaskLocationHint( - new HashSet(Arrays.asList(input.getLocations())), null); + new HashSet<>(Arrays.asList(input.getLocations())), null); } } catch (IOException e) { throw new RuntimeException(e); @@ -399,7 +386,7 @@ public TaskLocationHint apply(org.apache.hadoop.mapred.InputSplit input) { } } else { return TaskLocationHint.createTaskLocationHint( - new HashSet(Arrays.asList(input.getLocations())), + new HashSet<>(Arrays.asList(input.getLocations())), null); } } catch (IOException e) { @@ -413,20 +400,20 @@ public TaskLocationHint apply(org.apache.hadoop.mapred.InputSplit input) { @SuppressWarnings({ "rawtypes", "unchecked" }) private static org.apache.hadoop.mapreduce.InputSplit[] generateNewSplits( JobContext jobContext, boolean groupSplits, boolean sortSplits, - int numTasks) throws ClassNotFoundException, IOException, + int numTasks) throws IOException, InterruptedException { Configuration conf = jobContext.getConfiguration(); // This is the real input format. - org.apache.hadoop.mapreduce.InputFormat inputFormat = null; + org.apache.hadoop.mapreduce.InputFormat inputFormat; try { inputFormat = ReflectionUtils.newInstance(jobContext.getInputFormatClass(), conf); } catch (ClassNotFoundException e) { throw new TezUncheckedException(e); } - org.apache.hadoop.mapreduce.InputFormat finalInputFormat = inputFormat; + org.apache.hadoop.mapreduce.InputFormat finalInputFormat; // For grouping, the underlying InputFormatClass class is passed in as a parameter. // JobContext has this setup as TezGroupedSplitInputFormat @@ -443,7 +430,7 @@ private static org.apache.hadoop.mapreduce.InputSplit[] generateNewSplits( List array = finalInputFormat .getSplits(jobContext); - org.apache.hadoop.mapreduce.InputSplit[] splits = (org.apache.hadoop.mapreduce.InputSplit[]) array + org.apache.hadoop.mapreduce.InputSplit[] splits = array .toArray(new org.apache.hadoop.mapreduce.InputSplit[array.size()]); if (sortSplits) { @@ -469,7 +456,7 @@ private static org.apache.hadoop.mapred.InputSplit[] generateOldSplits( throw new TezUncheckedException(e); } - org.apache.hadoop.mapred.InputFormat finalInputFormat = inputFormat; + org.apache.hadoop.mapred.InputFormat finalInputFormat; if (groupSplits) { org.apache.hadoop.mapred.split.TezGroupedSplitsInputFormat groupedFormat = @@ -502,16 +489,8 @@ public int compare(org.apache.hadoop.mapreduce.InputSplit o1, try { long len1 = o1.getLength(); long len2 = o2.getLength(); - if (len1 < len2) { - return 1; - } else if (len1 == len2) { - return 0; - } else { - return -1; - } - } catch (IOException ie) { - throw new RuntimeException("exception in InputSplit compare", ie); - } catch (InterruptedException ie) { + return Long.compare(len2, len1); + } catch (IOException | InterruptedException ie) { throw new RuntimeException("exception in InputSplit compare", ie); } } @@ -528,13 +507,7 @@ public int compare(org.apache.hadoop.mapred.InputSplit o1, try { long len1 = o1.getLength(); long len2 = o2.getLength(); - if (len1 < len2) { - return 1; - } else if (len1 == len2) { - return 0; - } else { - return -1; - } + return Long.compare(len2, len1); } catch (IOException ie) { throw new RuntimeException("Problem getting input split size", ie); } @@ -549,10 +522,6 @@ public int compare(org.apache.hadoop.mapred.InputSplit o1, * @return InputSplitInfo containing the split files' information and the * location hints for each split generated to be used to determining parallelism of * the map stage. - * - * @throws IOException - * @throws InterruptedException - * @throws ClassNotFoundException */ private static InputSplitInfoDisk writeNewSplits(JobContext jobContext, Path inputSplitDir) throws IOException, InterruptedException, @@ -568,10 +537,10 @@ private static InputSplitInfoDisk writeNewSplits(JobContext jobContext, List locationHints = new ArrayList(splits.length); - for (int i = 0; i < splits.length; ++i) { + for (org.apache.hadoop.mapreduce.InputSplit split : splits) { locationHints.add( - TaskLocationHint.createTaskLocationHint(new HashSet( - Arrays.asList(splits[i].getLocations())), null) + TaskLocationHint.createTaskLocationHint(new HashSet( + Arrays.asList(split.getLocations())), null) ); } @@ -589,8 +558,6 @@ private static InputSplitInfoDisk writeNewSplits(JobContext jobContext, * @return InputSplitInfo containing the split files' information and the * number of splits generated to be used to determining parallelism of * the map stage. - * - * @throws IOException */ private static InputSplitInfoDisk writeOldSplits(JobConf jobConf, Path inputSplitDir) throws IOException { @@ -602,11 +569,11 @@ private static InputSplitInfoDisk writeOldSplits(JobConf jobConf, inputSplitDir.getFileSystem(jobConf), splits); List locationHints = - new ArrayList(splits.length); - for (int i = 0; i < splits.length; ++i) { + new ArrayList<>(splits.length); + for (InputSplit split : splits) { locationHints.add( - TaskLocationHint.createTaskLocationHint(new HashSet( - Arrays.asList(splits[i].getLocations())), null) + TaskLocationHint.createTaskLocationHint(new HashSet<>( + Arrays.asList(split.getLocations())), null) ); } @@ -637,10 +604,6 @@ private static InputSplitInfoDisk writeOldSplits(JobConf jobConf, * @return InputSplitInfo containing the split files' information and the * number of splits generated to be used to determining parallelism of * the map stage. - * - * @throws IOException - * @throws InterruptedException - * @throws ClassNotFoundException */ private static InputSplitInfoDisk generateInputSplits(Configuration conf, Path inputSplitsDir) throws IOException, InterruptedException, @@ -666,7 +629,6 @@ private static InputSplitInfoDisk generateInputSplits(Configuration conf, * @param conf Configuration * @param inputSplitInfo Information on location of split files * @param localResources LocalResources collection to be updated - * @throws IOException */ private static void updateLocalResourcesForInputSplits( Configuration conf, @@ -751,8 +713,8 @@ protected static UserPayload createMRInputPayload(Configuration conf, } private static UserPayload createMRInputPayload(ByteString bytes, - MRRuntimeProtos.MRSplitsProto mrSplitsProto, - boolean isGrouped, boolean isSorted) throws IOException { + MRRuntimeProtos.MRSplitsProto mrSplitsProto, + boolean isGrouped, boolean isSorted) { MRRuntimeProtos.MRInputUserPayloadProto.Builder userPayloadBuilder = MRRuntimeProtos.MRInputUserPayloadProto .newBuilder(); diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java index 3f5ad230cf..de38766cd3 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java @@ -25,13 +25,14 @@ import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; -public class MultiStageMRConfToTezTranslator { +public final class MultiStageMRConfToTezTranslator { + + private MultiStageMRConfToTezTranslator() {} /** * Given a single base MRR config, returns a list of complete stage * configurations. - * - * @param conf + * * @return list of complete stage configurations given Conifiguration */ @Private @@ -62,14 +63,13 @@ public static Configuration[] getStageConfs(Configuration conf) { } } - Configuration confs[] = new Configuration[numStages]; + Configuration[] confs = new Configuration[numStages]; Configuration nonItermediateConf = MultiStageMRConfigUtil.extractStageConf( conf, ""); + confs[0] = nonItermediateConf; if (numStages == 1) { - confs[0] = nonItermediateConf; confs[0].setBoolean(MRConfig.IS_MAP_PROCESSOR, true); } else { - confs[0] = nonItermediateConf; confs[numStages - 1] = new Configuration(nonItermediateConf); confs[numStages -1].setBoolean(MRConfig.IS_MAP_PROCESSOR, false); } diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfigUtil.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfigUtil.java index 13e0b860eb..23ffd3ef75 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfigUtil.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfigUtil.java @@ -18,14 +18,15 @@ package org.apache.tez.mapreduce.hadoop; -import java.util.Iterator; import java.util.Map.Entry; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; @Private -public class MultiStageMRConfigUtil { +public final class MultiStageMRConfigUtil { + + private MultiStageMRConfigUtil() {} ////////////////////////////////////////////////////////////////////////////// // Methods based on Stage Num // @@ -73,9 +74,7 @@ static Configuration extractStageConf(Configuration baseConf, String prefix) { Configuration strippedConf = new Configuration(false); Configuration conf = new Configuration(false); - Iterator> confEntries = baseConf.iterator(); - while (confEntries.hasNext()) { - Entry entry = confEntries.next(); + for (Entry entry : baseConf) { String key = entry.getKey(); if (key.startsWith(prefix)) { // Ignore keys for other intermediate stages in case of an initial or final stage. @@ -95,9 +94,7 @@ static Configuration extractStageConf(Configuration baseConf, } // Replace values from strippedConf into the finalConf. Override values // which may have been copied over from the baseConf root level. - Iterator> entries = strippedConf.iterator(); - while (entries.hasNext()) { - Entry entry = entries.next(); + for (Entry entry : strippedConf) { if (!Configuration.isDeprecated(entry.getKey())) { conf.set(entry.getKey(), entry.getValue()); } diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/TezTypeConverters.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/TezTypeConverters.java index 6f9c1c7600..0efde4ac70 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/TezTypeConverters.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/TezTypeConverters.java @@ -29,7 +29,9 @@ import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezTaskAttemptID; -public class TezTypeConverters { +public final class TezTypeConverters { + + private TezTypeConverters() {} // TODO Remove unused methods @@ -46,15 +48,13 @@ public static org.apache.hadoop.mapreduce.JobID toJobID(TezDAGID id) { public static TaskAttemptId toYarn(TezTaskAttemptID taskAttemptId) { TaskAttemptID mrTaskAttemptId = IDConverter .toMRTaskAttemptId(taskAttemptId); - TaskAttemptId mrv2TaskAttemptId = TypeConverter.toYarn(mrTaskAttemptId); - return mrv2TaskAttemptId; + return TypeConverter.toYarn(mrTaskAttemptId); } public static TezTaskAttemptID toTez(TaskAttemptId taskAttemptId) { TaskAttemptID mrTaskAttemptId = TypeConverter.fromYarn(taskAttemptId); - TezTaskAttemptID tezTaskAttemptId = IDConverter + return IDConverter .fromMRTaskAttemptId(mrTaskAttemptId); - return tezTaskAttemptId; } public static Counters fromTez(TezCounters tezCounters) { diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java index e64d273b44..ee907f5d74 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java @@ -176,7 +176,7 @@ private MRInputConfigBuilder setInputPaths(String inputPaths) { } private void initializeInputPath() { - Preconditions.checkState(inputFormatProvided == false, + Preconditions.checkState(!inputFormatProvided, "Should only be invoked when no inputFormat is provided"); if (org.apache.hadoop.mapred.FileInputFormat.class.isAssignableFrom(inputFormat) || FileInputFormat.class.isAssignableFrom(inputFormat)) { @@ -437,8 +437,6 @@ public static MRInputConfigBuilder createConfigBuilder(Configuration conf, private final ReentrantLock rrLock = new ReentrantLock(); private final Condition rrInited = rrLock.newCondition(); - - private volatile boolean eventReceived = false; private boolean readerCreated = false; @@ -537,24 +535,24 @@ inputRecordCounter, getContext().getApplicationId().getClusterTimestamp(), @Override public KeyValueReader getReader() throws IOException { Preconditions - .checkState(readerCreated == false, + .checkState(!readerCreated, "Only a single instance of record reader can be created for this input."); readerCreated = true; if (getNumPhysicalInputs() == 0) { return new KeyValueReader() { @Override - public boolean next() throws IOException { + public boolean next() { getContext().notifyProgress(); return false; } @Override - public Object getCurrentKey() throws IOException { + public Object getCurrentKey() { return null; } @Override - public Object getCurrentValue() throws IOException { + public Object getCurrentValue() { return null; } }; @@ -576,11 +574,11 @@ public void handleEvents(List inputEvents) throws Exception { throw new IllegalStateException( "Unexpected event. MRInput has been setup to receive 0 events"); } - if (eventReceived || inputEvents.size() != 1) { + + if (inputEvents.size() != 1) { throw new IllegalStateException( "MRInput expects only a single input. Received: current eventListSize: " - + inputEvents.size() + "Received previous input: " - + eventReceived); + + inputEvents.size() + "Received previous input: false"); } Event event = inputEvents.iterator().next(); Preconditions.checkArgument(event instanceof InputDataInformationEvent, diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRInputUtils.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRInputUtils.java index a2b87e04c9..e3c4f0e3a3 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRInputUtils.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRInputUtils.java @@ -43,14 +43,15 @@ * Helper methods for InputFormat based Inputs. Private to Tez. */ @Private -public class MRInputUtils { +public final class MRInputUtils { private static final Logger LOG = LoggerFactory.getLogger(MRInputUtils.class); + private MRInputUtils() {} + public static TaskSplitMetaInfo getSplits(Configuration conf, int index) throws IOException { - TaskSplitMetaInfo taskSplitMInfo = SplitMetaInfoReaderTez + return SplitMetaInfoReaderTez .getSplitMetaInfo(conf, FileSystem.getLocal(conf), index); - return taskSplitMInfo; } public static org.apache.hadoop.mapreduce.InputSplit getNewSplitDetailsFromEvent( @@ -78,12 +79,10 @@ public static org.apache.hadoop.mapreduce.InputSplit getNewSplitDetailsFromDisk( try { cls = (Class) jobConf.getClassByName(className); } catch (ClassNotFoundException ce) { - IOException wrap = new IOException("Split class " + className + " not found"); - wrap.initCause(ce); - throw wrap; + throw new IOException("Split class " + className + " not found", ce); } SerializationFactory factory = new SerializationFactory(jobConf); - Deserializer deserializer = (Deserializer) factory + Deserializer deserializer = factory .getDeserializer(cls); deserializer.open(inFile); org.apache.hadoop.mapreduce.InputSplit split = deserializer.deserialize(null); @@ -111,12 +110,10 @@ public static InputSplit getOldSplitDetailsFromDisk(TaskSplitIndex splitMetaInfo try { cls = (Class) jobConf.getClassByName(className); } catch (ClassNotFoundException ce) { - IOException wrap = new IOException("Split class " + className + " not found"); - wrap.initCause(ce); - throw wrap; + throw new IOException("Split class " + className + " not found", ce); } SerializationFactory factory = new SerializationFactory(jobConf); - Deserializer deserializer = (Deserializer) factory + Deserializer deserializer = factory .getDeserializer(cls); deserializer.open(inFile); org.apache.hadoop.mapred.InputSplit split = deserializer.deserialize(null); diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TezTestUtils.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TezTestUtils.java index 83c28dd7bb..5c90f8a9df 100644 --- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TezTestUtils.java +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TezTestUtils.java @@ -29,10 +29,11 @@ import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.runtime.api.InputInitializerContext; -import java.io.IOException; import java.util.Set; -public class TezTestUtils { +public final class TezTestUtils { + + private TezTestUtils() {} public static TezTaskAttemptID getMockTaskAttemptId( int jobId, int vertexId, int taskId, int taskAttemptId) { @@ -62,7 +63,7 @@ public static class TezRootInputInitializerContextForTest implements private final UserPayload payload; private final Configuration vertexConfig; - public TezRootInputInitializerContextForTest(UserPayload payload, Configuration vertexConfig) throws IOException { + public TezRootInputInitializerContextForTest(UserPayload payload, Configuration vertexConfig) { appId = ApplicationId.newInstance(1000, 200); this.payload = payload == null ? UserPayload.create(null) : payload; this.vertexConfig = vertexConfig; diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java index 6bfc5badd5..ba1acdf981 100644 --- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java @@ -71,9 +71,11 @@ import com.google.common.collect.HashMultimap; -public class MapUtils { +public final class MapUtils { private static final Logger LOG = LoggerFactory.getLogger(MapUtils.class); + + private MapUtils() {} public static void configureLocalDirs(Configuration conf, String localDir) throws IOException { @@ -101,7 +103,7 @@ public static void configureLocalDirs(Configuration conf, String localDir) // JOB_LOCAL_DIR doesn't exist on this host -- Create it. workDir = lDirAlloc.getLocalPathForWrite("work", conf); FileSystem lfs = FileSystem.getLocal(conf).getRaw(); - boolean madeDir = false; + boolean madeDir; try { madeDir = lfs.mkdirs(workDir); } catch (FileAlreadyExistsException e) { @@ -127,8 +129,8 @@ public static void configureLocalDirs(Configuration conf, String localDir) LOG.info("Generating data at path: " + file); // create a file with length entries @SuppressWarnings("deprecation") - SequenceFile.Writer writer = - SequenceFile.createWriter(fs, job, file, + SequenceFile.Writer writer = + SequenceFile.createWriter(fs, job, file, LongWritable.class, Text.class); try { Random r = new Random(System.currentTimeMillis()); @@ -144,8 +146,8 @@ public static void configureLocalDirs(Configuration conf, String localDir) writer.close(); } - SequenceFileInputFormat format = - new SequenceFileInputFormat(); + SequenceFileInputFormat format = + new SequenceFileInputFormat<>(); InputSplit[] splits = format.getSplits(job, 1); System.err.println("#split = " + splits.length + " ; " + "#locs = " + splits[0].getLocations().length + "; " + @@ -175,7 +177,7 @@ private static void writeSplitFiles(FileSystem fs, JobConf conf, String[] locations = split.getLocations(); - SplitMetaInfo info = null; + SplitMetaInfo info; info = new JobSplit.SplitMetaInfo(locations, offset, split.getLength()); Path jobSplitMetaInfoFile = new Path( @@ -209,7 +211,7 @@ public static LogicalIOProcessorRuntimeTask createLogicalTask(FileSystem fs, Pat MapProcessor.class.getName()).setUserPayload( TezUtils.createUserPayloadFromConf(jobConf)); - Token shuffleToken = new Token(); + Token shuffleToken = new Token<>(); TaskSpec taskSpec = new TaskSpec( TezTestUtils.getMockTaskAttemptId(0, 0, mapId, 0), @@ -218,18 +220,18 @@ public static LogicalIOProcessorRuntimeTask createLogicalTask(FileSystem fs, Pat inputSpecs, outputSpecs, null, null); - Map serviceConsumerMetadata = new HashMap(); + Map serviceConsumerMetadata = new HashMap<>(); String auxiliaryService = jobConf.get(TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID, TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID_DEFAULT); serviceConsumerMetadata.put(auxiliaryService, ShuffleUtils.convertJobTokenToBytes(shuffleToken)); - Map envMap = new HashMap(); + Map envMap = new HashMap<>(); ByteBuffer shufflePortBb = ByteBuffer.allocate(4).putInt(0, 8000); AuxiliaryServiceHelper .setServiceDataIntoEnv(auxiliaryService, shufflePortBb, envMap); - LogicalIOProcessorRuntimeTask task = new LogicalIOProcessorRuntimeTask( + return new LogicalIOProcessorRuntimeTask( taskSpec, 0, jobConf, @@ -237,8 +239,7 @@ public static LogicalIOProcessorRuntimeTask createLogicalTask(FileSystem fs, Pat umbilical, serviceConsumerMetadata, envMap, - HashMultimap.create(), null, "", new ExecutionContextImpl("localhost"), + HashMultimap.create(), null, "", new ExecutionContextImpl("localhost"), Runtime.getRuntime().maxMemory(), true, new DefaultHadoopShim(), sharedExecutor); - return task; } } diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/utils/Utils.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/utils/Utils.java index 94b50a6c5c..7cf1e1af6b 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/utils/Utils.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/utils/Utils.java @@ -42,11 +42,12 @@ import java.util.List; @InterfaceAudience.Private -public class Utils { +public final class Utils { private static final String LOG4J_CONFIGURATION = "log4j.configuration"; private static final org.slf4j.Logger LOG = LoggerFactory.getLogger(Utils.class); + private Utils() {} /** * Parse tez counters from json diff --git a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java index b9457a0761..49c78eab7e 100644 --- a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java +++ b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java @@ -60,7 +60,9 @@ import com.google.common.collect.Lists; -public class HistoryEventTimelineConversion { +public final class HistoryEventTimelineConversion { + + private HistoryEventTimelineConversion() {} private static void validateEvent(HistoryEvent event) { if (!event.isHistoryEvent()) { diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/common/ProtoConverters.java b/tez-runtime-internals/src/main/java/org/apache/tez/common/ProtoConverters.java index e4285708a5..2c9ad86018 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/common/ProtoConverters.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/common/ProtoConverters.java @@ -30,7 +30,7 @@ import org.apache.tez.runtime.api.events.VertexManagerEvent; import org.apache.tez.runtime.api.events.EventProtos.VertexManagerEventProto; -public class ProtoConverters { +public final class ProtoConverters { public static EventProtos.CustomProcessorEventProto convertCustomProcessorEventToProto( CustomProcessorEvent event) { @@ -168,4 +168,5 @@ public static InputInitializerEvent convertRootInputInitializerEventFromProto( return event; } + private ProtoConverters() {} } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/common/TezConverterUtils.java b/tez-runtime-internals/src/main/java/org/apache/tez/common/TezConverterUtils.java index 02dc69c3e5..933115757d 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/common/TezConverterUtils.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/common/TezConverterUtils.java @@ -28,7 +28,9 @@ import org.apache.tez.runtime.api.TaskFailureType; import org.apache.tez.runtime.internals.api.events.SystemEventProtos.TaskFailureTypeProto; -public class TezConverterUtils { +public final class TezConverterUtils { + + private TezConverterUtils() {} /** * return a {@link URI} from a given url @@ -36,7 +38,6 @@ public class TezConverterUtils { * @param url * url to convert * @return path from {@link URL} - * @throws URISyntaxException */ @Private public static URI getURIFromYarnURL(URL url) throws URISyntaxException { diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TaskExecutionTestHelpers.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TaskExecutionTestHelpers.java index 626d178cec..b6000ccae9 100644 --- a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TaskExecutionTestHelpers.java +++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TaskExecutionTestHelpers.java @@ -21,6 +21,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; @@ -49,10 +50,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class TaskExecutionTestHelpers { +public final class TaskExecutionTestHelpers { public static final String HEARTBEAT_EXCEPTION_STRING = "HeartbeatException"; + private TaskExecutionTestHelpers() {} + // Uses static fields for signaling. Ensure only used by one test at a time. public static class TestProcessor extends AbstractLogicalIOProcessor { @@ -184,9 +187,7 @@ public static void awaitCompletion() throws InterruptedException { LOG.info("Await completion"); processorLock.lock(); try { - if (completed) { - return; - } else { + if (!completed) { completionCondition.await(); } } finally { @@ -295,14 +296,14 @@ public static class TezTaskUmbilicalForTest implements TezTaskUmbilicalProtocol private static final Logger LOG = LoggerFactory.getLogger(TezTaskUmbilicalForTest.class); - private final List requestEvents = new LinkedList(); + private final List requestEvents = new LinkedList<>(); private final ReentrantLock umbilicalLock = new ReentrantLock(); private final Condition eventCondition = umbilicalLock.newCondition(); private boolean pendingEvent = false; private boolean eventEnacted = false; - volatile int getTaskInvocations = 0; + private final AtomicInteger taskInvocations = new AtomicInteger(0); private boolean shouldThrowException = false; private boolean shouldSendDieSignal = false; @@ -461,20 +462,20 @@ public void verifyTaskSuccessEvent() { } @Override - public long getProtocolVersion(String protocol, long clientVersion) throws IOException { + public long getProtocolVersion(String protocol, long clientVersion) { return 0; } @Override public ProtocolSignature getProtocolSignature(String protocol, long clientVersion, - int clientMethodsHash) throws IOException { + int clientMethodsHash) { return null; } @Override public ContainerTask getTask(ContainerContext containerContext) throws IOException { // Return shouldDie = true - getTaskInvocations++; + taskInvocations.incrementAndGet(); return new ContainerTask(null, true, null, null, false); } @@ -511,18 +512,20 @@ public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request) throws IOExce umbilicalLock.unlock(); } } + + public int getTaskInvocations() { + return taskInvocations.get(); + } } @SuppressWarnings("deprecation") public static ContainerId createContainerId(ApplicationId appId) { ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1); - ContainerId containerId = ContainerId.newInstance(appAttemptId, 1); - return containerId; + return ContainerId.newInstance(appAttemptId, 1); } public static TaskReporter createTaskReporter(ApplicationId appId, TezTaskUmbilicalForTest umbilical) { - TaskReporter taskReporter = new TaskReporter(umbilical, 100, 1000, 100, new AtomicLong(0), + return new TaskReporter(umbilical, 100, 1000, 100, new AtomicLong(0), createContainerId(appId).toString()); - return taskReporter; } } diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestContainerExecution.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestContainerExecution.java index c3c4705c15..a570ab8243 100644 --- a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestContainerExecution.java +++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestContainerExecution.java @@ -51,7 +51,7 @@ public void testGetTaskShouldDie() throws InterruptedException, ExecutionExcepti ListenableFuture getTaskFuture = executor.submit(containerReporter); getTaskFuture.get(); - assertEquals(1, umbilical.getTaskInvocations); + assertEquals(1, umbilical.getTaskInvocations()); } finally { executor.shutdownNow(); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/common/TezRuntimeFrameworkConfigs.java b/tez-runtime-library/src/main/java/org/apache/tez/common/TezRuntimeFrameworkConfigs.java index 62bc232b88..61c0fcd317 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/common/TezRuntimeFrameworkConfigs.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/common/TezRuntimeFrameworkConfigs.java @@ -27,7 +27,7 @@ * Configuration parameters which are internal to the Inputs and Outputs which exist in the Runtime */ @Private -public class TezRuntimeFrameworkConfigs { +public final class TezRuntimeFrameworkConfigs { private static final String TEZ_RUNTIME_FRAMEWORK_PREFIX = "tez.runtime.framework."; @@ -41,4 +41,6 @@ public class TezRuntimeFrameworkConfigs { public static final String TEZ_RUNTIME_METRICS_SESSION_ID = TEZ_RUNTIME_FRAMEWORK_PREFIX + "metrics.session.id"; public static final String TEZ_RUNTIME_METRICS_SESSION_ID_DEFAULT = ""; + + private TezRuntimeFrameworkConfigs() {} } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java index 1c747af2bb..de28286d9b 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java @@ -47,23 +47,22 @@ @Public @Evolving @ConfigurationClass(templateFileName = "tez-runtime-default-template.xml") -public class TezRuntimeConfiguration { +public final class TezRuntimeConfiguration { private static final String TEZ_RUNTIME_PREFIX = "tez.runtime."; - private static final Set tezRuntimeKeys = new HashSet(); - private static Set umnodifiableTezRuntimeKeySet; - private static final Set otherKeys = new HashSet(); - private static Set unmodifiableOtherKeySet; - private static Configuration defaultConf = new Configuration(false); - private static final Map tezRuntimeConfMap = new HashMap(); - private static final Map otherConfMap = new HashMap(); + private static final Set TEZ_RUNTIME_KEYS = new HashSet<>(); + private static final Set UMNODIFIABLE_TEZ_RUNTIME_KEY_SET; + private static final Set OTHER_KEYS = new HashSet<>(); + private static final Set UNMODIFIABLE_OTHER_KEY_SET; + private static final Configuration DEFAULT_CONF = new Configuration(false); + private static final Map TEZ_RUNTIME_CONF_MAP = new HashMap<>(); + private static final Map OTHER_CONF_MAP = new HashMap<>(); /** * Prefixes from Hadoop configuration which are allowed. */ - private static final List allowedPrefixes = new ArrayList(); - private static List unmodifiableAllowedPrefixes; + private static final List ALLOWED_PREFIXES = new ArrayList<>(); static { @@ -71,6 +70,8 @@ public class TezRuntimeConfiguration { TezRuntimeConfiguration.TEZ_RUNTIME_PIPELINED_SORTER_SORT_THREADS); } + private TezRuntimeConfiguration() {} + /** * Configuration key to enable/disable IFile readahead. */ @@ -250,7 +251,7 @@ public class TezRuntimeConfiguration { @Private @Unstable @ConfigurationProperty(type = "integer") - /** + /* * Expert setting made available only for debugging. Do not change it. Sets * the number of retries before giving up on downloading from source * attempt by consumer. Code internally handles the threshold if set to -1. @@ -264,7 +265,7 @@ public class TezRuntimeConfiguration { @Private @Unstable @ConfigurationProperty(type = "float") - /** + /* * Expert setting made available only for debugging. Do not change it. Setting * to determine if failures happened across a percentage of nodes. This * helps in determining if the consumer has to be restarted on continuous @@ -280,7 +281,7 @@ public class TezRuntimeConfiguration { @Private @Unstable @ConfigurationProperty(type = "integer") - /** + /* * Expert setting made available only for debugging. Do not change it. Setting * to determine if the consumer has to be restarted on continuous * failures across nodes. Used along with {@link @@ -294,7 +295,7 @@ public class TezRuntimeConfiguration { @Private @Unstable @ConfigurationProperty(type = "float") - /** + /* * Expert setting made available only for debugging. Do not change it. * Maximum percentage of time (compared to overall progress), the fetcher is * allowed before concluding that it is stalled. @@ -307,7 +308,7 @@ public class TezRuntimeConfiguration { @Private @Unstable @ConfigurationProperty(type = "float") - /** + /* * Expert setting made available only for debugging. Do not change it. * Fraction to determine whether the shuffle has progressed enough or not * If it has not progressed enough, it could be qualified for the consumer. @@ -321,7 +322,7 @@ public class TezRuntimeConfiguration { @Private @Unstable @ConfigurationProperty(type = "float") - /** + /* * Expert setting made available only for debugging. Do not change it. * Provides threshold for determining whether fetching has to be marked * unhealthy based on the ratio of (failures/(failures+completed)) @@ -335,7 +336,7 @@ public class TezRuntimeConfiguration { @Private @Unstable @ConfigurationProperty(type = "boolean") - /** + /* * Expert setting made available only for debugging. Do not change it. * Provides threshold for determining whether fetching has to be marked * unhealthy based on the ratio of (failures/(failures+completed)) @@ -432,7 +433,7 @@ public class TezRuntimeConfiguration { * "host1#Map_1#100": from host1 for Map 1 source tasks with 100% likelihood * "host1#Map_1#100#fail_only_first": as above but only for input attempts with index 0 */ - @ConfigurationProperty(type = "string") + @ConfigurationProperty() public static final String TEZ_RUNTIME_SHUFFLE_FETCH_TESTING_ERRORS_CONFIG = TEZ_RUNTIME_PREFIX + "shuffle.fetch.testing.errors.config"; public static final String TEZ_RUNTIME_SHUFFLE_FETCH_TESTING_ERRORS_CONFIG_DEFAULT = "*#50"; @@ -608,130 +609,130 @@ public class TezRuntimeConfiguration { public static final long TEZ_RUNTIME_RECORDS_BEFORE_PROGRESS_DEFAULT = 10000; static { - tezRuntimeKeys.add(TEZ_RUNTIME_IFILE_READAHEAD); - tezRuntimeKeys.add(TEZ_RUNTIME_IFILE_READAHEAD_BYTES); - tezRuntimeKeys.add(TEZ_RUNTIME_IO_FILE_BUFFER_SIZE); - tezRuntimeKeys.add(TEZ_RUNTIME_IO_SORT_FACTOR); - tezRuntimeKeys.add(TEZ_RUNTIME_SORT_SPILL_PERCENT); - tezRuntimeKeys.add(TEZ_RUNTIME_IO_SORT_MB); - tezRuntimeKeys.add(TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES); - tezRuntimeKeys.add(TEZ_RUNTIME_COMBINE_MIN_SPILLS); - tezRuntimeKeys.add(TEZ_RUNTIME_PIPELINED_SORTER_SORT_THREADS); - tezRuntimeKeys.add( + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_IFILE_READAHEAD); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_IFILE_READAHEAD_BYTES); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_IO_FILE_BUFFER_SIZE); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_IO_SORT_FACTOR); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SORT_SPILL_PERCENT); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_IO_SORT_MB); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_COMBINE_MIN_SPILLS); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_PIPELINED_SORTER_SORT_THREADS); + TEZ_RUNTIME_KEYS.add( TEZ_RUNTIME_PIPELINED_SORTER_MIN_BLOCK_SIZE_IN_MB); - tezRuntimeKeys.add(TEZ_RUNTIME_PIPELINED_SORTER_LAZY_ALLOCATE_MEMORY); - tezRuntimeKeys.add(TEZ_RUNTIME_UNORDERED_OUTPUT_BUFFER_SIZE_MB); - tezRuntimeKeys.add(TEZ_RUNTIME_UNORDERED_OUTPUT_MAX_PER_BUFFER_SIZE_BYTES); - tezRuntimeKeys.add(TEZ_RUNTIME_PARTITIONER_CLASS); - tezRuntimeKeys.add(TEZ_RUNTIME_COMBINER_CLASS); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_USE_ASYNC_HTTP); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES_LIMIT); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_FETCH_MAX_TASK_OUTPUT_AT_ONCE); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_NOTIFY_READERROR); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_CONNECT_TIMEOUT); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_KEEP_ALIVE_ENABLED); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_KEEP_ALIVE_MAX_CONNECTIONS); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_BUFFER_SIZE); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_ENABLE_SSL); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_FETCH_VERIFY_DISK_CHECKSUM); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_FETCH_BUFFER_PERCENT); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_MERGE_PERCENT); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_MEMTOMEM_SEGMENTS); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_ENABLE_MEMTOMEM); - tezRuntimeKeys.add + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_PIPELINED_SORTER_LAZY_ALLOCATE_MEMORY); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_UNORDERED_OUTPUT_BUFFER_SIZE_MB); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_UNORDERED_OUTPUT_MAX_PER_BUFFER_SIZE_BYTES); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_PARTITIONER_CLASS); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_COMBINER_CLASS); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_USE_ASYNC_HTTP); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES_LIMIT); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_FETCH_MAX_TASK_OUTPUT_AT_ONCE); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_NOTIFY_READERROR); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_CONNECT_TIMEOUT); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_KEEP_ALIVE_ENABLED); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_KEEP_ALIVE_MAX_CONNECTIONS); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_BUFFER_SIZE); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_ENABLE_SSL); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_FETCH_VERIFY_DISK_CHECKSUM); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_FETCH_BUFFER_PERCENT); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_MERGE_PERCENT); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_MEMTOMEM_SEGMENTS); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_ENABLE_MEMTOMEM); + TEZ_RUNTIME_KEYS.add (TEZ_RUNTIME_SHUFFLE_ACCEPTABLE_HOST_FETCH_FAILURE_FRACTION); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_MIN_FAILURES_PER_HOST); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_MAX_STALL_TIME_FRACTION); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_SOURCE_ATTEMPT_ABORT_LIMIT); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_MAX_ALLOWED_FAILED_FETCH_ATTEMPT_FRACTION); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_MIN_REQUIRED_PROGRESS_FRACTION); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_FAILED_CHECK_SINCE_LAST_COMPLETION); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_FETCH_TESTING_ERRORS_CONFIG); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_FETCH_ENABLE_TESTING_ERRORS); - tezRuntimeKeys.add(TEZ_RUNTIME_REPORT_PARTITION_STATS); - tezRuntimeKeys.add(TEZ_RUNTIME_INPUT_POST_MERGE_BUFFER_PERCENT); - tezRuntimeKeys.add(TEZ_RUNTIME_GROUP_COMPARATOR_CLASS); - tezRuntimeKeys.add(TEZ_RUNTIME_INTERNAL_SORTER_CLASS); - tezRuntimeKeys.add(TEZ_RUNTIME_KEY_COMPARATOR_CLASS); - tezRuntimeKeys.add(TEZ_RUNTIME_KEY_CLASS); - tezRuntimeKeys.add(TEZ_RUNTIME_VALUE_CLASS); - tezRuntimeKeys.add(TEZ_RUNTIME_COMPRESS); - tezRuntimeKeys.add(TEZ_RUNTIME_COMPRESS_CODEC); - tezRuntimeKeys.add(TEZ_RUNTIME_KEY_SECONDARY_COMPARATOR_CLASS); - tezRuntimeKeys.add(TEZ_RUNTIME_EMPTY_PARTITION_INFO_VIA_EVENTS_ENABLED); - tezRuntimeKeys.add(TEZ_RUNTIME_PIPELINED_SHUFFLE_ENABLED); - tezRuntimeKeys.add(TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT); - tezRuntimeKeys.add(TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_ENABLED); - tezRuntimeKeys.add(TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_MAX_SIZE); - tezRuntimeKeys.add(TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_SUPPORT_IN_MEM_FILE); - tezRuntimeKeys.add(TEZ_RUNTIME_RECORDS_BEFORE_PROGRESS); - tezRuntimeKeys.add(TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH); - tezRuntimeKeys.add(TEZ_RUNTIME_OPTIMIZE_SHARED_FETCH); - tezRuntimeKeys.add(TEZ_RUNTIME_CONVERT_USER_PAYLOAD_TO_HISTORY_TEXT); - tezRuntimeKeys.add(TEZ_RUNTIME_SORTER_CLASS); - tezRuntimeKeys.add(TEZ_RUNTIME_CLEANUP_FILES_ON_INTERRUPT); - tezRuntimeKeys.add(TEZ_RUNTIME_UNORDERED_PARTITIONED_KVWRITER_BUFFER_MERGE_PERCENT); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_FETCHER_USE_SHARED_POOL); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_HOST_PENALTY_TIME_LIMIT_MS); - tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_BATCH_WAIT); - - defaultConf.addResource("core-default.xml"); - defaultConf.addResource("core-site.xml"); - defaultConf.addResource("tez-site.xml"); - - for (Map.Entry confEntry : defaultConf) { - if (tezRuntimeKeys.contains(confEntry.getKey())) { - tezRuntimeConfMap.put(confEntry.getKey(), confEntry.getValue()); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_MIN_FAILURES_PER_HOST); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_MAX_STALL_TIME_FRACTION); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_SOURCE_ATTEMPT_ABORT_LIMIT); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_MAX_ALLOWED_FAILED_FETCH_ATTEMPT_FRACTION); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_MIN_REQUIRED_PROGRESS_FRACTION); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_FAILED_CHECK_SINCE_LAST_COMPLETION); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_FETCH_TESTING_ERRORS_CONFIG); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_FETCH_ENABLE_TESTING_ERRORS); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_REPORT_PARTITION_STATS); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_INPUT_POST_MERGE_BUFFER_PERCENT); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_GROUP_COMPARATOR_CLASS); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_INTERNAL_SORTER_CLASS); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_KEY_COMPARATOR_CLASS); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_KEY_CLASS); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_VALUE_CLASS); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_COMPRESS); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_COMPRESS_CODEC); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_KEY_SECONDARY_COMPARATOR_CLASS); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_EMPTY_PARTITION_INFO_VIA_EVENTS_ENABLED); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_PIPELINED_SHUFFLE_ENABLED); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_ENABLED); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_MAX_SIZE); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_SUPPORT_IN_MEM_FILE); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_RECORDS_BEFORE_PROGRESS); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_OPTIMIZE_SHARED_FETCH); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_CONVERT_USER_PAYLOAD_TO_HISTORY_TEXT); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SORTER_CLASS); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_CLEANUP_FILES_ON_INTERRUPT); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_UNORDERED_PARTITIONED_KVWRITER_BUFFER_MERGE_PERCENT); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_FETCHER_USE_SHARED_POOL); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_HOST_PENALTY_TIME_LIMIT_MS); + TEZ_RUNTIME_KEYS.add(TEZ_RUNTIME_SHUFFLE_BATCH_WAIT); + + DEFAULT_CONF.addResource("core-default.xml"); + DEFAULT_CONF.addResource("core-site.xml"); + DEFAULT_CONF.addResource("tez-site.xml"); + + for (Map.Entry confEntry : DEFAULT_CONF) { + if (TEZ_RUNTIME_KEYS.contains(confEntry.getKey())) { + TEZ_RUNTIME_CONF_MAP.put(confEntry.getKey(), confEntry.getValue()); } else { // TODO TEZ-1232 Filter out parameters from TezConfiguration, and Task specific confs - otherConfMap.put(confEntry.getKey(), confEntry.getValue()); - otherKeys.add(confEntry.getKey()); + OTHER_CONF_MAP.put(confEntry.getKey(), confEntry.getValue()); + OTHER_KEYS.add(confEntry.getKey()); } } // Do NOT need all prefixes from the following list. Only specific ones are allowed // "hadoop.", "hadoop.security", "io.", "fs.", "ipc.", "net.", "file.", "dfs.", "ha.", "s3.", "nfs3.", "rpc.", "ssl." - allowedPrefixes.add("io."); - allowedPrefixes.add("file."); - allowedPrefixes.add("fs."); - allowedPrefixes.add("ssl."); - - umnodifiableTezRuntimeKeySet = Collections.unmodifiableSet(tezRuntimeKeys); - unmodifiableOtherKeySet = Collections.unmodifiableSet(otherKeys); - unmodifiableAllowedPrefixes = Collections.unmodifiableList(allowedPrefixes); + ALLOWED_PREFIXES.add("io."); + ALLOWED_PREFIXES.add("file."); + ALLOWED_PREFIXES.add("fs."); + ALLOWED_PREFIXES.add("ssl."); + + UMNODIFIABLE_TEZ_RUNTIME_KEY_SET = Collections.unmodifiableSet(TEZ_RUNTIME_KEYS); + UNMODIFIABLE_OTHER_KEY_SET = Collections.unmodifiableSet(OTHER_KEYS); + List unmodifiableAllowedPrefixes = Collections.unmodifiableList(ALLOWED_PREFIXES); } @Private public static Set getRuntimeConfigKeySet() { - return umnodifiableTezRuntimeKeySet; + return UMNODIFIABLE_TEZ_RUNTIME_KEY_SET; } @Private public static Set getRuntimeAdditionalConfigKeySet() { - return unmodifiableOtherKeySet; + return UNMODIFIABLE_OTHER_KEY_SET; } @Private public static List getAllowedPrefixes() { - return allowedPrefixes; + return ALLOWED_PREFIXES; } @Private public static Map getTezRuntimeConfigDefaults() { - return Collections.unmodifiableMap(tezRuntimeConfMap); + return Collections.unmodifiableMap(TEZ_RUNTIME_CONF_MAP); } @Private public static Map getOtherConfigDefaults() { - return Collections.unmodifiableMap(otherConfMap); + return Collections.unmodifiableMap(OTHER_CONF_MAP); } public enum ReportPartitionStats { @Deprecated - /** + /* * Don't report partition stats. It is the same as NONE. * It is defined to maintain backward compatibility given * Configuration @link{#TEZ_RUNTIME_REPORT_PARTITION_STATS} used @@ -740,7 +741,7 @@ public enum ReportPartitionStats { DISABLED("false"), @Deprecated - /** + /* * Report partition stats. It is the same as MEMORY_OPTIMIZED. * It is defined to maintain backward compatibility given * Configuration @link{#TEZ_RUNTIME_REPORT_PARTITION_STATS} used @@ -766,7 +767,7 @@ public enum ReportPartitionStats { private final String type; - private ReportPartitionStats(String type) { + ReportPartitionStats(String type) { this.type = type; } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java index 74e74f2780..3bb85e33c4 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java @@ -37,7 +37,9 @@ @SuppressWarnings({"unchecked", "rawtypes"}) @InterfaceAudience.Private -public class ConfigUtils { +public final class ConfigUtils { + + private ConfigUtils() {} public static Class getIntermediateOutputCompressorClass( Configuration conf, Class defaultValue) { @@ -64,31 +66,27 @@ public static boolean shouldCompressIntermediateOutput(Configuration conf) { } public static Class getIntermediateOutputValueClass(Configuration conf) { - Class retv = (Class) conf.getClass( + return (Class) conf.getClass( TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS, null, Object.class); - return retv; } public static Class getIntermediateInputValueClass(Configuration conf) { - Class retv = (Class) conf.getClass( + return (Class) conf.getClass( TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS, null, Object.class); - return retv; } public static Class getIntermediateOutputKeyClass(Configuration conf) { - Class retv = (Class) conf.getClass( + return (Class) conf.getClass( TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, null, Object.class); - return retv; } public static Class getIntermediateInputKeyClass(Configuration conf) { - Class retv = (Class) conf.getClass( + return (Class) conf.getClass( TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, null, Object.class); - return retv; } public static RawComparator getIntermediateOutputKeyComparator(Configuration conf) { @@ -135,7 +133,7 @@ public static boolean useNewApi(Configuration conf) { public static Map extractConfigurationMap(Map confMap, Set allowedKeys) { Objects.requireNonNull(confMap, "ConfMap cannot be null"); Objects.requireNonNull(allowedKeys, "Valid key set cannot be empty"); - Map map = new HashMap(); + Map map = new HashMap<>(); for (Map.Entry entry : confMap.entrySet()) { if (allowedKeys.contains(entry.getKey())) { map.put(entry.getKey(), entry.getValue()); @@ -215,11 +213,11 @@ public static void mergeConfs(Configuration destConf, Configuration srcConf) { private static Map extractConfigurationMapInternal( Iterable> iterable, List> validKeySets, List allowedPrefixes) { - Set validKeys = new HashSet(); + Set validKeys = new HashSet<>(); for (Set set : validKeySets) { validKeys.addAll(set); } - Map localConfMap = new HashMap(); + Map localConfMap = new HashMap<>(); for (Map.Entry entry : iterable) { if (validKeys.contains(entry.getKey())) { localConfMap.put(entry.getKey(), entry.getValue()); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/Constants.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/Constants.java index 81921b2e85..e6cf73944f 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/Constants.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/Constants.java @@ -20,7 +20,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; @Private -public class Constants { +public final class Constants { // TODO NEWTEZ Check which of these constants are expecting specific pieces of information which are being removed - like taskAttemptId @@ -64,4 +64,5 @@ public class Constants { public static final String TEZ_RUNTIME_TASK_OUTPUT_MANAGER = "tez.runtime.task.local.output.manager"; + private Constants() {} } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java index a75925c5fe..32e76f4c1c 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java @@ -46,7 +46,7 @@ import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutputFiles; @Private -public class TezRuntimeUtils { +public final class TezRuntimeUtils { private static final Logger LOG = LoggerFactory .getLogger(TezRuntimeUtils.class); @@ -55,6 +55,8 @@ public class TezRuntimeUtils { //ShufflePort by default for ContainerLaunchers public static final int INVALID_PORT = -1; + private TezRuntimeUtils() {} + public static String getTaskIdentifier(String vertexName, int taskIndex) { return String.format("%s_%06d", vertexName, taskIndex); } @@ -85,26 +87,17 @@ public static Combiner instantiateCombiner(Configuration conf, TaskContext taskC throw new IOException("Unable to load combiner class: " + className); } - Combiner combiner = null; + Combiner combiner; - Constructor ctor; - try { - ctor = clazz.getConstructor(TaskContext.class); - combiner = ctor.newInstance(taskContext); - } catch (SecurityException e) { - throw new IOException(e); - } catch (NoSuchMethodException e) { - throw new IOException(e); - } catch (IllegalArgumentException e) { - throw new IOException(e); - } catch (InstantiationException e) { - throw new IOException(e); - } catch (IllegalAccessException e) { - throw new IOException(e); - } catch (InvocationTargetException e) { - throw new IOException(e); - } - return combiner; + Constructor ctor; + try { + ctor = clazz.getConstructor(TaskContext.class); + combiner = ctor.newInstance(taskContext); + } catch (SecurityException | NoSuchMethodException | IllegalArgumentException | InstantiationException + | IllegalAccessException | InvocationTargetException e) { + throw new IOException(e); + } + return combiner; } @SuppressWarnings("unchecked") @@ -123,31 +116,22 @@ public static Partitioner instantiatePartitioner(Configuration conf) LOG.debug("Using partitioner class: " + clazz.getName()); } - Partitioner partitioner = null; + Partitioner partitioner; try { Constructor ctorWithConf = clazz .getConstructor(Configuration.class); partitioner = ctorWithConf.newInstance(conf); - } catch (SecurityException e) { + } catch (SecurityException | IllegalArgumentException | InstantiationException | IllegalAccessException + | InvocationTargetException e) { throw new IOException(e); } catch (NoSuchMethodException e) { try { // Try a 0 argument constructor. partitioner = clazz.newInstance(); - } catch (InstantiationException e1) { - throw new IOException(e1); - } catch (IllegalAccessException e1) { + } catch (InstantiationException | IllegalAccessException e1) { throw new IOException(e1); } - } catch (IllegalArgumentException e) { - throw new IOException(e); - } catch (InstantiationException e) { - throw new IOException(e); - } catch (IllegalAccessException e) { - throw new IOException(e); - } catch (InvocationTargetException e) { - throw new IOException(e); } return partitioner; } @@ -158,10 +142,9 @@ public static TezTaskOutput instantiateTaskOutputManager(Configuration conf, Out try { Constructor ctor = clazz.getConstructor(Configuration.class, String.class, int.class); ctor.setAccessible(true); - TezTaskOutput instance = (TezTaskOutput) ctor.newInstance(conf, + return (TezTaskOutput) ctor.newInstance(conf, outputContext.getUniqueIdentifier(), outputContext.getDagIdentifier()); - return instance; } catch (Exception e) { throw new TezUncheckedException( "Unable to instantiate configured TezOutputFileManager: " @@ -183,7 +166,7 @@ public static URL constructBaseURIForShuffleHandlerDagComplete( sb.append("&job="); sb.append(appId.replace("application", "job")); sb.append("&dag="); - sb.append(String.valueOf(dagIdentifier)); + sb.append(dagIdentifier); return new URL(sb.toString()); } @@ -200,9 +183,9 @@ public static URL constructBaseURIForShuffleHandlerVertexComplete( sb.append("&job="); sb.append(appId.replace("application", "job")); sb.append("&dag="); - sb.append(String.valueOf(dagIdentifier)); + sb.append(dagIdentifier); sb.append("&vertex="); - sb.append(String.valueOf(vertexIndentifier)); + sb.append(vertexIndentifier); return new URL(sb.toString()); } @@ -219,9 +202,9 @@ public static URL constructBaseURIForShuffleHandlerTaskAttemptFailed( sb.append("&job="); sb.append(appId.replace("application", "job")); sb.append("&dag="); - sb.append(String.valueOf(dagIdentifier)); + sb.append(dagIdentifier); sb.append("&map="); - sb.append(String.valueOf(taskAttemptIdentifier)); + sb.append(taskAttemptIdentifier); return new URL(sb.toString()); } @@ -269,10 +252,9 @@ public static HttpConnectionParams getHttpConnectionParams(Configuration conf) { } } - HttpConnectionParams httpConnParams = new HttpConnectionParams(keepAlive, + return new HttpConnectionParams(keepAlive, keepAliveMaxConnections, connectionTimeout, readTimeout, bufferSize, sslShuffle, sslFactory); - return httpConnParams; } public static BaseHttpConnection getHttpConnection(boolean asyncHttp, URL url, @@ -288,13 +270,9 @@ public static BaseHttpConnection getHttpConnection(boolean asyncHttp, URL url, public static int deserializeShuffleProviderMetaData(ByteBuffer meta) throws IOException { - DataInputByteBuffer in = new DataInputByteBuffer(); - try { + try (DataInputByteBuffer in = new DataInputByteBuffer()) { in.reset(meta); - int port = in.readInt(); - return port; - } finally { - in.close(); + return in.readInt(); } } } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/SecureShuffleUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/SecureShuffleUtils.java index 67b8de27c5..19fbf2d084 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/SecureShuffleUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/SecureShuffleUtils.java @@ -37,13 +37,14 @@ */ @InterfaceAudience.Private @InterfaceStability.Unstable -public class SecureShuffleUtils { +public final class SecureShuffleUtils { public static final String HTTP_HEADER_URL_HASH = "UrlHash"; public static final String HTTP_HEADER_REPLY_URL_HASH = "ReplyHash"; + + private SecureShuffleUtils() {} /** * Base64 encoded hash of msg - * @param msg */ public static String generateHash(byte[] msg, SecretKey key) { return new String(Base64.encodeBase64(generateByteHash(msg, key)), Charsets.UTF_8); @@ -51,7 +52,7 @@ public static String generateHash(byte[] msg, SecretKey key) { /** * calculate hash of msg - * @param msg + * * @return byte array containing computed hash of message */ private static byte[] generateByteHash(byte[] msg, SecretKey key) { @@ -63,9 +64,6 @@ private static byte[] generateByteHash(byte[] msg, SecretKey key) { * This is only meant to be used when a process needs to verify against multiple different keys * (ShuffleHandler for instance) * - * @param hash - * @param msg - * @param key * @return true when hashes match; false otherwise */ private static boolean verifyHash(byte[] hash, byte[] msg, SecretKey key) { @@ -75,9 +73,7 @@ private static boolean verifyHash(byte[] hash, byte[] msg, SecretKey key) { /** * verify that hash equals to HMacHash(msg) - * @param hash - * @param msg - * @param mgr JobTokenSecretManager + * * @return true when hashes match; false otherwise */ private static boolean verifyHash(byte[] hash, byte[] msg, JobTokenSecretManager mgr) { @@ -87,14 +83,10 @@ private static boolean verifyHash(byte[] hash, byte[] msg, JobTokenSecretManager /** * Aux util to calculate hash of a String - * @param enc_str - * @param mgr JobTokenSecretManager - * @return Base64 encodedHash - * @throws IOException + * */ - public static String hashFromString(String enc_str, JobTokenSecretManager mgr) - throws IOException { - return new String(Base64.encodeBase64(mgr.computeHash(enc_str.getBytes(Charsets.UTF_8))), Charsets.UTF_8); + public static String hashFromString(String encStr, JobTokenSecretManager mgr) { + return new String(Base64.encodeBase64(mgr.computeHash(encStr.getBytes(Charsets.UTF_8))), Charsets.UTF_8); } /** @@ -106,13 +98,12 @@ public static String hashFromString(String enc_str, JobTokenSecretManager mgr) * @param base64Hash base64 encoded hash * @param msg the message * @param key the key to use to generate the hash from the message - * @throws IOException */ public static void verifyReply(String base64Hash, String msg, SecretKey key) throws IOException { byte[] hash = Base64.decodeBase64(base64Hash.getBytes(Charsets.UTF_8)); boolean res = verifyHash(hash, msg.getBytes(Charsets.UTF_8), key); - if(res != true) { + if(!res) { throw new IOException("Verification of the hashReply failed"); } } @@ -120,7 +111,7 @@ public static void verifyReply(String base64Hash, String msg, SecretKey key) thr /** * verify that base64Hash is same as HMacHash(msg) * @param base64Hash (Base64 encoded hash) - * @param msg + * @param msg the message * @throws IOException if not the same */ public static void verifyReply(String base64Hash, String msg, JobTokenSecretManager mgr) @@ -129,14 +120,14 @@ public static void verifyReply(String base64Hash, String msg, JobTokenSecretMana boolean res = verifyHash(hash, msg.getBytes(Charsets.UTF_8), mgr); - if(res != true) { + if(!res) { throw new IOException("Verification of the hashReply failed"); } } /** * Shuffle specific utils - build string for encoding from URL - * @param url + * * @return string for encoding */ public static String buildMsgFrom(URL url) { @@ -145,11 +136,10 @@ public static String buildMsgFrom(URL url) { /** * Shuffle specific utils - build string for encoding from URL - * @param uri_path - * @param uri_query + * * @return string for encoding */ private static String buildMsgFrom(String uri_path, String uri_query, int port) { - return String.valueOf(port) + uri_path + "?" + uri_query; + return port + uri_path + "?" + uri_query; } } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java index ca8b84724e..c5fb651167 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java @@ -72,7 +72,7 @@ import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto; import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DetailedPartitionStatsProto; -public class ShuffleUtils { +public final class ShuffleUtils { private static final Logger LOG = LoggerFactory.getLogger(ShuffleUtils.class); private static final long MB = 1024l * 1024l; @@ -94,6 +94,8 @@ protected FastNumberFormat initialValue() { } }; + private ShuffleUtils() {} + public static SecretKey getJobTokenSecretFromTokenBytes(ByteBuffer meta) throws IOException { DataInputByteBuffer in = new DataInputByteBuffer(); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java index 8f673185b0..671a426b66 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java @@ -62,7 +62,7 @@ */ @InterfaceAudience.Private @InterfaceStability.Unstable -public class IFile { +public final class IFile { private static final Logger LOG = LoggerFactory.getLogger(IFile.class); public static final int EOF_MARKER = -1; // End of File Marker public static final int RLE_MARKER = -2; // Repeat same key marker @@ -74,6 +74,8 @@ public class IFile { private static final String INCOMPLETE_READ = "Requested to read %d got %d"; private static final String REQ_BUFFER_SIZE_TOO_LARGE = "Size of data %d is greater than the max allowed of %d"; + private IFile() {} + /** * IFileWriter which stores data in memory for specified limit, beyond * which it falls back to file based writer. It creates files lazily on @@ -91,34 +93,22 @@ public class IFile { */ public static class FileBackedInMemIFileWriter extends Writer { - private FileSystem fs; + private final FileSystem fs; private boolean bufferFull; // For lazy creation of file - private TezTaskOutput taskOutput; + private final TezTaskOutput taskOutput; private int totalSize; private Path outputPath; - private CompressionCodec fileCodec; - private BoundedByteArrayOutputStream cacheStream; + private final CompressionCodec fileCodec; + private final BoundedByteArrayOutputStream cacheStream; private static final int checksumSize = IFileOutputStream.getCheckSumSize(); /** * Note that we do not allow compression in in-mem stream. * When spilled over to file, compression gets enabled. - * - * @param keySerialization - * @param valSerialization - * @param fs - * @param taskOutput - * @param keyClass - * @param valueClass - * @param codec - * @param writesCounter - * @param serializedBytesCounter - * @param cacheSize - * @throws IOException */ public FileBackedInMemIFileWriter(Serialization keySerialization, Serialization valSerialization, FileSystem fs, TezTaskOutput taskOutput, @@ -151,7 +141,6 @@ boolean shouldWriteToDisk() { /** * Create in mem stream. In it is too small, adjust it's size * - * @param size * @return in memory stream */ public static BoundedByteArrayOutputStream createBoundedBuffer(int size) { @@ -168,8 +157,6 @@ public static BoundedByteArrayOutputStream createBoundedBuffer(int size) { * out. * 3. Create relevant file based writer. * 4. Write header and then real data. - * - * @throws IOException */ private void resetToFileBasedWriter() throws IOException { // Close out stream, so that data checksums are written. @@ -409,7 +396,7 @@ public void close() throws IOException { // Write EOF_MARKER for key/value length WritableUtils.writeVInt(out, EOF_MARKER); WritableUtils.writeVInt(out, EOF_MARKER); - decompressedBytesWritten += 2 * WritableUtils.getVIntSize(EOF_MARKER); + decompressedBytesWritten += 2L * WritableUtils.getVIntSize(EOF_MARKER); //account for header bytes decompressedBytesWritten += HEADER.length; @@ -451,10 +438,6 @@ public void close() throws IOException { * one, send IFile.REPEAT_KEY as key parameter. Should not call this method with * IFile.REPEAT_KEY as the first key. It is caller's responsibility to ensure that correct * key/value type checks and key/value length (non-negative) checks are done properly. - * - * @param key - * @param value - * @throws IOException */ public void append(Object key, Object value) throws IOException { int keyLength = 0; @@ -493,9 +476,6 @@ public void append(Object key, Object value) throws IOException { /** * Appends the value to previous key. Assumes that the caller has already done relevant checks * for identical keys. Also, no validations are done in this method - * - * @param value - * @throws IOException */ public void appendValue(Object value) throws IOException { valueSerializer.serialize(value); @@ -511,9 +491,6 @@ public void appendValue(Object value) throws IOException { * for identical keys. Also, no validations are done in this method. It is caller's responsibility * to pass non-negative key/value lengths. Otherwise,IndexOutOfBoundsException could be * thrown at runtime. - * - * @param value - * @throws IOException */ public void appendValue(DataInputBuffer value) throws IOException { int valueLength = value.getLength() - value.getPosition(); @@ -527,9 +504,6 @@ public void appendValue(DataInputBuffer value) throws IOException { /** * Appends the value to previous key. Assumes that the caller has already done relevant checks * for identical keys. Also, no validations are done in this method - * - * @param valuesItr - * @throws IOException */ public void appendValues(Iterator valuesItr) throws IOException { while(valuesItr.hasNext()) { @@ -539,12 +513,6 @@ public void appendValues(Iterator valuesItr) throws IOException { /** * Append key and its associated set of values. - * - * @param key - * @param valuesItr - * @param - * @param - * @throws IOException */ public void appendKeyValues(K key, Iterator valuesItr) throws IOException { if (valuesItr.hasNext()) { @@ -561,11 +529,6 @@ public void appendKeyValues(K key, Iterator valuesItr) throws IOExcept * one, send IFile.REPEAT_KEY as key parameter. Should not call this method with * IFile.REPEAT_KEY as the first key. It is caller's responsibility to pass non-negative * key/value lengths. Otherwise,IndexOutOfBoundsException could be thrown at runtime. - * - * - * @param key - * @param value - * @throws IOException */ public void append(DataInputBuffer key, DataInputBuffer value) throws IOException { int keyLength = key.getLength() - key.getPosition(); @@ -623,7 +586,7 @@ protected void writeKVPair(byte[] keyData, int keyPos, int keyLength, } protected void writeRLE(DataOutputStream out) throws IOException { - /** + /* * To strike a balance between 2 use cases (lots of unique KV in stream * vs lots of identical KV in stream), we start off by writing KV pair. * If subsequent KV is identical, we write RLE marker along with V_END_MARKER @@ -638,7 +601,7 @@ protected void writeRLE(DataOutputStream out) throws IOException { } protected void writeValueMarker(DataOutputStream out) throws IOException { - /** + /* * Write V_END_MARKER only in RLE scenario. This will * save space in conditions where lots of unique KV pairs are found in the * stream. @@ -706,7 +669,7 @@ public enum KeyState {NO_KEY, NEW_KEY, SAME_KEY} protected int recNo = 1; protected int originalKeyLength; protected int prevKeyLength; - byte keyBytes[] = new byte[0]; + private byte[] keyBytes = new byte[0]; protected int currentKeyLength; protected int currentValueLength; @@ -720,7 +683,6 @@ public enum KeyState {NO_KEY, NEW_KEY, SAME_KEY} * checksum bytes for the data at the end of the file. * @param codec codec * @param readsCounter Counter for records read from disk - * @throws IOException */ public Reader(FileSystem fs, Path file, CompressionCodec codec, @@ -739,7 +701,6 @@ public Reader(FileSystem fs, Path file, * bytes. * @param codec codec * @param readsCounter Counter for records read from disk - * @throws IOException */ public Reader(InputStream in, long length, CompressionCodec codec, @@ -748,7 +709,7 @@ public Reader(InputStream in, long length, int bufferSize) throws IOException { this(in, ((in != null) ? (length - HEADER.length) : length), codec, readsCounter, bytesReadCounter, readAhead, readAheadLength, - bufferSize, ((in != null) ? isCompressedFlagEnabled(in) : false)); + bufferSize, (in != null && isCompressedFlagEnabled(in))); if (in != null && bytesReadCounter != null) { bytesReadCounter.increment(IFile.HEADER.length); } @@ -762,7 +723,6 @@ public Reader(InputStream in, long length, * bytes. * @param codec codec * @param readsCounter Counter for records read from disk - * @throws IOException */ public Reader(InputStream in, long length, CompressionCodec codec, @@ -799,14 +759,6 @@ public Reader(InputStream in, long length, /** * Read entire ifile content to memory. - * - * @param buffer - * @param in - * @param compressedLength - * @param codec - * @param ifileReadAhead - * @param ifileReadAheadLength - * @throws IOException */ public static void readToMemory(byte[] buffer, InputStream in, int compressedLength, CompressionCodec codec, boolean ifileReadAhead, int ifileReadAheadLength) @@ -825,7 +777,6 @@ public static void readToMemory(byte[] buffer, InputStream in, int compressedLen compressedLength); } else { LOG.warn("Could not obtain decompressor from CodecPool"); - in = checksumIn; } } try { @@ -863,7 +814,6 @@ public static void readToMemory(byte[] buffer, InputStream in, int compressedLen * @param in the input stream containing the IFile data * @param length the amount of data to read from the input * @return the number of bytes copied - * @throws IOException */ public static long readToDisk(OutputStream out, InputStream in, long length, boolean ifileReadAhead, int ifileReadAheadLength) @@ -908,7 +858,6 @@ public long getPosition() throws IOException { * @param off offset * @param len length of buffer * @return the no. of bytes read - * @throws IOException */ private int readData(byte[] buf, int off, int len) throws IOException { int bytesRead = 0; @@ -949,7 +898,6 @@ protected void readKeyValueLength(DataInput dIn) throws IOException { * @param dIn * @return true if key length and value length were set to the next * false if end of file (EOF) marker was reached - * @throws IOException */ protected boolean positionToNextRecord(DataInput dIn) throws IOException { // Sanity check @@ -1087,9 +1035,7 @@ public void close() throws IOException { } } - public void reset(int offset) { - return; - } + public void reset(int offset) {} public void disableChecksumValidation() { checksumIn.disableChecksumValidation(); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java index 12e5735199..d22988533c 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java @@ -61,14 +61,16 @@ @InterfaceAudience.Private @InterfaceStability.Unstable @SuppressWarnings({"unchecked", "rawtypes"}) -public class TezMerger { +public final class TezMerger { private static final Logger LOG = LoggerFactory.getLogger(TezMerger.class); // Local directories - private static LocalDirAllocator lDirAlloc = + private static final LocalDirAllocator L_DIR_ALLOC = new LocalDirAllocator(TezRuntimeFrameworkConfigs.LOCAL_DIRS); + private TezMerger() {} + public static TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, SerializationContext serializationContext, @@ -110,7 +112,7 @@ TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, mergePhase); } - public static + public static TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, SerializationContext serializationContext, List segments, @@ -144,7 +146,7 @@ public static TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, readsCounter, writesCounter, bytesReadCounter, mergePhase); } - public static + public static TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, SerializationContext serializationContext, CompressionCodec codec, @@ -167,7 +169,7 @@ TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, mergePhase); } - public static + public static TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, SerializationContext serializationContext, CompressionCodec codec, @@ -189,7 +191,7 @@ TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, mergePhase); } - public static + public static TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, SerializationContext serializationContext, CompressionCodec codec, @@ -211,7 +213,7 @@ TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, mergePhase); } - public static + public static void writeFile(TezRawKeyValueIterator records, Writer writer, Progressable progressable, long recordsBeforeProgress) throws IOException, InterruptedException { @@ -228,7 +230,7 @@ void writeFile(TezRawKeyValueIterator records, Writer writer, if (((recordCtr++) % recordsBeforeProgress) == 0) { progressable.progress(); if (Thread.currentThread().isInterrupted()) { - /** + /* * Takes care DefaultSorter.mergeParts, MergeManager's merger threads, * PipelinedSorter's flush(). This is not expensive check as it is carried out every * 10000 records or so. @@ -250,7 +252,7 @@ static class KeyValueBuffer { private int position; private int length; - public KeyValueBuffer(byte buf[], int position, int length) { + KeyValueBuffer(byte[] buf, int position, int length) { reset(buf, position, length); } @@ -279,7 +281,7 @@ public static class Segment { static final byte[] EMPTY_BYTES = new byte[0]; Reader reader = null; final KeyValueBuffer key = new KeyValueBuffer(EMPTY_BYTES, 0, 0); - TezCounter mapOutputsCounter = null; + private TezCounter mapOutputsCounter; public Segment(Reader reader, TezCounter mapOutputsCounter) { this.reader = reader; @@ -358,15 +360,17 @@ void reinitReader(int offset) throws IOException { @InterfaceStability.Unstable public static class DiskSegment extends Segment { - FileSystem fs = null; - Path file = null; - boolean preserve = false; // Signifies whether the segment should be kept after a merge is complete. Checked in the close method. - CompressionCodec codec = null; - long segmentOffset = 0; - long segmentLength = -1; + private FileSystem fs; + private Path file; + + // Signifies whether the segment should be kept after a merge is complete. Checked in the close method. + private boolean preserve; + private CompressionCodec codec; + private long segmentOffset; + private long segmentLength; boolean ifileReadAhead; int ifileReadAheadLength; - int bufferSize = -1; + private int bufferSize; public DiskSegment(FileSystem fs, Path file, CompressionCodec codec, boolean ifileReadAhead, @@ -389,7 +393,7 @@ public DiskSegment(FileSystem fs, Path file, long segmentOffset, long segmentLength, CompressionCodec codec, boolean ifileReadAhead, int ifileReadAheadLength, int bufferSize, - boolean preserve) throws IOException { + boolean preserve) { this(fs, file, segmentOffset, segmentLength, codec, ifileReadAhead, ifileReadAheadLength, bufferSize, preserve, null); } @@ -397,8 +401,7 @@ public DiskSegment(FileSystem fs, Path file, public DiskSegment(FileSystem fs, Path file, long segmentOffset, long segmentLength, CompressionCodec codec, boolean ifileReadAhead, int ifileReadAheadLength, int bufferSize, - boolean preserve, TezCounter mergedMapOutputsCounter) - throws IOException { + boolean preserve, TezCounter mergedMapOutputsCounter) { super(null, mergedMapOutputsCounter); this.fs = fs; this.file = file; @@ -460,7 +463,7 @@ void reinitReader(int offset) throws IOException { } @VisibleForTesting - static class MergeQueue + static class MergeQueue extends PriorityQueue implements TezRawKeyValueIterator { final Configuration conf; final FileSystem fs; @@ -471,7 +474,7 @@ static class MergeQueue static final int ifileBufferSize = TezRuntimeConfiguration.TEZ_RUNTIME_IFILE_BUFFER_SIZE_DEFAULT; static final long recordsBeforeProgress = TezRuntimeConfiguration.TEZ_RUNTIME_RECORDS_BEFORE_PROGRESS_DEFAULT; - List segments = new ArrayList(); + private List segments = new ArrayList<>(); final RawComparator comparator; @@ -537,7 +540,7 @@ public MergeQueue(Configuration conf, FileSystem fs, } // Sort segments on file-lengths - Collections.sort(segments, segmentComparator); + segments.sort(segmentComparator); } public MergeQueue(Configuration conf, FileSystem fs, @@ -566,7 +569,7 @@ public MergeQueue(Configuration conf, FileSystem fs, this.reporter = reporter; this.considerFinalMergeForProgress = considerFinalMergeForProgress; if (sortSegments) { - Collections.sort(segments, segmentComparator); + segments.sort(segmentComparator); } this.checkForSameKeys = checkForSameKeys; this.codec = codec; @@ -596,7 +599,7 @@ private void adjustPriorityQueue(Segment reader) throws IOException{ long startPos = reader.getPosition(); if (checkForSameKeys) { if (hasNext == null) { - /** + /* * hasNext can be null during first iteration & prevKey is initialized here. * In cases of NO_KEY/NEW_KEY, we readjust the queue later. If new segment/file is found * during this process, we need to compare keys for RLE across segment boundaries. @@ -606,7 +609,7 @@ private void adjustPriorityQueue(Segment reader) throws IOException{ } else { //indicates a key has been read already if (hasNext != KeyState.SAME_KEY) { - /** + /* * Store previous key before reading next for later key comparisons. * If all keys in a segment are unique, it would always hit this code path and key copies * are wasteful in such condition, as these comparisons are mainly done for RLE. @@ -635,9 +638,6 @@ private void adjustPriorityQueue(Segment reader) throws IOException{ /** * Check if the previous key is same as the next top segment's key. * This would be useful to compute whether same key is spread across multiple segments. - * - * @param current - * @throws IOException */ void compareKeyWithNextTopKey(Segment current) throws IOException { Segment nextTop = top(); @@ -759,7 +759,7 @@ TezRawKeyValueIterator merge(SerializationContext serializationContext, factor += inMem; } List segmentsToMerge = - new ArrayList(); + new ArrayList<>(); int segmentsConsidered = 0; int numSegmentsToConsider = factor; long startBytes = 0; // starting bytes of segments of this merge @@ -816,8 +816,8 @@ TezRawKeyValueIterator merge(SerializationContext serializationContext, // the 3rd phase of reduce task. totalBytesProcessed = 0; totalBytes = 0; - for (int i = 0; i < segmentsToMerge.size(); i++) { - totalBytes += segmentsToMerge.get(i).getLength(); + for (Segment segment : segmentsToMerge) { + totalBytes += segment.getLength(); } } if (totalBytes != 0) //being paranoid @@ -860,7 +860,7 @@ TezRawKeyValueIterator merge(SerializationContext serializationContext, Path tmpFilename = new Path(tmpDir, "intermediate").suffix("." + passNo); - Path outputFile = lDirAlloc.getLocalPathForWrite( + Path outputFile = L_DIR_ALLOC.getLocalPathForWrite( tmpFilename.toString(), approxOutputSize, conf); @@ -933,7 +933,7 @@ private static int getPassFactor(int factor, int passNo, int numSegments) { */ private List getSegmentDescriptors(int numDescriptors) { if (numDescriptors > segments.size()) { - List subList = new ArrayList(segments); + List subList = new ArrayList<>(segments); segments.clear(); return subList; } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/Utils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/Utils.java index c1b44a29d3..768ac6e8c1 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/Utils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/Utils.java @@ -22,7 +22,9 @@ import org.apache.tez.dag.api.EdgeProperty; @Private -class Utils { +final class Utils { + + private Utils() {} /** * Modify the EdgeProperty to set the history text if available diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/BufferUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/BufferUtils.java index a1685edcfa..9f31a09fa2 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/BufferUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/BufferUtils.java @@ -25,7 +25,10 @@ import org.apache.hadoop.io.DataOutputBuffer; @Private -public class BufferUtils { +public final class BufferUtils { + + private BufferUtils() {} + public static int compare(DataInputBuffer buf1, DataInputBuffer buf2) { byte[] b1 = buf1.getData(); byte[] b2 = buf2.getData(); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/FastByteComparisons.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/FastByteComparisons.java index 4bd2552688..3743c2a2ff 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/FastByteComparisons.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/FastByteComparisons.java @@ -1,6 +1,6 @@ package org.apache.tez.runtime.library.utils; -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -37,6 +37,8 @@ */ final class FastByteComparisons { + private FastByteComparisons() {} + /** * Lexicographically compare two byte arrays. */ @@ -48,8 +50,8 @@ public static int compareTo(byte[] b1, int s1, int l1, byte[] b2, int s2, private interface Comparer { - abstract public int compareTo(T buffer1, int offset1, int length1, - T buffer2, int offset2, int length2); + int compareTo(T buffer1, int offset1, int length1, + T buffer2, int offset2, int length2); } private static Comparer lexicographicalComparerJavaImpl() { @@ -131,12 +133,10 @@ public Object run() { Field f = Unsafe.class.getDeclaredField("theUnsafe"); f.setAccessible(true); return f.get(null); - } catch (NoSuchFieldException e) { + } catch (NoSuchFieldException | IllegalAccessException e) { // It doesn't matter what we throw; // it's swallowed in getBestComparer(). throw new Error(); - } catch (IllegalAccessException e) { - throw new Error(); } } }); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestTezMerger.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestTezMerger.java index c6574d74f4..6512853c59 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestTezMerger.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestTezMerger.java @@ -59,6 +59,7 @@ import java.util.Map; import java.util.Random; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; @@ -66,24 +67,24 @@ public class TestTezMerger { private static final Logger LOG = LoggerFactory.getLogger(TestTezMerger.class); - private static Configuration defaultConf = new Configuration(); - private static FileSystem localFs = null; - private static Path workDir = null; - private static RawComparator comparator = null; - private static Random rnd = new Random(); + private static final Configuration DEFAULT_CONF = new Configuration(); + private static FileSystem localFs; + private static Path workDir; + private static RawComparator comparator; + private static final Random RND = new Random(); private static final String SAME_KEY = "SAME_KEY"; private static final String DIFF_KEY = "DIFF_KEY"; //store the generated data for final verification - private static ListMultimap verificationDataSet = LinkedListMultimap.create(); + private static final ListMultimap VERIFICATION_DATA_SET = LinkedListMultimap.create(); - private MergeManager merger = mock(MergeManager.class); + private final MergeManager merger = mock(MergeManager.class); static { - defaultConf.set("fs.defaultFS", "file:///"); + DEFAULT_CONF.set("fs.defaultFS", "file:///"); try { - localFs = FileSystem.getLocal(defaultConf); + localFs = FileSystem.getLocal(DEFAULT_CONF); workDir = new Path( new Path(System.getProperty("test.build.data", "/tmp")), TestTezMerger.class.getName()) .makeQualified(localFs.getUri(), localFs.getWorkingDirectory()); @@ -91,12 +92,12 @@ public class TestTezMerger { } catch (IOException e) { throw new RuntimeException(e); } - defaultConf.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, IntWritable.class.getName()); - defaultConf.set(TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS, LongWritable.class.getName()); + DEFAULT_CONF.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, IntWritable.class.getName()); + DEFAULT_CONF.set(TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS, LongWritable.class.getName()); Path baseDir = new Path(workDir, TestMergeManager.class.getName()); String localDirs = baseDir.toString(); - defaultConf.setStrings(TezRuntimeFrameworkConfigs.LOCAL_DIRS, localDirs); - comparator = ConfigUtils.getIntermediateInputKeyComparator(defaultConf); + DEFAULT_CONF.setStrings(TezRuntimeFrameworkConfigs.LOCAL_DIRS, localDirs); + comparator = ConfigUtils.getIntermediateInputKeyComparator(DEFAULT_CONF); } @AfterClass @@ -106,7 +107,7 @@ public static void cleanup() throws Exception { @Test(timeout = 80000) public void testMerge() throws Exception { - /** + /* * test with number of files, keys per file and mergefactor */ @@ -128,11 +129,10 @@ public void testMerge() throws Exception { merge(5, 1000, 100); //Create random mix of files (empty files + files with keys) - List pathList = new LinkedList(); - pathList.clear(); - pathList.addAll(createIFiles(Math.max(2, rnd.nextInt(20)), 0)); - pathList.addAll(createIFiles(Math.max(2, rnd.nextInt(20)), Math.max(2, rnd.nextInt(10)))); - merge(pathList, Math.max(2, rnd.nextInt(10))); + List pathList = new LinkedList<>(); + pathList.addAll(createIFiles(Math.max(2, RND.nextInt(20)), 0)); + pathList.addAll(createIFiles(Math.max(2, RND.nextInt(20)), Math.max(2, RND.nextInt(10)))); + merge(pathList, Math.max(2, RND.nextInt(10))); } private Path createIFileWithTextData(List data) throws IOException { @@ -150,10 +150,6 @@ private Path createIFileWithTextData(List data) throws IOException { /** * Verify if the records are as per the expected data set - * - * @param records - * @param expectedResult - * @throws IOException */ private void verify(TezRawKeyValueIterator records, String[][] expectedResult) throws IOException { @@ -168,7 +164,7 @@ private void verify(TezRawKeyValueIterator records, String[][] expectedResult) Text v = new Text(); v.readFields(value); - assertTrue(k.toString().equals(expectedResult[i][0])); + assertEquals(k.toString(), expectedResult[i][0]); String correctResult = expectedResult[i][1]; @@ -186,7 +182,7 @@ private void verify(TezRawKeyValueIterator records, String[][] expectedResult) @Test(timeout = 5000) public void testWithCustomComparator_WithEmptyStrings() throws Exception { - List pathList = new LinkedList(); + List pathList = new LinkedList<>(); List data = Lists.newLinkedList(); //Merge datasets with custom comparator RawComparator rc = new CustomComparator(); @@ -236,7 +232,7 @@ public void testWithCustomComparator_WithEmptyStrings() throws Exception { @Test(timeout = 5000) public void testWithCustomComparator_No_RLE() throws Exception { - List pathList = new LinkedList(); + List pathList = new LinkedList<>(); List data = Lists.newLinkedList(); //Merge datasets with custom comparator RawComparator rc = new CustomComparator(); @@ -285,14 +281,13 @@ public void testWithCustomComparator_No_RLE() throws Exception { @Test(timeout = 5000) public void testWithCustomComparator_RLE_acrossFiles() throws Exception { - List pathList = new LinkedList(); + List pathList = new LinkedList<>(); List data = Lists.newLinkedList(); LOG.info("Test with custom comparator with RLE spanning across segment boundaries"); //Test with 2 files, where the RLE keys can span across files //First file - data.clear(); data.add("0"); data.add("0"); pathList.add(createIFileWithTextData(data)); @@ -325,14 +320,13 @@ public void testWithCustomComparator_RLE_acrossFiles() throws Exception { @Test(timeout = 5000) public void testWithCustomComparator_mixedFiles() throws Exception { - List pathList = new LinkedList(); + List pathList = new LinkedList<>(); List data = Lists.newLinkedList(); LOG.info("Test with custom comparator with mixed set of segments (empty, non-empty etc)"); //Test with 2 files, where the RLE keys can span across files //First file - data.clear(); data.add("0"); pathList.add(createIFileWithTextData(data)); @@ -374,7 +368,7 @@ public void testWithCustomComparator_mixedFiles() throws Exception { @Test(timeout = 5000) public void testWithCustomComparator_RLE() throws Exception { - List pathList = new LinkedList(); + List pathList = new LinkedList<>(); List data = Lists.newLinkedList(); LOG.info("Test with custom comparator 2 files one containing RLE and also other segment " @@ -382,7 +376,6 @@ public void testWithCustomComparator_RLE() throws Exception { //Test with 2 files, same keys in middle of file //First file - data.clear(); data.add("1"); data.add("2"); data.add("2"); @@ -413,7 +406,7 @@ public void testWithCustomComparator_RLE() throws Exception { @Test(timeout = 5000) public void testWithCustomComparator_RLE2() throws Exception { - List pathList = new LinkedList(); + List pathList = new LinkedList<>(); List data = Lists.newLinkedList(); LOG.info( @@ -421,7 +414,6 @@ public void testWithCustomComparator_RLE2() throws Exception { //Test with 3 files, same keys in middle of file //First file - data.clear(); data.add("0"); data.add("1"); data.add("1"); @@ -462,7 +454,7 @@ public void testWithCustomComparator_RLE2() throws Exception { @Test(timeout = 5000) public void testWithCustomComparator() throws Exception { - List pathList = new LinkedList(); + List pathList = new LinkedList<>(); List data = Lists.newLinkedList(); LOG.info( @@ -470,7 +462,6 @@ public void testWithCustomComparator() throws Exception { //Test with 3 files //First file - data.clear(); data.add("0"); pathList.add(createIFileWithTextData(data)); @@ -500,14 +491,13 @@ public void testWithCustomComparator() throws Exception { @Test(timeout = 5000) public void testWithCustomComparator_RLE3() throws Exception { - List pathList = new LinkedList(); + List pathList = new LinkedList<>(); List data = Lists.newLinkedList(); LOG.info("Test with custom comparator"); //Test with 3 files, same keys in middle of file //First file - data.clear(); data.add("0"); pathList.add(createIFileWithTextData(data)); @@ -535,7 +525,7 @@ public void testWithCustomComparator_RLE3() throws Exception { @Test(timeout = 5000) public void testWithCustomComparator_allEmptyFiles() throws Exception { - List pathList = new LinkedList(); + List pathList = new LinkedList<>(); List data = Lists.newLinkedList(); LOG.info("Test with custom comparator where all files are empty"); @@ -561,16 +551,10 @@ public void testWithCustomComparator_allEmptyFiles() throws Exception { /** * Merge the data sets - * - * @param pathList - * @param rc - * @return - * @throws IOException */ private TezRawKeyValueIterator merge(List pathList, RawComparator rc) throws IOException, InterruptedException { - TezMerger merger = new TezMerger(); - TezRawKeyValueIterator records = merger.merge(defaultConf, localFs, + TezRawKeyValueIterator records = TezMerger.merge(DEFAULT_CONF, localFs, new SerializationContext(IntWritable.class, LongWritable.class, new WritableSerialization(), new WritableSerialization()), null, false, 0, 1024, pathList.toArray(new Path[pathList.size()]), true, 4, @@ -604,8 +588,7 @@ private void merge(int fileCount, int keysPerFile, int mergeFactor) throws Excep private void merge(List pathList, int mergeFactor, RawComparator rc) throws Exception { //Merge datasets - TezMerger merger = new TezMerger(); - TezRawKeyValueIterator records = merger.merge(defaultConf, localFs, + TezRawKeyValueIterator records = TezMerger.merge(DEFAULT_CONF, localFs, new SerializationContext(IntWritable.class, LongWritable.class, new WritableSerialization(), new WritableSerialization()), null, false, 0, 1024, pathList.toArray(new Path[pathList.size()]), true, mergeFactor, @@ -613,7 +596,7 @@ private void merge(List pathList, int mergeFactor, RawComparator rc) throw new Reporter(), null, null, null, new Progress()); verifyData(records); - verificationDataSet.clear(); + VERIFICATION_DATA_SET.clear(); } private void verifyData(TezRawKeyValueIterator records) throws IOException { @@ -632,9 +615,9 @@ private void verifyData(TezRawKeyValueIterator records) throws IOException { if (records.isSameKey()) { LOG.info("\tSame Key : key=" + k.get() + ", val=" + v.get()); //More than one key should be present in the source data - assertTrue(verificationDataSet.get(k.get()).size() > 1); + assertTrue(VERIFICATION_DATA_SET.get(k.get()).size() > 1); //Ensure this is same as the previous key we saw - assertTrue("previousKey=" + pk + ", current=" + k.get(), pk == k.get()); + assertEquals("previousKey=" + pk + ", current=" + k.get(), pk, k.get()); } else { LOG.info("key=" + k.get() + ", val=" + v.get()); } @@ -645,21 +628,20 @@ private void verifyData(TezRawKeyValueIterator records) throws IOException { } //Verify if the number of distinct entries is the same in source and the test - assertTrue("dataMap=" + dataMap.keySet().size() + ", verificationSet=" + - verificationDataSet.keySet().size(), - dataMap.keySet().size() == verificationDataSet.keySet().size()); + assertEquals("dataMap=" + dataMap.keySet().size() + ", verificationSet=" + + VERIFICATION_DATA_SET.keySet().size(), dataMap.keySet().size(), VERIFICATION_DATA_SET.keySet().size()); //Verify with source data - for (Integer key : verificationDataSet.keySet()) { - assertTrue("Data size for " + key + " not matching with source; dataSize:" + dataMap - .get(key).intValue() + ", source:" + verificationDataSet.get(key).size(), - dataMap.get(key).intValue() == verificationDataSet.get(key).size()); + for (Integer key : VERIFICATION_DATA_SET.keySet()) { + assertEquals("Data size for " + key + " not matching with source; dataSize:" + dataMap + .get(key) + ", source:" + VERIFICATION_DATA_SET.get(key).size(), + (int) dataMap.get(key), VERIFICATION_DATA_SET.get(key).size()); } //Verify if every key has the same number of repeated items in the source dataset as well for (Map.Entry entry : dataMap.entrySet()) { - assertTrue(entry.getKey() + "", verificationDataSet.get(entry.getKey()).size() == entry - .getValue()); + assertEquals(entry.getKey() + "", VERIFICATION_DATA_SET.get(entry.getKey()).size(), (int) entry + .getValue()); } LOG.info("******************"); @@ -683,17 +665,17 @@ public void testMergeSegments() throws Exception { segments.addAll(createInMemorySegments(10, 100)); segments.addAll(createDiskSegments(10, 100)); mergeSegments(segments, 5, true); - verificationDataSet.clear(); + VERIFICATION_DATA_SET.clear(); segments.clear(); segments.addAll(createDiskSegments(10, 100)); mergeSegments(segments, 5, true); - verificationDataSet.clear(); + VERIFICATION_DATA_SET.clear(); segments.clear(); segments.addAll(createInMemorySegments(3, 100)); mergeSegments(segments, 5, false); - verificationDataSet.clear(); + VERIFICATION_DATA_SET.clear(); segments.clear(); } @@ -701,7 +683,7 @@ public void testMergeSegments() throws Exception { private void mergeSegments(List segmentList, int mergeFactor, boolean hasDiskSegments) throws Exception { //Merge datasets - TezMerger.MergeQueue mergeQueue = new TezMerger.MergeQueue(defaultConf, localFs, segmentList, + TezMerger.MergeQueue mergeQueue = new TezMerger.MergeQueue(DEFAULT_CONF, localFs, segmentList, comparator, new Reporter(), false, false); TezRawKeyValueIterator records = mergeQueue.merge( @@ -716,7 +698,7 @@ private void mergeSegments(List segmentList, int mergeFactor, int diskBufLen = mergeQueue.diskIFileValue.getLength(); assertTrue(diskBufLen + " disk buf length should be > 0", (hasDiskSegments == diskBufLen > 0)); - verificationDataSet.clear(); + VERIFICATION_DATA_SET.clear(); } private List createInMemorySegments(int segmentCount, int keysPerSegment) @@ -750,7 +732,7 @@ private void populateData(IntWritable intKey, LongWritable longVal, DataInputBuf longVal.write(v); key.reset(k.getData(), 0, k.getLength()); value.reset(v.getData(), 0, v.getLength()); - verificationDataSet.put(intKey.get(), longVal.get()); + VERIFICATION_DATA_SET.put(intKey.get(), longVal.get()); } private List createDiskSegments(int segmentCount, int keysPerSegment) throws @@ -780,7 +762,7 @@ static Path writeIFile(int keysPerFile, int repeatCount) throws for (Integer key : dataSet.keySet()) { for (Long value : dataSet.get(key)) { writer.append(new IntWritable(key), new LongWritable(value)); - verificationDataSet.put(key, value); + VERIFICATION_DATA_SET.put(key, value); } } writer.close(); @@ -793,7 +775,6 @@ static Path writeIFile(int keysPerFile, int repeatCount) throws * * @param keyCount approximate number of keys to be created * @param repeatCount number of times a key should be repeated - * @return */ static TreeMultimap createDataForIFile(int keyCount, int repeatCount) { TreeMultimap dataSet = TreeMultimap.create(); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java index 573b3e9a18..73a55d942f 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java @@ -35,7 +35,10 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -class OutputTestHelpers { +final class OutputTestHelpers { + + private OutputTestHelpers() {} + static OutputContext createOutputContext() throws IOException { OutputContext outputContext = mock(OutputContext.class); Configuration conf = new TezConfiguration(); @@ -47,7 +50,7 @@ static OutputContext createOutputContext() throws IOException { doReturn("destinationVertex").when(outputContext).getDestinationVertexName(); doReturn(payLoad).when(outputContext).getUserPayload(); doReturn(workingDirs).when(outputContext).getWorkDirs(); - doReturn(200 * 1024 * 1024l).when(outputContext).getTotalMemoryAvailableToTask(); + doReturn(200 * 1024 * 1024L).when(outputContext).getTotalMemoryAvailableToTask(); doReturn(counters).when(outputContext).getCounters(); doReturn(statsReporter).when(outputContext).getStatisticsReporter(); doReturn(new Configuration(false)).when(outputContext).getContainerConfiguration(); @@ -58,7 +61,7 @@ static OutputContext createOutputContext(Configuration conf, Configuration userP throws IOException { OutputContext ctx = mock(OutputContext.class); doAnswer(new Answer() { - @Override public Void answer(InvocationOnMock invocation) throws Throwable { + @Override public Void answer(InvocationOnMock invocation) { long requestedSize = (Long) invocation.getArguments()[0]; MemoryUpdateCallbackHandler callback = (MemoryUpdateCallbackHandler) invocation .getArguments()[1]; @@ -72,7 +75,7 @@ static OutputContext createOutputContext(Configuration conf, Configuration userP doReturn("destinationVertex").when(ctx).getDestinationVertexName(); doReturn("UUID").when(ctx).getUniqueIdentifier(); doReturn(new String[] { workingDir.toString() }).when(ctx).getWorkDirs(); - doReturn(200 * 1024 * 1024l).when(ctx).getTotalMemoryAvailableToTask(); + doReturn(200 * 1024 * 1024L).when(ctx).getTotalMemoryAvailableToTask(); doReturn(new TezCounters()).when(ctx).getCounters(); OutputStatisticsReporter statsReporter = mock(OutputStatisticsReporter.class); doReturn(statsReporter).when(ctx).getStatisticsReporter(); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/testutils/KVDataGen.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/testutils/KVDataGen.java index 960d371078..62fd2bf174 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/testutils/KVDataGen.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/testutils/KVDataGen.java @@ -25,10 +25,12 @@ import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Text; -public class KVDataGen { +public final class KVDataGen { static Random rnd = new Random(); + private KVDataGen() {} + public static List generateTestData(boolean repeatKeys) { return generateTestData(true, rnd.nextInt(100)); } @@ -38,7 +40,6 @@ public static List generateTestData(boolean repeatKeys) { * * @param sorted whether data should be sorted by key * @param repeatCount number of keys to be repeated - * @return */ public static List generateTestData(boolean sorted, int repeatCount) { return generateTestDataOfKeySize(sorted, 5, repeatCount); @@ -50,10 +51,9 @@ public static List generateTestData(boolean sorted, int repeatCount) { * @param sorted whether data should be sorted by key * @param keys number of keys * @param repeatCount number of keys to be repeated - * @return */ public static List generateTestDataOfKeySize(boolean sorted, int keys, int repeatCount) { - List data = new LinkedList(); + List data = new LinkedList<>(); Random rnd = new Random(); KVPair kvp = null; for (int i = 0; i < keys; i++) { @@ -81,8 +81,8 @@ public static List generateTestDataOfKeySize(boolean sorted, int keys, i } public static class KVPair { - private Text key; - private IntWritable value; + private final Text key; + private final IntWritable value; public KVPair(Text key, IntWritable value) { this.key = key; diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/testutils/RandomTextGenerator.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/testutils/RandomTextGenerator.java index c1a05d9932..d6d15720a3 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/testutils/RandomTextGenerator.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/testutils/RandomTextGenerator.java @@ -22,12 +22,14 @@ import java.util.Random; -public class RandomTextGenerator { +public final class RandomTextGenerator { static int minWordsInKey = 10; static int wordsInKeyRange = 100; static final Random random = new Random(); + private RandomTextGenerator() {} + public static Text generateSentence() { int noWordsKey = minWordsInKey + (wordsInKeyRange != 0 ? random.nextInt(wordsInKeyRange) : 0); @@ -35,10 +37,10 @@ public static Text generateSentence() { } public static Text generateSentence(int noWords) { - StringBuffer sentence = new StringBuffer(); + StringBuilder sentence = new StringBuilder(); String space = " "; for (int i = 0; i < noWords; ++i) { - sentence.append(words[random.nextInt(words.length)]); + sentence.append(WORDS[random.nextInt(WORDS.length)]); sentence.append(space); } return new Text(sentence.toString()); @@ -49,7 +51,7 @@ public static Text generateSentence(int noWords) { *

* A random list of 100 words from /usr/share/dict/words */ - private static String[] words = { + private static final String[] WORDS = { "diurnalness", "Homoiousian", "spiranthic", "tetragynian", "silverhead", "ungreat", diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/ExampleDriver.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/ExampleDriver.java index cdbdf13db5..cc22f6a4fa 100644 --- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/ExampleDriver.java +++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/ExampleDriver.java @@ -38,11 +38,13 @@ * A description of an example program based on its class and a * human-readable description. */ -public class ExampleDriver { +public final class ExampleDriver { private static final DecimalFormat formatter = new DecimalFormat("###.##%"); - public static void main(String argv[]){ + private ExampleDriver() {} + + public static void main(String[] argv){ int exitCode = -1; ProgramDriver pgd = new ProgramDriver(); try { @@ -110,9 +112,9 @@ public static void printDAGStatus(DAGClient dagClient, String[] vertexNames, DAGStatus dagStatus = dagClient.getDAGStatus( (displayDAGCounters ? opts : null)); Progress progress = dagStatus.getDAGProgress(); - double vProgressFloat = 0.0f; + double vProgressFloat; if (progress != null) { - System.out.println(""); + System.out.println(); System.out.println("DAG: State: " + dagStatus.getState() + " Progress: " diff --git a/tez-tests/src/test/java/org/apache/tez/test/SimpleTestDAG.java b/tez-tests/src/test/java/org/apache/tez/test/SimpleTestDAG.java index 90a8cf61ad..36fad117ed 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/SimpleTestDAG.java +++ b/tez-tests/src/test/java/org/apache/tez/test/SimpleTestDAG.java @@ -38,11 +38,13 @@ * v2 * */ -public class SimpleTestDAG { +public final class SimpleTestDAG { static Resource defaultResource = Resource.newInstance(100, 0); public static String TEZ_SIMPLE_DAG_NUM_TASKS = "tez.simple-test-dag.num-tasks"; public static int TEZ_SIMPLE_DAG_NUM_TASKS_DEFAULT = 2; + + private SimpleTestDAG() {} public static DAG createDAG(String name, Configuration conf) throws Exception { @@ -76,10 +78,6 @@ public static DAG createDAG(Configuration conf) throws Exception { * v4 v5 * \ / * v6 - * @param name - * @param conf - * @return - * @throws Exception */ public static DAG createDAGForVertexOrder(String name, Configuration conf) throws Exception{ UserPayload payload = UserPayload.create(null); diff --git a/tez-tests/src/test/java/org/apache/tez/test/SimpleTestDAG3Vertices.java b/tez-tests/src/test/java/org/apache/tez/test/SimpleTestDAG3Vertices.java index a48b2d696b..5873b66978 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/SimpleTestDAG3Vertices.java +++ b/tez-tests/src/test/java/org/apache/tez/test/SimpleTestDAG3Vertices.java @@ -40,11 +40,13 @@ * v3 * */ -public class SimpleTestDAG3Vertices { +public final class SimpleTestDAG3Vertices { static Resource defaultResource = Resource.newInstance(100, 0); public static String TEZ_SIMPLE_DAG_NUM_TASKS = "tez.simple-test-dag-3-vertices.num-tasks"; public static int TEZ_SIMPLE_DAG_NUM_TASKS_DEFAULT = 2; + + private SimpleTestDAG3Vertices() {} public static DAG createDAG(String name, Configuration conf) throws Exception { diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestDriver.java b/tez-tests/src/test/java/org/apache/tez/test/TestDriver.java index bf04fd5369..6da14dfee1 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestDriver.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestDriver.java @@ -20,9 +20,11 @@ import org.apache.hadoop.util.ProgramDriver; -public class TestDriver { +public final class TestDriver { - public static void main(String argv[]){ + private TestDriver() {} + + public static void main(String[] argv){ int exitCode = -1; ProgramDriver pgd = new ProgramDriver(); try { diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestFaultTolerance.java b/tez-tests/src/test/java/org/apache/tez/test/TestFaultTolerance.java index e2fc53f69a..b7ba9dd589 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestFaultTolerance.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestFaultTolerance.java @@ -30,6 +30,7 @@ import org.apache.tez.runtime.library.cartesianproduct.CartesianProductConfig; import org.apache.tez.runtime.library.cartesianproduct.CartesianProductEdgeManager; import org.apache.tez.runtime.library.cartesianproduct.CartesianProductVertexManager; +import org.apache.tez.test.dag.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,11 +52,6 @@ import org.apache.tez.dag.api.Vertex; import org.apache.tez.dag.api.client.DAGClient; import org.apache.tez.dag.api.client.DAGStatus; -import org.apache.tez.test.dag.SimpleReverseVTestDAG; -import org.apache.tez.test.dag.SimpleVTestDAG; -import org.apache.tez.test.dag.SixLevelsFailingDAG; -import org.apache.tez.test.dag.ThreeLevelsFailingDAG; -import org.apache.tez.test.dag.TwoLevelsFailingDAG; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Ignore; @@ -411,21 +407,24 @@ public void testMultiVersionInputFailureWithoutExit() throws Exception { @Test (timeout=60000) public void testTwoLevelsFailingDAGSuccess() throws Exception { Configuration testConf = new Configuration(); - DAG dag = TwoLevelsFailingDAG.createDAG("testTwoLevelsFailingDAGSuccess", testConf); + DAG dag = new FailingDagBuilder(FailingDagBuilder.Levels.TWO) + .withName("testTwoLevelsFailingDAGSuccess").withConf(testConf).build(); runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED); } @Test (timeout=60000) public void testThreeLevelsFailingDAGSuccess() throws Exception { Configuration testConf = new Configuration(); - DAG dag = ThreeLevelsFailingDAG.createDAG("testThreeLevelsFailingDAGSuccess", testConf); + DAG dag = new FailingDagBuilder(FailingDagBuilder.Levels.THREE) + .withName("testThreeLevelsFailingDAGSuccess").withConf(testConf).build(); runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED); } @Test (timeout=60000) public void testSixLevelsFailingDAGSuccess() throws Exception { Configuration testConf = new Configuration(); - DAG dag = SixLevelsFailingDAG.createDAG("testSixLevelsFailingDAGSuccess", testConf); + DAG dag = new FailingDagBuilder(FailingDagBuilder.Levels.SIX) + .withName("testSixLevelsFailingDAGSuccess").withConf(testConf).build(); runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED); } @@ -462,8 +461,9 @@ public void testThreeLevelsFailingDAG2VerticesHaveFailedAttemptsDAGSucceeds() th TestProcessor.TEZ_FAILING_PROCESSOR_VERIFY_TASK_INDEX, "l3v1"), "0"); testConf.setInt(TestProcessor.getVertexConfName( TestProcessor.TEZ_FAILING_PROCESSOR_VERIFY_VALUE, "l3v1", 0), 15); - - DAG dag = ThreeLevelsFailingDAG.createDAG("testThreeLevelsFailingDAG2VerticesHaveFailedAttemptsDAGSucceeds", testConf); + + DAG dag = new FailingDagBuilder(FailingDagBuilder.Levels.THREE) + .withName("testThreeLevelsFailingDAG2VerticesHaveFailedAttemptsDAGSucceeds").withConf(testConf).build(); runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED); } @@ -801,7 +801,8 @@ public void testRandomFailingTasks() throws Exception { Configuration testConf = new Configuration(false); testConf.setBoolean(TestProcessor.TEZ_FAILING_PROCESSOR_DO_RANDOM_FAIL, true); testConf.setFloat(TestProcessor.TEZ_FAILING_PROCESSOR_RANDOM_FAIL_PROBABILITY, 0.5f); - DAG dag = SixLevelsFailingDAG.createDAG("testRandomFailingTasks", testConf); + DAG dag = new FailingDagBuilder(FailingDagBuilder.Levels.SIX) + .withName("testRandomFailingTasks").withConf(testConf).build(); runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED); } @@ -811,7 +812,8 @@ public void testRandomFailingInputs() throws Exception { Configuration testConf = new Configuration(false); testConf.setBoolean(TestInput.TEZ_FAILING_INPUT_DO_RANDOM_FAIL, true); testConf.setFloat(TestInput.TEZ_FAILING_INPUT_RANDOM_FAIL_PROBABILITY, 0.5f); - DAG dag = SixLevelsFailingDAG.createDAG("testRandomFailingInputs", testConf); + DAG dag = new FailingDagBuilder(FailingDagBuilder.Levels.SIX) + .withName("testRandomFailingInputs").withConf(testConf).build(); runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED); } diff --git a/tez-tests/src/test/java/org/apache/tez/test/dag/FailingDagBuilder.java b/tez-tests/src/test/java/org/apache/tez/test/dag/FailingDagBuilder.java new file mode 100644 index 0000000000..69f7ba8112 --- /dev/null +++ b/tez-tests/src/test/java/org/apache/tez/test/dag/FailingDagBuilder.java @@ -0,0 +1,166 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.test.dag; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.tez.common.TezUtils; +import org.apache.tez.dag.api.*; +import org.apache.tez.test.TestInput; +import org.apache.tez.test.TestOutput; +import org.apache.tez.test.TestProcessor; + +import java.io.IOException; +import java.util.function.BiConsumer; + +/** + * A builder for a DAG with vertices divided into a maximum of 6 levels. + * Vertex name is "lv". Level/vertex numbers start at 1. + * Each vertex has failing processor and failing inputs. + * The builder can accept Tez Configuration to indicate failing patterns. + * The number of levels in the built DAG can be configured. + *

+ * DAG is shown with a diagram below. + * Each vertex has its degree of parallelism indicated in a bracket following its name. + * Each edge annotates with data movement (s = scatter/gather, b = broadcast) + *

+ * l1v1(1) l1v2(2) l1v3(3) l1v4(2) + * |s |s |s |b + * | | | | + * l2v1(1) l2v2(3) l2v3(2) l2v4(3) + * \s /s \b |s /s + * \ / \ | / + * l3v1(4) l3v2(4) + * \s /s + * \ / + * l4v1 (10) + * /s |s \s + * / | \ + * l5v1(2) l5v2(4) l5v3(1) + * \s |s /s + * \ | / + * l6v1(4) + * + */ + +public class FailingDagBuilder { + + private final static Resource DEFAULT_RESOURCE = org.apache.hadoop.yarn.api.records.Resource.newInstance(100, 0); + + private final Levels levels; + private String name; + private Configuration conf; + + public enum Levels { + TWO("TwoLevelsFailingDAG", (dag, payload) -> { + Vertex l1v1 = Vertex.create("l1v1", TestProcessor.getProcDesc(payload), 1, DEFAULT_RESOURCE); + Vertex l2v1 = Vertex.create("l2v1", TestProcessor.getProcDesc(payload), 1, DEFAULT_RESOURCE); + addVerticesAndEdgeInternal(dag, l1v1, l2v1, EdgeProperty.DataMovementType.SCATTER_GATHER, payload); + Vertex l1v2 = Vertex.create("l1v2", TestProcessor.getProcDesc(payload), 2, DEFAULT_RESOURCE); + Vertex l2v2 = Vertex.create("l2v2", TestProcessor.getProcDesc(payload), 3, DEFAULT_RESOURCE); + addVerticesAndEdgeInternal(dag, l1v2, l2v2, EdgeProperty.DataMovementType.SCATTER_GATHER, payload); + Vertex l1v3 = Vertex.create("l1v3", TestProcessor.getProcDesc(payload), 3, DEFAULT_RESOURCE); + Vertex l2v3 = Vertex.create("l2v3", TestProcessor.getProcDesc(payload), 2, DEFAULT_RESOURCE); + addVerticesAndEdgeInternal(dag, l1v3, l2v3, EdgeProperty.DataMovementType.SCATTER_GATHER, payload); + Vertex l1v4 = Vertex.create("l1v4", TestProcessor.getProcDesc(payload), 2, DEFAULT_RESOURCE); + Vertex l2v4 = Vertex.create("l2v4", TestProcessor.getProcDesc(payload), 3, DEFAULT_RESOURCE); + addVerticesAndEdgeInternal(dag, l1v4, l2v4, EdgeProperty.DataMovementType.BROADCAST, payload); + }), + THREE("ThreeLevelsFailingDAG", (dag, payload) -> { + TWO.levelAdder.accept(dag, payload); + Vertex l3v1 = Vertex.create("l3v1", TestProcessor.getProcDesc(payload), 4, DEFAULT_RESOURCE); + dag.addVertex(l3v1); + addEdge(dag, dag.getVertex("l2v1"), l3v1, EdgeProperty.DataMovementType.SCATTER_GATHER, payload); + addEdge(dag, dag.getVertex("l2v2"), l3v1, EdgeProperty.DataMovementType.SCATTER_GATHER, payload); + Vertex l3v2 = Vertex.create("l3v2", TestProcessor.getProcDesc(payload), 4, DEFAULT_RESOURCE); + dag.addVertex(l3v2); + addEdge(dag, dag.getVertex("l2v2"), l3v2, EdgeProperty.DataMovementType.BROADCAST, payload); + addEdge(dag, dag.getVertex("l2v3"), l3v2, EdgeProperty.DataMovementType.SCATTER_GATHER, payload); + addEdge(dag, dag.getVertex("l2v4"), l3v2, EdgeProperty.DataMovementType.SCATTER_GATHER, payload); + }), + SIX("SixLevelsFailingDAG", (dag, payload) -> { + THREE.levelAdder.accept(dag, payload); + Vertex l4v1 = Vertex.create("l4v1", TestProcessor.getProcDesc(payload), 10, DEFAULT_RESOURCE); + dag.addVertex(l4v1); + addEdge(dag, dag.getVertex("l3v1"), l4v1, EdgeProperty.DataMovementType.SCATTER_GATHER, payload); + addEdge(dag, dag.getVertex("l3v2"), l4v1, EdgeProperty.DataMovementType.SCATTER_GATHER, payload); + Vertex l5v1 = Vertex.create("l5v1", TestProcessor.getProcDesc(payload), 2, DEFAULT_RESOURCE); + dag.addVertex(l5v1); + addEdge(dag, l4v1, l5v1, EdgeProperty.DataMovementType.SCATTER_GATHER, payload); + Vertex l5v2 = Vertex.create("l5v2", TestProcessor.getProcDesc(payload), 4, DEFAULT_RESOURCE); + dag.addVertex(l5v2); + addEdge(dag, l4v1, l5v2, EdgeProperty.DataMovementType.SCATTER_GATHER, payload); + Vertex l5v3 = Vertex.create("l5v3", TestProcessor.getProcDesc(payload), 1, DEFAULT_RESOURCE); + dag.addVertex(l5v3); + addEdge(dag, l4v1, l5v3, EdgeProperty.DataMovementType.SCATTER_GATHER, payload); + Vertex l6v1 = Vertex.create("l6v1", TestProcessor.getProcDesc(payload), 4, DEFAULT_RESOURCE); + dag.addVertex(l6v1); + addEdge(dag, l5v1, l6v1, EdgeProperty.DataMovementType.SCATTER_GATHER, payload); + addEdge(dag, l5v2, l6v1, EdgeProperty.DataMovementType.SCATTER_GATHER, payload); + addEdge(dag, l5v3, l6v1, EdgeProperty.DataMovementType.SCATTER_GATHER, payload); + }); + + private final String defaultName; + private final BiConsumer levelAdder; + Levels(String defaultName, BiConsumer levelAdder) { + this.defaultName = defaultName; + this.levelAdder = levelAdder; + } + + private static void addVerticesAndEdgeInternal( + DAG dag, Vertex v1, Vertex v2, EdgeProperty.DataMovementType dataMovementType, UserPayload payload) { + dag.addVertex(v1).addVertex(v2); + addEdge(dag, v1, v2, dataMovementType, payload); + } + + private static void addEdge( + DAG dag, Vertex v1, Vertex v2, EdgeProperty.DataMovementType dataMovementType, UserPayload payload) { + dag.addEdge(Edge.create(v1, v2, + EdgeProperty.create(dataMovementType, + EdgeProperty.DataSourceType.PERSISTED, + EdgeProperty.SchedulingType.SEQUENTIAL, + TestOutput.getOutputDesc(payload), + TestInput.getInputDesc(payload)))); + } + } + + public FailingDagBuilder(Levels levels) { + this.levels = levels; + this.name = levels.defaultName; + } + + public FailingDagBuilder withConf(Configuration config) { + conf = config; + return this; + } + + public FailingDagBuilder withName(String dagName) { + name = dagName; + return this; + } + + public DAG build() throws IOException { + UserPayload payload = UserPayload.create(null); + if (conf != null) { + payload = TezUtils.createUserPayloadFromConf(conf); + } + + DAG dag = DAG.create(name); + + levels.levelAdder.accept(dag, payload); + + return dag; + } +} diff --git a/tez-tests/src/test/java/org/apache/tez/test/dag/MultiAttemptDAG.java b/tez-tests/src/test/java/org/apache/tez/test/dag/MultiAttemptDAG.java index cdf69e69b2..f2d02727be 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/dag/MultiAttemptDAG.java +++ b/tez-tests/src/test/java/org/apache/tez/test/dag/MultiAttemptDAG.java @@ -66,7 +66,7 @@ import java.util.List; import java.util.concurrent.atomic.AtomicInteger; -public class MultiAttemptDAG { +public final class MultiAttemptDAG { private static final Logger LOG = LoggerFactory.getLogger(MultiAttemptDAG.class); @@ -80,9 +80,11 @@ public class MultiAttemptDAG { "tez.multi-attempt-dag.use-failing-committer"; public static boolean MULTI_ATTEMPT_DAG_USE_FAILING_COMMITTER_DEFAULT = false; + private MultiAttemptDAG() {} + public static class FailOnAttemptVertexManagerPlugin extends VertexManagerPlugin { private int numSourceTasks = 0; - private AtomicInteger numCompletions = new AtomicInteger(); + private final AtomicInteger numCompletions = new AtomicInteger(); private boolean tasksScheduled = false; public FailOnAttemptVertexManagerPlugin(VertexManagerPluginContext context) { @@ -114,7 +116,7 @@ private synchronized void maybeScheduleTasks() { && !tasksScheduled) { tasksScheduled = true; String payload = new String(getContext().getUserPayload().deepCopyAsArray()); - int successAttemptId = Integer.valueOf(payload); + int successAttemptId = Integer.parseInt(payload); LOG.info("Checking whether to crash AM or schedule tasks" + ", vertex: " + getContext().getVertexName() + ", successfulAttemptID=" + successAttemptId @@ -150,7 +152,7 @@ public void onVertexManagerEventReceived(VertexManagerEvent vmEvent) { @Override public void onRootVertexInitialized(String inputName, InputDescriptor inputDescriptor, List events) { - List inputInfoEvents = new ArrayList(); + List inputInfoEvents = new ArrayList<>(); for (Event event: events) { if (event instanceof InputDataInformationEvent) { inputInfoEvents.add((InputDataInformationEvent)event); @@ -178,12 +180,12 @@ public void initialize() throws Exception { } @Override - public void setupOutput() throws Exception { + public void setupOutput() { } @Override - public void commitOutput() throws Exception { + public void commitOutput() { if (failOnCommit) { LOG.info("Committer causing AM to shutdown"); Runtime.getRuntime().halt(-1); @@ -191,7 +193,7 @@ public void commitOutput() throws Exception { } @Override - public void abortOutput(State finalState) throws Exception { + public void abortOutput(State finalState) { } @@ -212,11 +214,7 @@ public byte[] toUserPayload() { public void fromUserPayload(byte[] userPayload) { int failInt = Ints.fromByteArray(userPayload); - if (failInt == 0) { - failOnCommit = false; - } else { - failOnCommit = true; - } + failOnCommit = failInt != 0; } } } @@ -229,14 +227,13 @@ public TestRootInputInitializer(InputInitializerContext initializerContext) { @Override public List initialize() throws Exception { - List events = new ArrayList(); + List events = new ArrayList<>(); events.add(InputDataInformationEvent.createWithSerializedPayload(0, ByteBuffer.allocate(0))); return events; } @Override - public void handleInputInitializerEvent(List events) - throws Exception { + public void handleInputInitializerEvent(List events) { throw new UnsupportedOperationException("Not supported"); } } @@ -250,7 +247,7 @@ public FailingInputInitializer(InputInitializerContext initializerContext) { @Override public List initialize() throws Exception { try { - Thread.sleep(2000l); + Thread.sleep(2000L); } catch (InterruptedException e) { // Ignore } @@ -262,8 +259,7 @@ public List initialize() throws Exception { } @Override - public void handleInputInitializerEvent(List events) throws - Exception { + public void handleInputInitializerEvent(List events) { throw new UnsupportedOperationException("Not supported"); } } @@ -276,7 +272,7 @@ public NoOpInput(InputContext inputContext, int numPhysicalInputs) { @Override public List initialize() throws Exception { - getContext().requestInitialMemory(1l, new MemoryUpdateCallback() { + getContext().requestInitialMemory(1L, new MemoryUpdateCallback() { @Override public void memoryAssigned(long assignedSize) {} }); @@ -289,12 +285,12 @@ public void start() throws Exception { } @Override - public Reader getReader() throws Exception { + public Reader getReader() { return null; } @Override - public void handleEvents(List inputEvents) throws Exception { + public void handleEvents(List inputEvents) { } @@ -313,7 +309,7 @@ public NoOpOutput(OutputContext outputContext, @Override public List initialize() throws Exception { - getContext().requestInitialMemory(1l, new MemoryUpdateCallback() { + getContext().requestInitialMemory(1L, new MemoryUpdateCallback() { @Override public void memoryAssigned(long assignedSize) { } @@ -327,7 +323,7 @@ public void start() throws Exception { } @Override - public Writer getWriter() throws Exception { + public Writer getWriter() { return null; } @@ -361,13 +357,13 @@ public static DAG createDAG(String name, // Make each vertex manager fail on appropriate attempt v1.setVertexManagerPlugin(VertexManagerPluginDescriptor.create( FailOnAttemptVertexManagerPlugin.class.getName()) - .setUserPayload(UserPayload.create(ByteBuffer.wrap(new String("1").getBytes())))); + .setUserPayload(UserPayload.create(ByteBuffer.wrap("1".getBytes())))); v2.setVertexManagerPlugin(VertexManagerPluginDescriptor.create( FailOnAttemptVertexManagerPlugin.class.getName()) - .setUserPayload(UserPayload.create(ByteBuffer.wrap(new String("2").getBytes())))); + .setUserPayload(UserPayload.create(ByteBuffer.wrap("2".getBytes())))); v3.setVertexManagerPlugin(VertexManagerPluginDescriptor.create( FailOnAttemptVertexManagerPlugin.class.getName()) - .setUserPayload(UserPayload.create(ByteBuffer.wrap(new String("3").getBytes())))); + .setUserPayload(UserPayload.create(ByteBuffer.wrap("3".getBytes())))); dag.addVertex(v1).addVertex(v2).addVertex(v3); dag.addEdge(Edge.create(v1, v2, EdgeProperty.create(DataMovementType.SCATTER_GATHER, diff --git a/tez-tests/src/test/java/org/apache/tez/test/dag/SimpleReverseVTestDAG.java b/tez-tests/src/test/java/org/apache/tez/test/dag/SimpleReverseVTestDAG.java index 4c8771cdbe..29d4d0b978 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/dag/SimpleReverseVTestDAG.java +++ b/tez-tests/src/test/java/org/apache/tez/test/dag/SimpleReverseVTestDAG.java @@ -40,11 +40,13 @@ * v2 v3 * */ -public class SimpleReverseVTestDAG { +public final class SimpleReverseVTestDAG { static Resource defaultResource = Resource.newInstance(100, 0); public static String TEZ_SIMPLE_REVERSE_V_DAG_NUM_TASKS = "tez.simple-reverse-v-test-dag.num-tasks"; public static int TEZ_SIMPLE_REVERSE_V_DAG_NUM_TASKS_DEFAULT = 2; + + private SimpleReverseVTestDAG() {} public static DAG createDAG(String name, Configuration conf) throws Exception { diff --git a/tez-tests/src/test/java/org/apache/tez/test/dag/SimpleVTestDAG.java b/tez-tests/src/test/java/org/apache/tez/test/dag/SimpleVTestDAG.java index a4eb95e059..c6e815fa52 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/dag/SimpleVTestDAG.java +++ b/tez-tests/src/test/java/org/apache/tez/test/dag/SimpleVTestDAG.java @@ -40,11 +40,13 @@ * v3 * */ -public class SimpleVTestDAG { +public final class SimpleVTestDAG { static Resource defaultResource = Resource.newInstance(100, 0); public static String TEZ_SIMPLE_V_DAG_NUM_TASKS = "tez.simple-v-test-dag.num-tasks"; public static int TEZ_SIMPLE_V_DAG_NUM_TASKS_DEFAULT = 2; + + private SimpleVTestDAG() {} public static DAG createDAG(String name, Configuration conf) throws Exception { diff --git a/tez-tests/src/test/java/org/apache/tez/test/dag/SixLevelsFailingDAG.java b/tez-tests/src/test/java/org/apache/tez/test/dag/SixLevelsFailingDAG.java deleted file mode 100644 index 036bedfdb4..0000000000 --- a/tez-tests/src/test/java/org/apache/tez/test/dag/SixLevelsFailingDAG.java +++ /dev/null @@ -1,95 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tez.test.dag; -import org.apache.hadoop.conf.Configuration; -import org.apache.tez.common.TezUtils; -import org.apache.tez.dag.api.DAG; -import org.apache.tez.dag.api.Vertex; -import org.apache.tez.dag.api.EdgeProperty.DataMovementType; -import org.apache.tez.test.TestProcessor; - -/** - * A DAG with vertices divided into 6 levels. - * Vertex name is "lv". Level/vertex numbers start at 1. - * Each vertex has failing processor and failing inputs. The constructor can accept Tez Configuration to indicate failing patterns. - * - * DAG is shown with a diagram below. - * Each vertex has its degree of parallelism indicated in a bracket following its name. - * Each edge annotates with data movement (s = scatter/gather, b = broadcast) - * - * l1v1(1) l1v2(2) l1v3(3) l1v4(2) - * |s |s |s |b - * | | | | - * l2v1(1) l2v2(3) l2v3(2) l2v4(3) - * \s /s \b |s /s - * \ / \ | / - * l3v1(4) l3v2(4) - * \s /s - * \ / - * l4v1 (10) - * /s |s \s - * / | \ - * l5v1(2) l5v2(4) l5v3(1) - * \s |s /s - * \ | / - * l6v1(4) - * - */ -public class SixLevelsFailingDAG extends ThreeLevelsFailingDAG { - - protected static Vertex l4v1; - protected static Vertex l5v1, l5v2, l5v3; - protected static Vertex l6v1; - - protected static void addDAGVerticesAndEdges() { - ThreeLevelsFailingDAG.addDAGVerticesAndEdges(); - l4v1 = Vertex.create("l4v1", TestProcessor.getProcDesc(payload), 10, defaultResource); - dag.addVertex(l4v1); - addEdge(l3v1, l4v1, DataMovementType.SCATTER_GATHER); - addEdge(l3v2, l4v1, DataMovementType.SCATTER_GATHER); - l5v1 = Vertex.create("l5v1", TestProcessor.getProcDesc(payload), 2, defaultResource); - dag.addVertex(l5v1); - addEdge(l4v1, l5v1, DataMovementType.SCATTER_GATHER); - l5v2 = Vertex.create("l5v2", TestProcessor.getProcDesc(payload), 4, defaultResource); - dag.addVertex(l5v2); - addEdge(l4v1, l5v2, DataMovementType.SCATTER_GATHER); - l5v3 = Vertex.create("l5v3", TestProcessor.getProcDesc(payload), 1, defaultResource); - dag.addVertex(l5v3); - addEdge(l4v1, l5v3, DataMovementType.SCATTER_GATHER); - l6v1 = Vertex.create("l6v1", TestProcessor.getProcDesc(payload), 4, defaultResource); - dag.addVertex(l6v1); - addEdge(l5v1, l6v1, DataMovementType.SCATTER_GATHER); - addEdge(l5v2, l6v1, DataMovementType.SCATTER_GATHER); - addEdge(l5v3, l6v1, DataMovementType.SCATTER_GATHER); - } - - public static DAG createDAG(String name, - Configuration conf) throws Exception { - if (conf != null) { - payload = TezUtils.createUserPayloadFromConf(conf); - } - dag = DAG.create(name); - addDAGVerticesAndEdges(); - return dag; - } - - public static DAG createDAG(Configuration conf) throws Exception { - return createDAG("SixLevelsFailingDAG", conf); - } -} diff --git a/tez-tests/src/test/java/org/apache/tez/test/dag/ThreeLevelsFailingDAG.java b/tez-tests/src/test/java/org/apache/tez/test/dag/ThreeLevelsFailingDAG.java deleted file mode 100644 index 7f2e4f8ecd..0000000000 --- a/tez-tests/src/test/java/org/apache/tez/test/dag/ThreeLevelsFailingDAG.java +++ /dev/null @@ -1,75 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tez.test.dag; -import org.apache.hadoop.conf.Configuration; -import org.apache.tez.common.TezUtils; -import org.apache.tez.dag.api.DAG; -import org.apache.tez.dag.api.Vertex; -import org.apache.tez.dag.api.EdgeProperty.DataMovementType; -import org.apache.tez.test.TestProcessor; - -/** - * A DAG with vertices divided into 3 levels. - * Vertex name is "lv". Level/vertex numbers start at 1. - * Each vertex has failing processor and failing inputs. The constructor can accept Tez Configuration to indicate failing patterns. - * - * DAG is shown with a diagram below. - * Each vertex has its degree of parallelism indicated in a bracket following its name. - * Each edge annotates with data movement (s = scatter/gather, b = broadcast) - * - * l1v1(1) l1v2(2) l1v3(3) l1v4(2) - * |s |s |s |b - * | | | | - * l2v1(1) l2v2(3) l2v3(2) l2v4(3) - * \s /s \b |s /s - * \ / \ | / - * l3v1(4) l3v2(4) - * - */ -public class ThreeLevelsFailingDAG extends TwoLevelsFailingDAG { - - protected static Vertex l3v1, l3v2; - - protected static void addDAGVerticesAndEdges() { - TwoLevelsFailingDAG.addDAGVerticesAndEdges(); - l3v1 = Vertex.create("l3v1", TestProcessor.getProcDesc(payload), 4, defaultResource); - dag.addVertex(l3v1); - addEdge(l2v1, l3v1, DataMovementType.SCATTER_GATHER); - addEdge(l2v2, l3v1, DataMovementType.SCATTER_GATHER); - l3v2 = Vertex.create("l3v2", TestProcessor.getProcDesc(payload), 4, defaultResource); - dag.addVertex(l3v2); - addEdge(l2v2, l3v2, DataMovementType.BROADCAST); - addEdge(l2v3, l3v2, DataMovementType.SCATTER_GATHER); - addEdge(l2v4, l3v2, DataMovementType.SCATTER_GATHER); - } - - public static DAG createDAG(String name, - Configuration conf) throws Exception { - if (conf != null) { - payload = TezUtils.createUserPayloadFromConf(conf); - } - dag = DAG.create(name); - addDAGVerticesAndEdges(); - return dag; - } - - public static DAG createDAG(Configuration conf) throws Exception { - return createDAG("ThreeLevelsFailingDAG", conf); - } -} diff --git a/tez-tests/src/test/java/org/apache/tez/test/dag/TwoLevelsFailingDAG.java b/tez-tests/src/test/java/org/apache/tez/test/dag/TwoLevelsFailingDAG.java deleted file mode 100644 index 151e3855e1..0000000000 --- a/tez-tests/src/test/java/org/apache/tez/test/dag/TwoLevelsFailingDAG.java +++ /dev/null @@ -1,113 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tez.test.dag; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.tez.common.TezUtils; -import org.apache.tez.dag.api.DAG; -import org.apache.tez.dag.api.Edge; -import org.apache.tez.dag.api.EdgeProperty; -import org.apache.tez.dag.api.UserPayload; -import org.apache.tez.dag.api.Vertex; -import org.apache.tez.dag.api.EdgeProperty.DataMovementType; -import org.apache.tez.dag.api.EdgeProperty.DataSourceType; -import org.apache.tez.dag.api.EdgeProperty.SchedulingType; -import org.apache.tez.test.TestInput; -import org.apache.tez.test.TestOutput; -import org.apache.tez.test.TestProcessor; - -/** - * A DAG with vertices divided into 2 levels. - * Vertex name is "lv". Level/vertex numbers start at 1. - * Each vertex has failing processor and failing inputs. The constructor can accept Tez Configuration to indicate failing patterns. - * - * DAG is shown with a diagram below. - * Each vertex has its degree of parallelism indicated in a bracket following its name. - * Each edge annotates with data movement (s = scatter/gather, b = broadcast) - * - * l1v1(1) l1v2(2) l1v3(3) l1v4(2) - * |s |s |s |b - * | | | | - * l2v1(1) l2v2(3) l2v3(2) l2v4(3) - * - */ -public class TwoLevelsFailingDAG { - static Resource defaultResource = Resource.newInstance(100, 0); - protected static DAG dag; - protected static UserPayload payload = UserPayload.create(null); - protected static Vertex l1v1, l1v2, l1v3, l1v4; - protected static Vertex l2v1, l2v2, l2v3, l2v4; - - public static DAG createDAG(String name, - Configuration conf) throws Exception { - if (conf != null) { - payload = TezUtils.createUserPayloadFromConf(conf); - } - dag = DAG.create(name); - addDAGVerticesAndEdges(); - return dag; - } - - protected static void addDAGVerticesAndEdges() { - l1v1 = Vertex.create("l1v1", TestProcessor.getProcDesc(payload), 1, defaultResource); - l2v1 = Vertex.create("l2v1", TestProcessor.getProcDesc(payload), 1, defaultResource); - addVerticesAndEdgeInternal(l1v1, l2v1, DataMovementType.SCATTER_GATHER); - l1v2 = Vertex.create("l1v2", TestProcessor.getProcDesc(payload), 2, defaultResource); - l2v2 = Vertex.create("l2v2", TestProcessor.getProcDesc(payload), 3, defaultResource); - addVerticesAndEdgeInternal(l1v2, l2v2, DataMovementType.SCATTER_GATHER); - l1v3 = Vertex.create("l1v3", TestProcessor.getProcDesc(payload), 3, defaultResource); - l2v3 = Vertex.create("l2v3", TestProcessor.getProcDesc(payload), 2, defaultResource); - addVerticesAndEdgeInternal(l1v3, l2v3, DataMovementType.SCATTER_GATHER); - l1v4 = Vertex.create("l1v4", TestProcessor.getProcDesc(payload), 2, defaultResource); - l2v4 = Vertex.create("l2v4", TestProcessor.getProcDesc(payload), 3, defaultResource); - addVerticesAndEdgeInternal(l1v4, l2v4, DataMovementType.BROADCAST); - } - - /** - * Adds 2 vertices and an edge connecting them. - * Given two vertices must not exist. - * - * @param v1 vertice 1 - * @param v2 vertice 2 - * @param dataMovementType Data movement type - */ - protected static void addVerticesAndEdgeInternal(Vertex v1, Vertex v2, DataMovementType dataMovementType) { - dag.addVertex(v1).addVertex(v2); - addEdge(v1, v2, dataMovementType); - } - - /** - * Adds an edge to given 2 vertices. - * @param v1 vertice 1 - * @param v2 vertice 2 - * @param dataMovementType Data movement type - */ - protected static void addEdge(Vertex v1, Vertex v2, DataMovementType dataMovementType) { - dag.addEdge(Edge.create(v1, v2, - EdgeProperty.create(dataMovementType, - DataSourceType.PERSISTED, - SchedulingType.SEQUENTIAL, - TestOutput.getOutputDesc(payload), - TestInput.getInputDesc(payload)))); - } - - public static DAG createDAG(Configuration conf) throws Exception { - return createDAG("TwoLevelsFailingDAG", conf); - } -} diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/AnalyzerDriver.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/AnalyzerDriver.java index 294527cd3e..9eda46294e 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/AnalyzerDriver.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/AnalyzerDriver.java @@ -20,7 +20,9 @@ import org.apache.hadoop.util.ProgramDriver; -public class AnalyzerDriver { +public final class AnalyzerDriver { + + private AnalyzerDriver() {} public static void main(String argv[]){ int exitCode = -1; diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/utils/Utils.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/utils/Utils.java index 8bcf2650f6..3b3e639461 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/utils/Utils.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/utils/Utils.java @@ -30,10 +30,12 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; -public class Utils { +public final class Utils { private static Pattern sanitizeLabelPattern = Pattern.compile("[:\\-\\W]+"); + private Utils() {} + public static String getShortClassName(String className) { int pos = className.lastIndexOf("."); if (pos != -1 && pos < className.length() - 1) { diff --git a/tez-tools/tez-javadoc-tools/src/main/java/org/apache/tez/tools/javadoc/doclet/ConfigStandardDoclet.java b/tez-tools/tez-javadoc-tools/src/main/java/org/apache/tez/tools/javadoc/doclet/ConfigStandardDoclet.java index 6cc9ced865..0713467c73 100644 --- a/tez-tools/tez-javadoc-tools/src/main/java/org/apache/tez/tools/javadoc/doclet/ConfigStandardDoclet.java +++ b/tez-tools/tez-javadoc-tools/src/main/java/org/apache/tez/tools/javadoc/doclet/ConfigStandardDoclet.java @@ -19,7 +19,6 @@ package org.apache.tez.tools.javadoc.doclet; import java.io.IOException; -import java.util.HashMap; import java.util.Map; import org.apache.hadoop.classification.InterfaceAudience.Private; @@ -41,11 +40,13 @@ import com.sun.javadoc.RootDoc; import com.sun.tools.doclets.standard.Standard; -public class ConfigStandardDoclet { +public final class ConfigStandardDoclet { private static final String DEBUG_SWITCH = "-debug"; private static boolean debugMode = false; + private ConfigStandardDoclet() {} + public static LanguageVersion languageVersion() { return LanguageVersion.JAVA_1_5; } @@ -63,14 +64,15 @@ public static boolean start(RootDoc root) { for (String opt : opts) { if (opt.equals(DEBUG_SWITCH)) { debugMode = true; + break; } } } logMessage("Running doclet " + ConfigStandardDoclet.class.getSimpleName()); ClassDoc[] classes = root.classes(); - for (int i = 0; i < classes.length; ++i) { - processDoc(classes[i]); + for (ClassDoc aClass : classes) { + processDoc(aClass); } return true; @@ -184,11 +186,9 @@ private static void processDoc(ClassDoc doc) { ConfigurationProperty.class.getCanonicalName())) { configProperty.isValidConfigProp = true; - boolean foundType = false; for (ElementValuePair element : annotationDesc.elementValues()) { if (element.element().name().equals("type")) { configProperty.type = stripQuotes(element.value().toString()); - foundType = true; } else { logMessage("Unhandled annotation property: " + element.element().name()); } From 25fc8c488bdffca979a846ee6f019c9fbcf6ea58 Mon Sep 17 00:00:00 2001 From: Aman Raj <104416558+amanraj2520@users.noreply.github.com> Date: Mon, 28 Nov 2022 16:45:59 +0530 Subject: [PATCH 404/512] Upgraded is-my-json-valid version to 2.20.3 (#250) (Aman Raj reviewed Laszlo Bodor) --- tez-ui/src/main/webapp/package.json | 3 ++- tez-ui/src/main/webapp/yarn.lock | 11 ++++++++--- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json index 3500b8aaff..5fa135dbe7 100644 --- a/tez-ui/src/main/webapp/package.json +++ b/tez-ui/src/main/webapp/package.json @@ -70,6 +70,7 @@ "**/jsprim/json-schema": "0.4.0", "jsonpointer": "4.1.0", "cryptiles": "4.1.2", - "lodash.merge": "4.6.2" + "lodash.merge": "4.6.2", + "is-my-json-valid": "2.20.3" } } diff --git a/tez-ui/src/main/webapp/yarn.lock b/tez-ui/src/main/webapp/yarn.lock index 2df3036ac9..980e984288 100644 --- a/tez-ui/src/main/webapp/yarn.lock +++ b/tez-ui/src/main/webapp/yarn.lock @@ -2672,12 +2672,17 @@ is-integer@^1.0.4: dependencies: is-finite "^1.0.0" -is-my-json-valid@^2.12.4: - version "2.16.0" - resolved "https://registry.yarnpkg.com/is-my-json-valid/-/is-my-json-valid-2.16.0.tgz#f079dd9bfdae65ee2038aae8acbc86ab109e3693" +is-my-ip-valid@^1.0.0: + version "1.0.1" + resolved "https://registry.yarnpkg.com/is-my-ip-valid/-/is-my-ip-valid-1.0.1.tgz#f7220d1146257c98672e6fba097a9f3f2d348442" + +is-my-json-valid@2.20.3, is-my-json-valid@^2.12.4: + version "2.20.3" + resolved "https://registry.yarnpkg.com/is-my-json-valid/-/is-my-json-valid-2.20.3.tgz#7e72dfd435b7341bc4ba4caa44ccd5703a8f8e19" dependencies: generate-function "^2.0.0" generate-object-property "^1.1.0" + is-my-ip-valid "^1.0.0" jsonpointer "^4.0.0" xtend "^4.0.0" From 34d68100737b6e83a43f2a6837d3026eea5070bf Mon Sep 17 00:00:00 2001 From: Mayank Kunwar <55212670+mayankkunwar@users.noreply.github.com> Date: Fri, 2 Dec 2022 14:09:46 +0530 Subject: [PATCH 405/512] TEZ-4458:Upgrade Bouncy Castle to 1.70 due to high CVEs (#253) (Mayank Kunwar reviewed by Laszlo Bodor) --- pom.xml | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 5e03d2af85..68d5d204da 100644 --- a/pom.xml +++ b/pom.xml @@ -65,6 +65,7 @@ 1.7.36 3.21.1 0.7.45 + 1.70 ${env.PROTOC_PATH} scm:git:https://gitbox.apache.org/repos/asf/tez.git 1.8.0 @@ -758,7 +759,13 @@ org.bouncycastle bcprov-jdk15on - 1.65 + ${bouncycastle.version} + test + + + org.bouncycastle + bcpkix-jdk15on + ${bouncycastle.version} test From 4fd2db36cee9e21ab41193e216e820bacadec157 Mon Sep 17 00:00:00 2001 From: Arnout Engelen Date: Mon, 2 Jan 2023 09:49:20 +0100 Subject: [PATCH 406/512] TEZ-4464: Update some links to https (#260) (Arnout Engelen reviewed by Laszlo Bodor) --- docs/pom.xml | 16 ++++++++-------- docs/src/site/markdown/by-laws.md | 12 ++++++------ docs/src/site/markdown/index.md | 2 +- docs/src/site/markdown/install_pre_0_5_0.md | 2 +- docs/src/site/markdown/privacy-policy.md | 4 ++-- docs/src/site/markdown/talks.md | 8 ++++---- docs/src/site/markdown/tez-ui.md | 2 +- docs/src/site/markdown/tez_yarn_timeline.md | 2 +- docs/src/site/site.xml | 18 +++++++++--------- 9 files changed, 33 insertions(+), 33 deletions(-) diff --git a/docs/pom.xml b/docs/pom.xml index d82f003c41..405902fc7f 100644 --- a/docs/pom.xml +++ b/docs/pom.xml @@ -21,7 +21,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd"> Tez Tez is an application framework which allows for a complex directed-acyclic-graph of tasks for processing data and is built atop Apache Hadoop YARN. - http://tez.apache.org/ + https://tez.apache.org/ 4.0.0 @@ -35,14 +35,14 @@ Apache 2 - http://www.apache.org/licenses/LICENSE-2.0.txt + https://www.apache.org/licenses/LICENSE-2.0.txt repo Jira - http://issues.apache.org/jira/browse/TEZ + https://issues.apache.org/jira/browse/TEZ @@ -53,7 +53,7 @@ mailto:user-unsubscribe@tez.apache.org mailto:user@tez.apache.org - http://mail-archives.apache.org/mod_mbox/tez-user/ + https://mail-archives.apache.org/mod_mbox/tez-user/ Development list @@ -62,7 +62,7 @@ mailto:dev-unsubscribe@tez.apache.org mailto:dev@tez.apache.org - http://mail-archives.apache.org/mod_mbox/tez-dev/ + https://mail-archives.apache.org/mod_mbox/tez-dev/ Commit list @@ -71,7 +71,7 @@ mailto:commits-unsubscribe@tez.apache.org mailto:commits@tez.apache.org - http://mail-archives.apache.org/mod_mbox/tez-commits/ + https://mail-archives.apache.org/mod_mbox/tez-commits/ JIRA Issues list @@ -80,7 +80,7 @@ mailto:issues-unsubscribe@tez.apache.org mailto:issues@tez.apache.org - http://mail-archives.apache.org/mod_mbox/tez-issues/ + https://mail-archives.apache.org/mod_mbox/tez-issues/ @@ -440,7 +440,7 @@ Apache Software Foundation - http://www.apache.org/ + https://www.apache.org/ diff --git a/docs/src/site/markdown/by-laws.md b/docs/src/site/markdown/by-laws.md index 8037c349b8..4ea47d544b 100644 --- a/docs/src/site/markdown/by-laws.md +++ b/docs/src/site/markdown/by-laws.md @@ -124,9 +124,9 @@ This section describes the various actions which are undertaken within the proje Votes are open for a period of a minimum of 3 days (excluding weekend days) to allow all active voters time to consider the vote. For any votes requiring full consensus or a 2/3 majority, the vote should remain open for a minimum of 1 week. Votes relating to code changes are not subject to a strict timetable but should be made as timely as possible. -[Apache Software Foundation]: http://www.apache.org/foundation/ -[Incubator project]: http://incubator.apache.org/ -[Foundation FAQ]: http://www.apache.org/foundation/faq.html -[Committer FAQ]: http://www.apache.org/dev/committers.html -[CLA]: http://www.apache.org/licenses/icla.txt -[set of roles]: http://www.apache.org/foundation/how-it-works.html#roles +[Apache Software Foundation]: https://www.apache.org/foundation/ +[Incubator project]: https://incubator.apache.org/ +[Foundation FAQ]: https://www.apache.org/foundation/faq.html +[Committer FAQ]: https://www.apache.org/dev/committers.html +[CLA]: https://www.apache.org/licenses/icla.txt +[set of roles]: https://www.apache.org/foundation/how-it-works.html#roles diff --git a/docs/src/site/markdown/index.md b/docs/src/site/markdown/index.md index d41a98b12b..fc47f533a1 100644 --- a/docs/src/site/markdown/index.md +++ b/docs/src/site/markdown/index.md @@ -23,7 +23,7 @@ Introduction The Apache TEZ® project is aimed at building an application framework which allows for a complex directed-acyclic-graph of tasks for processing data. It is currently built atop -[Apache Hadoop YARN](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html). +[Apache Hadoop YARN](https://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html). The 2 main design themes for Tez are: diff --git a/docs/src/site/markdown/install_pre_0_5_0.md b/docs/src/site/markdown/install_pre_0_5_0.md index 494ff54270..5123c14014 100644 --- a/docs/src/site/markdown/install_pre_0_5_0.md +++ b/docs/src/site/markdown/install_pre_0_5_0.md @@ -20,7 +20,7 @@ [Install instructions for Tez (post 0.5.0)](./install.html) ----------------------------------------------------------------------------------- -Install/Deploy Instructions for Tez release pre-0.5.0 E.g. [(Tez-0.4.1)](http://archive.apache.org/dist/incubator/tez/tez-0.4.1-incubating/) +Install/Deploy Instructions for Tez release pre-0.5.0 E.g. [(Tez-0.4.1)](https://archive.apache.org/dist/incubator/tez/tez-0.4.1-incubating/) -------------------------------------------------------------------------------------------------------------------------------------------------- 1. Deploy Apache Hadoop using either the 2.2.0 release or a compatible diff --git a/docs/src/site/markdown/privacy-policy.md b/docs/src/site/markdown/privacy-policy.md index 95825d0604..dbcd29fc1b 100644 --- a/docs/src/site/markdown/privacy-policy.md +++ b/docs/src/site/markdown/privacy-policy.md @@ -35,9 +35,9 @@ the following: 5. The addresses of pages from where you followed a link to our site. Part of this information is gathered using a tracking cookie set by the -[Google Analytics](http://www.google.com/analytics/) service and handled +[Google Analytics](https://www.google.com/analytics/) service and handled by Google as described in their [privacy -policy](http://www.google.com/privacy.html). See your browser +policy](https://www.google.com/privacy.html). See your browser documentation for instructions on how to disable the cookie if you prefer not to share this data with Google. diff --git a/docs/src/site/markdown/talks.md b/docs/src/site/markdown/talks.md index a29d553dfc..ef4e7c2960 100644 --- a/docs/src/site/markdown/talks.md +++ b/docs/src/site/markdown/talks.md @@ -21,12 +21,12 @@ Talks ----- - Apache Tez : Accelerating Hadoop Query Processing by Bikas Saha and Hitesh Shah at [Hadoop Summit 2014, San Jose, CA, USA](http://hadoopsummit.org/san-jose/) - - [Slides](http://www.slideshare.net/Hadoop_Summit/w-1205phall1saha) - - [Video](http://www.youtube.com/watch?v=yf_hBiZy3nk) + - [Slides](https://www.slideshare.net/Hadoop_Summit/w-1205phall1saha) + - [Video](https://www.youtube.com/watch?v=yf_hBiZy3nk) User Meetup Recordings ---------------------- - [Recording](https://hortonworks.webex.com/hortonworks/ldr.php?AT=pb&SP=MC&rID=125516477&rKey=d147a3c924b64496) - from [Meetup on July 31st, 2013](http://www.meetup.com/Apache-Tez-User-Group/events/130852782/) - at [Hortonworks Inc](http://hortonworks.com) + from [Meetup on July 31st, 2013](https://www.meetup.com/Apache-Tez-User-Group/events/130852782/) + at [Hortonworks Inc](https://hortonworks.com) diff --git a/docs/src/site/markdown/tez-ui.md b/docs/src/site/markdown/tez-ui.md index 3855fbbb97..423018fdf5 100644 --- a/docs/src/site/markdown/tez-ui.md +++ b/docs/src/site/markdown/tez-ui.md @@ -128,7 +128,7 @@ yarn-site.xml ... ``` -__For more detailed information (setup, configuration, deployment), please refer to the [Apache Hadoop Documentation on the Application Timeline Server](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/TimelineServer.html)__ +__For more detailed information (setup, configuration, deployment), please refer to the [Apache Hadoop Documentation on the Application Timeline Server](https://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/TimelineServer.html)__ __For general information on the compatibility matrix of the Tez UI with YARN TimelineServer, please refer to the [Tez - Timeline Server Guide](tez_yarn_timeline.html)__ diff --git a/docs/src/site/markdown/tez_yarn_timeline.md b/docs/src/site/markdown/tez_yarn_timeline.md index 745f65c044..cbe28b4966 100644 --- a/docs/src/site/markdown/tez_yarn_timeline.md +++ b/docs/src/site/markdown/tez_yarn_timeline.md @@ -18,7 +18,7 @@ ## YARN Timeline Background -Initial support for [YARN Timeline](http://hadoop.apache.org/docs/r2.4.0/hadoop-yarn/hadoop-yarn-site/TimelineServer.html) was introduced in Apache Hadoop 2.4.0. Support for ACLs in Timeline was introduced in Apache Hadoop 2.6.0. Support for Timeline was introduced in Tez in 0.5.x ( with some experimental support in 0.4.x ). However, Tez ACLs integration with Timeline is only available from Tez 0.6.0 onwards. +Initial support for [YARN Timeline](https://hadoop.apache.org/docs/r2.4.0/hadoop-yarn/hadoop-yarn-site/TimelineServer.html) was introduced in Apache Hadoop 2.4.0. Support for ACLs in Timeline was introduced in Apache Hadoop 2.6.0. Support for Timeline was introduced in Tez in 0.5.x ( with some experimental support in 0.4.x ). However, Tez ACLs integration with Timeline is only available from Tez 0.6.0 onwards. ## How Tez Uses YARN Timeline diff --git a/docs/src/site/site.xml b/docs/src/site/site.xml index 5964cbc877..84fba95304 100644 --- a/docs/src/site/site.xml +++ b/docs/src/site/site.xml @@ -42,7 +42,7 @@ ./images/ApacheTezLogo_lowres.png 25% 25% - http://tez.apache.org/ + https://tez.apache.org/ @@ -94,13 +94,13 @@ - +

- + @@ -116,12 +116,12 @@ - + - + @@ -142,10 +142,10 @@ - - - - + + + +