diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java index 7115b67a2..8533d7afb 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java @@ -24,8 +24,8 @@ import com.uber.hoodie.cli.TableHeader; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; -import com.uber.hoodie.common.table.log.HoodieCompactedLogRecordScanner; import com.uber.hoodie.common.table.log.HoodieLogFormat; +import com.uber.hoodie.common.table.log.HoodieMergedLogRecordScanner; import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock; import com.uber.hoodie.common.table.log.block.HoodieCorruptBlock; import com.uber.hoodie.common.table.log.block.HoodieLogBlock; @@ -187,7 +187,7 @@ public class HoodieLogFileCommand implements CommandMarker { if (shouldMerge) { System.out.println("===========================> MERGING RECORDS <==================="); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, HoodieCLI.tableMetadata.getBasePath(), logFilePaths, readerSchema, HoodieCLI.tableMetadata.getActiveTimeline().getCommitTimeline().lastInstant().get() .getTimestamp(), diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIterator.java b/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIterator.java deleted file mode 100644 index 586164084..000000000 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIterator.java +++ /dev/null @@ -1,209 +0,0 @@ -/* - * Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com) - * - * 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 com.uber.hoodie.func; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.uber.hoodie.exception.HoodieException; -import java.util.Iterator; -import java.util.Optional; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.Semaphore; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Function; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.util.SizeEstimator; - -/** - * Used for buffering input records. Buffer limit is controlled by {@link #bufferMemoryLimit}. It - * internally samples every {@link #RECORD_SAMPLING_RATE}th record and adjusts number of records in - * buffer accordingly. This is done to ensure that we don't OOM. - * - * @param input payload data type - * @param output payload data type - */ -public class BufferedIterator implements Iterator { - - // interval used for polling records in the queue. - public static final int RECORD_POLL_INTERVAL_SEC = 5; - // rate used for sampling records to determine avg record size in bytes. - public static final int RECORD_SAMPLING_RATE = 64; - // maximum records that will be cached - private static final int RECORD_CACHING_LIMIT = 128 * 1024; - private static Logger logger = LogManager.getLogger(BufferedIterator.class); - // It indicates number of records to cache. We will be using sampled record's average size to - // determine how many - // records we should cache and will change (increase/decrease) permits accordingly. - @VisibleForTesting - public final Semaphore rateLimiter = new Semaphore(1); - // used for sampling records with "RECORD_SAMPLING_RATE" frequency. - public final AtomicLong samplingRecordCounter = new AtomicLong(-1); - // internal buffer to cache buffered records. - private final LinkedBlockingQueue> buffer = new - LinkedBlockingQueue<>(); - // maximum amount of memory to be used for buffering records. - private final long bufferMemoryLimit; - // original iterator from where records are read for buffering. - private final Iterator inputIterator; - // it holds the root cause of the exception in case either buffering records (reading from - // inputIterator) fails or - // thread reading records from buffer fails. - private final AtomicReference hasFailed = new AtomicReference(null); - // used for indicating that all the records from buffer are read successfully. - private final AtomicBoolean isDone = new AtomicBoolean(false); - // indicates rate limit (number of records to cache). it is updated whenever there is a change - // in avg record size. - @VisibleForTesting - public int currentRateLimit = 1; - // indicates avg record size in bytes. It is updated whenever a new record is sampled. - @VisibleForTesting - public long avgRecordSizeInBytes = 0; - // indicates number of samples collected so far. - private long numSamples = 0; - // next record to be read from buffer. - private O nextRecord; - // Function to transform the input payload to the expected output payload - private Function bufferedIteratorTransform; - - public BufferedIterator(final Iterator iterator, final long bufferMemoryLimit, - final Function bufferedIteratorTransform) { - this.inputIterator = iterator; - this.bufferMemoryLimit = bufferMemoryLimit; - this.bufferedIteratorTransform = bufferedIteratorTransform; - } - - @VisibleForTesting - public int size() { - return this.buffer.size(); - } - - // It samples records with "RECORD_SAMPLING_RATE" frequency and computes average record size in - // bytes. It is used - // for determining how many maximum records to buffer. Based on change in avg size it may - // increase or decrease - // available permits. - private void adjustBufferSizeIfNeeded(final I record) throws InterruptedException { - if (this.samplingRecordCounter.incrementAndGet() % RECORD_SAMPLING_RATE != 0) { - return; - } - final long recordSizeInBytes = SizeEstimator.estimate(record); - final long newAvgRecordSizeInBytes = Math - .max(1, (avgRecordSizeInBytes * numSamples + recordSizeInBytes) / (numSamples + 1)); - final int newRateLimit = (int) Math - .min(RECORD_CACHING_LIMIT, Math.max(1, this.bufferMemoryLimit / newAvgRecordSizeInBytes)); - - // If there is any change in number of records to cache then we will either release (if it increased) or acquire - // (if it decreased) to adjust rate limiting to newly computed value. - if (newRateLimit > currentRateLimit) { - rateLimiter.release(newRateLimit - currentRateLimit); - } else if (newRateLimit < currentRateLimit) { - rateLimiter.acquire(currentRateLimit - newRateLimit); - } - currentRateLimit = newRateLimit; - avgRecordSizeInBytes = newAvgRecordSizeInBytes; - numSamples++; - } - - // inserts record into internal buffer. It also fetches insert value from the record to offload - // computation work on to - // buffering thread. - private void insertRecord(I t) throws Exception { - rateLimiter.acquire(); - adjustBufferSizeIfNeeded(t); - // We are retrieving insert value in the record buffering thread to offload computation - // around schema validation - // and record creation to it. - final O payload = bufferedIteratorTransform.apply(t); - buffer.put(Optional.of(payload)); - } - - private void readNextRecord() { - rateLimiter.release(); - Optional newRecord; - while (true) { - try { - throwExceptionIfFailed(); - newRecord = buffer.poll(RECORD_POLL_INTERVAL_SEC, TimeUnit.SECONDS); - if (newRecord != null) { - break; - } - } catch (InterruptedException e) { - logger.error("error reading records from BufferedIterator", e); - throw new HoodieException(e); - } - } - if (newRecord.isPresent()) { - this.nextRecord = newRecord.get(); - } else { - // We are done reading all the records from internal iterator. - this.isDone.set(true); - this.nextRecord = null; - } - } - - public void startBuffering() throws Exception { - logger.info("starting to buffer records"); - try { - while (inputIterator.hasNext()) { - // We need to stop buffering if buffer-reader has failed and exited. - throwExceptionIfFailed(); - insertRecord(inputIterator.next()); - } - // done buffering records notifying buffer-reader. - buffer.put(Optional.empty()); - } catch (Exception e) { - logger.error("error buffering records", e); - // Used for notifying buffer-reader thread of the failed operation. - markAsFailed(e); - throw e; - } - logger.info("finished buffering records"); - } - - @Override - public boolean hasNext() { - if (this.nextRecord == null && !this.isDone.get()) { - readNextRecord(); - } - return !this.isDone.get(); - } - - @Override - public O next() { - Preconditions.checkState(hasNext() && this.nextRecord != null); - final O ret = this.nextRecord; - this.nextRecord = null; - return ret; - } - - private void throwExceptionIfFailed() { - if (this.hasFailed.get() != null) { - throw new HoodieException("operation has failed", this.hasFailed.get()); - } - } - - public void markAsFailed(Exception e) { - this.hasFailed.set(e); - // release the permits so that if the buffering thread is waiting for permits then it will - // get it. - this.rateLimiter.release(RECORD_CACHING_LIMIT + 1); - } -} \ No newline at end of file diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIteratorExecutor.java b/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIteratorExecutor.java deleted file mode 100644 index fea0f28bb..000000000 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIteratorExecutor.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com) - * - * 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 com.uber.hoodie.func; - -import com.uber.hoodie.config.HoodieWriteConfig; -import com.uber.hoodie.exception.HoodieException; -import java.util.Iterator; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import java.util.function.Function; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.TaskContext; -import org.apache.spark.TaskContext$; - -/** - * Executor for a BufferedIterator operation. This class takes as input the input iterator which - * needs to be buffered, the runnable function that needs to be executed in the reader thread and - * return the transformed output based on the writer function - */ -public class BufferedIteratorExecutor { - - private static Logger logger = LogManager.getLogger(BufferedIteratorExecutor.class); - - // Executor service used for launching writer thread. - final ExecutorService writerService; - // Used for buffering records which is controlled by HoodieWriteConfig#WRITE_BUFFER_LIMIT_BYTES. - final BufferedIterator bufferedIterator; - // Need to set current spark thread's TaskContext into newly launched thread so that new - // thread can access - // TaskContext properties. - final TaskContext sparkThreadTaskContext; - - public BufferedIteratorExecutor(final HoodieWriteConfig hoodieConfig, final Iterator inputItr, - final Function bufferedIteratorTransform, - final ExecutorService writerService) { - this.sparkThreadTaskContext = TaskContext.get(); - this.writerService = writerService; - this.bufferedIterator = new BufferedIterator<>(inputItr, hoodieConfig.getWriteBufferLimitBytes(), - bufferedIteratorTransform); - } - - /** - * Starts buffering and executing the writer function - */ - public Future start(Function writerFunction) { - try { - Future future = writerService.submit( - () -> { - logger.info("starting hoodie writer thread"); - // Passing parent thread's TaskContext to newly launched thread for it to access original TaskContext - // properties. - TaskContext$.MODULE$.setTaskContext(sparkThreadTaskContext); - try { - E result = writerFunction.apply(bufferedIterator); - logger.info("hoodie write is done; notifying reader thread"); - return result; - } catch (Exception e) { - logger.error("error writing hoodie records", e); - bufferedIterator.markAsFailed(e); - throw e; - } - }); - bufferedIterator.startBuffering(); - return future; - } catch (Exception e) { - throw new HoodieException(e); - } - } - - public boolean isRemaining() { - return bufferedIterator.hasNext(); - } -} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java b/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java index 36aba5305..6fdf41c3c 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java @@ -19,27 +19,25 @@ package com.uber.hoodie.func; import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.util.queue.BoundedInMemoryExecutor; +import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueConsumer; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieException; -import com.uber.hoodie.func.payload.AbstractBufferedIteratorPayload; -import com.uber.hoodie.func.payload.HoodieRecordBufferedIteratorPayload; import com.uber.hoodie.io.HoodieCreateHandle; import com.uber.hoodie.io.HoodieIOHandle; import com.uber.hoodie.table.HoodieTable; +import java.io.IOException; import java.util.ArrayList; import java.util.HashSet; import java.util.Iterator; -import java.util.LinkedList; import java.util.List; import java.util.Optional; import java.util.Set; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; import java.util.function.Function; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; import org.apache.spark.TaskContext; +import scala.Tuple2; /** * Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new @@ -52,7 +50,6 @@ public class LazyInsertIterable extends private final String commitTime; private final HoodieTable hoodieTable; private Set partitionsCleaned; - private HoodieCreateHandle handle; public LazyInsertIterable(Iterator> sortedRecordItr, HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable) { @@ -63,57 +60,68 @@ public class LazyInsertIterable extends this.hoodieTable = hoodieTable; } - @Override - protected void start() { - } - /** * Transformer function to help transform a HoodieRecord. This transformer is used by BufferedIterator to offload some * expensive operations of transformation to the reader thread. - * @param schema - * @param - * @return */ - public static Function, AbstractBufferedIteratorPayload> - bufferedItrPayloadTransform(Schema schema) { - return (hoodieRecord) -> new HoodieRecordBufferedIteratorPayload(hoodieRecord, schema); + static Function, + Tuple2, Optional>> getTransformFunction(Schema schema) { + return hoodieRecord -> { + try { + return new Tuple2, Optional>(hoodieRecord, + hoodieRecord.getData().getInsertValue(schema)); + } catch (IOException e) { + throw new HoodieException(e); + } + }; + } + + @Override + protected void start() { } @Override protected List computeNext() { // Executor service used for launching writer thread. - final ExecutorService writerService = Executors.newFixedThreadPool(1); + BoundedInMemoryExecutor, + Tuple2, Optional>, List> bufferedIteratorExecutor = null; try { - Function> function = (bufferedIterator) -> { - List statuses = new LinkedList<>(); - statuses.addAll(handleWrite(bufferedIterator)); - return statuses; - }; - BufferedIteratorExecutor, AbstractBufferedIteratorPayload, List> - bufferedIteratorExecutor = new BufferedIteratorExecutor(hoodieConfig, inputItr, - bufferedItrPayloadTransform(HoodieIOHandle.createHoodieWriteSchema(hoodieConfig)), - writerService); - Future> writerResult = bufferedIteratorExecutor.start(function); - final List result = writerResult.get(); + final Schema schema = HoodieIOHandle.createHoodieWriteSchema(hoodieConfig); + bufferedIteratorExecutor = + new SparkBoundedInMemoryExecutor<>(hoodieConfig, inputItr, + new InsertHandler(), getTransformFunction(schema)); + final List result = bufferedIteratorExecutor.execute(); assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining(); return result; } catch (Exception e) { throw new HoodieException(e); } finally { - writerService.shutdownNow(); + if (null != bufferedIteratorExecutor) { + bufferedIteratorExecutor.shutdownNow(); + } } } - private List handleWrite( - final BufferedIterator, AbstractBufferedIteratorPayload> bufferedIterator) { - List statuses = new ArrayList<>(); - while (bufferedIterator.hasNext()) { - final HoodieRecordBufferedIteratorPayload payload = (HoodieRecordBufferedIteratorPayload) bufferedIterator - .next(); - final HoodieRecord insertPayload = (HoodieRecord) payload.getInputPayload(); + @Override + protected void end() { + + } + + /** + * Consumes stream of hoodie records from in-memory queue and + * writes to one or more create-handles + */ + private class InsertHandler extends + BoundedInMemoryQueueConsumer, Optional>, List> { + + private final List statuses = new ArrayList<>(); + private HoodieCreateHandle handle; + + @Override + protected void consumeOneRecord(Tuple2, Optional> payload) { + final HoodieRecord insertPayload = payload._1(); // clean up any partial failures - if (!partitionsCleaned - .contains(insertPayload.getPartitionPath())) { + if (!partitionsCleaned.contains(insertPayload.getPartitionPath())) { // This insert task could fail multiple times, but Spark will faithfully retry with // the same data again. Thus, before we open any files under a given partition, we // first delete any files in the same partitionPath written by same Spark partition @@ -127,33 +135,30 @@ public class LazyInsertIterable extends handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath()); } - if (handle.canWrite(((HoodieRecord) payload.getInputPayload()))) { + if (handle.canWrite(payload._1())) { // write the payload, if the handle has capacity - handle.write(insertPayload, (Optional) payload.getOutputPayload(), payload.exception); + handle.write(insertPayload, payload._2()); } else { // handle is full. statuses.add(handle.close()); // Need to handle the rejected payload & open new handle handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath()); - handle.write(insertPayload, - (Optional) payload.getOutputPayload(), - payload.exception); // we should be able to write 1 payload. + handle.write(insertPayload, payload._2()); // we should be able to write 1 payload. } } - // If we exited out, because we ran out of records, just close the pending handle. - if (!bufferedIterator.hasNext()) { + @Override + protected void finish() { if (handle != null) { statuses.add(handle.close()); } + handle = null; + assert statuses.size() > 0; } - assert statuses.size() > 0 && !bufferedIterator.hasNext(); // should never return empty statuses - return statuses; - } - - @Override - protected void end() { - + @Override + protected List getResult() { + return statuses; + } } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/ParquetReaderIterator.java b/hoodie-client/src/main/java/com/uber/hoodie/func/ParquetReaderIterator.java index ed81da035..2ccc74320 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/ParquetReaderIterator.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/ParquetReaderIterator.java @@ -16,6 +16,7 @@ package com.uber.hoodie.func; +import com.uber.hoodie.common.util.queue.BoundedInMemoryQueue; import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; import java.util.Iterator; @@ -23,7 +24,7 @@ import org.apache.parquet.hadoop.ParquetReader; /** * This class wraps a parquet reader and provides an iterator based api to - * read from a parquet file. This is used in {@link BufferedIterator} + * read from a parquet file. This is used in {@link BoundedInMemoryQueue} */ public class ParquetReaderIterator implements Iterator { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/SparkBoundedInMemoryExecutor.java b/hoodie-client/src/main/java/com/uber/hoodie/func/SparkBoundedInMemoryExecutor.java new file mode 100644 index 000000000..76e35827f --- /dev/null +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/SparkBoundedInMemoryExecutor.java @@ -0,0 +1,57 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * 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 com.uber.hoodie.func; + +import com.uber.hoodie.common.util.queue.BoundedInMemoryExecutor; +import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueConsumer; +import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueProducer; +import com.uber.hoodie.common.util.queue.IteratorBasedQueueProducer; +import com.uber.hoodie.config.HoodieWriteConfig; +import java.util.Iterator; +import java.util.Optional; +import java.util.function.Function; +import org.apache.spark.TaskContext; +import org.apache.spark.TaskContext$; + +public class SparkBoundedInMemoryExecutor extends BoundedInMemoryExecutor { + + // Need to set current spark thread's TaskContext into newly launched thread so that new thread can access + // TaskContext properties. + final TaskContext sparkThreadTaskContext; + + public SparkBoundedInMemoryExecutor(final HoodieWriteConfig hoodieConfig, final Iterator inputItr, + BoundedInMemoryQueueConsumer consumer, + Function bufferedIteratorTransform) { + this(hoodieConfig, new IteratorBasedQueueProducer<>(inputItr), consumer, bufferedIteratorTransform); + } + + public SparkBoundedInMemoryExecutor(final HoodieWriteConfig hoodieConfig, + BoundedInMemoryQueueProducer producer, + BoundedInMemoryQueueConsumer consumer, + Function bufferedIteratorTransform) { + super(hoodieConfig.getWriteBufferLimitBytes(), producer, + Optional.of(consumer), bufferedIteratorTransform); + this.sparkThreadTaskContext = TaskContext.get(); + } + + public void preExecute() { + // Passing parent thread's TaskContext to newly launched thread for it to access original TaskContext properties. + TaskContext$.MODULE$.setTaskContext(sparkThreadTaskContext); + } +} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/payload/AbstractBufferedIteratorPayload.java b/hoodie-client/src/main/java/com/uber/hoodie/func/payload/AbstractBufferedIteratorPayload.java deleted file mode 100644 index 4a7e32224..000000000 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/payload/AbstractBufferedIteratorPayload.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com) - * - * 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 com.uber.hoodie.func.payload; - -/** - * @param Input data type for BufferedIterator - * @param Output data type for BufferedIterator - */ -public abstract class AbstractBufferedIteratorPayload { - - // input payload for iterator - protected I inputPayload; - // output payload for iterator, this is used in cases where the output payload is computed - // from the input payload and most of this computation is off-loaded to the reader - protected O outputPayload; - - public AbstractBufferedIteratorPayload(I record) { - this.inputPayload = record; - } - - public I getInputPayload() { - return inputPayload; - } - - public O getOutputPayload() { - return outputPayload; - } -} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/payload/HoodieRecordBufferedIteratorPayload.java b/hoodie-client/src/main/java/com/uber/hoodie/func/payload/HoodieRecordBufferedIteratorPayload.java deleted file mode 100644 index a79f0e01a..000000000 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/payload/HoodieRecordBufferedIteratorPayload.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com) - * - * 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 com.uber.hoodie.func.payload; - -import com.uber.hoodie.common.model.HoodieRecord; -import com.uber.hoodie.common.model.HoodieRecordPayload; -import java.util.Optional; -import org.apache.avro.Schema; -import org.apache.avro.generic.IndexedRecord; - -/** - * BufferedIteratorPayload that takes HoodieRecord as input and transforms to output Optional - * @param - */ -public class HoodieRecordBufferedIteratorPayload - extends AbstractBufferedIteratorPayload, Optional> { - - // It caches the exception seen while fetching insert value. - public Optional exception = Optional.empty(); - - public HoodieRecordBufferedIteratorPayload(HoodieRecord record, Schema schema) { - super(record); - try { - this.outputPayload = record.getData().getInsertValue(schema); - } catch (Exception e) { - this.exception = Optional.of(e); - } - } - - public Optional getException() { - return exception; - } -} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java index c2ee5f8c0..0384db71c 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java @@ -90,15 +90,9 @@ public class HoodieCreateHandle extends HoodieIOH /** * Perform the actual writing of the given record into the backing file. */ - public void write(HoodieRecord record, Optional insertValue, - Optional getInsertValueException) { + public void write(HoodieRecord record, Optional avroRecord) { Optional recordMetadata = record.getData().getMetadata(); try { - // throws exception if there was any exception while fetching insert value - if (getInsertValueException.isPresent()) { - throw getInsertValueException.get(); - } - Optional avroRecord = insertValue; if (avroRecord.isPresent()) { storageWriter.writeAvroWithMetadata(avroRecord.get(), record); // update the new location of record, so we know where to find it next diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index 8e28b8d98..679d5fa8e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -24,7 +24,9 @@ import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.model.HoodieWriteStat.RuntimeStats; import com.uber.hoodie.common.table.TableFileSystemView; +import com.uber.hoodie.common.util.DefaultSizeEstimator; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.HoodieRecordSizeEstimator; import com.uber.hoodie.common.util.ReflectionUtils; import com.uber.hoodie.common.util.collection.ExternalSpillableMap; import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; @@ -143,7 +145,8 @@ public class HoodieMergeHandle extends HoodieIOHa logger.info("MaxMemoryPerPartitionMerge => " + config.getMaxMemoryPerPartitionMerge()); this.keyToNewRecords = new ExternalSpillableMap<>(config.getMaxMemoryPerPartitionMerge(), config.getSpillableMapBasePath(), new StringConverter(), - new HoodieRecordConverter(schema, config.getPayloadClass())); + new HoodieRecordConverter(schema, config.getPayloadClass()), + new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); } catch (IOException io) { throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index cbf06d936..e3d00f75e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -28,7 +28,7 @@ import com.uber.hoodie.common.model.HoodieWriteStat.RuntimeStats; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.TableFileSystemView; -import com.uber.hoodie.common.table.log.HoodieCompactedLogRecordScanner; +import com.uber.hoodie.common.table.log.HoodieMergedLogRecordScanner; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.config.HoodieWriteConfig; @@ -115,7 +115,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor { .filterCompletedInstants().lastInstant().get().getTimestamp(); log.info("MaxMemoryPerCompaction => " + config.getMaxMemoryPerCompaction()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, metaClient.getBasePath(), operation.getDeltaFilePaths(), readerSchema, maxInstantTime, config.getMaxMemoryPerCompaction(), config.getCompactionLazyBlockReadEnabled(), config.getCompactionReverseLogReadEnabled(), config.getMaxDFSStreamBufferSize(), @@ -131,7 +131,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor { Iterable> resultIterable = () -> result; return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream) .map(s -> { - s.getStat().setTotalUpdatedRecordsCompacted(scanner.getTotalRecordsToUpdate()); + s.getStat().setTotalUpdatedRecordsCompacted(scanner.getNumMergedRecordsInLog()); s.getStat().setTotalLogFilesCompacted(scanner.getTotalLogFiles()); s.getStat().setTotalLogRecords(scanner.getTotalLogRecords()); s.getStat().setPartitionPath(operation.getPartitionPath()); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index bdf40f79a..ab194c4f5 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -33,17 +33,16 @@ import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.queue.BoundedInMemoryExecutor; +import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueConsumer; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.HoodieNotSupportedException; import com.uber.hoodie.exception.HoodieUpsertException; -import com.uber.hoodie.func.BufferedIterator; -import com.uber.hoodie.func.BufferedIteratorExecutor; import com.uber.hoodie.func.LazyInsertIterable; import com.uber.hoodie.func.ParquetReaderIterator; -import com.uber.hoodie.func.payload.AbstractBufferedIteratorPayload; -import com.uber.hoodie.func.payload.GenericRecordBufferedIteratorPayload; +import com.uber.hoodie.func.SparkBoundedInMemoryExecutor; import com.uber.hoodie.io.HoodieCleanHelper; import com.uber.hoodie.io.HoodieMergeHandle; import java.io.IOException; @@ -58,9 +57,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; import java.util.stream.Collectors; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -182,16 +178,6 @@ public class HoodieCopyOnWriteTable extends Hoodi return handleUpdateInternal(upsertHandle, commitTime, fileLoc); } - /** - * Transformer function to help transform a GenericRecord. This transformer is used by BufferedIterator to offload - * some expensive operations of transformation to the reader thread. - * - */ - public static java.util.function.Function - bufferedItrPayloadTransform() { - return (genericRecord) -> new GenericRecordBufferedIteratorPayload(genericRecord); - } - protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, String commitTime, String fileLoc) throws IOException { @@ -202,23 +188,19 @@ public class HoodieCopyOnWriteTable extends Hoodi AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getSchema()); ParquetReader reader = AvroParquetReader.builder(upsertHandle.getOldFilePath()) .withConf(getHadoopConf()).build(); - final ExecutorService writerService = Executors.newFixedThreadPool(1); + BoundedInMemoryExecutor wrapper = null; try { - java.util.function.Function runnableFunction = (bufferedIterator) -> { - handleWrite(bufferedIterator, upsertHandle); - return null; - }; - BufferedIteratorExecutor wrapper = - new BufferedIteratorExecutor(config, new ParquetReaderIterator(reader), bufferedItrPayloadTransform(), - writerService); - Future writerResult = wrapper.start(runnableFunction); - writerResult.get(); + wrapper = new SparkBoundedInMemoryExecutor(config, new ParquetReaderIterator(reader), + new UpdateHandler(upsertHandle), x -> x); + wrapper.execute(); } catch (Exception e) { throw new HoodieException(e); } finally { reader.close(); upsertHandle.close(); - writerService.shutdownNow(); + if (null != wrapper) { + wrapper.shutdownNow(); + } } } @@ -231,15 +213,6 @@ public class HoodieCopyOnWriteTable extends Hoodi .iterator(); } - private void handleWrite(final BufferedIterator bufferedIterator, - final HoodieMergeHandle upsertHandle) { - while (bufferedIterator.hasNext()) { - final GenericRecordBufferedIteratorPayload payload = (GenericRecordBufferedIteratorPayload) bufferedIterator - .next(); - upsertHandle.write(payload.getOutputPayload()); - } - } - protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc, Iterator> recordItr) { return new HoodieMergeHandle<>(config, commitTime, this, recordItr, fileLoc); @@ -493,6 +466,32 @@ public class HoodieCopyOnWriteTable extends Hoodi UPDATE, INSERT } + /** + * Consumer that dequeues records from queue and sends to Merge Handle + */ + private static class UpdateHandler extends BoundedInMemoryQueueConsumer { + + private final HoodieMergeHandle upsertHandle; + + private UpdateHandler(HoodieMergeHandle upsertHandle) { + this.upsertHandle = upsertHandle; + } + + @Override + protected void consumeOneRecord(GenericRecord record) { + upsertHandle.write(record); + } + + @Override + protected void finish() { + } + + @Override + protected Void getResult() { + return null; + } + } + private static class PartitionCleanStat implements Serializable { private final String partitionPath; diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIteratorExecutor.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryExecutor.java similarity index 55% rename from hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIteratorExecutor.java rename to hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryExecutor.java index 5c3695b8b..385afaabf 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIteratorExecutor.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryExecutor.java @@ -16,39 +16,35 @@ package com.uber.hoodie.func; +import static com.uber.hoodie.func.LazyInsertIterable.getTransformFunction; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; +import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueConsumer; import com.uber.hoodie.config.HoodieWriteConfig; import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.function.Function; +import java.util.Optional; +import org.apache.avro.generic.IndexedRecord; import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import scala.Tuple2; -public class TestBufferedIteratorExecutor { +public class TestBoundedInMemoryExecutor { private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator(); private final String commitTime = HoodieActiveTimeline.createNewCommitTime(); - private ExecutorService executorService = null; - - @Before - public void beforeTest() { - this.executorService = Executors.newFixedThreadPool(1); - } + private SparkBoundedInMemoryExecutor>, Integer> executor = null; @After public void afterTest() { - if (this.executorService != null) { - this.executorService.shutdownNow(); - this.executorService = null; + if (this.executor != null) { + this.executor.shutdownNow(); + this.executor = null; } } @@ -59,21 +55,32 @@ public class TestBufferedIteratorExecutor { HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024); - BufferedIteratorExecutor bufferedIteratorExecutor = new BufferedIteratorExecutor(hoodieWriteConfig, - hoodieRecords.iterator(), LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema), - executorService); - Function function = (bufferedIterator) -> { - Integer count = 0; - while (bufferedIterator.hasNext()) { - count++; - bufferedIterator.next(); - } - return count; - }; - Future future = bufferedIteratorExecutor.start(function); + BoundedInMemoryQueueConsumer>, Integer> consumer = + new BoundedInMemoryQueueConsumer>, Integer>() { + + private int count = 0; + + @Override + protected void consumeOneRecord(Tuple2> record) { + count++; + } + + @Override + protected void finish() { + } + + @Override + protected Integer getResult() { + return count; + } + }; + + executor = new SparkBoundedInMemoryExecutor(hoodieWriteConfig, + hoodieRecords.iterator(), consumer, getTransformFunction(HoodieTestDataGenerator.avroSchema)); + int result = executor.execute(); // It should buffer and write 100 records - Assert.assertEquals((int) future.get(), 100); + Assert.assertEquals(result, 100); // There should be no remaining records in the buffer - Assert.assertFalse(bufferedIteratorExecutor.isRemaining()); + Assert.assertFalse(executor.isRemaining()); } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryQueue.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryQueue.java new file mode 100644 index 000000000..b555b8167 --- /dev/null +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryQueue.java @@ -0,0 +1,336 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * 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 com.uber.hoodie.func; + +import static com.uber.hoodie.func.LazyInsertIterable.getTransformFunction; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.uber.hoodie.common.HoodieTestDataGenerator; +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; +import com.uber.hoodie.common.util.DefaultSizeEstimator; +import com.uber.hoodie.common.util.SizeEstimator; +import com.uber.hoodie.common.util.queue.BoundedInMemoryQueue; +import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueProducer; +import com.uber.hoodie.common.util.queue.FunctionBasedQueueProducer; +import com.uber.hoodie.common.util.queue.IteratorBasedQueueProducer; +import com.uber.hoodie.exception.HoodieException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.Semaphore; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.avro.generic.IndexedRecord; +import org.apache.commons.io.FileUtils; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import scala.Tuple2; + +public class TestBoundedInMemoryQueue { + + private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator(); + private final String commitTime = HoodieActiveTimeline.createNewCommitTime(); + private ExecutorService executorService = null; + + @Before + public void beforeTest() { + this.executorService = Executors.newFixedThreadPool(2); + } + + @After + public void afterTest() { + if (this.executorService != null) { + this.executorService.shutdownNow(); + this.executorService = null; + } + } + + // Test to ensure that we are reading all records from queue iterator in the same order + // without any exceptions. + @SuppressWarnings("unchecked") + @Test(timeout = 60000) + public void testRecordReading() throws Exception { + final int numRecords = 128; + final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); + final BoundedInMemoryQueue>> queue = new BoundedInMemoryQueue(FileUtils.ONE_KB, + getTransformFunction(HoodieTestDataGenerator.avroSchema)); + // Produce + Future resFuture = + executorService.submit(() -> { + new IteratorBasedQueueProducer<>(hoodieRecords.iterator()).produce(queue); + queue.close(); + return true; + }); + final Iterator originalRecordIterator = hoodieRecords.iterator(); + int recordsRead = 0; + while (queue.iterator().hasNext()) { + final HoodieRecord originalRecord = originalRecordIterator.next(); + final Optional originalInsertValue = originalRecord.getData() + .getInsertValue(HoodieTestDataGenerator.avroSchema); + final Tuple2> payload = queue.iterator().next(); + // Ensure that record ordering is guaranteed. + Assert.assertEquals(originalRecord, payload._1()); + // cached insert value matches the expected insert value. + Assert.assertEquals(originalInsertValue, + payload._1().getData().getInsertValue(HoodieTestDataGenerator.avroSchema)); + recordsRead++; + } + Assert.assertFalse(queue.iterator().hasNext() || originalRecordIterator.hasNext()); + // all the records should be read successfully. + Assert.assertEquals(numRecords, recordsRead); + // should not throw any exceptions. + resFuture.get(); + } + + /** + * Test to ensure that we are reading all records from queue iterator when we have multiple producers + */ + @SuppressWarnings("unchecked") + @Test(timeout = 60000) + public void testCompositeProducerRecordReading() throws Exception { + final int numRecords = 1000; + final int numProducers = 40; + final List> recs = new ArrayList<>(); + + final BoundedInMemoryQueue>> queue = + new BoundedInMemoryQueue(FileUtils.ONE_KB, getTransformFunction(HoodieTestDataGenerator.avroSchema)); + + // Record Key to + Map> keyToProducerAndIndexMap = new HashMap<>(); + + for (int i = 0; i < numProducers; i++) { + List pRecs = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); + int j = 0; + for (HoodieRecord r : pRecs) { + Assert.assertTrue(!keyToProducerAndIndexMap.containsKey(r.getRecordKey())); + keyToProducerAndIndexMap.put(r.getRecordKey(), new Tuple2<>(i, j)); + j++; + } + recs.add(pRecs); + } + + List> producers = new ArrayList<>(); + for (int i = 0; i < recs.size(); i++) { + final List r = recs.get(i); + // Alternate between pull and push based iterators + if (i % 2 == 0) { + producers.add(new IteratorBasedQueueProducer<>(r.iterator())); + } else { + producers.add(new FunctionBasedQueueProducer((buf) -> { + Iterator itr = r.iterator(); + while (itr.hasNext()) { + try { + buf.insertRecord(itr.next()); + } catch (Exception e) { + throw new HoodieException(e); + } + } + return true; + })); + } + } + + final List> futureList = producers.stream().map(producer -> { + return executorService.submit(() -> { + producer.produce(queue); + return true; + }); + }).collect(Collectors.toList()); + + // Close queue + Future closeFuture = executorService.submit(() -> { + try { + for (Future f : futureList) { + f.get(); + } + queue.close(); + } catch (Exception e) { + throw new RuntimeException(e); + } + return true; + }); + + // Used to ensure that consumer sees the records generated by a single producer in FIFO order + Map lastSeenMap = IntStream.range(0, numProducers).boxed() + .collect(Collectors.toMap(Function.identity(), x -> -1)); + Map countMap = IntStream.range(0, numProducers).boxed() + .collect(Collectors.toMap(Function.identity(), x -> 0)); + + // Read recs and ensure we have covered all producer recs. + while (queue.iterator().hasNext()) { + final Tuple2> payload = queue.iterator().next(); + final HoodieRecord rec = payload._1(); + Tuple2 producerPos = keyToProducerAndIndexMap.get(rec.getRecordKey()); + Integer lastSeenPos = lastSeenMap.get(producerPos._1()); + countMap.put(producerPos._1(), countMap.get(producerPos._1()) + 1); + lastSeenMap.put(producerPos._1(), lastSeenPos + 1); + // Ensure we are seeing the next record generated + Assert.assertEquals(lastSeenPos + 1, producerPos._2().intValue()); + } + + for (int i = 0; i < numProducers; i++) { + // Ensure we have seen all the records for each producers + Assert.assertEquals(Integer.valueOf(numRecords), countMap.get(i)); + } + + //Ensure Close future is done + closeFuture.get(); + } + + // Test to ensure that record queueing is throttled when we hit memory limit. + @SuppressWarnings("unchecked") + @Test(timeout = 60000) + public void testMemoryLimitForBuffering() throws Exception { + final int numRecords = 128; + final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); + // maximum number of records to keep in memory. + final int recordLimit = 5; + final SizeEstimator>> sizeEstimator = + new DefaultSizeEstimator<>(); + final long objSize = sizeEstimator.sizeEstimate( + getTransformFunction(HoodieTestDataGenerator.avroSchema).apply(hoodieRecords.get(0))); + final long memoryLimitInBytes = recordLimit * objSize; + final BoundedInMemoryQueue>> queue = + new BoundedInMemoryQueue(memoryLimitInBytes, + getTransformFunction(HoodieTestDataGenerator.avroSchema)); + + // Produce + Future resFuture = executorService.submit(() -> { + new IteratorBasedQueueProducer<>(hoodieRecords.iterator()).produce(queue); + return true; + }); + // waiting for permits to expire. + while (!isQueueFull(queue.rateLimiter)) { + Thread.sleep(10); + } + Assert.assertEquals(0, queue.rateLimiter.availablePermits()); + Assert.assertEquals(recordLimit, queue.currentRateLimit); + Assert.assertEquals(recordLimit, queue.size()); + Assert.assertEquals(recordLimit - 1, queue.samplingRecordCounter.get()); + + // try to read 2 records. + Assert.assertEquals(hoodieRecords.get(0), queue.iterator().next()._1()); + Assert.assertEquals(hoodieRecords.get(1), queue.iterator().next()._1()); + + // waiting for permits to expire. + while (!isQueueFull(queue.rateLimiter)) { + Thread.sleep(10); + } + // No change is expected in rate limit or number of queued records. We only expect + // queueing thread to read + // 2 more records into the queue. + Assert.assertEquals(0, queue.rateLimiter.availablePermits()); + Assert.assertEquals(recordLimit, queue.currentRateLimit); + Assert.assertEquals(recordLimit, queue.size()); + Assert.assertEquals(recordLimit - 1 + 2, queue.samplingRecordCounter.get()); + } + + // Test to ensure that exception in either queueing thread or BufferedIterator-reader thread + // is propagated to + // another thread. + @SuppressWarnings("unchecked") + @Test(timeout = 60000) + public void testException() throws Exception { + final int numRecords = 256; + final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); + final SizeEstimator>> sizeEstimator = + new DefaultSizeEstimator<>(); + // queue memory limit + final long objSize = sizeEstimator.sizeEstimate( + getTransformFunction(HoodieTestDataGenerator.avroSchema).apply(hoodieRecords.get(0))); + final long memoryLimitInBytes = 4 * objSize; + + // first let us throw exception from queueIterator reader and test that queueing thread + // stops and throws + // correct exception back. + BoundedInMemoryQueue>> queue1 = + new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.avroSchema)); + + // Produce + Future resFuture = executorService.submit(() -> { + new IteratorBasedQueueProducer<>(hoodieRecords.iterator()).produce(queue1); + return true; + }); + + // waiting for permits to expire. + while (!isQueueFull(queue1.rateLimiter)) { + Thread.sleep(10); + } + // notify queueing thread of an exception and ensure that it exits. + final Exception e = new Exception("Failing it :)"); + queue1.markAsFailed(e); + try { + resFuture.get(); + Assert.fail("exception is expected"); + } catch (ExecutionException e1) { + Assert.assertEquals(HoodieException.class, e1.getCause().getClass()); + Assert.assertEquals(e, e1.getCause().getCause()); + } + + // second let us raise an exception while doing record queueing. this exception should get + // propagated to + // queue iterator reader. + final RuntimeException expectedException = new RuntimeException("failing record reading"); + final Iterator mockHoodieRecordsIterator = mock(Iterator.class); + when(mockHoodieRecordsIterator.hasNext()).thenReturn(true); + when(mockHoodieRecordsIterator.next()).thenThrow(expectedException); + BoundedInMemoryQueue>> queue2 = + new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.avroSchema)); + + // Produce + Future res = executorService.submit(() -> { + try { + new IteratorBasedQueueProducer<>(mockHoodieRecordsIterator).produce(queue2); + } catch (Exception ex) { + queue2.markAsFailed(ex); + throw ex; + } + return true; + }); + + try { + queue2.iterator().hasNext(); + Assert.fail("exception is expected"); + } catch (Exception e1) { + Assert.assertEquals(expectedException, e1.getCause()); + } + // queueing thread should also have exited. make sure that it is not running. + try { + res.get(); + Assert.fail("exception is expected"); + } catch (ExecutionException e2) { + Assert.assertEquals(expectedException, e2.getCause()); + } + } + + private boolean isQueueFull(Semaphore rateLimiter) { + return (rateLimiter.availablePermits() == 0 && rateLimiter.hasQueuedThreads()); + } +} diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIterator.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIterator.java deleted file mode 100644 index e55db1b05..000000000 --- a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIterator.java +++ /dev/null @@ -1,203 +0,0 @@ -/* - * Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com) - * - * 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 com.uber.hoodie.func; - -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -import com.uber.hoodie.common.HoodieTestDataGenerator; -import com.uber.hoodie.common.model.HoodieRecord; -import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; -import com.uber.hoodie.exception.HoodieException; -import com.uber.hoodie.func.payload.AbstractBufferedIteratorPayload; -import com.uber.hoodie.func.payload.HoodieRecordBufferedIteratorPayload; -import java.io.IOException; -import java.util.Iterator; -import java.util.List; -import java.util.Optional; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.Semaphore; -import org.apache.avro.generic.IndexedRecord; -import org.apache.commons.io.FileUtils; -import org.apache.spark.util.SizeEstimator; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -public class TestBufferedIterator { - - private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator(); - private final String commitTime = HoodieActiveTimeline.createNewCommitTime(); - private ExecutorService recordReader = null; - - @Before - public void beforeTest() { - this.recordReader = Executors.newFixedThreadPool(1); - } - - @After - public void afterTest() { - if (this.recordReader != null) { - this.recordReader.shutdownNow(); - this.recordReader = null; - } - } - - // Test to ensure that we are reading all records from buffered iterator in the same order - // without any exceptions. - @Test(timeout = 60000) - public void testRecordReading() throws IOException, ExecutionException, InterruptedException { - final int numRecords = 128; - final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); - final BufferedIterator bufferedIterator = new BufferedIterator(hoodieRecords.iterator(), FileUtils.ONE_KB, - LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema)); - Future result = recordReader.submit(() -> { - bufferedIterator.startBuffering(); - return true; - }); - final Iterator originalRecordIterator = hoodieRecords.iterator(); - int recordsRead = 0; - while (bufferedIterator.hasNext()) { - final HoodieRecord originalRecord = originalRecordIterator.next(); - final Optional originalInsertValue = originalRecord.getData() - .getInsertValue(HoodieTestDataGenerator.avroSchema); - final HoodieRecordBufferedIteratorPayload payload = (HoodieRecordBufferedIteratorPayload) bufferedIterator.next(); - // Ensure that record ordering is guaranteed. - Assert.assertEquals(originalRecord, payload.getInputPayload()); - // cached insert value matches the expected insert value. - Assert.assertEquals(originalInsertValue, - ((HoodieRecord) payload.getInputPayload()).getData().getInsertValue(HoodieTestDataGenerator.avroSchema)); - recordsRead++; - } - Assert.assertFalse(bufferedIterator.hasNext() || originalRecordIterator.hasNext()); - // all the records should be read successfully. - Assert.assertEquals(numRecords, recordsRead); - // should not throw any exceptions. - Assert.assertTrue(result.get()); - } - - // Test to ensure that record buffering is throttled when we hit memory limit. - @Test(timeout = 60000) - public void testMemoryLimitForBuffering() throws IOException, InterruptedException { - final int numRecords = 128; - final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); - // maximum number of records to keep in memory. - final int recordLimit = 5; - final long memoryLimitInBytes = recordLimit * SizeEstimator.estimate(hoodieRecords.get(0)); - final BufferedIterator bufferedIterator = - new BufferedIterator(hoodieRecords.iterator(), memoryLimitInBytes, - LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema)); - Future result = recordReader.submit(() -> { - bufferedIterator.startBuffering(); - return true; - }); - // waiting for permits to expire. - while (!isQueueFull(bufferedIterator.rateLimiter)) { - Thread.sleep(10); - } - Assert.assertEquals(0, bufferedIterator.rateLimiter.availablePermits()); - Assert.assertEquals(recordLimit, bufferedIterator.currentRateLimit); - Assert.assertEquals(recordLimit, bufferedIterator.size()); - Assert.assertEquals(recordLimit - 1, bufferedIterator.samplingRecordCounter.get()); - - // try to read 2 records. - Assert.assertEquals(hoodieRecords.get(0), bufferedIterator.next().getInputPayload()); - Assert.assertEquals(hoodieRecords.get(1), bufferedIterator.next().getInputPayload()); - - // waiting for permits to expire. - while (!isQueueFull(bufferedIterator.rateLimiter)) { - Thread.sleep(10); - } - // No change is expected in rate limit or number of buffered records. We only expect - // buffering thread to read - // 2 more records into the buffer. - Assert.assertEquals(0, bufferedIterator.rateLimiter.availablePermits()); - Assert.assertEquals(recordLimit, bufferedIterator.currentRateLimit); - Assert.assertEquals(recordLimit, bufferedIterator.size()); - Assert.assertEquals(recordLimit - 1 + 2, bufferedIterator.samplingRecordCounter.get()); - } - - // Test to ensure that exception in either buffering thread or BufferedIterator-reader thread - // is propagated to - // another thread. - @Test(timeout = 60000) - public void testException() throws IOException, InterruptedException { - final int numRecords = 256; - final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); - // buffer memory limit - final long memoryLimitInBytes = 4 * SizeEstimator.estimate(hoodieRecords.get(0)); - - // first let us throw exception from bufferIterator reader and test that buffering thread - // stops and throws - // correct exception back. - BufferedIterator bufferedIterator1 = new BufferedIterator(hoodieRecords.iterator(), memoryLimitInBytes, - LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema)); - Future result = recordReader.submit(() -> { - bufferedIterator1.startBuffering(); - return true; - }); - // waiting for permits to expire. - while (!isQueueFull(bufferedIterator1.rateLimiter)) { - Thread.sleep(10); - } - // notify buffering thread of an exception and ensure that it exits. - final Exception e = new Exception("Failing it :)"); - bufferedIterator1.markAsFailed(e); - try { - result.get(); - Assert.fail("exception is expected"); - } catch (ExecutionException e1) { - Assert.assertEquals(HoodieException.class, e1.getCause().getClass()); - Assert.assertEquals(e, e1.getCause().getCause()); - } - - // second let us raise an exception while doing record buffering. this exception should get - // propagated to - // buffered iterator reader. - final RuntimeException expectedException = new RuntimeException("failing record reading"); - final Iterator mockHoodieRecordsIterator = mock(Iterator.class); - when(mockHoodieRecordsIterator.hasNext()).thenReturn(true); - when(mockHoodieRecordsIterator.next()).thenThrow(expectedException); - BufferedIterator bufferedIterator2 = new BufferedIterator(mockHoodieRecordsIterator, memoryLimitInBytes, - LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema)); - Future result2 = recordReader.submit(() -> { - bufferedIterator2.startBuffering(); - return true; - }); - try { - bufferedIterator2.hasNext(); - Assert.fail("exception is expected"); - } catch (Exception e1) { - Assert.assertEquals(expectedException, e1.getCause()); - } - // buffering thread should also have exited. make sure that it is not running. - try { - result2.get(); - Assert.fail("exception is expected"); - } catch (ExecutionException e2) { - Assert.assertEquals(expectedException, e2.getCause()); - } - } - - private boolean isQueueFull(Semaphore rateLimiter) { - return (rateLimiter.availablePermits() == 0 && rateLimiter.hasQueuedThreads()); - } -} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java similarity index 71% rename from hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java rename to hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java index 86da05b73..dd79fff92 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java @@ -19,7 +19,6 @@ package com.uber.hoodie.common.table.log; import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType.INSTANT_TIME; import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK; -import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; @@ -29,19 +28,14 @@ import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock; import com.uber.hoodie.common.table.log.block.HoodieCommandBlock; import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock; import com.uber.hoodie.common.table.log.block.HoodieLogBlock; -import com.uber.hoodie.common.util.HoodieTimer; import com.uber.hoodie.common.util.SpillableMapUtils; -import com.uber.hoodie.common.util.collection.ExternalSpillableMap; -import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; -import com.uber.hoodie.common.util.collection.converter.StringConverter; import com.uber.hoodie.exception.HoodieIOException; -import java.io.IOException; import java.util.ArrayDeque; import java.util.Arrays; import java.util.Deque; -import java.util.Iterator; +import java.util.HashSet; import java.util.List; -import java.util.Map; +import java.util.Set; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import org.apache.avro.Schema; @@ -53,24 +47,38 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; /** - * Scans through all the blocks in a list of HoodieLogFile and builds up a compacted/merged list of records which will - * be used as a lookup table when merging the base columnar file with the redo log file. NOTE: If readBlockLazily is + * Implements logic to scan log blocks and expose valid and deleted log records to subclass implementation. + * Subclass is free to either apply merging or expose raw data back to the caller. + * + * NOTE: If readBlockLazily is * turned on, does not merge, instead keeps reading log blocks and merges everything at once This is an optimization to * avoid seek() back and forth to read new block (forward seek()) and lazily read content of seen block (reverse and * forward seek()) during merge | | Read Block 1 Metadata | | Read Block 1 Data | | | Read Block 2 * Metadata | | Read Block 2 Data | | I/O Pass 1 | ..................... | I/O Pass 2 | ................. | | * | Read Block N Metadata | | Read Block N Data |

This results in two I/O passes over the log file. */ +public abstract class AbstractHoodieLogRecordScanner { -public class HoodieCompactedLogRecordScanner implements - Iterable> { + private static final Logger log = LogManager.getLogger(AbstractHoodieLogRecordScanner.class); - private static final Logger log = LogManager.getLogger(HoodieCompactedLogRecordScanner.class); - - // Final map of compacted/merged records - private final ExternalSpillableMap> records; // Reader schema for the records private final Schema readerSchema; + // Latest valid instant time + private final String latestInstantTime; + private final HoodieTableMetaClient hoodieTableMetaClient; + // Merge strategy to use when combining records from log + private final String payloadClassFQN; + // Log File Paths + private final List logFilePaths; + // Read Lazily flag + private final boolean readBlocksLazily; + // Reverse reader - Not implemented yet (NA -> Why do we need ?) + // but present here for plumbing for future implementation + private final boolean reverseReader; + // Buffer Size for log file reader + private final int bufferSize; + // FileSystem + private final FileSystem fs; // Total log files read - for metrics private AtomicLong totalLogFiles = new AtomicLong(0); // Total log blocks read - for metrics @@ -81,46 +89,47 @@ public class HoodieCompactedLogRecordScanner implements private AtomicLong totalRollbacks = new AtomicLong(0); // Total number of corrupt blocks written across all log files private AtomicLong totalCorruptBlocks = new AtomicLong(0); - // Total final list of compacted/merged records - private long totalRecordsToUpdate; - // Latest valid instant time - private String latestInstantTime; - private HoodieTableMetaClient hoodieTableMetaClient; - // Merge strategy to use when combining records from log - private String payloadClassFQN; // Store the last instant log blocks (needed to implement rollback) private Deque currentInstantLogBlocks = new ArrayDeque<>(); - // Stores the total time taken to perform reading and merging of log blocks - private long totalTimeTakenToReadAndMergeBlocks = 0L; - // A timer for calculating elapsed time in millis - public HoodieTimer timer = new HoodieTimer(); - public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, - Schema readerSchema, String latestInstantTime, Long maxMemorySizeInBytes, - boolean readBlocksLazily, boolean reverseReader, int bufferSize, String spillableMapBasePath) { + // Progress + private float progress = 0.0f; + + public AbstractHoodieLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, + Schema readerSchema, String latestInstantTime, + boolean readBlocksLazily, boolean reverseReader, int bufferSize) { this.readerSchema = readerSchema; this.latestInstantTime = latestInstantTime; this.hoodieTableMetaClient = new HoodieTableMetaClient(fs.getConf(), basePath); // load class from the payload fully qualified class name this.payloadClassFQN = this.hoodieTableMetaClient.getTableConfig().getPayloadClass(); this.totalLogFiles.addAndGet(logFilePaths.size()); - timer.startTimer(); + this.logFilePaths = logFilePaths; + this.readBlocksLazily = readBlocksLazily; + this.reverseReader = reverseReader; + this.fs = fs; + this.bufferSize = bufferSize; + } + /** + * Scan Log files + */ + public void scan() { try { - // Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize - this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath, - new StringConverter(), new HoodieRecordConverter(readerSchema, payloadClassFQN)); // iterate over the paths HoodieLogFormatReader logFormatReaderWrapper = new HoodieLogFormatReader(fs, logFilePaths.stream().map(logFile -> new HoodieLogFile(new Path(logFile))) .collect(Collectors.toList()), readerSchema, readBlocksLazily, reverseReader, bufferSize); - HoodieLogFile logFile; + Set scannedLogFiles = new HashSet<>(); while (logFormatReaderWrapper.hasNext()) { - logFile = logFormatReaderWrapper.getLogFile(); + HoodieLogFile logFile = logFormatReaderWrapper.getLogFile(); log.info("Scanning log file " + logFile); + scannedLogFiles.add(logFile); + totalLogFiles.set(scannedLogFiles.size()); // Use the HoodieLogFileReader to iterate through the blocks in the log file HoodieLogBlock r = logFormatReaderWrapper.next(); + totalLogBlocks.incrementAndGet(); if (r.getBlockType() != CORRUPT_BLOCK && !HoodieTimeline.compareTimestamps(r.getLogBlockHeader().get(INSTANT_TIME), this.latestInstantTime, @@ -134,7 +143,7 @@ public class HoodieCompactedLogRecordScanner implements if (isNewInstantBlock(r) && !readBlocksLazily) { // If this is an avro data block belonging to a different commit/instant, // then merge the last blocks and records into the main result - merge(records, currentInstantLogBlocks); + processQueuedBlocksForInstant(currentInstantLogBlocks, scannedLogFiles.size()); } // store the current block currentInstantLogBlocks.push(r); @@ -144,7 +153,7 @@ public class HoodieCompactedLogRecordScanner implements if (isNewInstantBlock(r) && !readBlocksLazily) { // If this is a delete data block belonging to a different commit/instant, // then merge the last blocks and records into the main result - merge(records, currentInstantLogBlocks); + processQueuedBlocksForInstant(currentInstantLogBlocks, scannedLogFiles.size()); } // store deletes so can be rolled back currentInstantLogBlocks.push(r); @@ -208,7 +217,6 @@ public class HoodieCompactedLogRecordScanner implements break; default: throw new UnsupportedOperationException("Command type not yet supported."); - } break; case CORRUPT_BLOCK: @@ -224,19 +232,14 @@ public class HoodieCompactedLogRecordScanner implements // merge the last read block when all the blocks are done reading if (!currentInstantLogBlocks.isEmpty()) { log.info("Merging the final data blocks"); - merge(records, currentInstantLogBlocks); + processQueuedBlocksForInstant(currentInstantLogBlocks, scannedLogFiles.size()); } - } catch (IOException e) { + // Done + progress = 1.0f; + } catch (Exception e) { + log.error("Got exception when reading log file", e); throw new HoodieIOException("IOException when reading log file "); } - this.totalRecordsToUpdate = records.size(); - this.totalTimeTakenToReadAndMergeBlocks = timer.endTimer(); - log.info("MaxMemoryInBytes allowed for compaction => " + maxMemorySizeInBytes); - log.info("Number of entries in MemoryBasedMap in ExternalSpillableMap => " + records.getInMemoryMapNumEntries()); - log.info("Total size in bytes of MemoryBasedMap in ExternalSpillableMap => " + records.getCurrentInMemoryMapSize()); - log.info("Number of entries in DiskBasedMap in ExternalSpillableMap => " + records.getDiskBasedMapNumEntries()); - log.info("Size of file spilled to disk => " + records.getSizeOfFileOnDiskInBytes()); - log.debug("Total time taken for scanning and compacting log files => " + totalTimeTakenToReadAndMergeBlocks); } /** @@ -250,66 +253,69 @@ public class HoodieCompactedLogRecordScanner implements } /** - * Iterate over the GenericRecord in the block, read the hoodie key and partition path and merge with the application - * specific payload if the same key was found before. Sufficient to just merge the log records since the base data is - * merged on previous compaction. Finally, merge this log block with the accumulated records + * Iterate over the GenericRecord in the block, read the hoodie key and partition path and + * call subclass processors to handle it. */ - private Map> merge( - HoodieAvroDataBlock dataBlock) throws IOException { - // TODO (NA) - Implemnt getRecordItr() in HoodieAvroDataBlock and use that here + private void processAvroDataBlock(HoodieAvroDataBlock dataBlock) throws Exception { + // TODO (NA) - Implement getRecordItr() in HoodieAvroDataBlock and use that here List recs = dataBlock.getRecords(); totalLogRecords.addAndGet(recs.size()); - recs.forEach(rec -> { - String key = ((GenericRecord) rec).get(HoodieRecord.RECORD_KEY_METADATA_FIELD) - .toString(); + for (IndexedRecord rec : recs) { HoodieRecord hoodieRecord = SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, this.payloadClassFQN); - if (records.containsKey(key)) { - // Merge and store the merged record - HoodieRecordPayload combinedValue = records.get(key).getData() - .preCombine(hoodieRecord.getData()); - records - .put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), - combinedValue)); - } else { - // Put the record as is - records.put(key, hoodieRecord); - } - }); - return records; + processNextRecord(hoodieRecord); + } } /** - * Merge the last seen log blocks with the accumulated records + * Process next record + * + * @param hoodieRecord Hoodie Record to process */ - private void merge(Map> records, - Deque lastBlocks) throws IOException { + protected abstract void processNextRecord(HoodieRecord hoodieRecord) + throws Exception; + + /** + * Process next deleted key + * + * @param key Deleted record key + */ + protected abstract void processNextDeletedKey(String key); + + /** + * Process the set of log blocks belonging to the last instant which is read fully. + */ + private void processQueuedBlocksForInstant(Deque lastBlocks, int numLogFilesSeen) + throws Exception { while (!lastBlocks.isEmpty()) { log.info("Number of remaining logblocks to merge " + lastBlocks.size()); // poll the element at the bottom of the stack since that's the order it was inserted HoodieLogBlock lastBlock = lastBlocks.pollLast(); switch (lastBlock.getBlockType()) { case AVRO_DATA_BLOCK: - merge((HoodieAvroDataBlock) lastBlock); + processAvroDataBlock((HoodieAvroDataBlock) lastBlock); break; case DELETE_BLOCK: // TODO : If delete is the only block written and/or records are present in parquet file // TODO : Mark as tombstone (optional.empty()) for data instead of deleting the entry - Arrays.stream(((HoodieDeleteBlock) lastBlock).getKeysToDelete()).forEach(records::remove); + Arrays.stream(((HoodieDeleteBlock) lastBlock).getKeysToDelete()).forEach(this::processNextDeletedKey); break; case CORRUPT_BLOCK: log.warn("Found a corrupt block which was not rolled back"); break; default: - //TODO : Need to understand if COMMAND_BLOCK has to be handled? break; } } + // At this step the lastBlocks are consumed. We track approximate progress by number of log-files seen + progress = numLogFilesSeen - 1 / logFilePaths.size(); } - @Override - public Iterator> iterator() { - return records.iterator(); + /** + * Return progress of scanning as a float between 0.0 to 1.0 + */ + public float getProgress() { + return progress; } public long getTotalLogFiles() { @@ -324,12 +330,8 @@ public class HoodieCompactedLogRecordScanner implements return totalLogBlocks.get(); } - public Map> getRecords() { - return records; - } - - public long getTotalRecordsToUpdate() { - return totalRecordsToUpdate; + protected String getPayloadClassFQN() { + return payloadClassFQN; } public long getTotalRollbacks() { @@ -339,9 +341,4 @@ public class HoodieCompactedLogRecordScanner implements public long getTotalCorruptBlocks() { return totalCorruptBlocks.get(); } - - public long getTotalTimeTakenToReadAndMergeBlocks() { - return totalTimeTakenToReadAndMergeBlocks; - } } - diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieMergedLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieMergedLogRecordScanner.java new file mode 100644 index 000000000..ed9766069 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieMergedLogRecordScanner.java @@ -0,0 +1,131 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * 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 com.uber.hoodie.common.table.log; + +import com.uber.hoodie.common.model.HoodieKey; +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.util.DefaultSizeEstimator; +import com.uber.hoodie.common.util.HoodieRecordSizeEstimator; +import com.uber.hoodie.common.util.HoodieTimer; +import com.uber.hoodie.common.util.collection.ExternalSpillableMap; +import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; +import com.uber.hoodie.common.util.collection.converter.StringConverter; +import com.uber.hoodie.exception.HoodieIOException; +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +/** + * Scans through all the blocks in a list of HoodieLogFile and builds up a compacted/merged list of records which will + * be used as a lookup table when merging the base columnar file with the redo log file. + * + * NOTE: If readBlockLazily is + * turned on, does not merge, instead keeps reading log blocks and merges everything at once This is an optimization to + * avoid seek() back and forth to read new block (forward seek()) and lazily read content of seen block (reverse and + * forward seek()) during merge | | Read Block 1 Metadata | | Read Block 1 Data | | | Read Block 2 + * Metadata | | Read Block 2 Data | | I/O Pass 1 | ..................... | I/O Pass 2 | ................. | | + * | Read Block N Metadata | | Read Block N Data |

This results in two I/O passes over the log file. + */ + +public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordScanner + implements Iterable> { + + private static final Logger log = LogManager.getLogger(HoodieMergedLogRecordScanner.class); + + // Final map of compacted/merged records + private final ExternalSpillableMap> records; + + // count of merged records in log + private long numMergedRecordsInLog; + + // Stores the total time taken to perform reading and merging of log blocks + private final long totalTimeTakenToReadAndMergeBlocks; + // A timer for calculating elapsed time in millis + public final HoodieTimer timer = new HoodieTimer(); + + @SuppressWarnings("unchecked") + public HoodieMergedLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, + Schema readerSchema, String latestInstantTime, Long maxMemorySizeInBytes, + boolean readBlocksLazily, boolean reverseReader, int bufferSize, String spillableMapBasePath) { + super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize); + try { + // Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize + this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath, + new StringConverter(), new HoodieRecordConverter(readerSchema, getPayloadClassFQN()), + new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(readerSchema)); + // Do the scan and merge + timer.startTimer(); + scan(); + this.totalTimeTakenToReadAndMergeBlocks = timer.endTimer(); + this.numMergedRecordsInLog = records.size(); + log.info("MaxMemoryInBytes allowed for compaction => " + maxMemorySizeInBytes); + log.info("Number of entries in MemoryBasedMap in ExternalSpillableMap => " + records + .getInMemoryMapNumEntries()); + log.info("Total size in bytes of MemoryBasedMap in ExternalSpillableMap => " + records + .getCurrentInMemoryMapSize()); + log.info("Number of entries in DiskBasedMap in ExternalSpillableMap => " + records + .getDiskBasedMapNumEntries()); + log.info("Size of file spilled to disk => " + records.getSizeOfFileOnDiskInBytes()); + } catch (IOException e) { + throw new HoodieIOException("IOException when reading log file "); + } + } + + @Override + public Iterator> iterator() { + return records.iterator(); + } + + public Map> getRecords() { + return records; + } + + public long getNumMergedRecordsInLog() { + return numMergedRecordsInLog; + } + + @Override + protected void processNextRecord(HoodieRecord hoodieRecord) { + String key = hoodieRecord.getRecordKey(); + if (records.containsKey(key)) { + // Merge and store the merged record + HoodieRecordPayload combinedValue = records.get(key).getData().preCombine(hoodieRecord.getData()); + records.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue)); + } else { + // Put the record as is + records.put(key, hoodieRecord); + } + } + + @Override + protected void processNextDeletedKey(String key) { + // TODO : If delete is the only block written and/or records are present in parquet file + // TODO : Mark as tombstone (optional.empty()) for data instead of deleting the entry + records.remove(key); + } + + public long getTotalTimeTakenToReadAndMergeBlocks() { + return totalTimeTakenToReadAndMergeBlocks; + } +} + diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieUnMergedLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieUnMergedLogRecordScanner.java new file mode 100644 index 000000000..98264352b --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieUnMergedLogRecordScanner.java @@ -0,0 +1,55 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * 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 com.uber.hoodie.common.table.log; + +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieRecordPayload; +import java.util.List; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; + +public class HoodieUnMergedLogRecordScanner extends AbstractHoodieLogRecordScanner { + + private final LogRecordScannerCallback callback; + + public HoodieUnMergedLogRecordScanner(FileSystem fs, String basePath, + List logFilePaths, Schema readerSchema, String latestInstantTime, + boolean readBlocksLazily, boolean reverseReader, int bufferSize, + LogRecordScannerCallback callback) { + super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize); + this.callback = callback; + } + + @Override + protected void processNextRecord(HoodieRecord hoodieRecord) throws Exception { + // Just call callback without merging + callback.apply(hoodieRecord); + } + + @Override + protected void processNextDeletedKey(String key) { + throw new IllegalStateException("Not expected to see delete records in this log-scan mode. Check Job Config"); + } + + @FunctionalInterface + public static interface LogRecordScannerCallback { + + public void apply(HoodieRecord record) throws Exception; + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java index 86b3f2698..17475155e 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java @@ -18,6 +18,7 @@ package com.uber.hoodie.common.table.log.block; import com.google.common.collect.Maps; import com.uber.hoodie.common.model.HoodieLogFile; +import com.uber.hoodie.common.table.log.HoodieMergedLogRecordScanner; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieIOException; import java.io.ByteArrayOutputStream; @@ -219,7 +220,7 @@ public abstract class HoodieLogBlock { /** * Read or Skip block content of a log block in the log file. Depends on lazy reading enabled in - * {@link com.uber.hoodie.common.table.log.HoodieCompactedLogRecordScanner} + * {@link HoodieMergedLogRecordScanner} */ public static byte[] readOrSkipContent(FSDataInputStream inputStream, Integer contentLength, boolean readBlockLazily) throws IOException { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/payload/GenericRecordBufferedIteratorPayload.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/DefaultSizeEstimator.java similarity index 55% rename from hoodie-client/src/main/java/com/uber/hoodie/func/payload/GenericRecordBufferedIteratorPayload.java rename to hoodie-common/src/main/java/com/uber/hoodie/common/util/DefaultSizeEstimator.java index 9d934d9ea..91145d08b 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/payload/GenericRecordBufferedIteratorPayload.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/DefaultSizeEstimator.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -14,18 +14,18 @@ * limitations under the License. */ -package com.uber.hoodie.func.payload; +package com.uber.hoodie.common.util; -import org.apache.avro.generic.GenericRecord; +import com.twitter.common.objectsize.ObjectSizeCalculator; /** - * BufferedIteratorPayload that takes GenericRecord as input and GenericRecord as output + * Default implementation of size-estimator that uses Twitter's ObjectSizeCalculator + * @param */ -public class GenericRecordBufferedIteratorPayload - extends AbstractBufferedIteratorPayload { +public class DefaultSizeEstimator implements SizeEstimator { - public GenericRecordBufferedIteratorPayload(GenericRecord record) { - super(record); - this.outputPayload = record; + @Override + public long sizeEstimate(T t) { + return ObjectSizeCalculator.getObjectSize(t); } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieRecordSizeEstimator.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieRecordSizeEstimator.java new file mode 100644 index 000000000..72bcc4fcd --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieRecordSizeEstimator.java @@ -0,0 +1,52 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * 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 com.uber.hoodie.common.util; + +import com.twitter.common.objectsize.ObjectSizeCalculator; +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieRecordPayload; +import org.apache.avro.Schema; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +/** + * Size Estimator for Hoodie record payload + * @param + */ +public class HoodieRecordSizeEstimator implements SizeEstimator> { + + private static Logger log = LogManager.getLogger(HoodieRecordSizeEstimator.class); + + // Schema used to get GenericRecord from HoodieRecordPayload then convert to bytes and vice-versa + private final Schema schema; + + public HoodieRecordSizeEstimator(Schema schema) { + this.schema = schema; + } + + @Override + public long sizeEstimate(HoodieRecord hoodieRecord) { + // Most HoodieRecords are bound to have data + schema. Although, the same schema object is shared amongst + // all records in the JVM. Calculate and print the size of the Schema and of the Record to + // note the sizes and differences. A correct estimation in such cases is handled in + /** {@link com.uber.hoodie.common.util.collection.ExternalSpillableMap} **/ + long sizeOfRecord = ObjectSizeCalculator.getObjectSize(hoodieRecord); + long sizeOfSchema = ObjectSizeCalculator.getObjectSize(schema); + log.info("SizeOfRecord => " + sizeOfRecord + " SizeOfSchema => " + sizeOfSchema); + return sizeOfRecord; + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/SizeEstimator.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/SizeEstimator.java new file mode 100644 index 000000000..6a6f597bf --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/SizeEstimator.java @@ -0,0 +1,31 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * 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 com.uber.hoodie.common.util; + +/** + * An interface to estimate the size of payload in memory + * @param + */ +public interface SizeEstimator { + + /** + * This method is used to estimate the size of a payload in memory. + * The default implementation returns the total allocated size, in bytes, of the object + * and all other objects reachable from it + */ + long sizeEstimate(T t); +} \ No newline at end of file diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java index 94aa1758d..79aa90ee6 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java @@ -20,7 +20,6 @@ import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.util.collection.DiskBasedMap; -import com.uber.hoodie.common.util.collection.converter.Converter; import com.uber.hoodie.common.util.collection.io.storage.SizeAwareDataOutputStream; import com.uber.hoodie.exception.HoodieCorruptedDataException; import java.io.IOException; @@ -99,8 +98,8 @@ public class SpillableMapUtils { * Compute a bytes representation of the payload by serializing the contents This is used to estimate the size of the * payload (either in memory or when written to disk) */ - public static long computePayloadSize(R value, Converter valueConverter) throws IOException { - return valueConverter.sizeEstimate(value); + public static long computePayloadSize(R value, SizeEstimator valueSizeEstimator) throws IOException { + return valueSizeEstimator.sizeEstimate(value); } /** diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java index e061bf444..081a889e7 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java @@ -17,6 +17,7 @@ package com.uber.hoodie.common.util.collection; import com.twitter.common.objectsize.ObjectSizeCalculator; +import com.uber.hoodie.common.util.SizeEstimator; import com.uber.hoodie.common.util.collection.converter.Converter; import com.uber.hoodie.exception.HoodieNotSupportedException; import java.io.IOException; @@ -56,6 +57,10 @@ public class ExternalSpillableMap implements Map { private final Converter keyConverter; // Value converter to convert value type to bytes private final Converter valueConverter; + // Size Estimator for key type + private final SizeEstimator keySizeEstimator; + // Size Estimator for key types + private final SizeEstimator valueSizeEstimator; // current space occupied by this map in-memory private Long currentInMemoryMapSize; // An estimate of the size of each payload written to this map @@ -64,7 +69,8 @@ public class ExternalSpillableMap implements Map { private boolean shouldEstimatePayloadSize = true; public ExternalSpillableMap(Long maxInMemorySizeInBytes, String baseFilePath, - Converter keyConverter, Converter valueConverter) throws IOException { + Converter keyConverter, Converter valueConverter, + SizeEstimator keySizeEstimator, SizeEstimator valueSizeEstimator) throws IOException { this.inMemoryMap = new HashMap<>(); this.diskBasedMap = new DiskBasedMap<>(baseFilePath, keyConverter, valueConverter); this.maxInMemorySizeInBytes = (long) Math @@ -72,6 +78,8 @@ public class ExternalSpillableMap implements Map { this.currentInMemoryMapSize = 0L; this.keyConverter = keyConverter; this.valueConverter = valueConverter; + this.keySizeEstimator = keySizeEstimator; + this.valueSizeEstimator = valueSizeEstimator; } /** @@ -146,7 +154,7 @@ public class ExternalSpillableMap implements Map { // At first, use the sizeEstimate of a record being inserted into the spillable map. // Note, the converter may over estimate the size of a record in the JVM this.estimatedPayloadSize = - keyConverter.sizeEstimate(key) + valueConverter.sizeEstimate(value); + keySizeEstimator.sizeEstimate(key) + valueSizeEstimator.sizeEstimate(value); log.info("Estimated Payload size => " + estimatedPayloadSize); } else if (shouldEstimatePayloadSize && inMemoryMap.size() % NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE == 0) { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/Converter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/Converter.java index b06651973..55168baa9 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/Converter.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/Converter.java @@ -31,9 +31,4 @@ public interface Converter { * This method is used to convert the serialized payload (in bytes) to the actual payload instance */ T getData(byte[] bytes); - - /** - * This method is used to estimate the size of a payload in memory - */ - long sizeEstimate(T t); } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java index 88bf7fcd0..ee6c90d9c 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java @@ -16,7 +16,6 @@ package com.uber.hoodie.common.util.collection.converter; -import com.twitter.common.objectsize.ObjectSizeCalculator; import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; @@ -87,16 +86,4 @@ public class HoodieRecordConverter implements throw new HoodieNotSerializableException("Cannot de-serialize value from bytes", io); } } - - @Override - public long sizeEstimate(HoodieRecord hoodieRecord) { - // Most HoodieRecords are bound to have data + schema. Although, the same schema object is shared amongst - // all records in the JVM. Calculate and print the size of the Schema and of the Record to - // note the sizes and differences. A correct estimation in such cases is handled in - /** {@link com.uber.hoodie.common.util.collection.ExternalSpillableMap} **/ - long sizeOfRecord = ObjectSizeCalculator.getObjectSize(hoodieRecord); - long sizeOfSchema = ObjectSizeCalculator.getObjectSize(schema); - log.info("SizeOfRecord => " + sizeOfRecord + " SizeOfSchema => " + sizeOfSchema); - return sizeOfRecord; - } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/StringConverter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/StringConverter.java index ea84a7dd9..7855484db 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/StringConverter.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/StringConverter.java @@ -16,7 +16,6 @@ package com.uber.hoodie.common.util.collection.converter; -import com.twitter.common.objectsize.ObjectSizeCalculator; import java.nio.charset.StandardCharsets; /** @@ -33,9 +32,4 @@ public class StringConverter implements Converter { public String getData(byte[] bytes) { return new String(bytes); } - - @Override - public long sizeEstimate(String s) { - return ObjectSizeCalculator.getObjectSize(s); - } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryExecutor.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryExecutor.java new file mode 100644 index 000000000..1e7664f48 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryExecutor.java @@ -0,0 +1,162 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * 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 com.uber.hoodie.common.util.queue; + +import com.uber.hoodie.common.util.DefaultSizeEstimator; +import com.uber.hoodie.common.util.SizeEstimator; +import com.uber.hoodie.exception.HoodieException; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.commons.lang3.concurrent.ConcurrentUtils; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +/** + * Executor which orchestrates concurrent producers and consumers communicating through a bounded in-memory queue. + * This class takes as input the size limit, queue producer(s), consumer and transformer + * and exposes API to orchestrate concurrent execution of these actors communicating through a central bounded queue + */ +public class BoundedInMemoryExecutor { + + private static Logger logger = LogManager.getLogger(BoundedInMemoryExecutor.class); + + // Executor service used for launching writer thread. + private final ExecutorService executorService; + // Used for buffering records which is controlled by HoodieWriteConfig#WRITE_BUFFER_LIMIT_BYTES. + private final BoundedInMemoryQueue queue; + // Producers + private final List> producers; + // Consumer + private final Optional> consumer; + + public BoundedInMemoryExecutor(final long bufferLimitInBytes, + BoundedInMemoryQueueProducer producer, + Optional> consumer, + final Function transformFunction) { + this(bufferLimitInBytes, Arrays.asList(producer), consumer, transformFunction, new DefaultSizeEstimator<>()); + } + + public BoundedInMemoryExecutor(final long bufferLimitInBytes, + List> producers, + Optional> consumer, + final Function transformFunction, + final SizeEstimator sizeEstimator) { + this.producers = producers; + this.consumer = consumer; + // Ensure single thread for each producer thread and one for consumer + this.executorService = Executors.newFixedThreadPool(producers.size() + 1); + this.queue = new BoundedInMemoryQueue<>(bufferLimitInBytes, transformFunction, sizeEstimator); + } + + /** + * Callback to implement environment specific behavior before executors (producers/consumer) + * run. + */ + public void preExecute() { + // Do Nothing in general context + } + + /** + * Start all Producers + */ + public ExecutorCompletionService startProducers() { + // Latch to control when and which producer thread will close the queue + final CountDownLatch latch = new CountDownLatch(producers.size()); + final ExecutorCompletionService completionService = + new ExecutorCompletionService(executorService); + producers.stream().map(producer -> { + return completionService.submit(() -> { + try { + preExecute(); + producer.produce(queue); + } catch (Exception e) { + logger.error("error consuming records", e); + queue.markAsFailed(e); + throw e; + } finally { + synchronized (latch) { + latch.countDown(); + if (latch.getCount() == 0) { + // Mark production as done so that consumer will be able to exit + queue.close(); + } + } + } + return true; + }); + }).collect(Collectors.toList()); + return completionService; + } + + /** + * Start only consumer + */ + private Future startConsumer() { + return consumer.map(consumer -> { + return executorService.submit( + () -> { + logger.info("starting consumer thread"); + preExecute(); + try { + E result = consumer.consume(queue); + logger.info("Queue Consumption is done; notifying producer threads"); + return result; + } catch (Exception e) { + logger.error("error consuming records", e); + queue.markAsFailed(e); + throw e; + } + }); + }).orElse(ConcurrentUtils.constantFuture(null)); + } + + /** + * Main API to run both production and consumption + */ + public E execute() { + try { + ExecutorCompletionService producerService = startProducers(); + Future future = startConsumer(); + // Wait for consumer to be done + return future.get(); + } catch (Exception e) { + throw new HoodieException(e); + } + } + + + public boolean isRemaining() { + return queue.iterator().hasNext(); + } + + public void shutdownNow() { + executorService.shutdownNow(); + } + + public BoundedInMemoryQueue getQueue() { + return queue; + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueue.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueue.java new file mode 100644 index 000000000..401924e96 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueue.java @@ -0,0 +1,273 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * 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 com.uber.hoodie.common.util.queue; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.uber.hoodie.common.util.DefaultSizeEstimator; +import com.uber.hoodie.common.util.SizeEstimator; +import com.uber.hoodie.exception.HoodieException; +import java.util.Iterator; +import java.util.Optional; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +/** + * Used for enqueueing input records. Queue limit is controlled by {@link #memoryLimit}. + * Unlike standard bounded queue implementations, this queue bounds the size by memory bytes occupied by its + * tenants. The standard implementation bounds by the number of entries in the queue. + * + * It internally samples every {@link #RECORD_SAMPLING_RATE}th record and adjusts number of records in + * queue accordingly. This is done to ensure that we don't OOM. + * + * This queue supports multiple producer single consumer pattern. + * + * @param input payload data type + * @param output payload data type + */ +public class BoundedInMemoryQueue implements Iterable { + + // interval used for polling records in the queue. + public static final int RECORD_POLL_INTERVAL_SEC = 1; + // rate used for sampling records to determine avg record size in bytes. + public static final int RECORD_SAMPLING_RATE = 64; + // maximum records that will be cached + private static final int RECORD_CACHING_LIMIT = 128 * 1024; + private static Logger logger = LogManager.getLogger(BoundedInMemoryQueue.class); + // It indicates number of records to cache. We will be using sampled record's average size to + // determine how many + // records we should cache and will change (increase/decrease) permits accordingly. + @VisibleForTesting + public final Semaphore rateLimiter = new Semaphore(1); + // used for sampling records with "RECORD_SAMPLING_RATE" frequency. + public final AtomicLong samplingRecordCounter = new AtomicLong(-1); + // internal queue for records. + private final LinkedBlockingQueue> queue = new + LinkedBlockingQueue<>(); + // maximum amount of memory to be used for queueing records. + private final long memoryLimit; + // it holds the root cause of the exception in case either queueing records (consuming from + // inputIterator) fails or + // thread reading records from queue fails. + private final AtomicReference hasFailed = new AtomicReference(null); + // used for indicating that all the records from queue are read successfully. + private final AtomicBoolean isReadDone = new AtomicBoolean(false); + // used for indicating that all records have been enqueued + private final AtomicBoolean isWriteDone = new AtomicBoolean(false); + // Function to transform the input payload to the expected output payload + private final Function transformFunction; + // Payload Size Estimator + private final SizeEstimator payloadSizeEstimator; + // Singleton (w.r.t this instance) Iterator for this queue + private final QueueIterator iterator; + // indicates rate limit (number of records to cache). it is updated whenever there is a change + // in avg record size. + @VisibleForTesting + public int currentRateLimit = 1; + // indicates avg record size in bytes. It is updated whenever a new record is sampled. + @VisibleForTesting + public long avgRecordSizeInBytes = 0; + // indicates number of samples collected so far. + private long numSamples = 0; + + /** + * Construct BoundedInMemoryQueue with default SizeEstimator + * + * @param memoryLimit MemoryLimit in bytes + * @param transformFunction Transformer Function to convert input payload type to stored payload type + */ + public BoundedInMemoryQueue(final long memoryLimit, final Function transformFunction) { + this(memoryLimit, transformFunction, new DefaultSizeEstimator() { + }); + } + + /** + * Construct BoundedInMemoryQueue with passed in size estimator + * + * @param memoryLimit MemoryLimit in bytes + * @param transformFunction Transformer Function to convert input payload type to stored payload type + * @param payloadSizeEstimator Payload Size Estimator + */ + public BoundedInMemoryQueue( + final long memoryLimit, + final Function transformFunction, + final SizeEstimator payloadSizeEstimator) { + this.memoryLimit = memoryLimit; + this.transformFunction = transformFunction; + this.payloadSizeEstimator = payloadSizeEstimator; + this.iterator = new QueueIterator(); + } + + @VisibleForTesting + public int size() { + return this.queue.size(); + } + + /** + * Samples records with "RECORD_SAMPLING_RATE" frequency and computes average record size in bytes. It is used + * for determining how many maximum records to queue. Based on change in avg size it ma increase or decrease + * available permits. + * + * @param payload Payload to size + */ + private void adjustBufferSizeIfNeeded(final O payload) throws InterruptedException { + if (this.samplingRecordCounter.incrementAndGet() % RECORD_SAMPLING_RATE != 0) { + return; + } + + final long recordSizeInBytes = payloadSizeEstimator.sizeEstimate(payload); + final long newAvgRecordSizeInBytes = Math + .max(1, (avgRecordSizeInBytes * numSamples + recordSizeInBytes) / (numSamples + 1)); + final int newRateLimit = (int) Math + .min(RECORD_CACHING_LIMIT, Math.max(1, this.memoryLimit / newAvgRecordSizeInBytes)); + + // If there is any change in number of records to cache then we will either release (if it increased) or acquire + // (if it decreased) to adjust rate limiting to newly computed value. + if (newRateLimit > currentRateLimit) { + rateLimiter.release(newRateLimit - currentRateLimit); + } else if (newRateLimit < currentRateLimit) { + rateLimiter.acquire(currentRateLimit - newRateLimit); + } + currentRateLimit = newRateLimit; + avgRecordSizeInBytes = newAvgRecordSizeInBytes; + numSamples++; + } + + /** + * Inserts record into queue after applying transformation + * + * @param t Item to be queueed + */ + public void insertRecord(I t) throws Exception { + // If already closed, throw exception + if (isWriteDone.get()) { + throw new IllegalStateException("Queue closed for enqueueing new entries"); + } + + // We need to stop queueing if queue-reader has failed and exited. + throwExceptionIfFailed(); + + rateLimiter.acquire(); + // We are retrieving insert value in the record queueing thread to offload computation + // around schema validation + // and record creation to it. + final O payload = transformFunction.apply(t); + adjustBufferSizeIfNeeded(payload); + queue.put(Optional.of(payload)); + } + + /** + * Checks if records are either available in the queue or expected to be written in future + */ + private boolean expectMoreRecords() { + return !isWriteDone.get() || (isWriteDone.get() && !queue.isEmpty()); + } + + /** + * Reader interface but never exposed to outside world as this is a single consumer queue. + * Reading is done through a singleton iterator for this queue. + */ + private Optional readNextRecord() { + if (this.isReadDone.get()) { + return Optional.empty(); + } + + rateLimiter.release(); + Optional newRecord = Optional.empty(); + while (expectMoreRecords()) { + try { + throwExceptionIfFailed(); + newRecord = queue.poll(RECORD_POLL_INTERVAL_SEC, TimeUnit.SECONDS); + if (newRecord != null) { + break; + } + } catch (InterruptedException e) { + logger.error("error reading records from queue", e); + throw new HoodieException(e); + } + } + if (newRecord != null && newRecord.isPresent()) { + return newRecord; + } else { + // We are done reading all the records from internal iterator. + this.isReadDone.set(true); + return Optional.empty(); + } + } + + /** + * Puts an empty entry to queue to denote termination + */ + public void close() throws InterruptedException { + // done queueing records notifying queue-reader. + isWriteDone.set(true); + } + + private void throwExceptionIfFailed() { + if (this.hasFailed.get() != null) { + throw new HoodieException("operation has failed", this.hasFailed.get()); + } + } + + /** + * API to allow producers and consumer to communicate termination due to failure + */ + public void markAsFailed(Exception e) { + this.hasFailed.set(e); + // release the permits so that if the queueing thread is waiting for permits then it will + // get it. + this.rateLimiter.release(RECORD_CACHING_LIMIT + 1); + } + + @Override + public Iterator iterator() { + return iterator; + } + + /** + * Iterator for the memory bounded queue + */ + private final class QueueIterator implements Iterator { + + // next record to be read from queue. + private O nextRecord; + + @Override + public boolean hasNext() { + if (this.nextRecord == null) { + Optional res = readNextRecord(); + this.nextRecord = res.orElse(null); + } + return this.nextRecord != null; + } + + @Override + public O next() { + Preconditions.checkState(hasNext() && this.nextRecord != null); + final O ret = this.nextRecord; + this.nextRecord = null; + return ret; + } + } +} \ No newline at end of file diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueueConsumer.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueueConsumer.java new file mode 100644 index 000000000..e1f985c2c --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueueConsumer.java @@ -0,0 +1,63 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * 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 com.uber.hoodie.common.util.queue; + +import java.util.Iterator; + + +/** + * Consume entries from queue and execute callback function + */ +public abstract class BoundedInMemoryQueueConsumer { + + /** + * API to de-queue entries to memory bounded queue + * + * @param queue In Memory bounded queue + */ + public O consume(BoundedInMemoryQueue queue) throws Exception { + Iterator iterator = queue.iterator(); + + while (iterator.hasNext()) { + consumeOneRecord(iterator.next()); + } + + // Notifies done + finish(); + + return getResult(); + } + + /** + * Consumer One record + */ + protected abstract void consumeOneRecord(I record); + + /** + * Notifies implementation that we have exhausted consuming records from queue + */ + protected abstract void finish(); + + /** + * Return result of consuming records so far + */ + protected abstract O getResult(); + + +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueueProducer.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueueProducer.java new file mode 100644 index 000000000..b7d918158 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueueProducer.java @@ -0,0 +1,35 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * 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 com.uber.hoodie.common.util.queue; + +/** + * Producer for BoundedInMemoryQueue. Memory Bounded Buffer supports + * multiple producers single consumer pattern. + * + * @param Input type for buffer items produced + */ +public interface BoundedInMemoryQueueProducer { + + /** + * API to enqueue entries to memory bounded queue + * + * @param queue In Memory bounded queue + */ + void produce(BoundedInMemoryQueue queue) throws Exception; +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/FunctionBasedQueueProducer.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/FunctionBasedQueueProducer.java new file mode 100644 index 000000000..3d4d0cee5 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/FunctionBasedQueueProducer.java @@ -0,0 +1,46 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * 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 com.uber.hoodie.common.util.queue; + +import java.util.function.Function; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +/** + * Buffer producer which allows custom functions to insert entries to queue. + * + * @param Type of entry produced for queue + */ +public class FunctionBasedQueueProducer implements BoundedInMemoryQueueProducer { + + private static final Logger logger = LogManager.getLogger(FunctionBasedQueueProducer.class); + + private final Function, Boolean> producerFunction; + + public FunctionBasedQueueProducer(Function, Boolean> producerFunction) { + this.producerFunction = producerFunction; + } + + @Override + public void produce(BoundedInMemoryQueue queue) { + logger.info("starting function which will enqueue records"); + producerFunction.apply(queue); + logger.info("finished function which will enqueue records"); + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/IteratorBasedQueueProducer.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/IteratorBasedQueueProducer.java new file mode 100644 index 000000000..f8161199e --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/IteratorBasedQueueProducer.java @@ -0,0 +1,49 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * 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 com.uber.hoodie.common.util.queue; + +import java.util.Iterator; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +/** + * Iterator based producer which pulls entry from iterator and produces items for the queue + * + * @param Item type produced for the buffer. + */ +public class IteratorBasedQueueProducer implements BoundedInMemoryQueueProducer { + + private static final Logger logger = LogManager.getLogger(IteratorBasedQueueProducer.class); + + // input iterator for producing items in the buffer. + private final Iterator inputIterator; + + public IteratorBasedQueueProducer(Iterator inputIterator) { + this.inputIterator = inputIterator; + } + + @Override + public void produce(BoundedInMemoryQueue queue) throws Exception { + logger.info("starting to buffer records"); + while (inputIterator.hasNext()) { + queue.insertRecord(inputIterator.next()); + } + logger.info("finished buffering records"); + } +} diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java index 192e6a51d..dcc2c4996 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java @@ -73,12 +73,11 @@ import org.junit.runners.Parameterized; @RunWith(Parameterized.class) public class HoodieLogFormatTest { + private static final String BASE_OUTPUT_PATH = "/tmp/"; + private static String basePath; private FileSystem fs; private Path partitionPath; - private static String basePath; private int bufferSize = 4096; - private static final String BASE_OUTPUT_PATH = "/tmp/"; - private Boolean readBlocksLazily = true; public HoodieLogFormatTest(Boolean readBlocksLazily) { @@ -87,7 +86,7 @@ public class HoodieLogFormatTest { @Parameterized.Parameters(name = "LogBlockReadMode") public static Collection data() { - return Arrays.asList(new Boolean[][] {{true}, {false}}); + return Arrays.asList(new Boolean[][]{{true}, {false}}); } @BeforeClass @@ -400,7 +399,7 @@ public class HoodieLogFormatTest { writer.close(); // scan all log blocks (across multiple log files) - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, logFiles.stream().map(logFile -> logFile.getPath().toString()).collect(Collectors.toList()), schema, "100", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); @@ -527,7 +526,7 @@ public class HoodieLogFormatTest { List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "100", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("", 200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); @@ -587,7 +586,7 @@ public class HoodieLogFormatTest { List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "102", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We read 200 records from 2 write batches", 200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); @@ -665,7 +664,7 @@ public class HoodieLogFormatTest { List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "103", 10240L, true, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We would read 200 records", 200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); @@ -719,7 +718,7 @@ public class HoodieLogFormatTest { List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "102", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We still would read 200 records", 200, scanner.getTotalLogRecords()); final List readKeys = new ArrayList<>(200); @@ -739,8 +738,8 @@ public class HoodieLogFormatTest { writer = writer.appendBlock(commandBlock); readKeys.clear(); - scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, "101", 10240L, readBlocksLazily, - false, bufferSize, BASE_OUTPUT_PATH); + scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "101", + 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 200 records after rollback of delete", 200, readKeys.size()); } @@ -800,7 +799,7 @@ public class HoodieLogFormatTest { "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); // all data must be rolled back before merge - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "100", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We would have scanned 0 records because of rollback", 0, scanner.getTotalLogRecords()); @@ -849,7 +848,7 @@ public class HoodieLogFormatTest { List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "100", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); } @@ -881,7 +880,7 @@ public class HoodieLogFormatTest { List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "100", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We still would read 100 records", 100, scanner.getTotalLogRecords()); final List readKeys = new ArrayList<>(100); @@ -931,7 +930,7 @@ public class HoodieLogFormatTest { List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "101", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); } @@ -1019,7 +1018,7 @@ public class HoodieLogFormatTest { List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "101", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java index fdf6ba7cd..83d20bd2b 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java @@ -27,6 +27,7 @@ import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.util.HoodieAvroUtils; +import com.uber.hoodie.common.util.HoodieRecordSizeEstimator; import com.uber.hoodie.common.util.SchemaTestUtil; import com.uber.hoodie.common.util.SpillableMapTestUtils; import com.uber.hoodie.common.util.SpillableMapUtils; @@ -156,14 +157,14 @@ public class TestDiskBasedMap { List hoodieRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1, schema); long payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0), - new HoodieRecordConverter(schema, HoodieAvroPayload.class.getName())); + new HoodieRecordSizeEstimator(schema)); assertTrue(payloadSize > 0); // Test sizeEstimator with hoodie metadata fields schema = HoodieAvroUtils.addMetadataFields(schema); hoodieRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1, schema); payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0), - new HoodieRecordConverter(schema, HoodieAvroPayload.class.getName())); + new HoodieRecordSizeEstimator(schema)); assertTrue(payloadSize > 0); // Following tests payloads without an Avro Schema in the Record @@ -175,7 +176,7 @@ public class TestDiskBasedMap { .map(r -> new HoodieRecord(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"), new AvroBinaryTestPayload(Optional.of((GenericRecord) r)))).collect(Collectors.toList()); payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0), - new HoodieRecordConverter(schema, AvroBinaryTestPayload.class.getName())); + new HoodieRecordSizeEstimator(schema)); assertTrue(payloadSize > 0); // Test sizeEstimator with hoodie metadata fields and without schema object in the payload @@ -188,7 +189,7 @@ public class TestDiskBasedMap { .of(HoodieAvroUtils.rewriteRecord((GenericRecord) r, simpleSchemaWithMetadata))))) .collect(Collectors.toList()); payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0), - new HoodieRecordConverter(schema, AvroBinaryTestPayload.class.getName())); + new HoodieRecordSizeEstimator(schema)); assertTrue(payloadSize > 0); } @@ -201,11 +202,11 @@ public class TestDiskBasedMap { // Test sizeEstimatorPerformance with simpleSchema Schema schema = SchemaTestUtil.getSimpleSchema(); List hoodieRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1, schema); - HoodieRecordConverter converter = - new HoodieRecordConverter(schema, HoodieAvroPayload.class.getName()); + HoodieRecordSizeEstimator sizeEstimator = + new HoodieRecordSizeEstimator(schema); HoodieRecord record = hoodieRecords.remove(0); long startTime = System.currentTimeMillis(); - SpillableMapUtils.computePayloadSize(record, converter); + SpillableMapUtils.computePayloadSize(record, sizeEstimator); long timeTaken = System.currentTimeMillis() - startTime; System.out.println("Time taken :" + timeTaken); assertTrue(timeTaken < 100); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java index 419cd1b41..7a5239f9d 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java @@ -25,7 +25,9 @@ import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; +import com.uber.hoodie.common.util.DefaultSizeEstimator; import com.uber.hoodie.common.util.HoodieAvroUtils; +import com.uber.hoodie.common.util.HoodieRecordSizeEstimator; import com.uber.hoodie.common.util.SchemaTestUtil; import com.uber.hoodie.common.util.SpillableMapTestUtils; import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; @@ -66,7 +68,8 @@ public class TestExternalSpillableMap { String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz)); //16B + new HoodieRecordConverter(schema, payloadClazz), + new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); @@ -88,7 +91,8 @@ public class TestExternalSpillableMap { ExternalSpillableMap> records = new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz)); //16B + new HoodieRecordConverter(schema, payloadClazz), + new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); @@ -126,7 +130,8 @@ public class TestExternalSpillableMap { ExternalSpillableMap> records = new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz)); //16B + new HoodieRecordConverter(schema, payloadClazz), + new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); // insert a bunch of records so that values spill to disk too @@ -181,7 +186,8 @@ public class TestExternalSpillableMap { ExternalSpillableMap> records = new ExternalSpillableMap<>(16L, FAILURE_OUTPUT_PATH, new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz)); //16B + new HoodieRecordConverter(schema, payloadClazz), + new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); @@ -200,7 +206,8 @@ public class TestExternalSpillableMap { ExternalSpillableMap> records = new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz)); //16B + new HoodieRecordConverter(schema, payloadClazz), + new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B List recordKeys = new ArrayList<>(); // Ensure we spill to disk @@ -253,7 +260,8 @@ public class TestExternalSpillableMap { ExternalSpillableMap> records = new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz)); //16B + new HoodieRecordConverter(schema, payloadClazz), + new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B List recordKeys = new ArrayList<>(); // Ensure we spill to disk diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/RecordReaderValueIterator.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/RecordReaderValueIterator.java new file mode 100644 index 000000000..59a83acef --- /dev/null +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/RecordReaderValueIterator.java @@ -0,0 +1,83 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * 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 com.uber.hoodie.hadoop; + +import com.uber.hoodie.exception.HoodieException; +import java.io.IOException; +import java.util.Iterator; +import java.util.NoSuchElementException; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.mapred.RecordReader; + +/** + * Provides Iterator Interface to iterate value entries read from record reader + * + * @param Key Type + * @param Value Type + */ +public class RecordReaderValueIterator implements Iterator { + + public static final Log LOG = LogFactory.getLog(RecordReaderValueIterator.class); + + private final RecordReader reader; + private V nextVal = null; + + /** + * Construct RecordReaderValueIterator + * + * @param reader reader + */ + public RecordReaderValueIterator(RecordReader reader) { + this.reader = reader; + } + + @Override + public boolean hasNext() { + if (nextVal == null) { + K key = reader.createKey(); + V val = reader.createValue(); + try { + boolean notDone = reader.next(key, val); + if (!notDone) { + return false; + } + this.nextVal = val; + } catch (IOException e) { + LOG.error("Got error reading next record from record reader"); + throw new HoodieException(e); + } + } + return true; + } + + @Override + public V next() { + if (!hasNext()) { + throw new NoSuchElementException("Make sure you are following iterator contract."); + } + V retVal = this.nextVal; + this.nextVal = null; + return retVal; + } + + public void close() throws IOException { + this.reader.close(); + } +} diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/SafeParquetRecordReaderWrapper.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/SafeParquetRecordReaderWrapper.java new file mode 100644 index 000000000..7fb9d67d1 --- /dev/null +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/SafeParquetRecordReaderWrapper.java @@ -0,0 +1,91 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * 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 com.uber.hoodie.hadoop; + +import java.io.IOException; +import org.apache.hadoop.io.ArrayWritable; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.RecordReader; + +/** + * Record Reader for parquet. Records read from this reader is safe to be + * buffered for concurrent processing. + * + * In concurrent producer/consumer pattern, where the record is read and buffered by one thread and processed in + * another thread, we need to ensure new instance of ArrayWritable is buffered. ParquetReader createKey/Value is unsafe + * as it gets reused for subsequent fetch. This wrapper makes ParquetReader safe for this use-case. + */ +public class SafeParquetRecordReaderWrapper implements RecordReader { + + // real Parquet reader to be wrapped + private final RecordReader parquetReader; + + // Value Class + private final Class valueClass; + + // Number of fields in Value Schema + private final int numValueFields; + + + public SafeParquetRecordReaderWrapper(RecordReader parquetReader) { + this.parquetReader = parquetReader; + ArrayWritable arrayWritable = parquetReader.createValue(); + this.valueClass = arrayWritable.getValueClass(); + this.numValueFields = arrayWritable.get().length; + } + + @Override + public boolean next(Void key, ArrayWritable value) throws IOException { + return parquetReader.next(key, value); + } + + @Override + public Void createKey() { + return parquetReader.createKey(); + } + + /** + * We could be in concurrent fetch and read env. + * We need to ensure new ArrayWritable as ParquetReader implementation reuses same + * ArrayWritable for all reads which will cause corruption when buffering. + * So, we create a new ArrayWritable here with Value class from parquetReader's value + * and an empty array. + */ + @Override + public ArrayWritable createValue() { + // Call createValue of parquetReader to get size and class type info only + Writable[] emptyWritableBuf = new Writable[numValueFields]; + return new ArrayWritable(valueClass, emptyWritableBuf); + } + + @Override + public long getPos() throws IOException { + return parquetReader.getPos(); + } + + @Override + public void close() throws IOException { + parquetReader.close(); + } + + @Override + public float getProgress() throws IOException { + return parquetReader.getProgress(); + } +} diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java new file mode 100644 index 000000000..383ce0cd9 --- /dev/null +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/AbstractRealtimeRecordReader.java @@ -0,0 +1,282 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * 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 com.uber.hoodie.hadoop.realtime; + +import com.uber.hoodie.exception.HoodieException; +import com.uber.hoodie.exception.HoodieIOException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.stream.Collectors; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericArray; +import org.apache.avro.generic.GenericFixed; +import org.apache.avro.generic.GenericRecord; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; +import org.apache.hadoop.hive.serde2.io.DoubleWritable; +import org.apache.hadoop.io.ArrayWritable; +import org.apache.hadoop.io.BooleanWritable; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.FloatWritable; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.JobConf; +import parquet.avro.AvroSchemaConverter; +import parquet.hadoop.ParquetFileReader; +import parquet.schema.MessageType; + +/** + * Record Reader implementation to merge fresh avro data with base parquet data, to support real + * time queries. + */ +public abstract class AbstractRealtimeRecordReader { + + // Fraction of mapper/reducer task memory used for compaction of log files + public static final String COMPACTION_MEMORY_FRACTION_PROP = "compaction.memory.fraction"; + public static final String DEFAULT_COMPACTION_MEMORY_FRACTION = "0.75"; + // used to choose a trade off between IO vs Memory when performing compaction process + // Depending on outputfile size and memory provided, choose true to avoid OOM for large file + // size + small memory + public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP = + "compaction.lazy.block.read.enabled"; + public static final String DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED = "true"; + + // Property to set the max memory for dfs inputstream buffer size + public static final String MAX_DFS_STREAM_BUFFER_SIZE_PROP = "hoodie.memory.dfs.buffer.max.size"; + // Setting this to lower value of 1 MB since no control over how many RecordReaders will be started in a mapper + public static final int DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE = 1 * 1024 * 1024; // 1 MB + // Property to set file path prefix for spillable file + public static final String SPILLABLE_MAP_BASE_PATH_PROP = "hoodie.memory.spillable.map.path"; + // Default file path prefix for spillable file + public static final String DEFAULT_SPILLABLE_MAP_BASE_PATH = "/tmp/"; + + public static final Log LOG = LogFactory.getLog(AbstractRealtimeRecordReader.class); + protected final HoodieRealtimeFileSplit split; + protected final JobConf jobConf; + private final MessageType baseFileSchema; + + // Schema handles + private Schema readerSchema; + private Schema writerSchema; + + public AbstractRealtimeRecordReader(HoodieRealtimeFileSplit split, JobConf job) { + this.split = split; + this.jobConf = job; + + LOG.info("cfg ==> " + job.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR)); + try { + baseFileSchema = readSchema(jobConf, split.getPath()); + init(); + } catch (IOException e) { + throw new HoodieIOException( + "Could not create HoodieRealtimeRecordReader on path " + this.split.getPath(), e); + } + } + + /** + * Reads the schema from the parquet file. This is different from ParquetUtils as it uses the + * twitter parquet to support hive 1.1.0 + */ + private static MessageType readSchema(Configuration conf, Path parquetFilePath) { + try { + return ParquetFileReader.readFooter(conf, parquetFilePath).getFileMetaData().getSchema(); + } catch (IOException e) { + throw new HoodieIOException("Failed to read footer for parquet " + parquetFilePath, e); + } + } + + protected static String arrayWritableToString(ArrayWritable writable) { + if (writable == null) { + return "null"; + } + + StringBuilder builder = new StringBuilder(); + Writable[] values = writable.get(); + builder.append(String.format("Size: %s,", values.length)); + for (Writable w : values) { + builder.append(w + " "); + } + return builder.toString(); + } + + /** + * Given a comma separated list of field names and positions at which they appear on Hive, return + * a ordered list of field names, that can be passed onto storage. + */ + public static List orderFields(String fieldNameCsv, String fieldOrderCsv, + String partitioningFieldsCsv) { + + String[] fieldOrders = fieldOrderCsv.split(","); + Set partitioningFields = Arrays.stream(partitioningFieldsCsv.split(",")) + .collect(Collectors.toSet()); + List fieldNames = Arrays.stream(fieldNameCsv.split(",")) + .filter(fn -> !partitioningFields.contains(fn)).collect(Collectors.toList()); + + // Hive does not provide ids for partitioning fields, so check for lengths excluding that. + if (fieldNames.size() != fieldOrders.length) { + throw new HoodieException(String + .format("Error ordering fields for storage read. #fieldNames: %d, #fieldPositions: %d", + fieldNames.size(), fieldOrders.length)); + } + TreeMap orderedFieldMap = new TreeMap<>(); + for (int ox = 0; ox < fieldOrders.length; ox++) { + orderedFieldMap.put(Integer.parseInt(fieldOrders[ox]), fieldNames.get(ox)); + } + return new ArrayList<>(orderedFieldMap.values()); + } + + /** + * Generate a reader schema off the provided writeSchema, to just project out the provided + * columns + */ + public static Schema generateProjectionSchema(Schema writeSchema, List fieldNames) { + List projectedFields = new ArrayList<>(); + for (String fn : fieldNames) { + Schema.Field field = writeSchema.getField(fn); + if (field == null) { + throw new HoodieException("Field " + fn + " not found log schema. Query cannot proceed!"); + } + projectedFields + .add(new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultValue())); + } + + return Schema.createRecord(projectedFields); + } + + /** + * Convert the projected read from delta record into an array writable + */ + public static Writable avroToArrayWritable(Object value, Schema schema) { + + // if value is null, make a NullWritable + if (value == null) { + return NullWritable.get(); + } + + switch (schema.getType()) { + case STRING: + return new Text(value.toString()); + case BYTES: + return new BytesWritable((byte[]) value); + case INT: + return new IntWritable((Integer) value); + case LONG: + return new LongWritable((Long) value); + case FLOAT: + return new FloatWritable((Float) value); + case DOUBLE: + return new DoubleWritable((Double) value); + case BOOLEAN: + return new BooleanWritable((Boolean) value); + case NULL: + return NullWritable.get(); + case RECORD: + GenericRecord record = (GenericRecord) value; + Writable[] values1 = new Writable[schema.getFields().size()]; + int index1 = 0; + for (Schema.Field field : schema.getFields()) { + values1[index1++] = avroToArrayWritable(record.get(field.name()), field.schema()); + } + return new ArrayWritable(Writable.class, values1); + case ENUM: + return new Text(value.toString()); + case ARRAY: + GenericArray arrayValue = (GenericArray) value; + Writable[] values2 = new Writable[arrayValue.size()]; + int index2 = 0; + for (Object obj : arrayValue) { + values2[index2++] = avroToArrayWritable(obj, schema.getElementType()); + } + return new ArrayWritable(Writable.class, values2); + case MAP: + Map mapValue = (Map) value; + Writable[] values3 = new Writable[mapValue.size()]; + int index3 = 0; + for (Object entry : mapValue.entrySet()) { + Map.Entry mapEntry = (Map.Entry) entry; + Writable[] mapValues = new Writable[2]; + mapValues[0] = new Text(mapEntry.getKey().toString()); + mapValues[1] = avroToArrayWritable(mapEntry.getValue(), schema.getValueType()); + values3[index3++] = new ArrayWritable(Writable.class, mapValues); + } + return new ArrayWritable(Writable.class, values3); + case UNION: + List types = schema.getTypes(); + if (types.size() != 2) { + throw new IllegalArgumentException("Only support union with 2 fields"); + } + Schema s1 = types.get(0); + Schema s2 = types.get(1); + if (s1.getType() == Schema.Type.NULL) { + return avroToArrayWritable(value, s2); + } else if (s2.getType() == Schema.Type.NULL) { + return avroToArrayWritable(value, s1); + } else { + throw new IllegalArgumentException("Only support union with null"); + } + case FIXED: + return new BytesWritable(((GenericFixed) value).bytes()); + default: + return null; + } + } + + /** + * Goes through the log files and populates a map with latest version of each key logged, since + * the base split was written. + */ + private void init() throws IOException { + writerSchema = new AvroSchemaConverter().convert(baseFileSchema); + List projectionFields = orderFields( + jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR), + jobConf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR), + jobConf.get("partition_columns", "")); + // TODO(vc): In the future, the reader schema should be updated based on log files & be able + // to null out fields not present before + readerSchema = generateProjectionSchema(writerSchema, projectionFields); + + LOG.info(String.format("About to read compacted logs %s for base split %s, projecting cols %s", + split.getDeltaFilePaths(), split.getPath(), projectionFields)); + } + + public Schema getReaderSchema() { + return readerSchema; + } + + public Schema getWriterSchema() { + return writerSchema; + } + + public long getMaxCompactionMemoryInBytes() { + return (long) Math.ceil(Double + .valueOf(jobConf.get(COMPACTION_MEMORY_FRACTION_PROP, DEFAULT_COMPACTION_MEMORY_FRACTION)) + * jobConf.getMemoryForMapTask()); + } +} diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java index 163cbe8f8..88a5fdd1e 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java @@ -18,339 +18,85 @@ package com.uber.hoodie.hadoop.realtime; -import com.uber.hoodie.common.model.HoodieRecord; -import com.uber.hoodie.common.model.HoodieRecordPayload; -import com.uber.hoodie.common.table.log.HoodieCompactedLogRecordScanner; -import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.exception.HoodieException; -import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.TreeMap; -import java.util.stream.Collectors; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericArray; -import org.apache.avro.generic.GenericFixed; -import org.apache.avro.generic.GenericRecord; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; -import org.apache.hadoop.hive.serde2.io.DoubleWritable; import org.apache.hadoop.io.ArrayWritable; -import org.apache.hadoop.io.BooleanWritable; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.FloatWritable; -import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RecordReader; -import parquet.avro.AvroSchemaConverter; -import parquet.hadoop.ParquetFileReader; -import parquet.schema.MessageType; /** - * Record Reader implementation to merge fresh avro data with base parquet data, to support real - * time queries. + * Realtime Record Reader which can do compacted (merge-on-read) record reading or + * unmerged reading (parquet and log files read in parallel) based on job configuration. */ public class HoodieRealtimeRecordReader implements RecordReader { - private final RecordReader parquetReader; - private final HoodieRealtimeFileSplit split; - private final JobConf jobConf; - - // Fraction of mapper/reducer task memory used for compaction of log files - public static final String COMPACTION_MEMORY_FRACTION_PROP = "compaction.memory.fraction"; - public static final String DEFAULT_COMPACTION_MEMORY_FRACTION = "0.75"; - - // used to choose a trade off between IO vs Memory when performing compaction process - // Depending on outputfile size and memory provided, choose true to avoid OOM for large file - // size + small memory - public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP = - "compaction.lazy.block.read.enabled"; - public static final String DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED = "true"; - - // Property to set the max memory for dfs inputstream buffer size - public static final String MAX_DFS_STREAM_BUFFER_SIZE_PROP = "hoodie.memory.dfs.buffer.max.size"; - // Setting this to lower value of 1 MB since no control over how many RecordReaders will be started in a mapper - public static final int DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE = 1 * 1024 * 1024; // 1 MB - // Property to set file path prefix for spillable file - public static final String SPILLABLE_MAP_BASE_PATH_PROP = "hoodie.memory.spillable.map.path"; - // Default file path prefix for spillable file - public static final String DEFAULT_SPILLABLE_MAP_BASE_PATH = "/tmp/"; - + // Property to enable parallel reading of parquet and log files without merging. + public static final String REALTIME_SKIP_MERGE_PROP = "hoodie.realtime.merge.skip"; + // By default, we do merged-reading + public static final String DEFAULT_REALTIME_SKIP_MERGE = "false"; public static final Log LOG = LogFactory.getLog(HoodieRealtimeRecordReader.class); - - private final HashMap deltaRecordMap; - private final MessageType baseFileSchema; + private final RecordReader reader; public HoodieRealtimeRecordReader(HoodieRealtimeFileSplit split, JobConf job, RecordReader realReader) { - this.split = split; - this.jobConf = job; - this.parquetReader = realReader; - this.deltaRecordMap = new HashMap<>(); + this.reader = constructRecordReader(split, job, realReader); + } - LOG.info("cfg ==> " + job.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR)); + public static boolean canSkipMerging(JobConf jobConf) { + return Boolean.valueOf(jobConf.get(REALTIME_SKIP_MERGE_PROP, DEFAULT_REALTIME_SKIP_MERGE)); + } + + /** + * Construct record reader based on job configuration + * + * @param split File Split + * @param jobConf Job Configuration + * @param realReader Parquet Record Reader + * @return Realtime Reader + */ + private static RecordReader constructRecordReader(HoodieRealtimeFileSplit split, + JobConf jobConf, RecordReader realReader) { try { - baseFileSchema = readSchema(jobConf, split.getPath()); - readAndCompactLog(jobConf); - } catch (IOException e) { - throw new HoodieIOException( - "Could not create HoodieRealtimeRecordReader on path " + this.split.getPath(), e); - } - } - - /** - * Reads the schema from the parquet file. This is different from ParquetUtils as it uses the - * twitter parquet to support hive 1.1.0 - */ - private static MessageType readSchema(Configuration conf, Path parquetFilePath) { - try { - return ParquetFileReader.readFooter(conf, parquetFilePath).getFileMetaData().getSchema(); - } catch (IOException e) { - throw new HoodieIOException("Failed to read footer for parquet " + parquetFilePath, e); - } - } - - - /** - * Goes through the log files and populates a map with latest version of each key logged, since - * the base split was written. - */ - private void readAndCompactLog(JobConf jobConf) throws IOException { - Schema writerSchema = new AvroSchemaConverter().convert(baseFileSchema); - List projectionFields = orderFields( - jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR), - jobConf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR), - jobConf.get("partition_columns", "")); - // TODO(vc): In the future, the reader schema should be updated based on log files & be able - // to null out fields not present before - Schema readerSchema = generateProjectionSchema(writerSchema, projectionFields); - - LOG.info(String.format("About to read compacted logs %s for base split %s, projecting cols %s", - split.getDeltaFilePaths(), split.getPath(), projectionFields)); - HoodieCompactedLogRecordScanner compactedLogRecordScanner = new HoodieCompactedLogRecordScanner( - FSUtils.getFs(split.getPath().toString(), jobConf), split.getBasePath(), - split.getDeltaFilePaths(), readerSchema, split.getMaxCommitTime(), (long) Math.ceil(Double - .valueOf(jobConf.get(COMPACTION_MEMORY_FRACTION_PROP, DEFAULT_COMPACTION_MEMORY_FRACTION)) - * jobConf.getMemoryForMapTask()), Boolean.valueOf(jobConf - .get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)), - false, jobConf.getInt(MAX_DFS_STREAM_BUFFER_SIZE_PROP, DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE), - jobConf.get(SPILLABLE_MAP_BASE_PATH_PROP, DEFAULT_SPILLABLE_MAP_BASE_PATH)); - // NOTE: HoodieCompactedLogRecordScanner will not return records for an in-flight commit - // but can return records for completed commits > the commit we are trying to read (if using - // readCommit() API) - for (HoodieRecord hoodieRecord : compactedLogRecordScanner) { - GenericRecord rec = (GenericRecord) hoodieRecord.getData().getInsertValue(readerSchema).get(); - String key = hoodieRecord.getRecordKey(); - // we assume, a later safe record in the log, is newer than what we have in the map & - // replace it. - // TODO : handle deletes here - ArrayWritable aWritable = (ArrayWritable) avroToArrayWritable(rec, writerSchema); - deltaRecordMap.put(key, aWritable); - if (LOG.isDebugEnabled()) { - LOG.debug("Log record : " + arrayWritableToString(aWritable)); + if (canSkipMerging(jobConf)) { + LOG.info("Enabling un-merged reading of realtime records"); + return new RealtimeUnmergedRecordReader(split, jobConf, realReader); } - } - } - - private static String arrayWritableToString(ArrayWritable writable) { - if (writable == null) { - return "null"; - } - - StringBuilder builder = new StringBuilder(); - Writable[] values = writable.get(); - builder.append(String.format("Size: %s,", values.length)); - for (Writable w : values) { - builder.append(w + " "); - } - return builder.toString(); - } - - /** - * Given a comma separated list of field names and positions at which they appear on Hive, return - * a ordered list of field names, that can be passed onto storage. - */ - public static List orderFields(String fieldNameCsv, String fieldOrderCsv, - String partitioningFieldsCsv) { - - String[] fieldOrders = fieldOrderCsv.split(","); - Set partitioningFields = Arrays.stream(partitioningFieldsCsv.split(",")) - .collect(Collectors.toSet()); - List fieldNames = Arrays.stream(fieldNameCsv.split(",")) - .filter(fn -> !partitioningFields.contains(fn)).collect(Collectors.toList()); - - // Hive does not provide ids for partitioning fields, so check for lengths excluding that. - if (fieldNames.size() != fieldOrders.length) { - throw new HoodieException(String - .format("Error ordering fields for storage read. #fieldNames: %d, #fieldPositions: %d", - fieldNames.size(), fieldOrders.length)); - } - TreeMap orderedFieldMap = new TreeMap<>(); - for (int ox = 0; ox < fieldOrders.length; ox++) { - orderedFieldMap.put(Integer.parseInt(fieldOrders[ox]), fieldNames.get(ox)); - } - return new ArrayList<>(orderedFieldMap.values()); - } - - /** - * Generate a reader schema off the provided writeSchema, to just project out the provided - * columns - */ - public static Schema generateProjectionSchema(Schema writeSchema, List fieldNames) { - List projectedFields = new ArrayList<>(); - for (String fn : fieldNames) { - Schema.Field field = writeSchema.getField(fn); - if (field == null) { - throw new HoodieException("Field " + fn + " not found log schema. Query cannot proceed!"); - } - projectedFields - .add(new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultValue())); - } - - return Schema.createRecord(projectedFields); - } - - /** - * Convert the projected read from delta record into an array writable - */ - public static Writable avroToArrayWritable(Object value, Schema schema) { - - // if value is null, make a NullWritable - if (value == null) { - return NullWritable.get(); - } - - switch (schema.getType()) { - case STRING: - return new Text(value.toString()); - case BYTES: - return new BytesWritable((byte[]) value); - case INT: - return new IntWritable((Integer) value); - case LONG: - return new LongWritable((Long) value); - case FLOAT: - return new FloatWritable((Float) value); - case DOUBLE: - return new DoubleWritable((Double) value); - case BOOLEAN: - return new BooleanWritable((Boolean) value); - case NULL: - return NullWritable.get(); - case RECORD: - GenericRecord record = (GenericRecord) value; - Writable[] values1 = new Writable[schema.getFields().size()]; - int index1 = 0; - for (Schema.Field field : schema.getFields()) { - values1[index1++] = avroToArrayWritable(record.get(field.name()), field.schema()); - } - return new ArrayWritable(Writable.class, values1); - case ENUM: - return new Text(value.toString()); - case ARRAY: - GenericArray arrayValue = (GenericArray) value; - Writable[] values2 = new Writable[arrayValue.size()]; - int index2 = 0; - for (Object obj : arrayValue) { - values2[index2++] = avroToArrayWritable(obj, schema.getElementType()); - } - return new ArrayWritable(Writable.class, values2); - case MAP: - Map mapValue = (Map) value; - Writable[] values3 = new Writable[mapValue.size()]; - int index3 = 0; - for (Object entry : mapValue.entrySet()) { - Map.Entry mapEntry = (Map.Entry) entry; - Writable[] mapValues = new Writable[2]; - mapValues[0] = new Text(mapEntry.getKey().toString()); - mapValues[1] = avroToArrayWritable(mapEntry.getValue(), schema.getValueType()); - values3[index3++] = new ArrayWritable(Writable.class, mapValues); - } - return new ArrayWritable(Writable.class, values3); - case UNION: - List types = schema.getTypes(); - if (types.size() != 2) { - throw new IllegalArgumentException("Only support union with 2 fields"); - } - Schema s1 = types.get(0); - Schema s2 = types.get(1); - if (s1.getType() == Schema.Type.NULL) { - return avroToArrayWritable(value, s2); - } else if (s2.getType() == Schema.Type.NULL) { - return avroToArrayWritable(value, s1); - } else { - throw new IllegalArgumentException("Only support union with null"); - } - case FIXED: - return new BytesWritable(((GenericFixed) value).bytes()); - default: - return null; + return new RealtimeCompactedRecordReader(split, jobConf, realReader); + } catch (IOException ex) { + LOG.error("Got exception when constructing record reader", ex); + throw new HoodieException(ex); } } @Override - public boolean next(Void aVoid, ArrayWritable arrayWritable) throws IOException { - // Call the underlying parquetReader.next - which may replace the passed in ArrayWritable - // with a new block of values - boolean result = this.parquetReader.next(aVoid, arrayWritable); - if (!result) { - // if the result is false, then there are no more records - return false; - } else { - // TODO(VC): Right now, we assume all records in log, have a matching base record. (which - // would be true until we have a way to index logs too) - // return from delta records map if we have some match. - String key = arrayWritable.get()[HoodieRealtimeInputFormat.HOODIE_RECORD_KEY_COL_POS] - .toString(); - if (LOG.isDebugEnabled()) { - LOG.debug(String.format("key %s, base values: %s, log values: %s", key, - arrayWritableToString(arrayWritable), arrayWritableToString(deltaRecordMap.get(key)))); - } - if (deltaRecordMap.containsKey(key)) { - // TODO(NA): Invoke preCombine here by converting arrayWritable to Avro ? - Writable[] replaceValue = deltaRecordMap.get(key).get(); - Writable[] originalValue = arrayWritable.get(); - System.arraycopy(replaceValue, 0, originalValue, 0, originalValue.length); - arrayWritable.set(originalValue); - } - return true; - } + public boolean next(Void key, ArrayWritable value) throws IOException { + return this.reader.next(key, value); } @Override public Void createKey() { - return parquetReader.createKey(); + return this.reader.createKey(); } @Override public ArrayWritable createValue() { - return parquetReader.createValue(); + return this.reader.createValue(); } @Override public long getPos() throws IOException { - return parquetReader.getPos(); + return this.reader.getPos(); } @Override public void close() throws IOException { - parquetReader.close(); + this.reader.close(); } @Override public float getProgress() throws IOException { - return parquetReader.getProgress(); + return this.reader.getProgress(); } } diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeCompactedRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeCompactedRecordReader.java new file mode 100644 index 000000000..c64933389 --- /dev/null +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeCompactedRecordReader.java @@ -0,0 +1,129 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * 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 com.uber.hoodie.hadoop.realtime; + +import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.table.log.HoodieMergedLogRecordScanner; +import com.uber.hoodie.common.util.FSUtils; +import java.io.IOException; +import java.util.HashMap; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.io.ArrayWritable; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.RecordReader; + +class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader implements + RecordReader { + + protected final RecordReader parquetReader; + private final HashMap deltaRecordMap; + + public RealtimeCompactedRecordReader(HoodieRealtimeFileSplit split, JobConf job, + RecordReader realReader) throws IOException { + super(split, job); + this.parquetReader = realReader; + this.deltaRecordMap = new HashMap<>(); + readAndCompactLog(); + } + + /** + * Goes through the log files and populates a map with latest version of each key logged, since + * the base split was written. + */ + private void readAndCompactLog() throws IOException { + HoodieMergedLogRecordScanner compactedLogRecordScanner = new HoodieMergedLogRecordScanner( + FSUtils.getFs(split.getPath().toString(), jobConf), split.getBasePath(), + split.getDeltaFilePaths(), getReaderSchema(), split.getMaxCommitTime(), getMaxCompactionMemoryInBytes(), + Boolean.valueOf(jobConf.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, + DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)), + false, jobConf.getInt(MAX_DFS_STREAM_BUFFER_SIZE_PROP, DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE), + jobConf.get(SPILLABLE_MAP_BASE_PATH_PROP, DEFAULT_SPILLABLE_MAP_BASE_PATH)); + // NOTE: HoodieCompactedLogRecordScanner will not return records for an in-flight commit + // but can return records for completed commits > the commit we are trying to read (if using + // readCommit() API) + for (HoodieRecord hoodieRecord : compactedLogRecordScanner) { + GenericRecord rec = (GenericRecord) hoodieRecord.getData().getInsertValue(getReaderSchema()).get(); + String key = hoodieRecord.getRecordKey(); + // we assume, a later safe record in the log, is newer than what we have in the map & + // replace it. + // TODO : handle deletes here + ArrayWritable aWritable = (ArrayWritable) avroToArrayWritable(rec, getWriterSchema()); + deltaRecordMap.put(key, aWritable); + if (LOG.isDebugEnabled()) { + LOG.debug("Log record : " + arrayWritableToString(aWritable)); + } + } + } + + @Override + public boolean next(Void aVoid, ArrayWritable arrayWritable) throws IOException { + // Call the underlying parquetReader.next - which may replace the passed in ArrayWritable + // with a new block of values + boolean result = this.parquetReader.next(aVoid, arrayWritable); + if (!result) { + // if the result is false, then there are no more records + return false; + } else { + // TODO(VC): Right now, we assume all records in log, have a matching base record. (which + // would be true until we have a way to index logs too) + // return from delta records map if we have some match. + String key = arrayWritable.get()[HoodieRealtimeInputFormat.HOODIE_RECORD_KEY_COL_POS] + .toString(); + if (LOG.isDebugEnabled()) { + LOG.debug(String.format("key %s, base values: %s, log values: %s", key, + arrayWritableToString(arrayWritable), arrayWritableToString(deltaRecordMap.get(key)))); + } + if (deltaRecordMap.containsKey(key)) { + // TODO(NA): Invoke preCombine here by converting arrayWritable to Avro ? + Writable[] replaceValue = deltaRecordMap.get(key).get(); + Writable[] originalValue = arrayWritable.get(); + System.arraycopy(replaceValue, 0, originalValue, 0, originalValue.length); + arrayWritable.set(originalValue); + } + return true; + } + } + + @Override + public Void createKey() { + return parquetReader.createKey(); + } + + @Override + public ArrayWritable createValue() { + return parquetReader.createValue(); + } + + @Override + public long getPos() throws IOException { + return parquetReader.getPos(); + } + + @Override + public void close() throws IOException { + parquetReader.close(); + } + + @Override + public float getProgress() throws IOException { + return parquetReader.getProgress(); + } +} diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeUnmergedRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeUnmergedRecordReader.java new file mode 100644 index 000000000..afddefc6e --- /dev/null +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeUnmergedRecordReader.java @@ -0,0 +1,142 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * 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 com.uber.hoodie.hadoop.realtime; + +import com.uber.hoodie.common.table.log.HoodieUnMergedLogRecordScanner; +import com.uber.hoodie.common.util.DefaultSizeEstimator; +import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.queue.BoundedInMemoryExecutor; +import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueProducer; +import com.uber.hoodie.common.util.queue.FunctionBasedQueueProducer; +import com.uber.hoodie.common.util.queue.IteratorBasedQueueProducer; +import com.uber.hoodie.hadoop.RecordReaderValueIterator; +import com.uber.hoodie.hadoop.SafeParquetRecordReaderWrapper; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.io.ArrayWritable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.RecordReader; + +class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader implements + RecordReader { + + // Log Record unmerged scanner + private final HoodieUnMergedLogRecordScanner logRecordScanner; + + // Parquet record reader + private final RecordReader parquetReader; + + // Parquet record iterator wrapper for the above reader + private final RecordReaderValueIterator parquetRecordsIterator; + + // Executor that runs the above producers in parallel + private final BoundedInMemoryExecutor executor; + + // Iterator for the buffer consumer + private final Iterator iterator; + + /** + * Construct a Unmerged record reader that parallely consumes both parquet and log records and buffers for upstream + * clients to consume + * + * @param split File split + * @param job Job Configuration + * @param realReader Parquet Reader + */ + public RealtimeUnmergedRecordReader(HoodieRealtimeFileSplit split, JobConf job, + RecordReader realReader) { + super(split, job); + this.parquetReader = new SafeParquetRecordReaderWrapper(realReader); + // Iterator for consuming records from parquet file + this.parquetRecordsIterator = new RecordReaderValueIterator<>(this.parquetReader); + this.executor = new BoundedInMemoryExecutor<>(getMaxCompactionMemoryInBytes(), getParallelProducers(), + Optional.empty(), x -> x, new DefaultSizeEstimator<>()); + // Consumer of this record reader + this.iterator = this.executor.getQueue().iterator(); + this.logRecordScanner = new HoodieUnMergedLogRecordScanner( + FSUtils.getFs(split.getPath().toString(), jobConf), split.getBasePath(), + split.getDeltaFilePaths(), getReaderSchema(), split.getMaxCommitTime(), Boolean.valueOf(jobConf + .get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)), + false, jobConf.getInt(MAX_DFS_STREAM_BUFFER_SIZE_PROP, DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE), + record -> { + // convert Hoodie log record to Hadoop AvroWritable and buffer + GenericRecord rec = (GenericRecord) record.getData().getInsertValue(getReaderSchema()).get(); + ArrayWritable aWritable = (ArrayWritable) avroToArrayWritable(rec, getWriterSchema()); + this.executor.getQueue().insertRecord(aWritable); + }); + // Start reading and buffering + this.executor.startProducers(); + } + + /** + * Setup log and parquet reading in parallel. Both write to central buffer. + */ + @SuppressWarnings("unchecked") + private List> getParallelProducers() { + List> producers = new ArrayList<>(); + producers.add(new FunctionBasedQueueProducer<>(buffer -> { + logRecordScanner.scan(); + return null; + })); + producers.add(new IteratorBasedQueueProducer<>(parquetRecordsIterator)); + return producers; + } + + @Override + public boolean next(Void key, ArrayWritable value) throws IOException { + if (!iterator.hasNext()) { + return false; + } + // Copy from buffer iterator and set to passed writable + value.set(iterator.next().get()); + return true; + } + + @Override + public Void createKey() { + return parquetReader.createKey(); + } + + @Override + public ArrayWritable createValue() { + return parquetReader.createValue(); + } + + @Override + public long getPos() throws IOException { + //TODO: vb - No logical way to represent parallel stream pos in a single long. + // Should we just return invalid (-1). Where is it used ? + return 0; + } + + @Override + public void close() throws IOException { + this.parquetRecordsIterator.close(); + this.executor.shutdownNow(); + } + + @Override + public float getProgress() throws IOException { + return Math.min(parquetReader.getProgress(), logRecordScanner.getProgress()); + } +} diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestRecordReaderValueIterator.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestRecordReaderValueIterator.java new file mode 100644 index 000000000..06fc41c99 --- /dev/null +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestRecordReaderValueIterator.java @@ -0,0 +1,105 @@ +/* + * Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * 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 com.uber.hoodie.hadoop; + +import groovy.lang.Tuple2; +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.RecordReader; +import org.junit.Assert; +import org.junit.Test; + +public class TestRecordReaderValueIterator { + + @Test + public void testValueIterator() { + String[] values = new String[]{ + "hoodie", + "efficient", + "new project", + "realtime", + "spark", + "dataset", + }; + List> entries = IntStream.range(0, values.length) + .boxed().map(idx -> new Tuple2<>(idx, values[idx])).collect(Collectors.toList()); + TestRecordReader reader = new TestRecordReader(entries); + RecordReaderValueIterator itr = new RecordReaderValueIterator(reader); + for (int i = 0; i < values.length; i++) { + Assert.assertTrue(itr.hasNext()); + Text val = itr.next(); + Assert.assertEquals(values[i], val.toString()); + } + Assert.assertFalse(itr.hasNext()); + } + + /** + * Simple replay record reader for unit-testing + */ + private static class TestRecordReader implements RecordReader { + + private final List> entries; + private int currIndex = 0; + + public TestRecordReader(List> entries) { + this.entries = entries; + } + + + @Override + public boolean next(IntWritable key, Text value) throws IOException { + if (currIndex >= entries.size()) { + return false; + } + key.set(entries.get(currIndex).getFirst()); + value.set(entries.get(currIndex).getSecond()); + currIndex++; + return true; + } + + @Override + public IntWritable createKey() { + return new IntWritable(); + } + + @Override + public Text createValue() { + return new Text(); + } + + @Override + public long getPos() throws IOException { + return currIndex; + } + + @Override + public void close() throws IOException { + + } + + @Override + public float getProgress() throws IOException { + return (currIndex * 1.0F) / entries.size(); + } + } +} diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java index ea9016bb9..a889e1a6e 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java @@ -35,8 +35,10 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; @@ -71,7 +73,7 @@ public class HoodieRealtimeRecordReaderTest { @Before public void setUp() { jobConf = new JobConf(); - jobConf.set(HoodieRealtimeRecordReader.MAX_DFS_STREAM_BUFFER_SIZE_PROP, String.valueOf(1 * 1024 * 1024)); + jobConf.set(AbstractRealtimeRecordReader.MAX_DFS_STREAM_BUFFER_SIZE_PROP, String.valueOf(1 * 1024 * 1024)); hadoopConf = HoodieTestUtils.getDefaultHadoopConf(); fs = FSUtils.getFs(basePath.getRoot().getAbsolutePath(), hadoopConf); } @@ -82,12 +84,18 @@ public class HoodieRealtimeRecordReaderTest { private HoodieLogFormat.Writer writeLogFile(File partitionDir, Schema schema, String fileId, String baseCommit, String newCommit, int numberOfRecords) throws InterruptedException, IOException { + return writeLogFile(partitionDir, schema, fileId, baseCommit, newCommit, numberOfRecords, 0); + } + + private HoodieLogFormat.Writer writeLogFile(File partitionDir, Schema schema, String fileId, + String baseCommit, String newCommit, int numberOfRecords, int offset) + throws InterruptedException, IOException { HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder() .onParentPath(new Path(partitionDir.getPath())) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId) .overBaseCommit(baseCommit).withFs(fs).build(); List records = new ArrayList<>(); - for (int i = 0; i < numberOfRecords; i++) { + for (int i = offset; i < offset + numberOfRecords; i++) { records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, newCommit, "fileid0")); } Schema writeSchema = records.get(0).getSchema(); @@ -142,8 +150,7 @@ public class HoodieRealtimeRecordReaderTest { jobConf.set("partition_columns", "datestr"); //validate record reader compaction - HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf, - reader); + HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader); //use reader to read base Parquet File and log file, merge in flight and return latest commit //here all 100 records should be updated, see above @@ -158,6 +165,90 @@ public class HoodieRealtimeRecordReaderTest { } } + @Test + public void testUnMergedReader() throws Exception { + // initial commit + Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema()); + HoodieTestUtils.initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(), + HoodieTableType.MERGE_ON_READ); + String commitTime = "100"; + final int numRecords = 1000; + final int firstBatchLastRecordKey = numRecords - 1; + final int secondBatchLastRecordKey = 2 * numRecords - 1; + File partitionDir = InputFormatTestUtil + .prepareParquetDataset(basePath, schema, 1, numRecords, commitTime); + InputFormatTestUtil.commit(basePath, commitTime); + // Add the paths + FileInputFormat.setInputPaths(jobConf, partitionDir.getPath()); + + // insert new records to log file + String newCommitTime = "101"; + HoodieLogFormat.Writer writer = writeLogFile(partitionDir, schema, "fileid0", commitTime, + newCommitTime, numRecords, numRecords); + long size = writer.getCurrentSize(); + writer.close(); + assertTrue("block - size should be > 0", size > 0); + + //create a split with baseFile (parquet file written earlier) and new log file(s) + String logFilePath = writer.getLogFile().getPath().toString(); + HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit( + new FileSplit(new Path(partitionDir + "/fileid0_1_" + commitTime + ".parquet"), 0, 1, + jobConf), basePath.getRoot().getPath(), Arrays.asList(logFilePath), newCommitTime); + + //create a RecordReader to be used by HoodieRealtimeRecordReader + RecordReader reader = + new MapredParquetInputFormat().getRecordReader( + new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null), + jobConf, null); + JobConf jobConf = new JobConf(); + List fields = schema.getFields(); + String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(",")); + String postions = fields.stream().map(f -> String.valueOf(f.pos())) + .collect(Collectors.joining(",")); + jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names); + jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions); + jobConf.set("partition_columns", "datestr"); + // Enable merge skipping. + jobConf.set("hoodie.realtime.merge.skip", "true"); + + //validate unmerged record reader + RealtimeUnmergedRecordReader recordReader = new RealtimeUnmergedRecordReader(split, jobConf, reader); + + //use reader to read base Parquet File and log file + //here all records should be present. Also ensure log records are in order. + Void key = recordReader.createKey(); + ArrayWritable value = recordReader.createValue(); + int numRecordsAtCommit1 = 0; + int numRecordsAtCommit2 = 0; + Set seenKeys = new HashSet<>(); + Integer lastSeenKeyFromLog = firstBatchLastRecordKey; + while (recordReader.next(key, value)) { + Writable[] values = value.get(); + String gotCommit = values[0].toString(); + String keyStr = values[2].toString(); + Integer gotKey = Integer.parseInt(keyStr.substring("key".length())); + if (gotCommit.equals(newCommitTime)) { + numRecordsAtCommit2++; + Assert.assertTrue(gotKey > firstBatchLastRecordKey); + Assert.assertTrue(gotKey <= secondBatchLastRecordKey); + Assert.assertEquals(gotKey.intValue(), lastSeenKeyFromLog + 1); + lastSeenKeyFromLog++; + } else { + numRecordsAtCommit1++; + Assert.assertTrue(gotKey >= 0); + Assert.assertTrue(gotKey <= firstBatchLastRecordKey); + } + // Ensure unique key + Assert.assertFalse(seenKeys.contains(gotKey)); + seenKeys.add(gotKey); + key = recordReader.createKey(); + value = recordReader.createValue(); + } + Assert.assertEquals(numRecords, numRecordsAtCommit1); + Assert.assertEquals(numRecords, numRecordsAtCommit2); + Assert.assertEquals(2 * numRecords, seenKeys.size()); + } + @Test public void testReaderWithNestedAndComplexSchema() throws Exception { // initial commit @@ -203,8 +294,7 @@ public class HoodieRealtimeRecordReaderTest { jobConf.set("partition_columns", "datestr"); // validate record reader compaction - HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf, - reader); + HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader); // use reader to read base Parquet File and log file, merge in flight and return latest commit // here the first 50 records should be updated, see above