Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[#1327]feat(tez): Support combine operation in sort for tez engine. #1328

Open
wants to merge 3 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
6 changes: 6 additions & 0 deletions client-tez/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -129,6 +129,12 @@
<artifactId>hadoop-minicluster</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.tez</groupId>
<artifactId>tez-mapreduce</artifactId>
<version>${tez.version}</version>
<scope>test</scope>
</dependency>
</dependencies>

<build>
Expand Down
Original file line number Diff line number Diff line change
@@ -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.runtime.library.common.sort.buffer;

import java.io.IOException;

import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.InMemoryWriter;

public class TezRssCombineOutputCollector<K, V> extends InMemoryWriter {
private WriteBuffer<K, V> writer;

public TezRssCombineOutputCollector(byte[] array) {
super(array);
}

public synchronized void setWriter(WriteBuffer<K, V> writer) {
this.writer = writer;
}

public synchronized void append(Object key, Object value) throws IOException {
writer.addRecord((K) key, (V) value);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,13 +19,17 @@

import java.io.IOException;
import java.io.OutputStream;
import java.util.Comparator;
import java.util.Arrays;
import java.util.Iterator;
import java.util.List;

import com.google.common.collect.Lists;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.RawComparator;
import org.apache.hadoop.io.WritableUtils;
import org.apache.hadoop.io.serializer.Serializer;
import org.apache.hadoop.util.Progress;
import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -89,36 +93,36 @@ public void clear() {
records.clear();
}

/** get data */
public synchronized byte[] getData() {
int extraSize = 0;
for (Record<K> record : records) {
extraSize += WritableUtils.getVIntSize(record.getKeyLength());
extraSize += WritableUtils.getVIntSize(record.getValueLength());
}
extraSize += WritableUtils.getVIntSize(-1);
extraSize += WritableUtils.getVIntSize(-1);
byte[] data = new byte[dataLength + extraSize];
int offset = 0;
public synchronized void sort() {
long startSort = System.currentTimeMillis();
if (this.isNeedSorted) {
records.sort(
new Comparator<Record<K>>() {
@Override
public int compare(Record<K> o1, Record<K> o2) {
return comparator.compare(
(o1, o2) ->
comparator.compare(
buffers.get(o1.getKeyIndex()).getBuffer(),
o1.getKeyOffSet(),
o1.getKeyLength(),
buffers.get(o2.getKeyIndex()).getBuffer(),
o2.getKeyOffSet(),
o2.getKeyLength());
}
});
o2.getKeyLength()));
}
long startCopy = System.currentTimeMillis();
sortTime += startCopy - startSort;
}

/** get data */
public synchronized byte[] getData() {
int extraSize = 0;
for (Record<K> record : records) {
extraSize += WritableUtils.getVIntSize(record.getKeyLength());
extraSize += WritableUtils.getVIntSize(record.getValueLength());
}
extraSize += WritableUtils.getVIntSize(-1);
extraSize += WritableUtils.getVIntSize(-1);
byte[] data = new byte[dataLength + extraSize];
int offset = 0;

final long startCopy = System.currentTimeMillis();
for (Record<K> record : records) {
offset = writeDataInt(data, offset, record.getKeyLength());
offset = writeDataInt(data, offset, record.getValueLength());
Expand Down Expand Up @@ -328,4 +332,96 @@ public int getSize() {
return size;
}
}

public static class BufferIterator<K, V> implements TezRawKeyValueIterator {
private final WriteBuffer<K, V> writeBuffer;
private final Iterator<Record<K>> iterator;

private final DataInputBuffer previousKeyBuffer = new DataInputBuffer();
private boolean hasPrevious = false;
private final DataInputBuffer keyBuffer = new DataInputBuffer();
private final DataInputBuffer valueBuffer = new DataInputBuffer();
private WriteBuffer.Record<K> currentRecord;

public BufferIterator(WriteBuffer<K, V> writeBuffer) {
this.writeBuffer = writeBuffer;
this.iterator = writeBuffer.records.iterator();
}

@Override
public DataInputBuffer getKey() {
WriteBuffer.WrappedBuffer keyWrappedBuffer =
writeBuffer.buffers.get(currentRecord.getKeyIndex());
byte[] rawData = keyWrappedBuffer.getBuffer();
keyBuffer.reset(rawData, currentRecord.getKeyOffSet(), currentRecord.getKeyLength());
return keyBuffer;
}

@Override
public DataInputBuffer getValue() {
WriteBuffer.WrappedBuffer valueWrappedBuffer =
writeBuffer.buffers.get(currentRecord.getKeyIndex());
byte[] rawData = valueWrappedBuffer.getBuffer();
int valueOffset = currentRecord.getKeyOffSet() + currentRecord.getKeyLength();
valueBuffer.reset(rawData, valueOffset, currentRecord.getValueLength());
return valueBuffer;
}

@Override
public boolean next() {
if (hasPrevious) {
int length =
Math.min(currentRecord.getKeyLength(), keyBuffer.getLength() - keyBuffer.getPosition());
if (length > 0) {
byte[] prevKeyData = new byte[length];
System.arraycopy(keyBuffer.getData(), keyBuffer.getPosition(), prevKeyData, 0, length);
previousKeyBuffer.reset(prevKeyData, 0, length);
}
}

if (iterator.hasNext()) {
currentRecord = iterator.next();
hasPrevious = true;
return true;
}
return false;
}

@Override
public boolean hasNext() {
return iterator.hasNext();
}

@Override
public void close() throws IOException {}

@Override
public Progress getProgress() {
return new Progress();
}

@Override
public boolean isSameKey() {
if (!hasPrevious) {
return false;
}

if (previousKeyBuffer.getLength() != keyBuffer.getLength()) {
return false;
}

byte[] prevKey = new byte[previousKeyBuffer.getLength()];
byte[] currKey = new byte[keyBuffer.getLength()];
System.arraycopy(
previousKeyBuffer.getData(),
previousKeyBuffer.getPosition(),
prevKey,
0,
previousKeyBuffer.getLength());
System.arraycopy(
keyBuffer.getData(), keyBuffer.getPosition(), currKey, 0, keyBuffer.getLength());

return Arrays.equals(prevKey, currKey);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,8 @@
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.combine.Combiner;
import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -100,6 +102,8 @@ public class WriteBufferManager<K, V> {
private final TezCounter mapOutputByteCounter;
private final TezCounter mapOutputRecordCounter;

protected final Combiner combiner;

/** WriteBufferManager */
public WriteBufferManager(
TezTaskAttemptID tezTaskAttemptID,
Expand Down Expand Up @@ -128,7 +132,8 @@ public WriteBufferManager(
int shuffleId,
boolean isNeedSorted,
TezCounter mapOutputByteCounter,
TezCounter mapOutputRecordCounter) {
TezCounter mapOutputRecordCounter,
Combiner combiner) {
this.tezTaskAttemptID = tezTaskAttemptID;
this.maxMemSize = maxMemSize;
this.appId = appId;
Expand Down Expand Up @@ -158,6 +163,8 @@ public WriteBufferManager(
this.mapOutputRecordCounter = mapOutputRecordCounter;
this.sendExecutorService =
Executors.newFixedThreadPool(sendThreadNum, ThreadUtils.getThreadFactory("send-thread"));

this.combiner = combiner;
}

/** add record */
Expand Down Expand Up @@ -240,9 +247,23 @@ public int compare(WriteBuffer<K, V> o1, WriteBuffer<K, V> o2) {
sendShuffleBlocks(shuffleBlocks);
}

private void prepareBufferForSend(List<ShuffleBlockInfo> shuffleBlocks, WriteBuffer buffer) {
private void prepareBufferForSend(
List<ShuffleBlockInfo> shuffleBlocks, WriteBuffer<K, V> buffer) {
buffers.remove(buffer.getPartitionId());
ShuffleBlockInfo block = createShuffleBlock(buffer);
buffer.sort();
ShuffleBlockInfo block;
if (combiner != null) {
try {
buffer = combineBuffer(buffer);
if (LOG.isDebugEnabled()) {
LOG.debug("Successfully finished combining.");
}
} catch (Exception e) {
Thread.currentThread().interrupt();
LOG.error("Error occurred while combining in Sort:", e);
}
}
block = createShuffleBlock(buffer);
buffer.clear();
shuffleBlocks.add(block);
allBlockIds.add(block.getBlockId());
Expand All @@ -252,6 +273,27 @@ private void prepareBufferForSend(List<ShuffleBlockInfo> shuffleBlocks, WriteBuf
partitionToBlocks.get(block.getPartitionId()).add(block.getBlockId());
}

public WriteBuffer<K, V> combineBuffer(WriteBuffer<K, V> buffer)
throws IOException, InterruptedException {
TezRawKeyValueIterator kvIterator = new WriteBuffer.BufferIterator<>(buffer);

TezRssCombineOutputCollector<K, V> combineCollector =
new TezRssCombineOutputCollector<>(new byte[] {});

WriteBuffer<K, V> newBuffer =
new WriteBuffer<>(
false,
buffer.getPartitionId(),
comparator,
maxSegmentSize,
keySerializer,
valSerializer);

combineCollector.setWriter(newBuffer);
combiner.combine(kvIterator, combineCollector);
return newBuffer;
}

private void sendShuffleBlocks(List<ShuffleBlockInfo> shuffleBlocks) {
sendExecutorService.submit(
new Runnable() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,8 @@
import org.apache.tez.common.RssTezUtils;
import org.apache.tez.dag.records.TezTaskAttemptID;
import org.apache.tez.runtime.api.OutputContext;
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.buffer.WriteBufferManager;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand All @@ -49,6 +51,8 @@ public class RssSorter extends ExternalSorter {
private Set<Long> failedBlockIds = Sets.newConcurrentHashSet();
private Map<Integer, List<ShuffleServerInfo>> partitionToServers;

protected final Combiner combiner;

private int[] numRecordsPerPartition;

/** Initialization */
Expand Down Expand Up @@ -137,6 +141,8 @@ public RssSorter(
LOG.info("bitmapSplitNum is {}", bitmapSplitNum);
}

this.combiner = TezRuntimeUtils.instantiateCombiner(this.conf, outputContext);

LOG.info("applicationAttemptId is {}", applicationAttemptId.toString());

bufferManager =
Expand Down Expand Up @@ -167,7 +173,8 @@ public RssSorter(
shuffleId,
true,
mapOutputByteCounter,
mapOutputRecordCounter);
mapOutputRecordCounter,
combiner);
LOG.info("Initialized WriteBufferManager.");
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -165,7 +165,8 @@ public RssUnSorter(
shuffleId,
false,
mapOutputByteCounter,
mapOutputRecordCounter);
mapOutputRecordCounter,
null);
LOG.info("Initialized WriteBufferManager.");
}

Expand Down