Yu-Wen Lai has posted comments on this change. ( 
http://gerrit.cloudera.org:8080/17858 )

Change subject: IMPALA-10923: Fine grained table refreshing at partition level 
events for transactional tables
......................................................................


Patch Set 7:

(11 comments)

http://gerrit.cloudera.org:8080/#/c/17858/1/fe/src/main/java/org/apache/impala/catalog/Catalog.java
File fe/src/main/java/org/apache/impala/catalog/Catalog.java:

http://gerrit.cloudera.org:8080/#/c/17858/1/fe/src/main/java/org/apache/impala/catalog/Catalog.java@97
PS1, Line 97: protected final ConcurrentHashMap<Long, Set<TableWriteId>> 
txnToWriteIds_ =
            :       new ConcurrentHashMap<>();
> Thanks for the clarification. "the new HMS API getAllWriteEventInfo only re
getAllWriteEventInfo just return the data stored in the table 
TXN_WRITE_NOTIFICATION_LOG. AFAIK, HS2 calls add_write_notification_log that 
inserts records into TXN_WRITE_NOTIFICATION_LOG only for DML for transactional 
tables. I tried  few queries locally like "drop constraint", and they advance 
write id but don't add write notification log.

I tried to reduce the memory footprint here by saving write ids for 
transactional partitioned tables only. Besides, this map's size is just 
proportional to the simultaneous open transactions. Despite I don't have any 
real data points, we might not have a huge number of simultaneous "open" 
transactions?


http://gerrit.cloudera.org:8080/#/c/17858/1/fe/src/main/java/org/apache/impala/catalog/Catalog.java@97
PS1, Line 97: protected final ConcurrentHashMap<Long, Set<TableWriteId>> 
txnToWriteIds_ =
            :       new ConcurrentHashMap<>();
> @Yu-Wen: In addition to what Vihang asked, how would we handle the followin
@Sourabh
Good question. Since I don't see a way to retrieve back the missing write id, 
we might accept that this write id remains open. When next time a request with 
writeIdList that has this write id as committed, we will reload the whole table 
because the writeIdList of the request is considered more recent. In some 
sense, the table cache is considered stale when the write id is not marked 
committed.


http://gerrit.cloudera.org:8080/#/c/17858/6/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
File fe/src/main/java/org/apache/impala/catalog/HdfsTable.java:

http://gerrit.cloudera.org:8080/#/c/17858/6/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java@2891
PS6, Line 2891: case
> do we need a default: clause which throws a exception?
Done


http://gerrit.cloudera.org:8080/#/c/17858/6/fe/src/main/java/org/apache/impala/catalog/Table.java
File fe/src/main/java/org/apache/impala/catalog/Table.java:

http://gerrit.cloudera.org:8080/#/c/17858/6/fe/src/main/java/org/apache/impala/catalog/Table.java@142
PS6, Line 142: volatile
> not sure why we need this?
@Vihang
I call getCreateEventId() in AllocWriteIdEvent without acquiring lock. Is there 
any chance createEventId will be set after the table is loaded? If not, we 
don't need this.


http://gerrit.cloudera.org:8080/#/c/17858/6/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/17858/6/fe/src/main/java/org/apache/impala/catalog/events/MetastoreEvents.java@2066
PS6, Line 2066: catalog_.removeWriteIds(txnId_);
> This line must be in finally block otherwise we are leaking memory in case
Thank you for catching this.


http://gerrit.cloudera.org:8080/#/c/17858/7/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/17858/7/fe/src/main/java/org/apache/impala/catalog/events/MetastoreEvents.java@2057
PS7, Line 2057:         commitTxnMessage_.addWriteEventInfo(writeEventInfoList);
> Why are we modifying commitTxnMesage? Can't we get all the required info fr
@Sourabh
I actually imitated the code from hive repl: 
https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/CommitTxnHandler.java#L166.
The upside is that I don't have to parse table and partition objects. It is 
done by CommitTxnMessage. As I can see from the hive code, it seems like this 
function is used like this by design.


http://gerrit.cloudera.org:8080/#/c/17858/7/fe/src/main/java/org/apache/impala/catalog/events/MetastoreEvents.java@2080
PS7, Line 2080:       
Preconditions.checkNotNull(commitTxnMessage_.getPartitions());
> Why are we checking for non null partitions? Wouldn't unpartitioned table h
As long as we have called addWriteEventInfo, this would be empty list even for 
unpartitioned table. So, this is just to check we have added write event info. 
I can change the check to other variables to avoid confusion.


http://gerrit.cloudera.org:8080/#/c/17858/6/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
File fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java:

http://gerrit.cloudera.org:8080/#/c/17858/6/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java@4256
PS6, Line 4256: !(table instanceof HdfsTable)
> do we need a check for MVs here?
Thanks for the reminder. I added a check in CommitTxnEvent.


http://gerrit.cloudera.org:8080/#/c/17858/6/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java@4271
PS6, Line 4271: // HMS doesn't set partition object's write id correctly, so we 
set
              :         // it here as workaround.
              :         long writeId = writeIds.get(it.nextIndex());
              :         Partition part = it.next();
              :         part.setWriteId(writeId);
> This could lead to confusion, for instance, we may be sending the partition
I need to check if there is another way to fulfill my need here. I use this 
because I reload partitions from "event"'s partition objects without getting 
latest partitions from HMS so I need to check write id to avoid reloading an 
older partition object. Meanwhile, this can ensure the event processing for 
CommitTxnEvent is idempotent.


http://gerrit.cloudera.org:8080/#/c/17858/7/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
File fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java:

http://gerrit.cloudera.org:8080/#/c/17858/7/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java@4160
PS7, Line 4160:       List<Partition> partsFromEvent, boolean loadFromEvent, 
boolean loadFileMetadata,
> Looks like if loadFromEvent is true, then in HdfsTable we avoid making HMS
OK. I will separate another API for this usage.


http://gerrit.cloudera.org:8080/#/c/17858/7/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java@4233
PS7, Line 4233:   public int 
addCommittedWriteIdsAndReloadPartitionsIfExist(long eventId, String dbName,
> This api is not making much sense to me here. More so since it is public.
Could you elaborate the concern? It seems like we don't directly manipulate 
cache in MetastoreEvents but expose the API from CatalogOpExecutor or 
CatalogServiceCatalog as a convention.



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I6ba07c9a338a25614690e314335ee4b801486da9
Gerrit-Change-Number: 17858
Gerrit-PatchSet: 7
Gerrit-Owner: Yu-Wen Lai <[email protected]>
Gerrit-Reviewer: Anonymous Coward <[email protected]>
Gerrit-Reviewer: Fucun Chu <[email protected]>
Gerrit-Reviewer: Impala Public Jenkins <[email protected]>
Gerrit-Reviewer: Sourabh Goyal <[email protected]>
Gerrit-Reviewer: Vihang Karajgaonkar <[email protected]>
Gerrit-Reviewer: Yu-Wen Lai <[email protected]>
Gerrit-Comment-Date: Wed, 20 Oct 2021 21:52:28 +0000
Gerrit-HasComments: Yes

Reply via email to