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 36: (23 comments) http://gerrit.cloudera.org:8080/#/c/21031/34/fe/src/compat-apache-hive-3/java/org/apache/impala/compat/MetastoreShim.java File fe/src/compat-apache-hive-3/java/org/apache/impala/compat/MetastoreShim.java: http://gerrit.cloudera.org:8080/#/c/21031/34/fe/src/compat-apache-hive-3/java/org/apache/impala/compat/MetastoreShim.java@659 PS34, Line 659: > nit: Apache Hive-3 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<WriteEventInfo> writeEventInfoList = : client.getHiveClient().getAllWriteEventInfo( : new GetAllWriteEventInfoRequest(event.txnId_)); What happen if client failed to talk with MetaStore here? http://gerrit.cloudera.org:8080/#/c/21031/36/fe/src/compat-hive-3/java/org/apache/impala/compat/MetastoreShim.java@1102 PS36, Line 1102: Event processing cannot " + : "continue Why event processing can not continue? Can you narrow down the error message here? http://gerrit.cloudera.org:8080/#/c/21031/34/fe/src/main/java/org/apache/impala/catalog/events/DBBarrierEvent.java File fe/src/main/java/org/apache/impala/catalog/events/DBBarrierEvent.java: http://gerrit.cloudera.org:8080/#/c/21031/34/fe/src/main/java/org/apache/impala/catalog/events/DBBarrierEvent.java@136 PS34, Line 136: */ > Please revisit the access level for some of these methods and turn them int Done 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: return; : } > This is obtaining usableLock_ twice. Is that OK? This is not addressed yet. http://gerrit.cloudera.org:8080/#/c/21031/36/fe/src/main/java/org/apache/impala/catalog/events/EventExecutorService.java File fe/src/main/java/org/apache/impala/catalog/events/EventExecutorService.java: http://gerrit.cloudera.org:8080/#/c/21031/36/fe/src/main/java/org/apache/impala/catalog/events/EventExecutorService.java@143 PS36, Line 143: void process(MetastoreEvent event) throws MetastoreNotificationException { This method handle multiple cases. Can you encapsulate each branch as its own method please? http://gerrit.cloudera.org:8080/#/c/21031/36/fe/src/main/java/org/apache/impala/catalog/events/ExternalEventsProcessor.java File fe/src/main/java/org/apache/impala/catalog/events/ExternalEventsProcessor.java: http://gerrit.cloudera.org:8080/#/c/21031/36/fe/src/main/java/org/apache/impala/catalog/events/ExternalEventsProcessor.java@49 PS36, Line 49: /** : * Holds the event processing till it is resumed again. It is not meant to change the : * event processor status. It gives the control to not process the events though event : * processor status is active. : * {@link org.apache.impala.catalog.events.ExternalEventsProcessor#hold()} and : * {@link org.apache.impala.catalog.events.ExternalEventsProcessor#resume()} can be : * used together if we want to measure the time taken to process the set of events : * deterministically in the end to end tests once all of the events are available : * to process. : */ : void hold(); Please contrast this against pause(). Why we need hold() if we have pause()? Should behavior of pause() changed instead if hierarchical event processing is enabled? Please make distinction between processing event vs queueing event. It looks like pause() drop all queued event and stop queueing new events. On the other hand, hold() just stop queueing new event but continue processing any that has been queued. Please clarify this. http://gerrit.cloudera.org:8080/#/c/21031/36/fe/src/main/java/org/apache/impala/catalog/events/ExternalEventsProcessor.java@61 PS36, Line 61: /** : * Resumes the event processing kept on hold. It is not meant to change the event : * processor status. It allows the event processor to process the events iff the event : * processor is active. : */ : void resume(); Please contrast this against stop(). Why we need resume if we have stop()? Should behavior of stop() changed instead if hierarchical event processing is enabled? http://gerrit.cloudera.org:8080/#/c/21031/34/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/34/fe/src/main/java/org/apache/impala/catalog/events/MetastoreEventsProcessor.java@779 PS34, Line 779: etrics_ > Adds: Done http://gerrit.cloudera.org:8080/#/c/21031/36/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/36/fe/src/main/java/org/apache/impala/catalog/events/MetastoreEventsProcessor.java@899 PS36, Line 899: eventExecutorService_.clear(); Why pause() translate to eventExecutorService_.clear()? Can you elaborate in method documentation what happen when clear is called? Everything still in queue will be discarded? What if we have dependent Pseudo event like RenameTableBarrierEvent, PseudoCommitTxnEvent or PseudoAbortTxnEvent, and only subset of them done processed? http://gerrit.cloudera.org:8080/#/c/21031/34/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/34/fe/src/main/java/org/apache/impala/catalog/events/RenameTableBarrierEvent.java@66 PS34, Line 66: private final State state_; : : /** : * Rename event processing state of alter table rename event. It is combined state of : * both pseudo-events. > I think using 2 Queue object per RenameTableBarrierEvent is overkill here. Done http://gerrit.cloudera.org:8080/#/c/21031/34/fe/src/main/java/org/apache/impala/catalog/events/RenameTableBarrierEvent.java@79 PS34, Line 79: // Whether pseudo create ta > Add precondition to check that pseudoEvent is either of DROP_TABLE or CREAT Done http://gerrit.cloudera.org:8080/#/c/21031/34/fe/src/main/java/org/apache/impala/catalog/events/RenameTableBarrierEvent.java@139 PS34, Line 139: Metastor > Add Precondition in this else block. Done 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: State RenameEventState is more descriptive. Also, I am more confident if all accessor method here have synchronized modifier. I understand that volatie should be atomic for single variable access/update, but I am not sure if its is still atomic if two volatile variables are accessed & updated concurrently without synchronized. It is better to avoid potential concurrency bug. Please create unit test just for this state class and validate return value of each method after subsequent setProcessed() call. http://gerrit.cloudera.org:8080/#/c/21031/36/fe/src/main/java/org/apache/impala/catalog/events/RenameTableBarrierEvent.java@109 PS36, Line 109: setProcessed Add Precondition that prevent True variable from turning back to False. http://gerrit.cloudera.org:8080/#/c/21031/36/fe/src/main/java/org/apache/impala/catalog/events/RenameTableBarrierEvent.java@113 PS36, Line 113: } else { Add Precondition in else block that eventType is CREATE_TABLE. http://gerrit.cloudera.org:8080/#/c/21031/36/fe/src/main/java/org/apache/impala/catalog/events/RenameTableBarrierEvent.java@157 PS36, Line 157: if (MetastoreEventType.DROP_TABLE.equals(getEventType())) { : return !state_.isDropProcessed(); : } : return state_.isDropProcessed() && !state_.isCreateProcessed(); This should be synchronized methods inside State object, like setProcessed(). 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: } : } : LOG.debug( > This also does not seem right to me. This is not addressed yet. Please change branch in L307 to Preconditon. Precondition.checkState(outstandingEventCount_ > 0); http://gerrit.cloudera.org:8080/#/c/21031/34/fe/src/main/java/org/apache/impala/catalog/events/TableEventExecutor.java@333 PS34, Line 333: Preconditions.checkState(tabl > This should be Precondition at the end of method rather than assignment. This is not addressed yet. 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: getOutstandingEventCount getOutstandingEventCount() is always valid to call, whether hierarchical event processing is enabled or not, right? http://gerrit.cloudera.org:8080/#/c/21031/36/fe/src/test/java/org/apache/impala/catalog/metastore/CatalogHmsSyncToLatestEventIdTest.java File fe/src/test/java/org/apache/impala/catalog/metastore/CatalogHmsSyncToLatestEventIdTest.java: http://gerrit.cloudera.org:8080/#/c/21031/36/fe/src/test/java/org/apache/impala/catalog/metastore/CatalogHmsSyncToLatestEventIdTest.java@106 PS36, Line 106: 10000L Why pollingFrequencyInMilliSec increased to 10s here? http://gerrit.cloudera.org:8080/#/c/21031/34/tests/custom_cluster/test_event_processing_perf.py File tests/custom_cluster/test_event_processing_perf.py: http://gerrit.cloudera.org:8080/#/c/21031/34/tests/custom_cluster/test_event_processing_perf.py@220 PS34, Line 220: > I suggest rename to __run_event_processing_tests to distinct from other met Done http://gerrit.cloudera.org:8080/#/c/21031/34/tests/custom_cluster/test_events_custom_configs.py File tests/custom_cluster/test_events_custom_configs.py: http://gerrit.cloudera.org:8080/#/c/21031/34/tests/custom_cluster/test_events_custom_configs.py@1445 PS34, Line 1445: rt_values = [ > I think 'rename_table_with_hierarchical_event_process' is more accurate? 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: 36 Gerrit-Owner: Anonymous Coward <[email protected]> Gerrit-Reviewer: Anonymous Coward <[email protected]> Gerrit-Reviewer: Anonymous Coward <[email protected]> Gerrit-Reviewer: Csaba Ringhofer <[email protected]> Gerrit-Reviewer: Impala Public Jenkins <[email protected]> Gerrit-Reviewer: Quanlong Huang <[email protected]> Gerrit-Reviewer: Riza Suminto <[email protected]> Gerrit-Reviewer: Sai Hemanth Gantasala <[email protected]> Gerrit-Comment-Date: Thu, 06 Feb 2025 02:06:36 +0000 Gerrit-HasComments: Yes
