Skip to content

Commit

Permalink
Explicitly release resources in LogFileReader and TestHoodieClientBase
Browse files Browse the repository at this point in the history
  • Loading branch information
bvaradar committed Sep 20, 2018
1 parent 2728f96 commit 4bd15b0
Show file tree
Hide file tree
Showing 15 changed files with 119 additions and 7 deletions.
Expand Up @@ -77,6 +77,7 @@ public String showCommits(
List<Comparable[]> readCommits = readRecords.stream().map(r -> (GenericRecord) r).map(r -> readCommit(r))
.collect(Collectors.toList());
allCommits.addAll(readCommits);
reader.close();
}

TableHeader header = new TableHeader().addTableHeaderField("CommitTime")
Expand Down
Expand Up @@ -131,6 +131,7 @@ public String showLogFileCommits(
totalEntries++;
}
}
reader.close();
}
List<Comparable[]> rows = new ArrayList<>();
int i = 0;
Expand Down Expand Up @@ -221,6 +222,7 @@ public String showLogFileRecords(@CliOption(key = {
}
}
}
reader.close();
if (allRecords.size() >= limit) {
break;
}
Expand Down
Expand Up @@ -73,6 +73,11 @@ private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) {
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build());
}

@Override
public void tearDown() throws IOException {
super.tearDown();
}

