1
0

Reducing memory footprint required in HoodieAvroDataBlock and HoodieAppendHandle

This commit is contained in:
Nishith Agarwal
2017-12-15 14:03:06 -08:00
committed by vinoth chandar
parent 85d32930cd
commit 937ae322ba
8 changed files with 174 additions and 48 deletions

View File

@@ -0,0 +1,63 @@
/*
* 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.storage;
import java.io.DataInputStream;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicInteger;
/**
* Keeps track of how many bytes were read from a DataInputStream
*/
public class SizeAwareDataInputStream {
private final DataInputStream dis;
private final AtomicInteger numberOfBytesRead;
public SizeAwareDataInputStream(DataInputStream dis) {
this.dis = dis;
this.numberOfBytesRead = new AtomicInteger(0);
}
public int readInt() throws IOException {
numberOfBytesRead.addAndGet(Integer.BYTES);
return dis.readInt();
}
public void readFully(byte b[], int off, int len) throws IOException {
numberOfBytesRead.addAndGet(len);
dis.readFully(b, off, len);
}
public void readFully(byte b[]) throws IOException {
numberOfBytesRead.addAndGet(b.length);
dis.readFully(b);
}
public int skipBytes(int n) throws IOException {
numberOfBytesRead.addAndGet(n);
return dis.skipBytes(n);
}
public void close() throws IOException {
dis.close();
}
public Integer getNumberOfBytesRead() {
return numberOfBytesRead.get();
}
}

View File

