[HUDI-312] Make docker hdfs cluster ephemeral. This is needed to fix flakiness in integration tests. Also, Fix DeltaStreamer hanging issue due to uncaught exception
This commit is contained in:
committed by
Balaji Varadarajan
parent
144ea4eedf
commit
a6390aefc4
@@ -30,6 +30,7 @@ import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
@@ -251,7 +252,9 @@ public class DeltaSync implements Serializable {
|
||||
} else {
|
||||
throw new HoodieDeltaStreamerException(
|
||||
"Unable to find previous checkpoint. Please double check if this table "
|
||||
+ "was indeed built via delta streamer ");
|
||||
+ "was indeed built via delta streamer. Last Commit :" + lastCommit + ", Instants :"
|
||||
+ commitTimelineOpt.get().getInstants().collect(Collectors.toList()) + ", CommitMetadata="
|
||||
+ commitMetadata.toJsonString());
|
||||
}
|
||||
}
|
||||
} else {
|
||||
|
||||
@@ -117,9 +117,15 @@ public class HoodieDeltaStreamer implements Serializable {
|
||||
log.info("Delta Sync shutting down");
|
||||
} else {
|
||||
log.info("Delta Streamer running only single round");
|
||||
deltaSyncService.getDeltaSync().syncOnce();
|
||||
deltaSyncService.close();
|
||||
log.info("Shut down deltastreamer");
|
||||
try {
|
||||
deltaSyncService.getDeltaSync().syncOnce();
|
||||
} catch (Exception ex) {
|
||||
log.error("Got error running delta sync once. Shutting down", ex);
|
||||
throw ex;
|
||||
} finally {
|
||||
deltaSyncService.close();
|
||||
log.info("Shut down deltastreamer");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user