@Test
public void testRollbackInflightIngestionWithPendingCompaction() throws Exception {
// Rollback inflight ingestion when there is pending compaction
Expand Down
5 changes: 5 additions & 0 deletions hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java
Expand Up @@ -85,6 +85,11 @@ public class TestCleaner extends TestHoodieClientBase {
private static final int BIG_BATCH_INSERT_SIZE = 500;
private static Logger logger = LogManager.getLogger(TestHoodieClientBase.class);

@Override
public void tearDown() throws IOException {
super.tearDown();
}

/**
* Helper method to do first batch of insert for clean by versions/commits tests
*
Expand Down
Expand Up @@ -37,6 +37,7 @@
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.table.HoodieTable;
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.spark.api.java.JavaRDD;
Expand All @@ -47,6 +48,11 @@
*/
public class TestClientRollback extends TestHoodieClientBase {

@Override
public void tearDown() throws IOException {
super.tearDown();
}

/**
* Test case for rollback-savepoint interaction
*/
Expand Down
Expand Up @@ -51,6 +51,8 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SQLContext;
Expand All @@ -63,10 +65,13 @@
*/
public class TestHoodieClientBase implements Serializable {

protected static Logger logger = LogManager.getLogger(TestHoodieClientBase.class);

protected transient JavaSparkContext jsc = null;
protected transient SQLContext sqlContext;
protected transient FileSystem fs;
protected String basePath = null;
protected TemporaryFolder folder = null;
protected transient HoodieTestDataGenerator dataGen = null;

@Before
Expand All @@ -78,10 +83,10 @@ public void init() throws IOException {
//SQLContext stuff
sqlContext = new SQLContext(jsc);

// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();
folder = new TemporaryFolder();
folder.create();
basePath = folder.getRoot().getAbsolutePath();

fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
if (fs instanceof LocalFileSystem) {
LocalFileSystem lfs = (LocalFileSystem) fs;
Expand All @@ -94,6 +99,33 @@ public void init() throws IOException {
dataGen = new HoodieTestDataGenerator();
}

@After
/**
* Properly release resources at end of each test
*/
public void tearDown() throws IOException {
if (null != sqlContext) {
logger.info("Clearing sql context cache of spark-session used in previous test-case");
sqlContext.clearCache();
}

if (null != jsc) {
logger.info("Closing spark context used in previous test-case");
jsc.close();
}

// Create a temp folder as the base path
if (null != folder) {
logger.info("Explicitly removing workspace used in previously run test-case");
folder.delete();
}

if (null != fs) {
logger.warn("Closing file-system instance used in previous test-run");
fs.close();
}
}

/**
* Get Default HoodieWriteConfig for tests
*
Expand Down
Expand Up @@ -43,6 +43,7 @@
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.table.HoodieTable;
import java.io.FileInputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
Expand All @@ -64,6 +65,11 @@
@SuppressWarnings("unchecked")
public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {

@Override
public void tearDown() throws IOException {
super.tearDown();
}

/**
* Test Auto Commit behavior for HoodieWriteClient insert API
*/
Expand Down
Expand Up @@ -20,6 +20,7 @@

import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.config.HoodieWriteConfig;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
Expand All @@ -35,6 +36,11 @@
*/
public class TestHoodieReadClient extends TestHoodieClientBase {

@Override
public void tearDown() throws IOException {
super.tearDown();
}

/**
* Test ReadFilter API after writing new records using HoodieWriteClient.insert
*/
Expand Down
Expand Up @@ -271,6 +271,7 @@ public void testArchiveDatasetWithArchival() throws IOException {

// verify in-flight instants after archive
verifyInflightInstants(metaClient, 3);
reader.close();
}

@Test
Expand Down
Expand Up @@ -30,6 +30,7 @@
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.util.SpillableMapUtils;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException;
import java.util.ArrayDeque;
import java.util.Arrays;
import java.util.Deque;
Expand Down Expand Up @@ -115,9 +116,10 @@ public AbstractHoodieLogRecordScanner(FileSystem fs, String basePath, List<Strin
* Scan Log files
*/
public void scan() {
HoodieLogFormatReader logFormatReaderWrapper = null;
try {
// iterate over the paths
HoodieLogFormatReader logFormatReaderWrapper =
logFormatReaderWrapper =
new HoodieLogFormatReader(fs,
logFilePaths.stream().map(logFile -> new HoodieLogFile(new Path(logFile)))
.collect(Collectors.toList()), readerSchema, readBlocksLazily, reverseReader, bufferSize);
Expand Down Expand Up @@ -239,6 +241,15 @@ public void scan() {
} catch (Exception e) {
log.error("Got exception when reading log file", e);
throw new HoodieIOException("IOException when reading log file ");
} finally {
try {
if (null != logFormatReaderWrapper) {
logFormatReaderWrapper.close();
}
} catch (IOException ioe) {
// Eat exception as we do not want to mask the original exception that can happen
log.error("Unable to close log format reader", ioe);
}
}
}

Expand Down
Expand Up @@ -62,6 +62,7 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
private long reverseLogFilePosition;
private long lastReverseLogFilePosition;
private boolean reverseReader;
private boolean closed = false;

HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize,
boolean readBlockLazily, boolean reverseReader) throws IOException {
Expand Down Expand Up @@ -95,13 +96,13 @@ public HoodieLogFile getLogFile() {
}

/**
* Close the inputstream when the JVM exits
* Close the inputstream if not closed when the JVM exits
*/
private void addShutDownHook() {
Runtime.getRuntime().addShutdownHook(new Thread() {
public void run() {
try {
inputStream.close();
close();
} catch (Exception e) {
log.warn("unable to close input stream for log file " + logFile, e);
// fail silently for any sort of exception
Expand Down Expand Up @@ -277,7 +278,10 @@ private long scanForNextAvailableBlockOffset() throws IOException {

@Override
public void close() throws IOException {
this.inputStream.close();
if (!closed) {
this.inputStream.close();
closed = true;
}
}

@Override
Expand Down
Expand Up @@ -20,6 +20,7 @@
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
Expand All @@ -29,6 +30,8 @@
public class HoodieLogFormatReader implements HoodieLogFormat.Reader {

private final List<HoodieLogFile> logFiles;
// Readers for previously scanned log-files that are still open
private final List<HoodieLogFileReader> prevReadersInOpenState;
private HoodieLogFileReader currentReader;
private final FileSystem fs;
private final Schema readerSchema;
Expand All @@ -46,14 +49,28 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
this.readBlocksLazily = readBlocksLazily;
this.reverseLogReader = reverseLogReader;
this.bufferSize = bufferSize;
this.prevReadersInOpenState = new ArrayList<>();
if (logFiles.size() > 0) {
HoodieLogFile nextLogFile = logFiles.remove(0);
this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false);
}
}

@Override
/**
* Note : In lazy mode, clients must ensure close() should be called only after processing
* all log-blocks as the underlying inputstream will be closed.
* TODO: We can introduce invalidate() API at HoodieLogBlock and this object can call invalidate on
* all returned log-blocks so that we check this scenario specifically in HoodieLogBlock
*/
public void close() throws IOException {

for (HoodieLogFileReader reader : prevReadersInOpenState) {
reader.close();
}

prevReadersInOpenState.clear();

if (currentReader != null) {
currentReader.close();
}
Expand All @@ -69,6 +86,12 @@ public boolean hasNext() {
} else if (logFiles.size() > 0) {
try {
HoodieLogFile nextLogFile = logFiles.remove(0);
// First close previous reader only if readBlockLazily is true
if (!readBlocksLazily) {
this.currentReader.close();
} else {
this.prevReadersInOpenState.add(currentReader);
}
this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily,
false);
} catch (IOException io) {
Expand Down
Expand Up @@ -304,6 +304,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect
dataBlockRead.getRecords().size());
assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords,
dataBlockRead.getRecords());
reader.close();
}

@SuppressWarnings("unchecked")
Expand Down Expand Up @@ -370,6 +371,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect
dataBlockRead.getRecords().size());
assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords3,
dataBlockRead.getRecords());
reader.close();
}

@SuppressWarnings("unchecked")
Expand Down Expand Up @@ -454,6 +456,8 @@ public void testAppendAndReadOnCorruptedLog() throws IOException, URISyntaxExcep
//assertEquals("", "something-random", new String(corruptBlock.getCorruptedBytes()));
assertFalse("There should be no more block left", reader.hasNext());

reader.close();

// Simulate another failure back to back
outputStream = fs.append(writer.getLogFile().getPath());
// create a block with
Expand Down Expand Up @@ -493,6 +497,7 @@ public void testAppendAndReadOnCorruptedLog() throws IOException, URISyntaxExcep
assertTrue("We should get the last block next", reader.hasNext());
reader.next();
assertFalse("We should have no more blocks left", reader.hasNext());
reader.close();
}


Expand Down Expand Up @@ -1097,7 +1102,7 @@ public void testMagicAndLogVersionsBackwardsCompatibility()
assertEquals(block.getBlockType(), HoodieLogBlockType.AVRO_DATA_BLOCK);
dBlock = (HoodieAvroDataBlock) block;
assertEquals(dBlock.getRecords().size(), 100);

reader.close();
}

@SuppressWarnings("unchecked")
Expand Down Expand Up @@ -1167,6 +1172,7 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect
dataBlockRead.getRecords());

assertFalse(reader.hasPrev());
reader.close();
}

@Test
Expand Down Expand Up @@ -1224,6 +1230,7 @@ public void testAppendAndReadOnCorruptedLogInReverse() throws IOException, URISy
e.printStackTrace();
// We should have corrupted block
}
reader.close();
}

@SuppressWarnings("unchecked")
Expand Down Expand Up @@ -1283,5 +1290,6 @@ record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect
dataBlockRead.getRecords());

assertFalse(reader.hasPrev());
reader.close();
}
}
Expand Up @@ -294,6 +294,7 @@ public static Schema readSchemaFromLogFile(FileSystem fs, Path path) throws IOEx
lastBlock = (HoodieAvroDataBlock) block;
}
}
reader.close();
if (lastBlock != null) {
return lastBlock.getSchema();
}
Expand Down
Expand Up @@ -447,6 +447,7 @@ public static MessageType readSchemaFromLogFile(FileSystem fs, Path path) throws
lastBlock = (HoodieAvroDataBlock) block;
}
}
reader.close();
if (lastBlock != null) {
return new parquet.avro.AvroSchemaConverter().convert(lastBlock.getSchema());
}
Expand Down

0 comments on commit 4bd15b0

Please sign in to comment.