Skip to content

Commit

Permalink
Merge 8d786f3 into fb6dffe
Browse files Browse the repository at this point in the history
  • Loading branch information
ajantha-bhat committed Aug 29, 2018
2 parents fb6dffe + 8d786f3 commit ef546e5
Show file tree
Hide file tree
Showing 2 changed files with 16 additions and 33 deletions.
Expand Up @@ -23,6 +23,7 @@
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.atomic.AtomicLong;

import org.apache.carbondata.core.constants.CarbonCommonConstants;
import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
Expand Down Expand Up @@ -460,27 +461,29 @@ public static class CarbonMultiRecordWriter extends CarbonRecordWriter {

private CarbonOutputIteratorWrapper[] iterators;

private int counter;
private AtomicLong counter;

CarbonMultiRecordWriter(CarbonOutputIteratorWrapper[] iterators,
DataLoadExecutor dataLoadExecutor, CarbonLoadModel loadModel, Future future,
ExecutorService executorService) {
super(null, dataLoadExecutor, loadModel, future, executorService);
this.iterators = iterators;
counter = new AtomicLong(0);
}

@Override public synchronized void write(NullWritable aVoid, ObjectArrayWritable objects)
@Override public void write(NullWritable aVoid, ObjectArrayWritable objects)
throws InterruptedException {
iterators[counter].write(objects.get());
if (++counter == iterators.length) {
//round robin reset
counter = 0;
int hash = (int) (counter.incrementAndGet() % iterators.length);
synchronized (iterators[hash]) {
iterators[hash].write(objects.get());
}
}

@Override public void close(TaskAttemptContext taskAttemptContext) throws InterruptedException {
for (CarbonOutputIteratorWrapper itr : iterators) {
itr.closeWriter(false);
for (int i = 0; i < iterators.length; i++) {
synchronized (iterators[i]) {
iterators[i].closeWriter(false);
}
}
super.close(taskAttemptContext);
}
Expand Down
Expand Up @@ -19,7 +19,6 @@
import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
Expand Down Expand Up @@ -64,10 +63,13 @@ public class InputProcessorStepWithNoConverterImpl extends AbstractDataLoadProce

private Map<Integer, GenericDataType> dataFieldsWithComplexDataType;

private short sdkUserCore;

public InputProcessorStepWithNoConverterImpl(CarbonDataLoadConfiguration configuration,
CarbonIterator<Object[]>[] inputIterators) {
super(configuration, null);
this.inputIterators = inputIterators;
sdkUserCore = configuration.getWritingCoresCount();
}

@Override public DataField[] getOutput() {
Expand Down Expand Up @@ -133,7 +135,8 @@ private int[] arrangeData(DataField[] dataFields, String[] header) {

@Override public Iterator<CarbonRowBatch>[] execute() {
int batchSize = CarbonProperties.getInstance().getBatchSize();
List<CarbonIterator<Object[]>>[] readerIterators = partitionInputReaderIterators();
List<CarbonIterator<Object[]>>[] readerIterators =
CarbonDataProcessorUtil.partitionInputReaderIterators(this.inputIterators, sdkUserCore);
Iterator<CarbonRowBatch>[] outIterators = new Iterator[readerIterators.length];
for (int i = 0; i < outIterators.length; i++) {
outIterators[i] =
Expand All @@ -144,29 +147,6 @@ private int[] arrangeData(DataField[] dataFields, String[] header) {
return outIterators;
}

/**
* Partition input iterators equally as per the number of threads.
*
* @return
*/
private List<CarbonIterator<Object[]>>[] partitionInputReaderIterators() {
// Get the number of cores configured in property.
int numberOfCores = CarbonProperties.getInstance().getNumberOfCores();
// Get the minimum of number of cores and iterators size to get the number of parallel threads
// to be launched.
int parallelThreadNumber = Math.min(inputIterators.length, numberOfCores);

List<CarbonIterator<Object[]>>[] iterators = new List[parallelThreadNumber];
for (int i = 0; i < parallelThreadNumber; i++) {
iterators[i] = new ArrayList<>();
}
// Equally partition the iterators as per number of threads
for (int i = 0; i < inputIterators.length; i++) {
iterators[i % parallelThreadNumber].add(inputIterators[i]);
}
return iterators;
}

@Override public void close() {
if (!closed) {
super.close();
Expand Down

0 comments on commit ef546e5

Please sign in to comment.