Riza Suminto has posted comments on this change. ( 
http://gerrit.cloudera.org:8080/21031 )

Change subject: IMPALA-12709: Add support for hierarchical metastore event 
processing
......................................................................


Patch Set 38:

(24 comments)

Thank you for your hard work on this patch. I will need another pass because 
the changes are pretty big since the last time I review.

This will be a new Impala subsystem that manage critical states.
My general comment is, if there is a tradeoff between performance and 
correctness, lets aim for ultimate correctness. The fact that 
EventExecutorService is multithreaded should guarantee some performance gain 
thatn current single threaded event processor, and we can continue improving 
performance in next iteration.

http://gerrit.cloudera.org:8080/#/c/21031/34//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/21031/34//COMMIT_MSG@63
PS34, Line 63: Following new events are added:
> Done
Done


http://gerrit.cloudera.org:8080/#/c/21031/36/fe/src/compat-hive-3/java/org/apache/impala/compat/MetastoreShim.java
File fe/src/compat-hive-3/java/org/apache/impala/compat/MetastoreShim.java:

http://gerrit.cloudera.org:8080/#/c/21031/36/fe/src/compat-hive-3/java/org/apache/impala/compat/MetastoreShim.java@1060
PS36, Line 1060:     List<PseudoCommitTxnEvent> pseudoEvents = new 
ArrayList<>();
               :     List<WriteEventInfo> writeEventInfoList;
               :     try (MetaStoreClientPool.MetaStoreClient client = event.g
> Have refactored a bit to make this same as without hierarchical event proce
Ack.
Not related to this patch, but it sounds like these two flags can not be both 
False?
invalidate_metadata_on_event_processing_failure
invalidate_global_metadata_on_event_processing_failure


http://gerrit.cloudera.org:8080/#/c/21031/34/fe/src/main/java/org/apache/impala/catalog/events/DBEventExecutor.java
File fe/src/main/java/org/apache/impala/catalog/events/DBEventExecutor.java:

http://gerrit.cloudera.org:8080/#/c/21031/34/fe/src/main/java/org/apache/impala/catalog/events/DBEventExecutor.java@244
PS34, Line 244:
              :
> Have used isUsable() method at all places instead of checking usable_ varia
Ack


http://gerrit.cloudera.org:8080/#/c/21031/34/fe/src/main/java/org/apache/impala/catalog/events/DBEventExecutor.java@438
PS34, Line 438:
              :
              :
> inEvents_ and barrierEvents_ are of type ConcurrentLinkedQueue(thread safe)
Thank you for your explanation. I have follow up comment in patch set 38.


http://gerrit.cloudera.org:8080/#/c/21031/34/fe/src/main/java/org/apache/impala/catalog/events/DBEventExecutor.java@515
PS34, Line 515:
> Please check the above comment's reply.
Ack


http://gerrit.cloudera.org:8080/#/c/21031/38/fe/src/main/java/org/apache/impala/catalog/events/DbEventExecutor.java
File fe/src/main/java/org/apache/impala/catalog/events/DbEventExecutor.java:

http://gerrit.cloudera.org:8080/#/c/21031/38/fe/src/main/java/org/apache/impala/catalog/events/DbEventExecutor.java@457
PS38, Line 457:   long getOutstandingEventCount() {
              :     long outstandingEventCount;
              :     synchronized (outstandingEventCountLock_) {
              :       outstandingEventCount = outstandingEventCount_;
              :     }
              :     return outstandingEventCount + tableEventExecutors_.stream()
              :         
.mapToLong(TableEventExecutor::getOutstandingEventCount).sum();
              :   }
Looking again, we might have a potential split-brain problem here by having 
separate outstandingEventCount_ counter.
Is it possible to count directly from the data structure?

Then outstandingEventCount_, incrOutstandingEventCount() and 
decrOutstandingEventCount() can be removed.
Access to both offer() and poll() against inEvents_ and dbEvents_ should 
subsequently hold dbEventExecutor_.outstandingEventCountLock_.


http://gerrit.cloudera.org:8080/#/c/21031/34/fe/src/main/java/org/apache/impala/catalog/events/MetastoreEvents.java
File fe/src/main/java/org/apache/impala/catalog/events/MetastoreEvents.java:

http://gerrit.cloudera.org:8080/#/c/21031/34/fe/src/main/java/org/apache/impala/catalog/events/MetastoreEvents.java@588
PS34, Line 588: protected static final String CLUSTER_WIDE_TARGET = 
"CLUSTER_WIDE";
> It is not added with this patch.
Ack


http://gerrit.cloudera.org:8080/#/c/21031/38/fe/src/main/java/org/apache/impala/catalog/events/MetastoreEventsProcessor.java
File 
fe/src/main/java/org/apache/impala/catalog/events/MetastoreEventsProcessor.java:

http://gerrit.cloudera.org:8080/#/c/21031/38/fe/src/main/java/org/apache/impala/catalog/events/MetastoreEventsProcessor.java@840
PS38, Line 840:   public EventExecutorService getEventExecutorService() {
Annotate that this may return null.

  public @Nullable EventExecutorService getEventExecutorService() {


http://gerrit.cloudera.org:8080/#/c/21031/38/fe/src/main/java/org/apache/impala/catalog/events/MetastoreEventsProcessor.java@844
PS38, Line 844:   @VisibleForTesting
              :   public void setEventExecutorService(EventExecutorService 
eventExecutorService) {
              :     eventExecutorService_ = eventExecutorService;
              :   }
Please add explicit comment that this is only for testing and must not be used 
anywhere else.

Should this add validation that eventExecutorService is not null?


http://gerrit.cloudera.org:8080/#/c/21031/38/fe/src/main/java/org/apache/impala/catalog/events/MetastoreEventsProcessor.java@1165
PS38, Line 1165:     } catch (Exception ex) {
               :       handleEventProcessException(ex);
               :     } finally {
               :       if (eventExecutorService_ != null) {
               :         eventExecutorService_.cleanup();
               :       }
               :     }
Here, Exception is followed by eventExecutorService_.cleanup().
But other call sites in TableEventExecutor.process() and 
DbEventExecutor.process() is not.

Why is that the case? Will there be behavior difference or correctness issue?


http://gerrit.cloudera.org:8080/#/c/21031/38/fe/src/main/java/org/apache/impala/catalog/events/MetastoreEventsProcessor.java@1173
PS38, Line 1173:
               :   public void handleEventProcessException(Exception ex) {
Please add comment for this method.


http://gerrit.cloudera.org:8080/#/c/21031/36/fe/src/main/java/org/apache/impala/catalog/events/RenameTableBarrierEvent.java
File 
fe/src/main/java/org/apache/impala/catalog/events/RenameTableBarrierEvent.java:

http://gerrit.cloudera.org:8080/#/c/21031/36/fe/src/main/java/org/apache/impala/catalog/events/RenameTableBarrierEvent.java@72
PS36, Line 72: nt pr
> Renamed class to RenameEventState.
Thank you for your explanation. Catalog and HMS event code manage catalog state 
where correctness is critical to maintain. To me, synchronized method guarantee 
correctness with minimal performance tradeoff. I'm leaving new comment in patch 
set 38.


http://gerrit.cloudera.org:8080/#/c/21031/36/fe/src/main/java/org/apache/impala/catalog/events/RenameTableBarrierEvent.java@109
PS36, Line 109: aram skipped
> Done. Have removed processed argument as we always need to pass true. And a
Done


http://gerrit.cloudera.org:8080/#/c/21031/36/fe/src/main/java/org/apache/impala/catalog/events/RenameTableBarrierEvent.java@113
PS36, Line 113:    Precon
> Done
Done


http://gerrit.cloudera.org:8080/#/c/21031/36/fe/src/main/java/org/apache/impala/catalog/events/RenameTableBarrierEvent.java@157
PS36, Line 157:    * Determines whether this event can be processed. 
TableProcessor uses this method to
              :    * check if the event can be processed before processing it.
              :    * @return True if the event can be processed. False otherwise
              :    */
> IMHO, just keeping dropProcessed_ and dropSkipped_ flags as volatile is eno
Ack


http://gerrit.cloudera.org:8080/#/c/21031/38/fe/src/main/java/org/apache/impala/catalog/events/RenameTableBarrierEvent.java
File 
fe/src/main/java/org/apache/impala/catalog/events/RenameTableBarrierEvent.java:

http://gerrit.cloudera.org:8080/#/c/21031/38/fe/src/main/java/org/apache/impala/catalog/events/RenameTableBarrierEvent.java@71
PS38, Line 71:   /**
             :    * Rename event processing state of alter table rename event. 
It is combined state of
             :    * both pseudo-events.
             :    */
Please add a comment in this class on why volatile variables are enough to 
ensure correctness. It will help warn us if future code changes is going to 
break the current premise.

What I understand from your explanation is, volatile boolean is sufficient 
since True value is not allowed to return back to initial (False) value.


http://gerrit.cloudera.org:8080/#/c/21031/38/fe/src/main/java/org/apache/impala/catalog/events/RenameTableBarrierEvent.java@82
PS38, Line 82:     // Whether pseudo create table event is skipped upon the 
event process.
             :     private boolean createSkipped_ = false;
             :
             :     // Whether pseudo create table event is processed.
             :     private boolean createProcessed_ = false;
Should this stay volatile like the rest?


http://gerrit.cloudera.org:8080/#/c/21031/34/fe/src/main/java/org/apache/impala/catalog/events/TableEventExecutor.java
File fe/src/main/java/org/apache/impala/catalog/events/TableEventExecutor.java:

http://gerrit.cloudera.org:8080/#/c/21031/34/fe/src/main/java/org/apache/impala/catalog/events/TableEventExecutor.java@312
PS34, Line 312:    */
              :   private void decrOutstandingEventCount() {
              :     synchronize
> events_ is of type ConcurrentLinkedQueue(thread safe).It is not explicitly
Thank you. I have a follow up comment in patch set 38.


http://gerrit.cloudera.org:8080/#/c/21031/34/fe/src/main/java/org/apache/impala/catalog/events/TableEventExecutor.java@333
PS34, Line 333:   }
> Please check the above comment's reply.
Ack


http://gerrit.cloudera.org:8080/#/c/21031/38/fe/src/main/java/org/apache/impala/catalog/events/TableEventExecutor.java
File fe/src/main/java/org/apache/impala/catalog/events/TableEventExecutor.java:

http://gerrit.cloudera.org:8080/#/c/21031/38/fe/src/main/java/org/apache/impala/catalog/events/TableEventExecutor.java@295
PS38, Line 295:   long getOutstandingEventCount() {
              :     synchronized (outstandingEventCountLock_) {
              :       return outstandingEventCount_;
              :     }
              :   }
Is it possible to count directly by summing events_.size() of all 
tableProcessors_?

Then outstandingEventCount_, incrOutstandingEventCount() and 
decrOutstandingEventCount() can be removed.
Access to both offer() and poll() against events_ should subsequently hold 
tableEventExecutor_.outstandingEventCountLock_.


http://gerrit.cloudera.org:8080/#/c/21031/38/fe/src/test/java/org/apache/impala/catalog/events/EventExecutorServiceTest.java
File 
fe/src/test/java/org/apache/impala/catalog/events/EventExecutorServiceTest.java:

http://gerrit.cloudera.org:8080/#/c/21031/38/fe/src/test/java/org/apache/impala/catalog/events/EventExecutorServiceTest.java@93
PS38, Line 93: eventExecutorService_
Assert that this is not null.


http://gerrit.cloudera.org:8080/#/c/21031/38/fe/src/test/java/org/apache/impala/catalog/events/EventExecutorServiceTest.java@114
PS38, Line 114:     
eventsProcessor_.setEventExecutorService(eventExecutorService_);
Please add comment why this is necessary.
I'm guessing that some tests replaced the internal EventExecutorService 
in-between tests?


http://gerrit.cloudera.org:8080/#/c/21031/38/fe/src/test/java/org/apache/impala/catalog/events/EventExecutorServiceTest.java@125
PS38, Line 125:     
assertEquals(MetastoreEventsProcessor.EventProcessorStatus.ACTIVE,
              :         eventsProcessor_.getStatus());
Also assert that test ends with empty eventsProcessor_?


http://gerrit.cloudera.org:8080/#/c/21031/36/fe/src/test/java/org/apache/impala/catalog/events/SynchronousHMSEventProcessorForTests.java
File 
fe/src/test/java/org/apache/impala/catalog/events/SynchronousHMSEventProcessorForTests.java:

http://gerrit.cloudera.org:8080/#/c/21031/36/fe/src/test/java/org/apache/impala/catalog/events/SynchronousHMSEventProcessorForTests.java@51
PS36, Line 51: ad.sleep(10);
> Correct. Removed explicit hierarchical processing enabled check now.
Done



--
To view, visit http://gerrit.cloudera.org:8080/21031
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I76d8a739f9db6d40f01028bfd786a85d83f9e5d6
Gerrit-Change-Number: 21031
Gerrit-PatchSet: 38
Gerrit-Owner: Anonymous Coward <k.venureddy2...@gmail.com>
Gerrit-Reviewer: Anonymous Coward <cclive1...@gmail.com>
Gerrit-Reviewer: Anonymous Coward <k.venureddy2...@gmail.com>
Gerrit-Reviewer: Csaba Ringhofer <csringho...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <impala-public-jenk...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <huangquanl...@gmail.com>
Gerrit-Reviewer: Riza Suminto <riza.sumi...@cloudera.com>
Gerrit-Reviewer: Sai Hemanth Gantasala <saihema...@cloudera.com>
Gerrit-Comment-Date: Thu, 27 Feb 2025 01:46:06 +0000
Gerrit-HasComments: Yes

Reply via email to