Apache9 commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1443671033
##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java:
##########
@@ -464,7 +464,6 @@ protected boolean performCompaction(FileDetails fd,
InternalScanner scanner, Cel
lastCleanCell = null;
lastCleanCellSeqId = 0;
}
- writer.append(c);
Review Comment:
Is this change related?
##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java:
##########
@@ -50,15 +50,15 @@ public interface StoreFileManager {
*/
@RestrictedApi(explanation = "Should only be called in StoreEngine", link =
"",
allowedOnPath =
".*(/org/apache/hadoop/hbase/regionserver/StoreEngine.java|/src/test/.*)")
- void loadFiles(List<HStoreFile> storeFiles);
+ void loadFiles(List<HStoreFile> storeFiles) throws IOException;
Review Comment:
Why we need to add the exception declaration here?
##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java:
##########
@@ -17,42 +17,45 @@
*/
package org.apache.hadoop.hbase.regionserver.compactions;
+import static
org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_COMPACTION_ENABLE_DUAL_FILE_WRITER_KEY;
+import static
org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_ENABLE_DUAL_FILE_WRITER;
+
import java.io.IOException;
import java.util.List;
import java.util.function.Consumer;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.regionserver.DualFileWriter;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.hadoop.hbase.security.User;
import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
/**
* Compact passed set of files. Create an instance and then call
* {@link #compact(CompactionRequestImpl, ThroughputController, User)}
*/
@InterfaceAudience.Private
-public class DefaultCompactor extends Compactor<StoreFileWriter> {
- private static final Logger LOG =
LoggerFactory.getLogger(DefaultCompactor.class);
+public class DefaultCompactor extends
AbstractMultiOutputCompactor<DualFileWriter> {
public DefaultCompactor(Configuration conf, HStore store) {
super(conf, store);
}
- private final CellSinkFactory<StoreFileWriter> writerFactory =
- new CellSinkFactory<StoreFileWriter>() {
+ private final CellSinkFactory<DualFileWriter> writerFactory =
Review Comment:
There is no store file writer any more?
##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java:
##########
@@ -17,42 +17,45 @@
*/
package org.apache.hadoop.hbase.regionserver.compactions;
+import static
org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_COMPACTION_ENABLE_DUAL_FILE_WRITER_KEY;
+import static
org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_ENABLE_DUAL_FILE_WRITER;
+
import java.io.IOException;
import java.util.List;
import java.util.function.Consumer;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.regionserver.DualFileWriter;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.hadoop.hbase.security.User;
import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
/**
* Compact passed set of files. Create an instance and then call
* {@link #compact(CompactionRequestImpl, ThroughputController, User)}
*/
@InterfaceAudience.Private
-public class DefaultCompactor extends Compactor<StoreFileWriter> {
- private static final Logger LOG =
LoggerFactory.getLogger(DefaultCompactor.class);
+public class DefaultCompactor extends
AbstractMultiOutputCompactor<DualFileWriter> {
public DefaultCompactor(Configuration conf, HStore store) {
super(conf, store);
}
- private final CellSinkFactory<StoreFileWriter> writerFactory =
- new CellSinkFactory<StoreFileWriter>() {
+ private final CellSinkFactory<DualFileWriter> writerFactory =
+ new CellSinkFactory<DualFileWriter>() {
@Override
- public StoreFileWriter createWriter(InternalScanner scanner, FileDetails
fd,
+ public DualFileWriter createWriter(InternalScanner scanner, FileDetails
fd,
boolean shouldDropBehind, boolean major, Consumer<Path>
writerCreationTracker)
throws IOException {
- return DefaultCompactor.this.createWriter(fd, shouldDropBehind, major,
- writerCreationTracker);
+ boolean enableDualFileWriter = conf.getBoolean(
Review Comment:
It seems a bit strange here, the flag is named enable dual file writer, but
no matter it is true or false, we always create a DualFileWriter...
I think here we should still return a StoreFileWriter? If dual file writer
is enabled, we return a DualFileWriter, otherwise we return the default store
file writer?
##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java:
##########
@@ -300,7 +300,7 @@ private double getMidStripeSplitRatio(long smallerSize,
long largerSize, long la
@Override
public Collection<HStoreFile> getFilesForScan(byte[] startRow, boolean
includeStartRow,
- byte[] stopRow, boolean includeStopRow) {
+ byte[] stopRow, boolean includeStopRow, boolean onlyLatestVersion) {
Review Comment:
The new parameter is not used?
##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java:
##########
@@ -41,7 +45,7 @@
* Default implementation of StoreFileManager. Not thread-safe.
*/
@InterfaceAudience.Private
-class DefaultStoreFileManager implements StoreFileManager {
+public class DefaultStoreFileManager implements StoreFileManager {
Review Comment:
Why we need to make this public?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]