vjagadish1989 commented on a change in pull request #1086: SAMZA-2256:
Hotstandby fix for exception while reading side inputs
URL: https://github.com/apache/samza/pull/1086#discussion_r295083551
##########
File path:
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java
##########
@@ -705,43 +714,42 @@ public void run() {
this.sideInputSystemConsumers.start();
// create a thread for sideInput reads
- Thread readSideInputs = new Thread(() -> {
- while (!shutDownSideInputRead) {
- IncomingMessageEnvelope envelope =
sideInputSystemConsumers.choose(true);
- if (envelope != null) {
-
- if (!envelope.isEndOfStream())
-
sideInputStorageManagers.get(envelope.getSystemStreamPartition()).process(envelope);
+ Thread readSideInputsThread = new Thread(() -> {
+ try {
+ while (!shutDownSideInputRead) {
+ IncomingMessageEnvelope envelope =
sideInputSystemConsumers.choose(true);
+ if (envelope != null) {
- checkSideInputCaughtUp(envelope.getSystemStreamPartition(),
envelope.getOffset(),
- SystemStreamMetadata.OffsetType.NEWEST,
envelope.isEndOfStream());
+ if (!envelope.isEndOfStream())
+
sideInputStorageManagers.get(envelope.getSystemStreamPartition()).process(envelope);
- } else {
- LOG.trace("No incoming message was available");
+ checkSideInputCaughtUp(envelope.getSystemStreamPartition(),
envelope.getOffset(),
+ SystemStreamMetadata.OffsetType.NEWEST,
envelope.isEndOfStream());
+ } else {
+ LOG.trace("No incoming message was available");
+ }
}
+ } catch (Exception e) {
+ LOG.error("Exception in reading side inputs", e);
+ sideInputException = Optional.of(e);
}
});
-
- readSideInputs.setDaemon(true);
- readSideInputs.setUncaughtExceptionHandler(new
Thread.UncaughtExceptionHandler() {
- @Override
- public void uncaughtException(Thread t, Throwable e) {
- sideInputException = Optional.of(e);
- sideInputsCaughtUp.countDown();
- }
- });
+ readSideInputsThread.setName(SIDEINPUTS_READ_THREAD_NAME);
+ readSideInputsThread.setDaemon(true);
try {
- readSideInputs.start();
- // Make the main thread wait until all sideInputs have been caughtup or
thrown an exception
- this.sideInputsCaughtUp.await();
+ readSideInputsThread.start();
+
+ // Make the main thread wait until all sideInputs have been caughtup or
an exception was thrown
+ while (!sideInputException.isPresent() &&
!this.sideInputsCaughtUp.await(SIDE_INPUT_READ_THREAD_TIMEOUT.getSeconds(),
TimeUnit.SECONDS)) {
Review comment:
Consider using a volatile field instead of Optional to track
`sideInputException`?
Personally don't prefer Optional field here since (1) optionality is
trivially obvious and (2) it wasn't the [intended
goal](http://mail.openjdk.java.net/pipermail/lambda-dev/2012-September/005952.html)
of Optional-api, which is to provide better composability when dealing with
Nullables from public-functions.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services