@@ -16,16 +16,9 @@
package com.uber.hoodie.common.table.log.block;
import com.uber.hoodie.common.storage.SizeAwareDataInputStream;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericDatumWriter;
@@ -35,18 +28,31 @@ import org.apache.avro.io.DecoderFactory;
import org.apache.avro.io.Encoder;
import org.apache.avro.io.EncoderFactory;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
/**
* DataBlock contains a list of records serialized using Avro. The Datablock contains 1. Compressed
* Writer Schema length 2. Compressed Writer Schema content 3. Total number of records in the block
* 4. Size of a record 5. Actual avro serialized content of the record
* DataBlock contains a list of records serialized using Avro.
* The Datablock contains
* 1. Compressed Writer Schema length
* 2. Compressed Writer Schema content
* 3. Total number of records in the block
* 4. Size of a record
* 5. Actual avro serialized content of the record
*/
public class HoodieAvroDataBlock extends HoodieLogBlock {
private List<IndexedRecord> records;
private Schema schema;
public HoodieAvroDataBlock(List<IndexedRecord> records, Schema schema,
Map<LogMetadataType, String> metadata) {
public HoodieAvroDataBlock(List<IndexedRecord> records, Schema schema, Map<LogMetadataType, String> metadata) {
super(metadata);
this.records = records;
this.schema = schema;
@@ -56,6 +62,7 @@ public class HoodieAvroDataBlock extends HoodieLogBlock {
this(records, schema, null);
}
//TODO : (na) lazily create IndexedRecords only when required
public List<IndexedRecord> getRecords() {
return records;
}
@@ -85,7 +92,9 @@ public class HoodieAvroDataBlock extends HoodieLogBlock {
output.writeInt(records.size());
// 4. Write the records
records.forEach(s -> {
Iterator<IndexedRecord> itr = records.iterator();
while (itr.hasNext()) {
IndexedRecord s = itr.next();
ByteArrayOutputStream temp = new ByteArrayOutputStream();
Encoder encoder = EncoderFactory.get().binaryEncoder(temp, null);
try {
@@ -99,10 +108,11 @@ public class HoodieAvroDataBlock extends HoodieLogBlock {
output.writeInt(size);
// Write the content
output.write(temp.toByteArray());
itr.remove();
} catch (IOException e) {
throw new HoodieIOException("IOException converting HoodieAvroDataBlock to bytes", e);
}
});
}
output.close();
return baos.toByteArray();
@@ -113,10 +123,10 @@ public class HoodieAvroDataBlock extends HoodieLogBlock {
return HoodieLogBlockType.AVRO_DATA_BLOCK;
}
public static HoodieLogBlock fromBytes(byte[] content, Schema readerSchema, boolean readMetadata)
throws IOException {
//TODO (na) - Break down content into smaller chunks of byte [] to be GC as they are used
public static HoodieLogBlock fromBytes(byte[] content, Schema readerSchema, boolean readMetadata) throws IOException {
DataInputStream dis = new DataInputStream(new ByteArrayInputStream(content));
SizeAwareDataInputStream dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(content)));
Map<LogMetadataType, String> metadata = null;
// 1. Read the metadata written out, if applicable
if (readMetadata) {
@@ -132,22 +142,20 @@ public class HoodieAvroDataBlock extends HoodieLogBlock {
readerSchema = writerSchema;
}
//TODO : (na) lazily create IndexedRecords only when required
GenericDatumReader<IndexedRecord> reader = new GenericDatumReader<>(writerSchema, readerSchema);
// 2. Get the total records
int totalRecords = dis.readInt();
List<IndexedRecord> records = new ArrayList<>(totalRecords);
// 3. Read the content
for (int i = 0; i < totalRecords; i++) {
// TODO - avoid bytes copy
for (int i=0;i<totalRecords;i++) {
int recordLength = dis.readInt();
byte[] recordData = new byte[recordLength];
dis.readFully(recordData, 0, recordLength);
Decoder decoder = DecoderFactory.get().binaryDecoder(recordData, null);
Decoder decoder = DecoderFactory.get().binaryDecoder(content, dis.getNumberOfBytesRead(), recordLength, null);
IndexedRecord record = reader.read(null, decoder);
records.add(record);
dis.skipBytes(recordLength);
}
dis.close();
return new HoodieAvroDataBlock(records, readerSchema, metadata);
}

View File

@@ -16,6 +16,8 @@
package com.uber.hoodie.common.table.log.block;
import com.uber.hoodie.common.storage.SizeAwareDataInputStream;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
@@ -65,7 +67,7 @@ public class HoodieCommandBlock extends HoodieLogBlock {
}
public static HoodieLogBlock fromBytes(byte[] content, boolean readMetadata) throws IOException {
DataInputStream dis = new DataInputStream(new ByteArrayInputStream(content));
SizeAwareDataInputStream dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(content)));
Map<LogMetadataType, String> metadata = null;
if (readMetadata) {
metadata = HoodieLogBlock.getLogMetadata(dis);

View File

@@ -16,6 +16,8 @@
package com.uber.hoodie.common.table.log.block;
import com.uber.hoodie.common.storage.SizeAwareDataInputStream;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
@@ -62,7 +64,7 @@ public class HoodieCorruptBlock extends HoodieLogBlock {
public static HoodieLogBlock fromBytes(byte[] content, int blockSize, boolean readMetadata)
throws IOException {
DataInputStream dis = new DataInputStream(new ByteArrayInputStream(content));
SizeAwareDataInputStream dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(content)));
Map<LogMetadataType, String> metadata = null;
int bytesRemaining = blockSize;
if (readMetadata) {

View File

@@ -23,6 +23,8 @@ import java.io.DataOutputStream;
import java.io.IOException;
import java.nio.charset.Charset;
import java.util.Map;
import com.uber.hoodie.common.storage.SizeAwareDataInputStream;
import org.apache.commons.lang3.StringUtils;
/**
@@ -64,7 +66,7 @@ public class HoodieDeleteBlock extends HoodieLogBlock {
}
public static HoodieLogBlock fromBytes(byte[] content, boolean readMetadata) throws IOException {
DataInputStream dis = new DataInputStream(new ByteArrayInputStream(content));
SizeAwareDataInputStream dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(content)));
Map<LogMetadataType, String> metadata = null;
if (readMetadata) {
metadata = HoodieLogBlock.getLogMetadata(dis);

View File

@@ -17,6 +17,7 @@
package com.uber.hoodie.common.table.log.block;
import com.google.common.collect.Maps;
import com.uber.hoodie.common.storage.SizeAwareDataInputStream;
import com.uber.hoodie.exception.HoodieException;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
@@ -90,7 +91,7 @@ public abstract class HoodieLogBlock {
/**
* Convert bytes to LogMetadata, follow the same order as {@link HoodieLogBlock#getLogMetadataBytes}
*/
public static Map<LogMetadataType, String> getLogMetadata(DataInputStream dis)
public static Map<LogMetadataType, String> getLogMetadata(SizeAwareDataInputStream dis)
throws IOException {
Map<LogMetadataType, String> metadata = Maps.newHashMap();