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
HBASE-25392 Direct insert compacted HFiles into data directory. #3389
Conversation
💔 -1 overall
This message was automatically generated. |
Change-Id: Iff62fd5d402429115b743dd22293c357cbf1a29e
💔 -1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nice change. I posted some minor comments.
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
Show resolved
Hide resolved
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
Show resolved
Hide resolved
...ver/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDefaultCompactor.java
Outdated
Show resolved
Hide resolved
Change-Id: I5ac86a27c87e956926860ad9fb3eaeed3be75d00
🎊 +1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
...r/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DirectInStoreCompactor.java
Outdated
Show resolved
Hide resolved
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DirectInStoreFlushContext.java
Outdated
Show resolved
Hide resolved
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DirectInStoreFlushContext.java
Outdated
Show resolved
Hide resolved
if (store.getHRegion().getRegionServerServices() != null) { | ||
favoredNodes = store.getHRegion().getRegionServerServices().getFavoredNodesForRegion( | ||
store.getHRegion().getRegionInfo().getEncodedName()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why not use this directly from the StoreContext: https://github.com/apache/hbase/blob/master/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreContext.java#L95
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I already need an instance of HStore, as you noted on your comment below. This HStore instance need is not exclusive from this Compactor extension, it is already defined on the parent classes single declared constructors, so it's used for other stuff in the hierarchy, not only the filereader creation. I don't think there's much value in create an extra StoreContext here when we have everything we need in HStore instance itself.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The StoreContext is initialized with the store so there is no need to create a context. I'm specifically suggesting to use the store.getStoreContext().getFavoredNodes() instead of having to expose the RSServices and have all this extra code here.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Wasn't finding store.getStoreContext()
before, since it was package private. Made it public on HStore and am calling it there now.
...r/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DirectInStoreCompactor.java
Outdated
Show resolved
Hide resolved
Change-Id: I7a79666b8856715b1ad4421162d2b4658c213f4d
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
* @param totalCompactedFilesSize the total size of compacted files. | ||
* @return true if the checks mentioned above pass and the cache is enabled, false otherwise. | ||
*/ | ||
public boolean enableCacheOnWrite(long totalCompactedFilesSize) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should we include something to distinguish that this is specifically for compactions in the method name since the CacheConfig is used in more places than just compactions?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Renamed it on last commit.
Compression.Algorithm compression, boolean includeMVCCReadpoint, boolean includesTag, | ||
boolean shouldDropBehind, long totalCompactedFilesSize) throws IOException { | ||
final CacheConfig writerCacheConf; | ||
// Don't cache data on write on compactions. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Update/remove this comment since now it depends on the configs
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Removed now.
Change-Id: I26d10e3cb52a14bc82f13647ff9921c92282a353
🎊 +1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
Change-Id: I40ca4a799f9f4a4e406f34a637cc75ec765a21d9
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
2nd pass (I was running out the door the first time I looked this over and was going to be repeating a bit from Stack, so I abstained).
It's nice to see this coming together, Wellington. Let's see if we can think through how we can clean up some of the smells Stack points out.
hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java
Show resolved
Hide resolved
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
Show resolved
Hide resolved
@@ -694,7 +700,7 @@ private void refreshStoreFilesInternal(Collection<StoreFileInfo> newFiles) throw | |||
refreshStoreSizeAndTotalBytes(); | |||
} | |||
|
|||
protected HStoreFile createStoreFileAndReader(final Path p) throws IOException { | |||
public HStoreFile createStoreFileAndReader(final Path p) throws IOException { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I keep trying to think about what the right abstraction to have for our internal API.
Is there a reason to not have the pluggable compaction code have a method like this to return the HStoreFile? Glancing, I don't see any obvious methods/classes which are package-private to HStore (that would require these methods to live here).
Although, I also see that HStore#openStoreFiles
is using one instance of createStoreFileAndReader
, so we couldn't completely move this out of HStore. Is it better to lift one method into Store
and leave other instances "internal" to HStore? That might help the smell a little, but an obvious solution isn't jumping out at me.
@@ -1192,7 +1195,7 @@ public StoreFileWriter createWriterInTmp(long maxKeyCount, Compression.Algorithm | |||
return builder.build(); | |||
} | |||
|
|||
HFileContext createFileContext(Compression.Algorithm compression, | |||
public HFileContext createFileContext(Compression.Algorithm compression, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
re-implement this in the DirectStoreCompactor
I feel like that would just result in having to change code in multiple places which is error-prone. What about moving this onto HFileContext
itself, or the Builder
. Create a method which lets you pass the Builder
an Store
and do most of this work there. That seems to be more appropriate to me (at a glance) than doing this in HStore.
@@ -135,7 +135,7 @@ public CompactionProgress getProgress() { | |||
/** Min SeqId to keep during a major compaction **/ | |||
public long minSeqIdToKeep = 0; | |||
/** Total size of the compacted files **/ | |||
private long totalCompactedFilesSize = 0; | |||
public long totalCompactedFilesSize = 0; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
suggestion: leave the variable private, make a public (if necessary) getter for this method, and make a package-private/private method to mutate the value.
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
Show resolved
Hide resolved
...ver/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDefaultCompactor.java
Show resolved
Hide resolved
when(mockStore.getRegionFileSystem()).thenReturn(mockRegionFS); | ||
when(mockRegionFS.getRegionDir()).thenReturn(filePath); | ||
when(mockStore.getColumnFamilyName()).thenReturn(cfName); | ||
HFileContext mockFileContext = mock(HFileContext.class); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What about making a real HFileContext
to avoid ~5 mockings?
when(mockStore.getHRegion()).thenReturn(mock(HRegion.class)); | ||
when(mockStore.getStoreContext()).thenReturn(new StoreContext.Builder(). | ||
withFavoredNodesSupplier(()-> null).build()); | ||
ColumnFamilyDescriptor mockDesc = mock(ColumnFamilyDescriptor.class); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Same here: real CFD?
...src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDirectStoreCompactor.java
Show resolved
Hide resolved
Change-Id: I618ba247caf013c30eb54ea9bf0e36d0d1baabe3
🎊 +1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
Change-Id: I120093272991de10d33f02903e7b5e51c6767978
💔 -1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
Change-Id: Ie439d3a0dc6d6e5006c37c72711b0f236aa38908
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think migrating the Function to some kind of named and documented interface is the only thing I see remaining. Good work, Wellington. I think I'm +1 after this.
My comment about StripeCompactor and DTCompactor can be deferred, I think. Doesn't need to complicate this further.
p -> { | ||
try { | ||
return this.createStoreFileAndReader((Path) p); | ||
}catch(IOException e){ |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit whitespace/checkstyle
but I like the solution!
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
Outdated
Show resolved
Hide resolved
…ctor Change-Id: I129da2790d72e0a1043f6359704bc9fc423bcc30
Yeah, had thought about it previously. Had just added it in the last commit.
Agreed. Right now, SC and DTC will only work with the default engine that does temp/rename. We can work later on support direct store for these type of compactions too. |
🎊 +1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1 on the latest changes here. Looks like we have some flakey tests, but only one of them didn't pass on re-run
[ERROR] org.apache.hadoop.hbase.master.TestClusterRestartFailover.test
[ERROR] Run 1: TestClusterRestartFailover.test:130 serverNode should be deleted after SCP finished expected null, but was:<f286fa1f49d4,39095,1626965264700/OFFLINE/regionCount=0/lock=java.util.concurrent.locks.ReentrantReadWriteLock@4a62266f[Write locks = 0, Read locks = 0]>
[ERROR] Run 2: TestClusterRestartFailover.test:134 Metrics Counters should be equal expected:<4> but was:<8>
[ERROR] Run 3: TestClusterRestartFailover.test:134 Metrics Counters should be equal expected:<4> but was:<12>
@saintstack and @z-york, what do you think about this change at this point? Want Wellington to wait on your re-reviews?
Continuing with the works of HBASE-24749. This one is about compactions, with the introduction of DirectInStoreCompactor that allows for writing compacted files direct in the store dir.
I had also renamed the Flusher and FlushContext classes introduced by HBASE-25391 to have more meaningful naming.