Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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,
Expand Down Expand Up @@ -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) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;
Expand All @@ -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;

Expand Down Expand Up @@ -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<Event> eventListWithDme = new LinkedList<>();
eventListWithDme.add(dataMovementEvent);
eventListWithDme.add(compositeRoutedDataMovementEvent);

try (MockedStatic<ShuffleUtils> 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<Void>) 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 {
Expand Down Expand Up @@ -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;
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,13 @@
# Licensed under the Apache License, Version 2.0 (the "License");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

does this extra file have any advantages compared to adding this below as a dependency:

<dependency>
    <groupId>org.mockito</groupId>
    <artifactId>mockito-inline</artifactId>
    <version>xxx</version>
    <scope>test</scope>
</dependency>

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@abstractdog Both options work, the difference is just the extra file versus extra dependency. As per this doc, mockito-inline artifact likely to be discontinued once mocking of final classes and methods gets integrated into the default mock maker. Please let me know if adding mockito-inline dependency looks better.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

thanks for clarifying, I'm fine with this file then

# 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