Skip to content

HIVE-28735: Propagated the exception properly from hash table-loader threads to main thread #5845

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 4 commits into
base: master
Choose a base branch
from

Conversation

Paramvir109
Copy link

@Paramvir109 Paramvir109 commented Jun 4, 2025

What changes were proposed in this pull request?

Need to properly propagate the exception which can occur while parallel loading of fast Hash tables by the threads.
In the current scenario the thread is silently dying causing incorrect(lesser number)of HT entries to be reported which is giving incorrect results

Why are the changes needed?

It fixes bug caused by https://issues.apache.org/jira/browse/HIVE-25149 that hides actual exceptions and gives incorrect results. Example query is added in the jira : https://issues.apache.org/jira/browse/HIVE-28735

Is the change a dependency upgrade?

No

Does this PR introduce any user-facing change?

No

How was this patch tested?

Tested using the sample query.

Hive 3

Caused by: java.lang.OutOfMemoryError: Java heap space
    at org.apache.hadoop.hive.ql.exec.vector.mapjoin.fast.VectorMapJoinFastLongHashTable.expandAndRehash(VectorMapJoinFastLongHashTable.java:166)
    at org.apache.hadoop.hive.ql.exec.vector.mapjoin.fast.VectorMapJoinFastLongHashTable.add(VectorMapJoinFastLongHashTable.java:100)
    at org.apache.hadoop.hive.ql.exec.vector.mapjoin.fast.VectorMapJoinFastLongHashTable.adaptPutRow(VectorMapJoinFastLongHashTable.java:91)
    at org.apache.hadoop.hive.ql.exec.vector.mapjoin.fast.VectorMapJoinFastLongHashMap.putRow(VectorMapJoinFastLongHashMap.java:147)
    at org.apache.hadoop.hive.ql.exec.vector.mapjoin.fast.VectorMapJoinFastTableContainer.putRow(VectorMapJoinFastTableContainer.java:184)
    at org.apache.hadoop.hive.ql.exec.vector.mapjoin.fast.VectorMapJoinFastHashTableLoader.load(VectorMapJoinFastHashTableLoader.java:130)
    at org.apache.hadoop.hive.ql.exec.MapJoinOperator.loadHashTableInternal(MapJoinOperator.java:388)
    at org.apache.hadoop.hive.ql.exec.MapJoinOperator.loadHashTable(MapJoinOperator.java:457)
    at org.apache.hadoop.hive.ql.exec.MapJoinOperator.lambda$initializeOp$0(MapJoinOperator.java:241)
    ... 3 more

Hive 4

Before the fix : Wrong results on each run
After the fix : Getting same OOM exception as in hive 3

Also added a unit test which verifies the same

@@ -105,6 +110,23 @@ public void init(ExecMapperContext context, MapredContext mrContext,
this.htLoadCounter = tezContext.getTezProcessorContext().getCounters().findCounter(counterGroup, counterName);
}

@VisibleForTesting
protected void initHTLoadingServiceForTest(Configuration conf, long estKeyCount, ExecutorService loadExecService) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@Paramvir109, that is production code; it shouldn't contain any test-specific logic. Please use mocking techniques instead

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hey @deniskuzZ, The method which I need to test - submitQueueDrainThreads is a private method hence I'm making a protected testing method which will actually call the private method. And initHTLoadingService method which is responsible for initialising all the class members(private) required for this test also has a private access in the class. I won't be able to mock private class members. I can use reflection to set the private class members but it won't be a good idea I believe

for (int partitionId = 0; partitionId < numLoadThreads; partitionId++) {
int finalPartitionId = partitionId;
this.loadExecService.submit(() -> {
Future<?> future = this.loadExecService.submit(() -> {
Copy link
Member

@deniskuzZ deniskuzZ Jun 6, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

please use CompletableFuture instead

List<CompletableFuture<Void>> loaderTasks = new ArrayList<>();
...................

CompletableFuture<Void> asyncTask = CompletableFuture.runAsync(() -> {
  try {
    LOG.info("Partition id {} with Queue size {}", finalPartitionId, loadBatchQueues[finalPartitionId].size());
    drainAndLoadForPartition(finalPartitionId, vectorMapJoinFastTableContainer);
  } catch (IOException | InterruptedException | SerDeException | HiveException e) {
    throw new RuntimeException("Failed to start HT Load thread", e);
  }
}, loadExecService);

.....................

loaderTasks.add(asyncTask);

try {
  CompletableFuture.allOf(loaderTasks.toArray(new CompletableFuture[0]))
    .get(2, TimeUnit.MINUTES);
} catch (TimeoutException e) {
  throw new HiveException("Failed to complete the hash table loader. Loading timed out.");
} catch (ExecutionException e) {
  throw new HiveException("One of the loader threads failed", e.getCause());
} catch (InterruptedException e) {
  Thread.currentThread().interrupt();
} finally {
  if (loadExecService != null && !loadExecService.isTerminated()) {
    loadExecService.shutdownNow();
  }
}

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ack, I'll update it.

Copy link

sonarqubecloud bot commented Jun 8, 2025

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants