Skip to content
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -303,6 +303,8 @@ private <R> Function2<Integer, Iterator<HoodieRecord<R>>, Iterator<HoodieRecord<
}
} catch (IOException e) {
throw new HoodieIndexException("Failed to Tag indexed locations because of exception with HBase Client", e);
} finally {
limiter.stop();
}
return taggedRecords.iterator();
};
Expand Down Expand Up @@ -338,8 +340,8 @@ private Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>> updateL
final long startTimeForPutsTask = DateTime.now().getMillis();
LOG.info("startTimeForPutsTask for this task: " + startTimeForPutsTask);

final RateLimiter limiter = RateLimiter.create(multiPutBatchSize, TimeUnit.SECONDS);
try (BufferedMutator mutator = hbaseConnection.getBufferedMutator(TableName.valueOf(tableName))) {
final RateLimiter limiter = RateLimiter.create(multiPutBatchSize, TimeUnit.SECONDS);
while (statusIterator.hasNext()) {
WriteStatus writeStatus = statusIterator.next();
List<Mutation> mutations = new ArrayList<>();
Expand Down Expand Up @@ -387,6 +389,8 @@ private Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>> updateL
LOG.info("hbase puts task time for this task: " + (endPutsTime - startTimeForPutsTask));
} catch (IOException e) {
throw new HoodieIndexException("Failed to Update Index locations because of exception with HBase Client", e);
} finally {
limiter.stop();
}
return writeStatusList.iterator();
};
Expand Down Expand Up @@ -586,10 +590,9 @@ public boolean rollbackCommit(String instantTime) {
hbaseConnection = getHBaseConnection();
}
}
final RateLimiter limiter = RateLimiter.create(multiPutBatchSize, TimeUnit.SECONDS);
try (HTable hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName));
BufferedMutator mutator = hbaseConnection.getBufferedMutator(TableName.valueOf(tableName))) {
final RateLimiter limiter = RateLimiter.create(multiPutBatchSize, TimeUnit.SECONDS);

Long rollbackTime = HoodieActiveTimeline.parseDateFromInstantTime(instantTime).getTime();
Long currentTime = new Date().getTime();
Scan scan = new Scan();
Expand Down Expand Up @@ -638,6 +641,8 @@ public boolean rollbackCommit(String instantTime) {
} catch (Exception e) {
LOG.error("hbase index roll back failed", e);
return false;
} finally {
limiter.stop();
}
return true;
}
Expand Down