From b1b3bbe140c44ef6417ef845fc8e0fb852c01c36 Mon Sep 17 00:00:00 2001 From: Vladimir Rodionov Date: Fri, 13 Sep 2019 13:14:54 -0700 Subject: [PATCH 01/15] HBASE-22749: Distributed MOB compactions --- .../apache/hadoop/hbase/master/HMaster.java | 33 +- .../master/MasterMobCompactionThread.java | 181 --- .../hbase/master/MasterRpcServices.java | 90 +- .../hbase/master/MobCompactionChore.java | 96 -- ...nerChore.java => MobFileCleanerChore.java} | 24 +- .../hbase/master/MobFileCompactionChore.java | 179 +++ .../hbase/mob/DefaultMobStoreCompactor.java | 654 ++++++++- .../hbase/mob/DefaultMobStoreFlusher.java | 27 + .../apache/hadoop/hbase/mob/MobConstants.java | 99 +- ...leCleaner.java => MobFileCleanerTool.java} | 8 +- .../apache/hadoop/hbase/mob/MobFileName.java | 33 +- .../hadoop/hbase/mob/MobStoreEngine.java | 12 +- .../org/apache/hadoop/hbase/mob/MobUtils.java | 610 +++++---- .../mob/compactions/MobCompactionRequest.java | 64 - .../hbase/mob/compactions/MobCompactor.java | 90 -- .../PartitionedMobCompactionRequest.java | 333 ----- .../compactions/PartitionedMobCompactor.java | 929 ------------- .../hadoop/hbase/regionserver/HMobStore.java | 41 +- .../hadoop/hbase/regionserver/HStore.java | 2 +- .../hadoop/hbase/regionserver/HStoreFile.java | 3 + .../hbase/regionserver/MobStoreScanner.java | 1 + .../hbase/regionserver/StoreFileInfo.java | 13 +- .../hbase/regionserver/StoreFileWriter.java | 23 + .../regionserver/compactions/Compactor.java | 2 +- .../hbase/mob/FaultyMobStoreCompactor.java | 355 +++++ .../hadoop/hbase/mob/MobStressTool.java | 80 ++ .../hbase/mob/TestExpiredMobFileCleaner.java | 2 +- .../hadoop/hbase/mob/TestMobCompaction.java | 344 +++++ .../hadoop/hbase/mob/TestMobFileName.java | 19 +- .../mob/compactions/TestMobCompactor.java | 1202 ----------------- .../TestPartitionedMobCompactionRequest.java | 65 - .../TestPartitionedMobCompactor.java | 958 ------------- .../hadoop/hbase/util/BaseTestHBaseFsck.java | 2 +- 33 files changed, 2106 insertions(+), 4468 deletions(-) delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobCompactionChore.java rename hbase-server/src/main/java/org/apache/hadoop/hbase/master/{ExpiredMobFileCleanerChore.java => MobFileCleanerChore.java} (80%) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java rename hbase-server/src/main/java/org/apache/hadoop/hbase/mob/{ExpiredMobFileCleaner.java => MobFileCleanerTool.java} (94%) delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactionRequest.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactionRequest.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactionRequest.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 5939053aec84..ee41974b6f88 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -394,9 +394,8 @@ public void run() { private LogCleaner logCleaner; private HFileCleaner hfileCleaner; private ReplicationBarrierCleaner replicationBarrierCleaner; - private ExpiredMobFileCleanerChore expiredMobFileCleanerChore; - private MobCompactionChore mobCompactChore; - private MasterMobCompactionThread mobCompactThread; + private MobFileCleanerChore mobFileCleanerChore; + private MobFileCompactionChore mobFileCompactionChore; // used to synchronize the mobCompactionStates private final IdLock mobCompactionLock = new IdLock(); // save the information of mob compactions in tables. @@ -1299,19 +1298,18 @@ public void updateConfigurationForQuotasObserver(Configuration conf) { } private void initMobCleaner() { - this.expiredMobFileCleanerChore = new ExpiredMobFileCleanerChore(this); - getChoreService().scheduleChore(expiredMobFileCleanerChore); + this.mobFileCleanerChore = new MobFileCleanerChore(this); + getChoreService().scheduleChore(mobFileCleanerChore); int mobCompactionPeriod = conf.getInt(MobConstants.MOB_COMPACTION_CHORE_PERIOD, MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD); if (mobCompactionPeriod > 0) { - this.mobCompactChore = new MobCompactionChore(this, mobCompactionPeriod); - getChoreService().scheduleChore(mobCompactChore); + this.mobFileCompactionChore = new MobFileCompactionChore(this); + getChoreService().scheduleChore(mobFileCompactionChore); } else { LOG .info("The period is " + mobCompactionPeriod + " seconds, MobCompactionChore is disabled"); } - this.mobCompactThread = new MasterMobCompactionThread(this); } /** @@ -1499,9 +1497,7 @@ protected void stopServiceThreads() { } } stopChores(); - if (this.mobCompactThread != null) { - this.mobCompactThread.close(); - } + super.stopServiceThreads(); if (cleanerPool != null) { cleanerPool.shutdownNow(); @@ -1622,8 +1618,8 @@ private void stopProcedureExecutor() { private void stopChores() { ChoreService choreService = getChoreService(); if (choreService != null) { - choreService.cancelChore(this.expiredMobFileCleanerChore); - choreService.cancelChore(this.mobCompactChore); + choreService.cancelChore(this.mobFileCleanerChore); + choreService.cancelChore(this.mobFileCompactionChore); choreService.cancelChore(this.balancerChore); choreService.cancelChore(this.normalizerChore); choreService.cancelChore(this.clusterStatusChore); @@ -3430,17 +3426,6 @@ public void reportMobCompactionEnd(TableName tableName) throws IOException { } } - /** - * Requests mob compaction. - * @param tableName The table the compact. - * @param columns The compacted columns. - * @param allFiles Whether add all mob files into the compaction. - */ - public void requestMobCompaction(TableName tableName, - List columns, boolean allFiles) throws IOException { - mobCompactThread.requestMobCompaction(conf, fs, tableName, columns, allFiles); - } - /** * Queries the state of the {@link LoadBalancerTracker}. If the balancer is not initialized, * false is returned. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java deleted file mode 100644 index 0779eeafe8b0..000000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java +++ /dev/null @@ -1,181 +0,0 @@ -/** - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.master; - -import java.io.IOException; -import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.master.locking.LockManager; -import org.apache.hadoop.hbase.mob.MobUtils; -import org.apache.hadoop.hbase.procedure2.LockType; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; - -/** - * The mob compaction thread used in {@link MasterRpcServices} - */ -@InterfaceAudience.Private -public class MasterMobCompactionThread { - static final Logger LOG = LoggerFactory.getLogger(MasterMobCompactionThread.class); - private final HMaster master; - private final Configuration conf; - private final ExecutorService mobCompactorPool; - private final ExecutorService masterMobPool; - - public MasterMobCompactionThread(HMaster master) { - this.master = master; - this.conf = master.getConfiguration(); - final String n = Thread.currentThread().getName(); - // this pool is used to run the mob compaction - this.masterMobPool = new ThreadPoolExecutor(1, 2, 60, - TimeUnit.SECONDS, new SynchronousQueue<>(), - new ThreadFactoryBuilder().setDaemon(true) - .setNameFormat(n + "-MasterMobCompaction-" + EnvironmentEdgeManager.currentTime()) - .build()); - ((ThreadPoolExecutor) this.masterMobPool).allowCoreThreadTimeOut(true); - // this pool is used in the mob compaction to compact the mob files by partitions - // in parallel - this.mobCompactorPool = MobUtils - .createMobCompactorThreadPool(master.getConfiguration()); - } - - /** - * Requests mob compaction - * @param conf The Configuration - * @param fs The file system - * @param tableName The table the compact - * @param columns The column descriptors - * @param allFiles Whether add all mob files into the compaction. - */ - public void requestMobCompaction(Configuration conf, FileSystem fs, TableName tableName, - List columns, boolean allFiles) throws IOException { - master.reportMobCompactionStart(tableName); - try { - masterMobPool.execute(new CompactionRunner(fs, tableName, columns, - allFiles, mobCompactorPool)); - } catch (RejectedExecutionException e) { - // in case the request is rejected by the pool - try { - master.reportMobCompactionEnd(tableName); - } catch (IOException e1) { - LOG.error("Failed to mark end of mob compaction", e1); - } - throw e; - } - if (LOG.isDebugEnabled()) { - LOG.debug("The mob compaction is requested for the columns " + columns - + " of the table " + tableName.getNameAsString()); - } - } - - private class CompactionRunner implements Runnable { - private FileSystem fs; - private TableName tableName; - private List hcds; - private boolean allFiles; - private ExecutorService pool; - - public CompactionRunner(FileSystem fs, TableName tableName, List hcds, - boolean allFiles, ExecutorService pool) { - super(); - this.fs = fs; - this.tableName = tableName; - this.hcds = hcds; - this.allFiles = allFiles; - this.pool = pool; - } - - @Override - public void run() { - // These locks are on dummy table names, and only used for compaction/mob file cleaning. - final LockManager.MasterLock lock = master.getLockManager().createMasterLock( - MobUtils.getTableLockName(tableName), LockType.EXCLUSIVE, - this.getClass().getName() + ": mob compaction"); - try { - for (ColumnFamilyDescriptor hcd : hcds) { - MobUtils.doMobCompaction(conf, fs, tableName, hcd, pool, allFiles, lock); - } - } catch (IOException e) { - LOG.error("Failed to perform the mob compaction", e); - } finally { - try { - master.reportMobCompactionEnd(tableName); - } catch (IOException e) { - LOG.error("Failed to mark end of mob compaction", e); - } - } - } - } - - /** - * Only interrupt once it's done with a run through the work loop. - */ - private void interruptIfNecessary() { - mobCompactorPool.shutdown(); - masterMobPool.shutdown(); - } - - /** - * Wait for all the threads finish. - */ - private void join() { - waitFor(mobCompactorPool, "Mob Compaction Thread"); - waitFor(masterMobPool, "Region Server Mob Compaction Thread"); - } - - /** - * Closes the MasterMobCompactionThread. - */ - public void close() { - interruptIfNecessary(); - join(); - } - - /** - * Wait for thread finish. - * @param t the thread to wait - * @param name the thread name. - */ - private void waitFor(ExecutorService t, String name) { - boolean done = false; - while (!done) { - try { - done = t.awaitTermination(60, TimeUnit.SECONDS); - LOG.info("Waiting for " + name + " to finish..."); - if (!done) { - t.shutdownNow(); - } - } catch (InterruptedException ie) { - LOG.warn("Interrupted waiting for " + name + " to finish..."); - } - } - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java index 06a99fa5432d..a411e85c47e7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java @@ -32,6 +32,7 @@ import java.util.Map.Entry; import java.util.Set; import java.util.stream.Collectors; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ClusterMetricsBuilder; @@ -45,7 +46,6 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.UnknownRegionException; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.MasterSwitchType; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; @@ -103,23 +103,6 @@ import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil; import org.apache.hadoop.hbase.security.access.UserPermission; import org.apache.hadoop.hbase.security.visibility.VisibilityController; -import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; -import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.ForeignExceptionUtil; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; -import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.zookeeper.KeeperException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; -import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; -import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; - import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter; import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos; @@ -342,6 +325,21 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription; +import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; +import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.ForeignExceptionUtil; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; +import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; +import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.zookeeper.KeeperException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Implements the master RPC services. @@ -1744,10 +1742,13 @@ public CompactRegionResponse compactRegion(final RpcController controller, master.checkInitialized(); byte[] regionName = request.getRegion().getValue().toByteArray(); TableName tableName = RegionInfo.getTable(regionName); + // TODO: support CompactType.MOB // if the region is a mob region, do the mob file compaction. if (MobUtils.isMobRegionName(tableName, regionName)) { checkHFileFormatVersionForMob(); - return compactMob(request, tableName); + //return compactMob(request, tableName); + //TODO: support CompactType.MOB + return super.compactRegion(controller, request); } else { return super.compactRegion(controller, request); } @@ -1790,57 +1791,6 @@ public GetRegionInfoResponse getRegionInfo(final RpcController controller, } } - /** - * Compacts the mob files in the current table. - * @param request the request. - * @param tableName the current table name. - * @return The response of the mob file compaction. - * @throws IOException - */ - private CompactRegionResponse compactMob(final CompactRegionRequest request, - TableName tableName) throws IOException { - if (!master.getTableStateManager().isTableState(tableName, TableState.State.ENABLED)) { - throw new DoNotRetryIOException("Table " + tableName + " is not enabled"); - } - boolean allFiles = false; - List compactedColumns = new ArrayList<>(); - ColumnFamilyDescriptor[] hcds = master.getTableDescriptors().get(tableName).getColumnFamilies(); - byte[] family = null; - if (request.hasFamily()) { - family = request.getFamily().toByteArray(); - for (ColumnFamilyDescriptor hcd : hcds) { - if (Bytes.equals(family, hcd.getName())) { - if (!hcd.isMobEnabled()) { - LOG.error("Column family " + hcd.getNameAsString() + " is not a mob column family"); - throw new DoNotRetryIOException("Column family " + hcd.getNameAsString() - + " is not a mob column family"); - } - compactedColumns.add(hcd); - } - } - } else { - for (ColumnFamilyDescriptor hcd : hcds) { - if (hcd.isMobEnabled()) { - compactedColumns.add(hcd); - } - } - } - if (compactedColumns.isEmpty()) { - LOG.error("No mob column families are assigned in the mob compaction"); - throw new DoNotRetryIOException( - "No mob column families are assigned in the mob compaction"); - } - if (request.hasMajor() && request.getMajor()) { - allFiles = true; - } - String familyLogMsg = (family != null) ? Bytes.toString(family) : ""; - if (LOG.isTraceEnabled()) { - LOG.trace("User-triggered mob compaction requested for table: " - + tableName.getNameAsString() + " for column family: " + familyLogMsg); - } - master.requestMobCompaction(tableName, compactedColumns, allFiles); - return CompactRegionResponse.newBuilder().build(); - } @Override public IsBalancerEnabledResponse isBalancerEnabled(RpcController controller, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobCompactionChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobCompactionChore.java deleted file mode 100644 index 6c5d677a86e5..000000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobCompactionChore.java +++ /dev/null @@ -1,96 +0,0 @@ -/** - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.master; - -import java.util.Map; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.TimeUnit; - -import org.apache.hadoop.hbase.ScheduledChore; -import org.apache.hadoop.hbase.TableDescriptors; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.client.TableDescriptor; -import org.apache.hadoop.hbase.client.TableState; -import org.apache.hadoop.hbase.master.locking.LockManager; -import org.apache.hadoop.hbase.mob.MobUtils; -import org.apache.hadoop.hbase.procedure2.LockType; - -/** - * The Class MobCompactChore for running compaction regularly to merge small mob files. - */ -@InterfaceAudience.Private -public class MobCompactionChore extends ScheduledChore { - - private static final Logger LOG = LoggerFactory.getLogger(MobCompactionChore.class); - private HMaster master; - private ExecutorService pool; - - public MobCompactionChore(HMaster master, int period) { - // use the period as initial delay. - super(master.getServerName() + "-MobCompactionChore", master, period, period, TimeUnit.SECONDS); - this.master = master; - this.pool = MobUtils.createMobCompactorThreadPool(master.getConfiguration()); - } - - @Override - protected void chore() { - try { - TableDescriptors htds = master.getTableDescriptors(); - Map map = htds.getAll(); - for (TableDescriptor htd : map.values()) { - if (!master.getTableStateManager().isTableState(htd.getTableName(), - TableState.State.ENABLED)) { - continue; - } - boolean reported = false; - try { - final LockManager.MasterLock lock = master.getLockManager().createMasterLock( - MobUtils.getTableLockName(htd.getTableName()), LockType.EXCLUSIVE, - this.getClass().getName() + ": mob compaction"); - for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) { - if (!hcd.isMobEnabled()) { - continue; - } - if (!reported) { - master.reportMobCompactionStart(htd.getTableName()); - reported = true; - } - MobUtils.doMobCompaction(master.getConfiguration(), master.getFileSystem(), - htd.getTableName(), hcd, pool, false, lock); - } - } finally { - if (reported) { - master.reportMobCompactionEnd(htd.getTableName()); - } - } - } - } catch (Exception e) { - LOG.error("Failed to compact mob files", e); - } - } - - @Override - protected synchronized void cleanup() { - super.cleanup(); - pool.shutdown(); - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java similarity index 80% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java index d37a80a1c3db..f6207e024e2f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java @@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.master.locking.LockManager; -import org.apache.hadoop.hbase.mob.ExpiredMobFileCleaner; +import org.apache.hadoop.hbase.mob.MobFileCleanerTool; import org.apache.hadoop.hbase.mob.MobConstants; import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.procedure2.LockType; @@ -39,27 +39,33 @@ * mob files. */ @InterfaceAudience.Private -public class ExpiredMobFileCleanerChore extends ScheduledChore { +public class MobFileCleanerChore extends ScheduledChore { - private static final Logger LOG = LoggerFactory.getLogger(ExpiredMobFileCleanerChore.class); + private static final Logger LOG = LoggerFactory.getLogger(MobFileCleanerChore.class); private final HMaster master; - private ExpiredMobFileCleaner cleaner; + private MobFileCleanerTool cleaner; + private volatile boolean running = false; - public ExpiredMobFileCleanerChore(HMaster master) { + public MobFileCleanerChore(HMaster master) { super(master.getServerName() + "-ExpiredMobFileCleanerChore", master, master.getConfiguration() .getInt(MobConstants.MOB_CLEANER_PERIOD, MobConstants.DEFAULT_MOB_CLEANER_PERIOD), master .getConfiguration().getInt(MobConstants.MOB_CLEANER_PERIOD, MobConstants.DEFAULT_MOB_CLEANER_PERIOD), TimeUnit.SECONDS); this.master = master; - cleaner = new ExpiredMobFileCleaner(); + cleaner = new MobFileCleanerTool(); cleaner.setConf(master.getConfiguration()); } @Override @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION", justification="Intentional") + protected void chore() { try { + if (running) { + LOG.warn(getName() +" is running already, skipping this attempt."); + return; + } TableDescriptors htds = master.getTableDescriptors(); Map map = htds.getAll(); for (TableDescriptor htd : map.values()) { @@ -78,9 +84,15 @@ protected void chore() { } } } + // Now clean obsolete files for a table + LOG.info("Cleaning obsolete MOB files ..."); + MobUtils.cleanupObsoleteMobFiles(master.getConfiguration(), htd.getTableName()); + LOG.info("Cleaning obsolete MOB files finished"); } } catch (Exception e) { LOG.error("Fail to clean the expired mob files", e); + } finally { + running = false; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java new file mode 100644 index 000000000000..b2c2206382b8 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java @@ -0,0 +1,179 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.master; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ScheduledChore; +import org.apache.hadoop.hbase.TableDescriptors; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.CompactionState; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableState; +import org.apache.hadoop.hbase.mob.MobConstants; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@InterfaceAudience.Private +public class MobFileCompactionChore extends ScheduledChore { + + private static final Logger LOG = LoggerFactory.getLogger(MobFileCompactionChore.class); + private final Configuration conf; + private final HMaster master; + private volatile boolean running = false; + private int regionBatchSize = 0;// not set - compact all + + public MobFileCompactionChore(HMaster master) { + super(master.getServerName() + "-MobFileCompactionChore", master, master.getConfiguration() + .getInt(MobConstants.MOB_COMPACTION_CHORE_PERIOD, + MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD), master + .getConfiguration().getInt(MobConstants.MOB_COMPACTION_CHORE_PERIOD, + MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD), TimeUnit.SECONDS); + this.master = master; + this.conf = master.getConfiguration(); + this.regionBatchSize = + master.getConfiguration().getInt(MobConstants.MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE, + MobConstants.DEFAULT_MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE); + + } + + @Override + protected void chore() { + + boolean reported = false; + + try (Connection conn = ConnectionFactory.createConnection(conf); + Admin admin = conn.getAdmin(); ) { + + if (running) { + LOG.warn(getName() +" is running already, skipping this attempt."); + return; + } + running = true; + TableDescriptors htds = master.getTableDescriptors(); + Map map = htds.getAll(); + for (TableDescriptor htd : map.values()) { + if (!master.getTableStateManager().isTableState(htd.getTableName(), + TableState.State.ENABLED)) { + continue; + } + for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) { + if (hcd.isMobEnabled()) { + if (!reported) { + master.reportMobCompactionStart(htd.getTableName()); + reported = true; + } + LOG.info(" Major compacting "+ htd.getTableName() + " cf=" + hcd.getNameAsString()); + if (regionBatchSize == MobConstants.DEFAULT_MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE) { + admin.majorCompact(htd.getTableName(), hcd.getName()); + } else { + performMajorCompactionInBatches(admin, htd, hcd); + } + } + } + if (reported) { + master.reportMobCompactionEnd(htd.getTableName()); + reported = false; + } + } + } catch (Exception e) { + LOG.error("Failed to compact", e); + } finally { + running = false; + } + } + + private void performMajorCompactionInBatches(Admin admin, TableDescriptor htd, + ColumnFamilyDescriptor hcd) throws IOException { + + List regions = admin.getRegions(htd.getTableName()); + if (regions.size() <= this.regionBatchSize) { + admin.majorCompact(htd.getTableName(), hcd.getName()); + return; + } + // Shuffle list of regions in case if they come ordered by region server + Collections.shuffle(regions); + // Create first batch + List toCompact = new ArrayList(); + for (int i=0; i < this.regionBatchSize; i++) { + toCompact.add(regions.remove(0)); + } + + // Start compaction now + for(RegionInfo ri: toCompact) { + startCompaction(admin, htd.getTableName(), ri); + } + + List compacted = new ArrayList(); + while(!toCompact.isEmpty()) { + // Check status of active compactions + for (RegionInfo ri: toCompact) { + if (admin.getCompactionStateForRegion(ri.getRegionName()) == CompactionState.NONE) { + LOG.info("Finished major compaction: table={} region={}", htd.getTableName(), + ri.getRegionNameAsString()); + compacted.add(ri); + } + } + // Update batch: remove compacted regions and add new ones + for (RegionInfo ri: compacted) { + toCompact.remove(ri); + if (regions.size() > 0) { + RegionInfo region = regions.remove(0); + startCompaction(admin, htd.getTableName(),region); + toCompact.add(region); + } + } + compacted.clear(); + try { + Thread.sleep(1000); + } catch(InterruptedException e) { + // swallow + } + } + LOG.info(" Finished major compacting "+ htd.getTableName() + " cf=" + hcd.getNameAsString()); + + } + + private void startCompaction(Admin admin, TableName table, RegionInfo region) + throws IOException { + + LOG.info("Started major compaction: table={} region={}", table, + region.getRegionNameAsString()); + admin.majorCompactRegion(region.getRegionName()); + // Wait until it really starts + while(admin.getCompactionStateForRegion(region.getRegionName()) == CompactionState.NONE) { + //TODO: what if we stuck here? + try { + Thread.sleep(10); + } catch (InterruptedException e) { + } + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java index e73a7d2226d7..408ca4d68d44 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java @@ -17,21 +17,36 @@ */ package org.apache.hadoop.hbase.mob; +import static org.apache.hadoop.hbase.regionserver.ScanType.COMPACT_DROP_DELETES; +import static org.apache.hadoop.hbase.regionserver.ScanType.COMPACT_RETAIN_DELETES; + +import java.io.FileNotFoundException; import java.io.IOException; import java.io.InterruptedIOException; import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.Date; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.PrivateCellUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.io.hfile.CacheConfig; +import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.CellSink; import org.apache.hadoop.hbase.regionserver.HMobStore; import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.KeyValueScanner; import org.apache.hadoop.hbase.regionserver.ScanInfo; @@ -48,6 +63,7 @@ import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -58,16 +74,33 @@ @InterfaceAudience.Private public class DefaultMobStoreCompactor extends DefaultCompactor { - private static final Logger LOG = LoggerFactory.getLogger(DefaultMobStoreCompactor.class); - private long mobSizeThreshold; - private HMobStore mobStore; + protected static final Logger LOG = LoggerFactory.getLogger(DefaultMobStoreCompactor.class); + protected long mobSizeThreshold; + protected HMobStore mobStore; + + // MOB file reference set + static ThreadLocal> mobRefSet = new ThreadLocal>() { + @Override + protected Set initialValue() { + return new HashSet(); + } + }; + + static ThreadLocal userRequest = new ThreadLocal () { + @Override + protected Boolean initialValue() { + return Boolean.FALSE; + } + }; + + static ThreadLocal> compSelections = + new ThreadLocal>(); private final InternalScannerFactory scannerFactory = new InternalScannerFactory() { @Override public ScanType getScanType(CompactionRequestImpl request) { - // retain the delete markers until they are expired. - return ScanType.COMPACT_RETAIN_DELETES; + return request.isAllFiles() ? COMPACT_DROP_DELETES : COMPACT_RETAIN_DELETES; } @Override @@ -103,9 +136,34 @@ public DefaultMobStoreCompactor(Configuration conf, HStore store) { mobSizeThreshold = store.getColumnFamilyDescriptor().getMobThreshold(); } + @Override - public List compact(CompactionRequestImpl request, ThroughputController throughputController, - User user) throws IOException { + public List compact(CompactionRequestImpl request, + ThroughputController throughputController, User user) throws IOException { + LOG.debug("Mob compaction: major=" + request.isMajor() + " isAll=" + request.isAllFiles() + + " priority=" + request.getPriority()); + if (request.getPriority() == HStore.PRIORITY_USER) { + userRequest.set(Boolean.TRUE); + } else { + userRequest.set(Boolean.FALSE); + } + LOG.debug("Mob compaction files: " + request.getFiles()); + // Check if generational MOB compaction + compSelections.set(null); + if (conf.get(MobConstants.MOB_COMPACTION_TYPE_KEY, MobConstants.DEFAULT_MOB_COMPACTION_TYPE) + .equals(MobConstants.GENERATIONAL_MOB_COMPACTION_TYPE)) { + if (request.isMajor() && request.getPriority() == HStore.PRIORITY_USER) { + // Compact MOBs + List mobFiles = getReferencedMobFiles(request.getFiles()); + if (mobFiles.size() > 0) { + Generations gens = Generations.build(mobFiles, conf); + List list = gens.getCompactionSelections(); + if (list.size() > 0) { + compSelections.set(list); + } + } + } + } return compact(request, scannerFactory, writerFactory, throughputController, user); } @@ -169,6 +227,35 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel long bytesWrittenProgressForCloseCheck = 0; long bytesWrittenProgressForLog = 0; long bytesWrittenProgressForShippedCall = 0; + // Clear old mob references + mobRefSet.get().clear(); + boolean isUserRequest = userRequest.get(); + boolean compactMOBs = major && isUserRequest; + boolean generationalMob = conf.get(MobConstants.MOB_COMPACTION_TYPE_KEY, + MobConstants.DEFAULT_MOB_COMPACTION_TYPE) + .equals(MobConstants.GENERATIONAL_MOB_COMPACTION_TYPE); + OutputMobWriters mobWriters = null; + + if (compactMOBs && generationalMob) { + List sel = compSelections.get(); + if (sel != null && sel.size() > 0) { + // Create output writers for compaction selections + mobWriters = new OutputMobWriters(sel); + int numWriters = mobWriters.getNumberOfWriters(); + List writers = new ArrayList(); + for (int i=0; i < numWriters; i++) { + StoreFileWriter sfw = mobStore.createWriterInTmp(new Date(fd.latestPutTs), fd.maxKeyCount, + compactionCompression, store.getRegionInfo().getStartKey(), true); + writers.add(sfw); + } + mobWriters.initOutputWriters(writers); + } + } + + boolean discardMobMiss = + conf.getBoolean(MobConstants.MOB_DISCARD_MISS_KEY, MobConstants.DEFAULT_MOB_DISCARD_MISS); + FileSystem fs = FileSystem.get(conf); + // Since scanner.next() can return 'false' but still be delivering data, // we have to use a do/while loop. List cells = new ArrayList<>(); @@ -183,16 +270,20 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel boolean hasMore; Path path = MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName()); byte[] fileName = null; - StoreFileWriter mobFileWriter = null, delFileWriter = null; - long mobCells = 0, deleteMarkersCount = 0; + StoreFileWriter mobFileWriter = null; + long mobCells = 0; long cellsCountCompactedToMob = 0, cellsCountCompactedFromMob = 0; long cellsSizeCompactedToMob = 0, cellsSizeCompactedFromMob = 0; boolean finished = false; + ScannerContext scannerContext = ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build(); throughputController.start(compactionName); - KeyValueScanner kvs = (scanner instanceof KeyValueScanner)? (KeyValueScanner)scanner : null; - long shippedCallSizeLimit = (long) numofFilesToCompact * this.store.getColumnFamilyDescriptor().getBlocksize(); + KeyValueScanner kvs = (scanner instanceof KeyValueScanner) ? (KeyValueScanner) scanner : null; + long shippedCallSizeLimit = + (long) numofFilesToCompact * this.store.getColumnFamilyDescriptor().getBlocksize(); + + MobCell mobCell = null; try { try { // If the mob file writer could not be created, directly write the cell to the store file. @@ -200,19 +291,15 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel compactionCompression, store.getRegionInfo().getStartKey(), true); fileName = Bytes.toBytes(mobFileWriter.getPath().getName()); } catch (IOException e) { - LOG.warn("Failed to create mob writer, " - + "we will continue the compaction by writing MOB cells directly in store files", e); + // Bailing out + LOG.error("Failed to create mob writer, ", e); + throw e; } - if (major) { - try { - delFileWriter = mobStore.createDelFileWriterInTmp(new Date(fd.latestPutTs), - fd.maxKeyCount, compactionCompression, store.getRegionInfo().getStartKey()); - } catch (IOException e) { - LOG.warn( - "Failed to create del writer, " - + "we will continue the compaction by writing delete markers directly in store files", - e); - } + if (compactMOBs) { + // Add the only reference we get for compact MOB case + // because new store file will have only one MOB reference + // in this case - of newly compacted MOB file + mobRefSet.get().add(mobFileWriter.getPath().getName()); } do { hasMore = scanner.next(cells, scannerContext); @@ -220,54 +307,113 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel now = EnvironmentEdgeManager.currentTime(); } for (Cell c : cells) { - if (major && CellUtil.isDelete(c)) { - if (MobUtils.isMobReferenceCell(c) || delFileWriter == null) { - // Directly write it to a store file - writer.append(c); + + if (compactMOBs) { + if (MobUtils.isMobReferenceCell(c)) { + String fName = MobUtils.getMobFileName(c); + Path pp = new Path(new Path(fs.getUri()), new Path(path, fName)); + + // Added to support migration + try { + mobCell = mobStore.resolve(c, true, false); + } catch (FileNotFoundException fnfe) { + if (discardMobMiss) { + LOG.error("Missing MOB cell: file=" + pp + " not found"); + continue; + } else { + throw fnfe; + } + } + + if (discardMobMiss && mobCell.getCell().getValueLength() == 0) { + LOG.error("Missing MOB cell value: file=" + pp + " cell=" + mobCell); + continue; + } + + if (mobCell.getCell().getValueLength() > mobSizeThreshold) { + // put the mob data back to the store file + PrivateCellUtil.setSequenceId(mobCell.getCell(), c.getSequenceId()); + if (generationalMob) { + //TODO: verify fName + StoreFileWriter stw = mobWriters.getOutputWriterForInputFile(fName); + if (stw != null) { + stw.append(mobCell.getCell()); + mobWriters.incrementMobCountForOutputWriter(stw, 1); + } else { + // leave mob cell in a MOB file which is not in compaction selections + } + } else { + mobFileWriter.append(mobCell.getCell()); + mobCells++; + } + writer.append(MobUtils.createMobRefCell(mobCell.getCell(), fileName, + this.mobStore.getRefCellTags())); + cellsCountCompactedFromMob++; + cellsSizeCompactedFromMob += mobCell.getCell().getValueLength(); + } else { + + // If MOB value is less than threshold, append it directly to a store file + PrivateCellUtil.setSequenceId(mobCell.getCell(), c.getSequenceId()); + writer.append(mobCell.getCell()); + } + } else { - // Add a ref tag to this cell and write it to a store file. - writer.append(MobUtils.createMobRefDeleteMarker(c)); - // Write the cell to a del file - delFileWriter.append(c); - deleteMarkersCount++; + // Not a MOB reference cell + int size = c.getValueLength(); + if (size > mobSizeThreshold) { + // This MOB cell comes from a regular store file + // therefore we store it in original mob output + mobFileWriter.append(c); + writer + .append(MobUtils.createMobRefCell(c, fileName, this.mobStore.getRefCellTags())); + mobCells++; + } else { + writer.append(c); + } } - } else if (mobFileWriter == null || c.getTypeByte() != KeyValue.Type.Put.getCode()) { - // If the mob file writer is null or the kv type is not put, directly write the cell + } else if (c.getTypeByte() != KeyValue.Type.Put.getCode()) { + // Not a major compaction or major with MOB disabled + // If the kv type is not put, directly write the cell // to the store file. writer.append(c); } else if (MobUtils.isMobReferenceCell(c)) { + // Not a major MOB compaction, Put MOB reference if (MobUtils.hasValidMobRefCellValue(c)) { int size = MobUtils.getMobValueLength(c); if (size > mobSizeThreshold) { // If the value size is larger than the threshold, it's regarded as a mob. Since // its value is already in the mob file, directly write this cell to the store file writer.append(c); + // Add MOB reference to a set + mobRefSet.get().add(MobUtils.getMobFileName(c)); } else { // If the value is not larger than the threshold, it's not regarded a mob. Retrieve - // the mob cell from the mob file, and write it back to the store file. Must - // close the mob scanner once the life cycle finished. - try (MobCell mobCell = mobStore.resolve(c, false)) { - if (mobCell.getCell().getValueLength() != 0) { - // put the mob data back to the store file - PrivateCellUtil.setSequenceId(mobCell.getCell(), c.getSequenceId()); - writer.append(mobCell.getCell()); - cellsCountCompactedFromMob++; - cellsSizeCompactedFromMob += mobCell.getCell().getValueLength(); - } else { - // If the value of a file is empty, there might be issues when retrieving, - // directly write the cell to the store file, and leave it to be handled by the - // next compaction. - writer.append(c); - } + // the mob cell from the mob file, and write it back to the store file. + mobCell = mobStore.resolve(c, true, false); + if (mobCell.getCell().getValueLength() != 0) { + // put the mob data back to the store file + PrivateCellUtil.setSequenceId(mobCell.getCell(), c.getSequenceId()); + writer.append(mobCell.getCell()); + cellsCountCompactedFromMob++; + cellsSizeCompactedFromMob += mobCell.getCell().getValueLength(); + } else { + // If the value of a file is empty, there might be issues when retrieving, + // directly write the cell to the store file, and leave it to be handled by the + // next compaction. + LOG.error("Empty value for: " + c); + writer.append(c); + // Add MOB reference to a set + mobRefSet.get().add(MobUtils.getMobFileName(c)); } } } else { - LOG.warn("The value format of the KeyValue " + c - + " is wrong, its length is less than " + Bytes.SIZEOF_INT); + // TODO ???? + LOG.error("Corrupted MOB reference: " + c); writer.append(c); } } else if (c.getValueLength() <= mobSizeThreshold) { - //If value size of a cell is not larger than the threshold, directly write to store file + // If the value size of a cell is not larger than the threshold, directly write it to + // the store file. writer.append(c); } else { // If the value size of a cell is larger than the threshold, it's regarded as a mob, @@ -275,13 +421,15 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel mobCells++; // append the original keyValue in the mob file. mobFileWriter.append(c); - Cell reference = MobUtils.createMobRefCell(c, fileName, - this.mobStore.getRefCellTags()); + Cell reference = MobUtils.createMobRefCell(c, fileName, this.mobStore.getRefCellTags()); // write the cell whose value is the path of a mob file to the store file. writer.append(reference); cellsCountCompactedToMob++; cellsSizeCompactedToMob += c.getValueLength(); + // Add ref we get for compact MOB case + mobRefSet.get().add(mobFileWriter.getPath().getName()); } + int len = c.getSerializedSize(); ++progress.currentCompactedKVs; progress.totalCompactedSize += len; @@ -302,7 +450,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel } } if (kvs != null && bytesWrittenProgressForShippedCall > shippedCallSizeLimit) { - ((ShipperListener)writer).beforeShipped(); + ((ShipperListener) writer).beforeShipped(); kvs.shipped(); bytesWrittenProgressForShippedCall = 0; } @@ -326,6 +474,9 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel progress.cancel(); throw new InterruptedIOException( "Interrupted while control throughput of compacting " + compactionName); + } catch (IOException t) { + LOG.error("Mob compaction failed for region: " + store.getRegionInfo().getEncodedName()); + throw t; } finally { // Clone last cell in the final because writer will append last cell when committing. If // don't clone here and once the scanner get closed, then the memory of last cell will be @@ -333,24 +484,19 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel ((ShipperListener) writer).beforeShipped(); throughputController.finish(compactionName); if (!finished && mobFileWriter != null) { + // Remove all MOB references because compaction failed + mobRefSet.get().clear(); + // Abort writer abortWriter(mobFileWriter); - } - if (!finished && delFileWriter != null) { - abortWriter(delFileWriter); - } - } - if (delFileWriter != null) { - if (deleteMarkersCount > 0) { - // If the del file is not empty, commit it. - // If the commit fails, the compaction is re-performed again. - delFileWriter.appendMetadata(fd.maxSeqId, major, deleteMarkersCount); - delFileWriter.close(); - mobStore.commitFile(delFileWriter.getPath(), path); - } else { - // If the del file is empty, delete it instead of committing. - abortWriter(delFileWriter); + //Check if other writers exist + if (mobWriters != null) { + for(StoreFileWriter w: mobWriters.getOutputWriters()) { + abortWriter(w); + } + } } } + // Commit or abort major mob writer if (mobFileWriter != null) { if (mobCells > 0) { // If the mob file is not empty, commit it. @@ -362,6 +508,20 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel abortWriter(mobFileWriter); } } + // Commit or abort generational writers + if (mobWriters != null) { + for (StoreFileWriter w: mobWriters.getOutputWriters()) { + Long mobs = mobWriters.getMobCountForOutputWriter(w); + if (mobs != null && mobs > 0) { + mobRefSet.get().add(w.getPath().getName()); + w.appendMetadata(fd.maxSeqId, major, mobs); + w.close(); + mobStore.commitFile(w.getPath(), path); + } else { + abortWriter(w); + } + } + } mobStore.updateCellsCountCompactedFromMob(cellsCountCompactedFromMob); mobStore.updateCellsCountCompactedToMob(cellsCountCompactedToMob); mobStore.updateCellsSizeCompactedFromMob(cellsSizeCompactedFromMob); @@ -369,4 +529,354 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel progress.complete(); return true; } + + protected static String createKey(TableName tableName, String encodedName, + String columnFamilyName) { + return tableName.getNameAsString()+ "_" + encodedName + "_"+ columnFamilyName; + } + + @Override + protected List commitWriter(StoreFileWriter writer, FileDetails fd, + CompactionRequestImpl request) throws IOException { + List newFiles = Lists.newArrayList(writer.getPath()); + writer.appendMetadata(fd.maxSeqId, request.isAllFiles(), request.getFiles()); + // Append MOB references + Set refSet = mobRefSet.get(); + writer.appendMobMetadata(refSet); + writer.close(); + return newFiles; + } + + private List getReferencedMobFiles(Collection storeFiles) { + Path mobDir = MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName()); + Set mobSet = new HashSet(); + for (HStoreFile sf: storeFiles) { + byte[] value = sf.getMetadataValue(HStoreFile.MOB_FILE_REFS); + if (value != null) { + String s = new String(value); + String[] all = s.split(","); + Collections.addAll(mobSet, all); + } + } + List retList = new ArrayList(); + for(String name: mobSet) { + retList.add(new Path(mobDir, name)); + } + return retList; + } +} + +class FileSelection implements Comparable { + + public final static String NULL_REGION = ""; + private Path path; + private long earliestTs; + private Configuration conf; + + public FileSelection(Path path, Configuration conf) throws IOException { + this.path = path; + this.conf = conf; + readEarliestTimestamp(); + } + + public String getEncodedRegionName() { + String fileName = path.getName(); + String[] parts = fileName.split("_"); + if (parts.length == 2) { + return parts[1]; + } else { + return NULL_REGION; + } + } + + public Path getPath() { + return path; + } + + public long getEarliestTimestamp() { + return earliestTs; + } + + private void readEarliestTimestamp() throws IOException { + FileSystem fs = path.getFileSystem(conf); + HStoreFile sf = new HStoreFile(fs, path, conf, CacheConfig.DISABLED, + BloomType.NONE, true); + sf.initReader(); + byte[] tsData = sf.getMetadataValue(HStoreFile.EARLIEST_PUT_TS); + if (tsData != null) { + this.earliestTs = Bytes.toLong(tsData); + } + sf.closeStoreFile(true); + } + + @Override + public int compareTo(FileSelection o) { + if (this.earliestTs > o.earliestTs) { + return +1; + } else if (this.earliestTs == o.earliestTs) { + return 0; + } else { + return -1; + } + } + +} + +class Generations { + + private List generations; + private Configuration conf; + + private Generations(List gens, Configuration conf) { + this.generations = gens; + this.conf = conf; + } + + List getCompactionSelections() throws IOException { + int maxTotalFiles = this.conf.getInt(MobConstants.MOB_COMPACTION_MAX_TOTAL_FILES_KEY, + MobConstants.DEFAULT_MOB_COMPACTION_MAX_TOTAL_FILES); + int currentTotal = 0; + List list = new ArrayList(); + + for (Generation g: generations) { + List sel = g.getCompactionSelections(conf); + int size = getSize(sel); + if ((currentTotal + size > maxTotalFiles) && currentTotal > 0) { + break; + } else { + currentTotal += size; + list.addAll(sel); + } + } + return list; + } + + private int getSize(List sel) { + int size = 0; + for(CompactionSelection cs: sel) { + size += cs.size(); + } + return size; + } + + static Generations build(List files, Configuration conf) throws IOException { + Map > map = new HashMap>(); + for(Path p: files) { + String key = getRegionNameFromFileName(p.getName()); + ArrayList list = map.get(key); + if (list == null) { + list = new ArrayList(); + map.put(key, list); + } + list.add(new FileSelection(p, conf)); + } + + List gens = new ArrayList(); + for (String key: map.keySet()) { + Generation g = new Generation(key); + List selFiles = map.get(key); + for(FileSelection fs: selFiles) { + g.addFile(fs); + } + gens.add(g); + } + // Sort all generation files one-by-one + for(Generation gg: gens) { + gg.sortFiles(); + } + // Sort generations + Collections.sort(gens); + return new Generations(gens, conf); + } + + static String getRegionNameFromFileName(String name) { + int index = name.lastIndexOf("_"); + if (index < 0) return Generation.GEN0; + return name.substring(index+1); + } +} + +class Generation implements Comparable { + + static final String GEN0 ="GEN0"; + private String regionName; + private long earliestTs = Long.MAX_VALUE; + private List files = new ArrayList<>(); + List compSelections; + + public Generation(String name) { + this.regionName = name; + } + + @SuppressWarnings("deprecation") + public List getCompactionSelections(Configuration conf) throws IOException { + + + int minFiles = conf.getInt(MobConstants.MOB_COMPACTION_MIN_FILES_KEY, + MobConstants.DEFAULT_MOB_COMPACTION_MIN_FILES); + int maxFiles = conf.getInt(MobConstants.MOB_COMPACTION_MAX_FILES_KEY, + MobConstants.DEFAULT_MOB_COMPACTION_MAX_FILES); + long maxSelectionSize = conf.getLong(MobConstants.MOB_COMPACTION_MAX_SELECTION_SIZE_KEY, + MobConstants.DEFAULT_MOB_COMPACTION_MAX_SELECTION_SIZE); + // Now it is ordered from oldest to newest ones + List rfiles = Lists.reverse(files); + List retList = new ArrayList(); + FileSystem fs = rfiles.get(0).getPath().getFileSystem(conf); + int off = 0; + while (off < rfiles.size()) { + if (fs.getLength(rfiles.get(off).getPath()) >= maxSelectionSize) { + off++; continue; + } + long selSize = 0; + int limit = Math.min(off + maxFiles, rfiles.size()); + int start = off; + List sel = new ArrayList(); + for (; off < limit; off++) { + Path p = rfiles.get(off).getPath(); + long fSize = fs.getLength(p); + if (selSize + fSize < maxSelectionSize) { + selSize+= fSize; + sel.add(new FileSelection(p, conf)); + } else { + if (sel.size() < minFiles) { + // discard + sel.clear(); + // advance by 1 + off = start +1; + } else { + // we have new selection + CompactionSelection cs = new CompactionSelection(sel); + retList.add(cs); + off++; + } + break; // continue outer loop + } + } + } + return retList; + } + + public boolean addFile(FileSelection f) { + if (f.getEncodedRegionName().equals(regionName)) { + files.add(f); + if (f.getEarliestTimestamp() < earliestTs) { + earliestTs = f.getEarliestTimestamp(); + } + return true; + } else { + return false; + } + } + + public void sortFiles() { + Collections.sort(files); + } + + public List getFiles() { + return files; + } + + public String getEncodedRegionName() { + return regionName; + } + + public long getEarliestTimestamp() { + return earliestTs; + } + + @Override + public int compareTo(Generation o) { + if (this.earliestTs > o.earliestTs) { + return +1; + } else if (this.earliestTs == o.earliestTs) { + return 0; + } else { + return -1; + } + } +} + +class CompactionSelection { + private static AtomicLong idGen = new AtomicLong(); + private List files; + private long id; + + public CompactionSelection(List files) { + this.files = files; + this.id = idGen.getAndIncrement(); + } + + public List getFiles() { + return files; + } + + public long getId() { + return id; + } + + int size() { + return files.size(); + } + +} + +class OutputMobWriters { + + /* + * Input MOB file name -> output file writer + */ + private Map writerMap = new HashMap(); + /* + * Output file name -> MOB counter + */ + private Map mapMobCounts = new HashMap(); + /* + * List of compaction selections + */ + private List compSelections; + + public OutputMobWriters(List compSelections) + { + this.compSelections = compSelections; + } + + int getNumberOfWriters() { + return compSelections.size(); + } + + StoreFileWriter getWriterForFile(String fileName) { + return writerMap.get(fileName); + } + + void initOutputWriters(List writers) { + for (int i = 0; i < writers.size(); i++) { + StoreFileWriter sw = writers.get(i); + mapMobCounts.put(sw.getPath().getName(), 0L); + CompactionSelection cs = compSelections.get(i); + for (FileSelection fs: cs.getFiles()) { + writerMap.put(fs.getPath().getName(), sw); + } + } + } + + Collection getOutputWriters() { + return writerMap.values(); + } + + StoreFileWriter getOutputWriterForInputFile(String name) { + return writerMap.get(name); + } + + long getMobCountForOutputWriter(StoreFileWriter writer) { + return mapMobCounts.get(writer.getPath().getName()); + } + + void incrementMobCountForOutputWriter(StoreFileWriter writer, int val) { + String key = writer.getPath().getName(); + Long count = mapMobCounts.get(key); + if (count == null) { + count = 0L; + } + mapMobCounts.put(key, count + val); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java index 3de7992cb121..2c0d9ff1c0ba 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java @@ -22,7 +22,9 @@ import java.io.InterruptedIOException; import java.util.ArrayList; import java.util.Date; +import java.util.HashSet; import java.util.List; +import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -37,6 +39,7 @@ import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.MemStoreSnapshot; import org.apache.hadoop.hbase.regionserver.ScannerContext; +import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFileWriter; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputControlUtil; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; @@ -69,6 +72,13 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher { private long mobCellValueSizeThreshold = 0; private Path targetPath; private HMobStore mobStore; + // MOB file reference set + static ThreadLocal> mobRefSet = new ThreadLocal>() { + @Override + protected Set initialValue() { + return new HashSet(); + } + }; public DefaultMobStoreFlusher(Configuration conf, HStore store) throws IOException { super(conf, store); @@ -188,6 +198,8 @@ protected void performMobFlush(MemStoreSnapshot snapshot, long cacheFlushId, throughputController.start(flushName); } IOException ioe = null; + // Clear all past MOB references + mobRefSet.get().clear(); try { do { hasMore = scanner.next(cells, scannerContext); @@ -244,9 +256,12 @@ protected void performMobFlush(MemStoreSnapshot snapshot, long cacheFlushId, status.setStatus("Flushing mob file " + store + ": closing flushed file"); mobFileWriter.close(); mobStore.commitFile(mobFileWriter.getPath(), targetPath); + LOG.debug(" FLUSH store file: " + writer.getPath()); mobStore.updateMobFlushCount(); mobStore.updateMobFlushedCellsCount(mobCount); mobStore.updateMobFlushedCellsSize(mobSize); + // Add mob reference to store file metadata + mobRefSet.get().add(mobFileWriter.getPath().getName()); } else { try { status.setStatus("Flushing mob file " + store + ": no mob cells, closing flushed file"); @@ -258,4 +273,16 @@ protected void performMobFlush(MemStoreSnapshot snapshot, long cacheFlushId, } } } + + protected void finalizeWriter(StoreFileWriter writer, long cacheFlushSeqNum, + MonitoredTask status) throws IOException { + // Write out the log sequence number that corresponds to this output + // hfile. Also write current time in metadata as minFlushTime. + // The hfile is current up to and including cacheFlushSeqNum. + status.setStatus("Flushing " + store + ": appending metadata"); + writer.appendMetadata(cacheFlushSeqNum, false); + writer.appendMobMetadata(mobRefSet.get()); + status.setStatus("Flushing " + store + ": closing flushed file"); + writer.close(); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java index 4afd75b9c680..59777e780b43 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java @@ -43,7 +43,7 @@ public final class MobConstants { public static final String MOB_REGION_NAME = ".mob"; public static final byte[] MOB_REGION_NAME_BYTES = Bytes.toBytes(MOB_REGION_NAME); - public static final String MOB_CLEANER_PERIOD = "hbase.master.mob.ttl.cleaner.period"; + public static final String MOB_CLEANER_PERIOD = "hbase.master.mob.cleaner.period"; public static final int DEFAULT_MOB_CLEANER_PERIOD = 24 * 60 * 60; // one day public static final String MOB_CACHE_EVICT_PERIOD = "hbase.mob.cache.evict.period"; @@ -55,33 +55,23 @@ public final class MobConstants { public static final long DEFAULT_MOB_CACHE_EVICT_PERIOD = 3600L; public final static String TEMP_DIR_NAME = ".tmp"; - public final static String BULKLOAD_DIR_NAME = ".bulkload"; public final static byte[] MOB_TABLE_LOCK_SUFFIX = Bytes.toBytes(".mobLock"); - public final static String EMPTY_STRING = ""; - /** - * If the size of a mob file is less than this value, it's regarded as a small file and needs to - * be merged in mob compaction. The default value is 1280MB. - */ - public static final String MOB_COMPACTION_MERGEABLE_THRESHOLD = - "hbase.mob.compaction.mergeable.threshold"; - public static final long DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD = 10 * 128 * 1024 * 1024; + /** - * The max number of del files that is allowed in the mob file compaction. In the mob - * compaction, when the number of existing del files is larger than this value, they are merged - * until number of del files is not larger this value. The default value is 3. + * The max number of a MOB table regions that is allowed in a batch of the mob compaction. + * By setting this number to a custom value, users can control the overall effect + * of a major compaction of a large MOB-enabled table. */ - public static final String MOB_DELFILE_MAX_COUNT = "hbase.mob.delfile.max.count"; - public static final int DEFAULT_MOB_DELFILE_MAX_COUNT = 3; + + public static final String MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE = + "hbase.mob.compaction.batch.size"; + /** - * The max number of the mob files that is allowed in a batch of the mob compaction. - * The mob compaction merges the small mob files to bigger ones. If the number of the - * small files is very large, it could lead to a "too many opened file handlers" in the merge. - * And the merge has to be split into batches. This value limits the number of mob files - * that are selected in a batch of the mob compaction. The default value is 100. + * Default is 0 - means no limit - all regions of a MOB table will be compacted at once */ - public static final String MOB_COMPACTION_BATCH_SIZE = - "hbase.mob.compaction.batch.size"; - public static final int DEFAULT_MOB_COMPACTION_BATCH_SIZE = 100; + + public static final int DEFAULT_MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE = 0; + /** * The period that MobCompactionChore runs. The unit is second. * The default value is one week. @@ -91,12 +81,67 @@ public final class MobConstants { public static final int DEFAULT_MOB_COMPACTION_CHORE_PERIOD = 24 * 60 * 60 * 7; // a week public static final String MOB_COMPACTOR_CLASS_KEY = "hbase.mob.compactor.class"; + /** - * The max number of threads used in MobCompactor. + * Mob compaction type: "full", "generational" + * "full" - run full major compaction (during migration) + * "generational" - optimized version */ - public static final String MOB_COMPACTION_THREADS_MAX = - "hbase.mob.compaction.threads.max"; - public static final int DEFAULT_MOB_COMPACTION_THREADS_MAX = 1; + public final static String MOB_COMPACTION_TYPE_KEY = "hbase.mob.compaction.type"; + + public final static String DEFAULT_MOB_COMPACTION_TYPE = "full"; + + public final static String GENERATIONAL_MOB_COMPACTION_TYPE = "generational"; + + public final static String FULL_MOB_COMPACTION_TYPE = "full"; + + + /** + * Maximum size of a MOB compaction selection + */ + public static final String MOB_COMPACTION_MAX_SELECTION_SIZE_KEY = + "hbase.mob.compactions.max.selection.size"; + /** + * Default maximum selection size = 1GB + */ + public static final long DEFAULT_MOB_COMPACTION_MAX_SELECTION_SIZE = 1024 * 1024 * 1024; + + + /** + * Minimum number of MOB files eligible for compaction + */ + public static final String MOB_COMPACTION_MIN_FILES_KEY = "hbase.mob.compactions.min.files"; + + public static final int DEFAULT_MOB_COMPACTION_MIN_FILES = 3; + + /** + * Maximum number of MOB files (in one selection) eligible for compaction + */ + + public static final String MOB_COMPACTION_MAX_FILES_KEY = "hbase.mob.compactions.max.files"; + + public static final int DEFAULT_MOB_COMPACTION_MAX_FILES = 100; + + /** + * Maximum number of MOB files allowed in MOB compaction (per region) + */ + + public static final String MOB_COMPACTION_MAX_TOTAL_FILES_KEY = + "hbase.mob.compactions.max.total.files"; + + public static final int DEFAULT_MOB_COMPACTION_MAX_TOTAL_FILES = 1000; + + public static final String MOB_DISCARD_MISS_KEY = "hbase.mob.discard.miss"; + + public static final boolean DEFAULT_MOB_DISCARD_MISS = false; + + /** + * Minimum age required for MOB file to be archived + */ + public static final String MOB_MINIMUM_FILE_AGE_TO_ARCHIVE_KEY = "mob.minimum.file.age.to.archive"; + + public static final long DEFAULT_MOB_MINIMUM_FILE_AGE_TO_ARCHIVE = 3600000; // 1 hour + private MobConstants() { } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerTool.java similarity index 94% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerTool.java index 42e78bbd007d..07c89355caf1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerTool.java @@ -43,9 +43,9 @@ * The cleaner to delete the expired MOB files. */ @InterfaceAudience.Private -public class ExpiredMobFileCleaner extends Configured implements Tool { +public class MobFileCleanerTool extends Configured implements Tool { - private static final Logger LOG = LoggerFactory.getLogger(ExpiredMobFileCleaner.class); + private static final Logger LOG = LoggerFactory.getLogger(MobFileCleanerTool.class); /** * Cleans the MOB files when they're expired and their min versions are 0. * If the latest timestamp of Cells in a MOB file is older than the TTL in the column family, @@ -72,12 +72,12 @@ public void cleanExpiredMobFiles(String tableName, ColumnFamilyDescriptor family public static void main(String[] args) throws Exception { Configuration conf = HBaseConfiguration.create(); - ToolRunner.run(conf, new ExpiredMobFileCleaner(), args); + ToolRunner.run(conf, new MobFileCleanerTool(), args); } private void printUsage() { System.err.println("Usage:\n" + "--------------------------\n" - + ExpiredMobFileCleaner.class.getName() + " tableName familyName"); + + MobFileCleanerTool.class.getName() + " tableName familyName"); System.err.println(" tableName The table name"); System.err.println(" familyName The column family name"); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileName.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileName.java index 3a29274285b6..d8b3544e6c7a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileName.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileName.java @@ -45,10 +45,13 @@ public final class MobFileName { private final String startKey; private final String uuid; private final String fileName; + // Encoded region name + private final String regionName; private static final int STARTKEY_END_INDEX = 32; private static final int DATE_END_INDEX = 40; private static final int UUID_END_INDEX = 72; + public static final String REGION_SEP = "_"; /** * @param startKey @@ -57,12 +60,14 @@ public final class MobFileName { * The string of the latest timestamp of cells in this file, the format is yyyymmdd. * @param uuid * The uuid + * @param regionName region's name */ - private MobFileName(byte[] startKey, String date, String uuid) { + private MobFileName(byte[] startKey, String date, String uuid, String regionName) { this.startKey = MD5Hash.getMD5AsHex(startKey, 0, startKey.length); this.uuid = uuid; this.date = date; - this.fileName = this.startKey + this.date + this.uuid; + this.regionName = regionName; + this.fileName = this.startKey + this.date + this.uuid + REGION_SEP + this.regionName; } /** @@ -72,12 +77,14 @@ private MobFileName(byte[] startKey, String date, String uuid) { * The string of the latest timestamp of cells in this file, the format is yyyymmdd. * @param uuid * The uuid + * @param regionName region's name */ - private MobFileName(String startKey, String date, String uuid) { + private MobFileName(String startKey, String date, String uuid, String regionName) { this.startKey = startKey; this.uuid = uuid; this.date = date; - this.fileName = this.startKey + this.date + this.uuid; + this.regionName = regionName; + this.fileName = this.startKey + this.date + this.uuid + REGION_SEP + this.regionName; } /** @@ -90,8 +97,8 @@ private MobFileName(String startKey, String date, String uuid) { * @param uuid The uuid. * @return An instance of a MobFileName. */ - public static MobFileName create(byte[] startKey, String date, String uuid) { - return new MobFileName(startKey, date, uuid); + public static MobFileName create(byte[] startKey, String date, String uuid, String regionName) { + return new MobFileName(startKey, date, uuid, regionName); } /** @@ -104,8 +111,8 @@ public static MobFileName create(byte[] startKey, String date, String uuid) { * @param uuid The uuid. * @return An instance of a MobFileName. */ - public static MobFileName create(String startKey, String date, String uuid) { - return new MobFileName(startKey, date, uuid); + public static MobFileName create(String startKey, String date, String uuid, String regionName) { + return new MobFileName(startKey, date, uuid, regionName); } /** @@ -119,7 +126,8 @@ public static MobFileName create(String fileName) { String startKey = fileName.substring(0, STARTKEY_END_INDEX); String date = fileName.substring(STARTKEY_END_INDEX, DATE_END_INDEX); String uuid = fileName.substring(DATE_END_INDEX, UUID_END_INDEX); - return new MobFileName(startKey, date, uuid); + String regionName = fileName.substring(UUID_END_INDEX+1); + return new MobFileName(startKey, date, uuid, regionName); } /** @@ -148,6 +156,13 @@ public String getStartKey() { return startKey; } + /** + * Gets region name + * @return region name + */ + public String getRegionName() { + return regionName; + } /** * Gets the date string. Its format is yyyymmdd. * @return The date string. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java index ee1fe7db14fc..6adb4b58ded5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java @@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine; import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.yetus.audience.InterfaceAudience; /** @@ -29,7 +30,7 @@ */ @InterfaceAudience.Private public class MobStoreEngine extends DefaultStoreEngine { - + public final static String DEFAULT_MOB_COMPACTOR_CLASS_KEY = "hbase.mob.default.compactor"; @Override protected void createStoreFlusher(Configuration conf, HStore store) throws IOException { // When using MOB, we use DefaultMobStoreFlusher always @@ -43,6 +44,13 @@ protected void createStoreFlusher(Configuration conf, HStore store) throws IOExc */ @Override protected void createCompactor(Configuration conf, HStore store) throws IOException { - compactor = new DefaultMobStoreCompactor(conf, store); + String className = + conf.get(DEFAULT_MOB_COMPACTOR_CLASS_KEY, DefaultMobStoreCompactor.class.getName()); + try { + compactor = ReflectionUtils.instantiateWithCustomCtor(className, + new Class[] { Configuration.class, HStore.class }, new Object[] { conf, store }); + } catch (Exception e) { + throw new IOException("Unable to load configured compactor '" + className + "'", e); + } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java index 304a62e8901b..73c6f25548dd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java @@ -23,23 +23,22 @@ import java.text.ParseException; import java.text.SimpleDateFormat; import java.util.ArrayList; +import java.util.Arrays; import java.util.Calendar; import java.util.Collection; import java.util.Date; +import java.util.HashSet; import java.util.List; import java.util.Optional; +import java.util.Set; import java.util.UUID; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.RejectedExecutionHandler; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.HConstants; @@ -49,8 +48,10 @@ import org.apache.hadoop.hbase.TagType; import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.backup.HFileArchiver; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.Scan; @@ -62,10 +63,6 @@ import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; -import org.apache.hadoop.hbase.master.locking.LockManager; -import org.apache.hadoop.hbase.mob.compactions.MobCompactor; -import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId; -import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactor; import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.HStoreFile; @@ -74,8 +71,7 @@ import org.apache.hadoop.hbase.util.ChecksumType; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.hbase.util.ReflectionUtils; -import org.apache.hadoop.hbase.util.Threads; +import org.apache.hadoop.hbase.util.HFileArchiveUtil; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -87,8 +83,7 @@ public final class MobUtils { private static final Logger LOG = LoggerFactory.getLogger(MobUtils.class); - private final static long WEEKLY_THRESHOLD_MULTIPLIER = 7; - private final static long MONTHLY_THRESHOLD_MULTIPLIER = 4 * WEEKLY_THRESHOLD_MULTIPLIER; + public static final String SEP = "_"; private static final ThreadLocal LOCAL_FORMAT = new ThreadLocal() { @@ -98,13 +93,6 @@ protected SimpleDateFormat initialValue() { } }; - private static final byte[] REF_DELETE_MARKER_TAG_BYTES; - static { - List tags = new ArrayList<>(); - tags.add(MobConstants.MOB_REF_TAG); - REF_DELETE_MARKER_TAG_BYTES = TagUtil.fromList(tags); - } - /** * Private constructor to keep this class from being instantiated. */ @@ -130,44 +118,6 @@ public static Date parseDate(String dateString) throws ParseException { return LOCAL_FORMAT.get().parse(dateString); } - /** - * Get the first day of the input date's month - * @param calendar Calendar object - * @param date The date to find out its first day of that month - * @return The first day in the month - */ - public static Date getFirstDayOfMonth(final Calendar calendar, final Date date) { - - calendar.setTime(date); - calendar.set(Calendar.HOUR_OF_DAY, 0); - calendar.set(Calendar.MINUTE, 0); - calendar.set(Calendar.SECOND, 0); - calendar.set(Calendar.MILLISECOND, 0); - calendar.set(Calendar.DAY_OF_MONTH, 1); - - Date firstDayInMonth = calendar.getTime(); - return firstDayInMonth; - } - - /** - * Get the first day of the input date's week - * @param calendar Calendar object - * @param date The date to find out its first day of that week - * @return The first day in the week - */ - public static Date getFirstDayOfWeek(final Calendar calendar, final Date date) { - - calendar.setTime(date); - calendar.set(Calendar.HOUR_OF_DAY, 0); - calendar.set(Calendar.MINUTE, 0); - calendar.set(Calendar.SECOND, 0); - calendar.set(Calendar.MILLISECOND, 0); - calendar.setFirstDayOfWeek(Calendar.MONDAY); - calendar.set(Calendar.DAY_OF_WEEK, Calendar.MONDAY); - - Date firstDayInWeek = calendar.getTime(); - return firstDayInWeek; - } /** * Whether the current cell is a mob reference cell. @@ -370,6 +320,23 @@ public static Path getMobHome(Configuration conf) { return getMobHome(hbaseDir); } + /** + * Gets region encoded name from a MOB file name + * @param name name of a MOB file + * @return encoded region name + * + */ + public static String getEncodedRegionNameFromMobFileName(String mobFileName) + { + int index = mobFileName.lastIndexOf(MobFileName.REGION_SEP); + if (index > 0) { + return mobFileName.substring(index+1); + } else { + return null; + } + } + + /** * Gets the root dir of the mob files under the qualified HBase root dir. * It's {rootDir}/mobdir. @@ -487,15 +454,6 @@ public static boolean isMobRegionName(TableName tableName, byte[] regionName) { return Bytes.equals(regionName, getMobRegionInfo(tableName).getRegionName()); } - /** - * Gets the working directory of the mob compaction. - * @param root The root directory of the mob compaction. - * @param jobName The current job name. - * @return The directory of the mob compaction for the current job. - */ - public static Path getCompactionWorkingPath(Path root, String jobName) { - return new Path(root, jobName); - } /** * Archives the mob files. @@ -513,6 +471,40 @@ public static void removeMobFiles(Configuration conf, FileSystem fs, TableName t storeFiles); } + /** + * Archives the mob files. + * @param conf The current configuration. + * @param tableName The table name. + * @param family The name of the column family. + * @param storeFiles The files to be archived. + * @throws IOException + */ + public static void removeMobFiles(Configuration conf, TableName tableName, + byte[] family, List storeFiles) throws IOException { + + if (storeFiles.size() == 0) { + // nothing to remove + LOG.debug("Skipping archiving old MOB file: collection is empty"); + return; + } + Path mobTableDir = FSUtils.getTableDir(MobUtils.getMobHome(conf), tableName); + FileSystem fs = storeFiles.get(0).getFileSystem(conf); + Path storeArchiveDir = HFileArchiveUtil.getStoreArchivePath(conf, getMobRegionInfo(tableName), + mobTableDir, family); + + for (Path p: storeFiles) { + Path archiveFilePath = new Path(storeArchiveDir, p.getName()); + if (fs.exists(archiveFilePath)) { + LOG.info(" MOB Cleaner skip archiving: " + p); + continue; + } + LOG.info(" MOB Cleaner archiving: " + p); + HFileArchiver.archiveStoreFile(conf, fs, getMobRegionInfo(tableName), mobTableDir, family, p); + } + } + + + /** * Creates a mob reference KeyValue. * The value of the mob reference KeyValue is mobCellValueSize + mobFileName. @@ -561,91 +553,43 @@ public static Cell createMobRefCell(Cell cell, byte[] fileName, byte[] refCellTa public static StoreFileWriter createWriter(Configuration conf, FileSystem fs, ColumnFamilyDescriptor family, String date, Path basePath, long maxKeyCount, Compression.Algorithm compression, String startKey, CacheConfig cacheConfig, - Encryption.Context cryptoContext, boolean isCompaction) + Encryption.Context cryptoContext, boolean isCompaction, String regionName) throws IOException { MobFileName mobFileName = MobFileName.create(startKey, date, - UUID.randomUUID().toString().replaceAll("-", "")); + UUID.randomUUID().toString().replaceAll("-", ""), regionName); return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression, cacheConfig, cryptoContext, isCompaction); } - /** - * Creates a writer for the ref file in temp directory. - * @param conf The current configuration. - * @param fs The current file system. - * @param family The descriptor of the current column family. - * @param basePath The basic path for a temp directory. - * @param maxKeyCount The key count. - * @param cacheConfig The current cache config. - * @param cryptoContext The encryption context. - * @param isCompaction If the writer is used in compaction. - * @return The writer for the mob file. - * @throws IOException - */ - public static StoreFileWriter createRefFileWriter(Configuration conf, FileSystem fs, - ColumnFamilyDescriptor family, Path basePath, long maxKeyCount, CacheConfig cacheConfig, - Encryption.Context cryptoContext, boolean isCompaction) - throws IOException { - return createWriter(conf, fs, family, - new Path(basePath, UUID.randomUUID().toString().replaceAll("-", "")), maxKeyCount, - family.getCompactionCompressionType(), cacheConfig, cryptoContext, - HStore.getChecksumType(conf), HStore.getBytesPerChecksum(conf), family.getBlocksize(), - family.getBloomFilterType(), isCompaction); - } - /** - * Creates a writer for the mob file in temp directory. - * @param conf The current configuration. - * @param fs The current file system. - * @param family The descriptor of the current column family. - * @param date The date string, its format is yyyymmmdd. - * @param basePath The basic path for a temp directory. - * @param maxKeyCount The key count. - * @param compression The compression algorithm. - * @param startKey The start key. - * @param cacheConfig The current cache config. - * @param cryptoContext The encryption context. - * @param isCompaction If the writer is used in compaction. - * @return The writer for the mob file. - * @throws IOException - */ - public static StoreFileWriter createWriter(Configuration conf, FileSystem fs, - ColumnFamilyDescriptor family, String date, Path basePath, long maxKeyCount, - Compression.Algorithm compression, byte[] startKey, CacheConfig cacheConfig, - Encryption.Context cryptoContext, boolean isCompaction) - throws IOException { - MobFileName mobFileName = MobFileName.create(startKey, date, - UUID.randomUUID().toString().replaceAll("-", "")); - return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression, - cacheConfig, cryptoContext, isCompaction); - } +// /** +// * Creates a writer for the mob file in temp directory. +// * @param conf The current configuration. +// * @param fs The current file system. +// * @param family The descriptor of the current column family. +// * @param date The date string, its format is yyyymmmdd. +// * @param basePath The basic path for a temp directory. +// * @param maxKeyCount The key count. +// * @param compression The compression algorithm. +// * @param startKey The start key. +// * @param cacheConfig The current cache config. +// * @param cryptoContext The encryption context. +// * @param isCompaction If the writer is used in compaction. +// * @return The writer for the mob file. +// * @throws IOException +// */ +// public static StoreFileWriter createWriter(Configuration conf, FileSystem fs, +// ColumnFamilyDescriptor family, String date, Path basePath, long maxKeyCount, +// Compression.Algorithm compression, byte[] startKey, CacheConfig cacheConfig, +// Encryption.Context cryptoContext, boolean isCompaction) +// throws IOException { +// MobFileName mobFileName = MobFileName.create(startKey, date, +// UUID.randomUUID().toString().replaceAll("-", "")); +// return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression, +// cacheConfig, cryptoContext, isCompaction); +// } + - /** - * Creates a writer for the del file in temp directory. - * @param conf The current configuration. - * @param fs The current file system. - * @param family The descriptor of the current column family. - * @param date The date string, its format is yyyymmmdd. - * @param basePath The basic path for a temp directory. - * @param maxKeyCount The key count. - * @param compression The compression algorithm. - * @param startKey The start key. - * @param cacheConfig The current cache config. - * @param cryptoContext The encryption context. - * @return The writer for the del file. - * @throws IOException - */ - public static StoreFileWriter createDelFileWriter(Configuration conf, FileSystem fs, - ColumnFamilyDescriptor family, String date, Path basePath, long maxKeyCount, - Compression.Algorithm compression, byte[] startKey, CacheConfig cacheConfig, - Encryption.Context cryptoContext) - throws IOException { - String suffix = UUID - .randomUUID().toString().replaceAll("-", "") + "_del"; - MobFileName mobFileName = MobFileName.create(startKey, date, suffix); - return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression, - cacheConfig, cryptoContext, true); - } /** * Creates a writer for the mob file in temp directory. @@ -827,70 +771,8 @@ public static TableName getTableLockName(TableName tn) { return TableName.valueOf(Bytes.add(tableName, MobConstants.MOB_TABLE_LOCK_SUFFIX)); } - /** - * Performs the mob compaction. - * @param conf the Configuration - * @param fs the file system - * @param tableName the table the compact - * @param hcd the column descriptor - * @param pool the thread pool - * @param allFiles Whether add all mob files into the compaction. - */ - public static void doMobCompaction(Configuration conf, FileSystem fs, TableName tableName, - ColumnFamilyDescriptor hcd, ExecutorService pool, boolean allFiles, LockManager.MasterLock lock) - throws IOException { - String className = conf.get(MobConstants.MOB_COMPACTOR_CLASS_KEY, - PartitionedMobCompactor.class.getName()); - // instantiate the mob compactor. - MobCompactor compactor = null; - try { - compactor = ReflectionUtils.instantiateWithCustomCtor(className, new Class[] { - Configuration.class, FileSystem.class, TableName.class, ColumnFamilyDescriptor.class, - ExecutorService.class }, new Object[] { conf, fs, tableName, hcd, pool }); - } catch (Exception e) { - throw new IOException("Unable to load configured mob file compactor '" + className + "'", e); - } - // compact only for mob-enabled column. - // obtain a write table lock before performing compaction to avoid race condition - // with major compaction in mob-enabled column. - try { - lock.acquire(); - compactor.compact(allFiles); - } catch (Exception e) { - LOG.error("Failed to compact the mob files for the column " + hcd.getNameAsString() - + " in the table " + tableName.getNameAsString(), e); - } finally { - lock.release(); - } - } - /** - * Creates a thread pool. - * @param conf the Configuration - * @return A thread pool. - */ - public static ExecutorService createMobCompactorThreadPool(Configuration conf) { - int maxThreads = conf.getInt(MobConstants.MOB_COMPACTION_THREADS_MAX, - MobConstants.DEFAULT_MOB_COMPACTION_THREADS_MAX); - if (maxThreads == 0) { - maxThreads = 1; - } - final SynchronousQueue queue = new SynchronousQueue<>(); - ThreadPoolExecutor pool = new ThreadPoolExecutor(1, maxThreads, 60, TimeUnit.SECONDS, queue, - Threads.newDaemonThreadFactory("MobCompactor"), new RejectedExecutionHandler() { - @Override - public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) { - try { - // waiting for a thread to pick up instead of throwing exceptions. - queue.put(r); - } catch (InterruptedException e) { - throw new RejectedExecutionException(e); - } - } - }); - ((ThreadPoolExecutor) pool).allowCoreThreadTimeOut(true); - return pool; - } + /** * Checks whether this table has mob-enabled columns. @@ -907,6 +789,143 @@ public static boolean hasMobColumns(TableDescriptor htd) { return false; } + /** + * Get list of Mob column families (if any exists) + * @param htd table descriptor + * @return list of Mob column families + */ + public static List getMobColumnFamilies(TableDescriptor htd){ + + List fams = new ArrayList(); + ColumnFamilyDescriptor[] hcds = htd.getColumnFamilies(); + for (ColumnFamilyDescriptor hcd : hcds) { + if (hcd.isMobEnabled()) { + fams.add(hcd); + } + } + return fams; + } + + /** + * Performs housekeeping file cleaning (called by MOB Cleaner chore) + * @param conf configuration + * @param table table name + * @throws IOException + */ + public static void cleanupObsoleteMobFiles(Configuration conf, TableName table) + throws IOException { + + try (final Connection conn = ConnectionFactory.createConnection(conf); + final Admin admin = conn.getAdmin();) { + TableDescriptor htd = admin.getDescriptor(table); + List list = getMobColumnFamilies(htd); + if (list.size() == 0) { + LOG.info("Skipping non-MOB table [" + table + "]"); + return; + } + Path rootDir = FSUtils.getRootDir(conf); + Path tableDir = FSUtils.getTableDir(rootDir, table); + // How safe is this call? + List regionDirs = FSUtils.getRegionDirs(FileSystem.get(conf), tableDir); + + Set allActiveMobFileName = new HashSet(); + FileSystem fs = FileSystem.get(conf); + for (Path regionPath: regionDirs) { + for (ColumnFamilyDescriptor hcd: list) { + String family = hcd.getNameAsString(); + Path storePath = new Path(regionPath, family); + boolean succeed = false; + Set regionMobs = new HashSet(); + while(!succeed) { + //TODO handle FNFE + RemoteIterator rit = fs.listLocatedStatus(storePath); + List storeFiles = new ArrayList(); + // Load list of store files first + while(rit.hasNext()) { + Path p = rit.next().getPath(); + if (fs.isFile(p)) { + storeFiles.add(p); + } + } + try { + for(Path pp: storeFiles) { + HStoreFile sf = new HStoreFile(fs, pp, conf, CacheConfig.DISABLED, + BloomType.NONE, true); + sf.initReader(); + byte[] mobRefData = sf.getMetadataValue(HStoreFile.MOB_FILE_REFS); + byte[] mobCellCountData = sf.getMetadataValue(HStoreFile.MOB_CELLS_COUNT); + byte[] bulkloadMarkerData = sf.getMetadataValue(HStoreFile.BULKLOAD_TASK_KEY); + if (mobRefData == null && (mobCellCountData != null || + bulkloadMarkerData == null)) { + LOG.info("Found old store file with no MOB_FILE_REFS: " + pp + +" - can not proceed until all old files will be MOB-compacted"); + return; + } else if (mobRefData == null) { + LOG.info("Skipping file without MOB references (can be bulkloaded file):"+ pp); + continue; + } + String[] mobs = new String(mobRefData).split(","); + regionMobs.addAll(Arrays.asList(mobs)); + } + } catch (FileNotFoundException e) { + //TODO + LOG.warn(e.getMessage()); + continue; + } + succeed = true; + } + // Add MOB refs for current region/family + allActiveMobFileName.addAll(regionMobs); + } // END column families + }//END regions + + // Now scan MOB directories and find MOB files with no references to them + long now = System.currentTimeMillis(); + long minAgeToArchive = conf.getLong(MobConstants.MOB_MINIMUM_FILE_AGE_TO_ARCHIVE_KEY, + MobConstants.DEFAULT_MOB_MINIMUM_FILE_AGE_TO_ARCHIVE); + for (ColumnFamilyDescriptor hcd: list) { + List toArchive = new ArrayList(); + String family = hcd.getNameAsString(); + Path dir = getMobFamilyPath(conf, table, family); + RemoteIterator rit = fs.listLocatedStatus(dir); + while(rit.hasNext()) { + LocatedFileStatus lfs = rit.next(); + Path p = lfs.getPath(); + if (!allActiveMobFileName.contains(p.getName())) { + // MOB is not in a list of active references, but it can be too + // fresh, skip it in this case + /*DEBUG*/ LOG.debug(" Age=" + (now - fs.getFileStatus(p).getModificationTime()) + + " MOB file="+ p); + if (now - fs.getFileStatus(p).getModificationTime() > minAgeToArchive) { + toArchive.add(p); + } else { + LOG.debug(" Skipping fresh file: " + p); + } + } + } + LOG.info(" MOB Cleaner found "+ toArchive.size()+" files for family="+family); + removeMobFiles(conf, table, family.getBytes(), toArchive); + LOG.info(" MOB Cleaner archived "+ toArchive.size()+" files"); + } + } + } + + + + + public static long getNumberOfMobFiles(Configuration conf, TableName tableName, String family) + throws IOException { + FileSystem fs = FileSystem.get(conf); + Path dir = getMobFamilyPath(conf, tableName, family); + FileStatus[] stat = fs.listStatus(dir); + for (FileStatus st: stat) { + LOG.info(" MOB Directory content: "+ st.getPath()); + } + LOG.info(" MOB Directory content total files: "+ stat.length); + + return stat.length; + } + /** * Indicates whether return null value when the mob file is missing or corrupt. * The information is set in the attribute "empty.value.on.mobcell.miss" of scan. @@ -923,14 +942,6 @@ public static boolean isReadEmptyValueOnMobCellMiss(Scan scan) { } } - /** - * Creates a mob ref delete marker. - * @param cell The current delete marker. - * @return A delete marker with the ref tag. - */ - public static Cell createMobRefDeleteMarker(Cell cell) { - return PrivateCellUtil.createCell(cell, TagUtil.concatTags(REF_DELETE_MARKER_TAG_BYTES, cell)); - } /** * Checks if the mob file is expired. @@ -962,86 +973,89 @@ public static boolean isMobFileExpired(ColumnFamilyDescriptor column, long curre return false; } - /** - * fill out partition id based on compaction policy and date, threshold... - * @param id Partition id to be filled out - * @param firstDayOfCurrentMonth The first day in the current month - * @param firstDayOfCurrentWeek The first day in the current week - * @param dateStr Date string from the mob file - * @param policy Mob compaction policy - * @param calendar Calendar object - * @param threshold Mob compaciton threshold configured - * @return true if the file needs to be excluded from compaction - */ - public static boolean fillPartitionId(final CompactionPartitionId id, - final Date firstDayOfCurrentMonth, final Date firstDayOfCurrentWeek, final String dateStr, - final MobCompactPartitionPolicy policy, final Calendar calendar, final long threshold) { - - boolean skipCompcation = false; - id.setThreshold(threshold); - if (threshold <= 0) { - id.setDate(dateStr); - return skipCompcation; - } - - long finalThreshold; - Date date; - try { - date = MobUtils.parseDate(dateStr); - } catch (ParseException e) { - LOG.warn("Failed to parse date " + dateStr, e); - id.setDate(dateStr); - return true; - } - - /* The algorithm works as follows: - * For monthly policy: - * 1). If the file's date is in past months, apply 4 * 7 * threshold - * 2). If the file's date is in past weeks, apply 7 * threshold - * 3). If the file's date is in current week, exclude it from the compaction - * For weekly policy: - * 1). If the file's date is in past weeks, apply 7 * threshold - * 2). If the file's date in currently, apply threshold - * For daily policy: - * 1). apply threshold - */ - if (policy == MobCompactPartitionPolicy.MONTHLY) { - if (date.before(firstDayOfCurrentMonth)) { - // Check overflow - if (threshold < (Long.MAX_VALUE / MONTHLY_THRESHOLD_MULTIPLIER)) { - finalThreshold = MONTHLY_THRESHOLD_MULTIPLIER * threshold; - } else { - finalThreshold = Long.MAX_VALUE; - } - id.setThreshold(finalThreshold); - - // set to the date for the first day of that month - id.setDate(MobUtils.formatDate(MobUtils.getFirstDayOfMonth(calendar, date))); - return skipCompcation; - } - } - - if ((policy == MobCompactPartitionPolicy.MONTHLY) || - (policy == MobCompactPartitionPolicy.WEEKLY)) { - // Check if it needs to apply weekly multiplier - if (date.before(firstDayOfCurrentWeek)) { - // Check overflow - if (threshold < (Long.MAX_VALUE / WEEKLY_THRESHOLD_MULTIPLIER)) { - finalThreshold = WEEKLY_THRESHOLD_MULTIPLIER * threshold; - } else { - finalThreshold = Long.MAX_VALUE; - } - id.setThreshold(finalThreshold); - id.setDate(MobUtils.formatDate(MobUtils.getFirstDayOfWeek(calendar, date))); - return skipCompcation; - } else if (policy == MobCompactPartitionPolicy.MONTHLY) { - skipCompcation = true; - } - } - - // Rest is daily - id.setDate(dateStr); - return skipCompcation; - } +//TODO : remove below after analysis + +// /** +// * fill out partition id based on compaction policy and date, threshold... +// * @param id Partition id to be filled out +// * @param firstDayOfCurrentMonth The first day in the current month +// * @param firstDayOfCurrentWeek The first day in the current week +// * @param dateStr Date string from the mob file +// * @param policy Mob compaction policy +// * @param calendar Calendar object +// * @param threshold Mob compaciton threshold configured +// * @return true if the file needs to be excluded from compaction +// */ +// public static boolean fillPartitionId(final CompactionPartitionId id, +// final Date firstDayOfCurrentMonth, final Date firstDayOfCurrentWeek, final String dateStr, +// final MobCompactPartitionPolicy policy, final Calendar calendar, final long threshold) { +// +// boolean skipCompcation = false; +// id.setThreshold(threshold); +// if (threshold <= 0) { +// id.setDate(dateStr); +// return skipCompcation; +// } +// +// long finalThreshold; +// Date date; +// try { +// date = MobUtils.parseDate(dateStr); +// } catch (ParseException e) { +// LOG.warn("Failed to parse date " + dateStr, e); +// id.setDate(dateStr); +// return true; +// } +// +// /* The algorithm works as follows: +// * For monthly policy: +// * 1). If the file's date is in past months, apply 4 * 7 * threshold +// * 2). If the file's date is in past weeks, apply 7 * threshold +// * 3). If the file's date is in current week, exclude it from the compaction +// * For weekly policy: +// * 1). If the file's date is in past weeks, apply 7 * threshold +// * 2). If the file's date in currently, apply threshold +// * For daily policy: +// * 1). apply threshold +// */ +// if (policy == MobCompactPartitionPolicy.MONTHLY) { +// if (date.before(firstDayOfCurrentMonth)) { +// // Check overflow +// if (threshold < (Long.MAX_VALUE / MONTHLY_THRESHOLD_MULTIPLIER)) { +// finalThreshold = MONTHLY_THRESHOLD_MULTIPLIER * threshold; +// } else { +// finalThreshold = Long.MAX_VALUE; +// } +// id.setThreshold(finalThreshold); +// +// // set to the date for the first day of that month +// id.setDate(MobUtils.formatDate(MobUtils.getFirstDayOfMonth(calendar, date))); +// return skipCompcation; +// } +// } +// +// if ((policy == MobCompactPartitionPolicy.MONTHLY) || +// (policy == MobCompactPartitionPolicy.WEEKLY)) { +// // Check if it needs to apply weekly multiplier +// if (date.before(firstDayOfCurrentWeek)) { +// // Check overflow +// if (threshold < (Long.MAX_VALUE / WEEKLY_THRESHOLD_MULTIPLIER)) { +// finalThreshold = WEEKLY_THRESHOLD_MULTIPLIER * threshold; +// } else { +// finalThreshold = Long.MAX_VALUE; +// } +// id.setThreshold(finalThreshold); +// +// id.setDate(MobUtils.formatDate(MobUtils.getFirstDayOfWeek(calendar, date))); +// return skipCompcation; +// } else if (policy == MobCompactPartitionPolicy.MONTHLY) { +// skipCompcation = true; +// } +// } +// +// // Rest is daily +// id.setDate(dateStr); +// return skipCompcation; +// } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactionRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactionRequest.java deleted file mode 100644 index 5025ed261229..000000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactionRequest.java +++ /dev/null @@ -1,64 +0,0 @@ -/** - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.mob.compactions; - -import org.apache.yetus.audience.InterfaceAudience; - -/** - * The compaction request for mob files. - */ -@InterfaceAudience.Private -public abstract class MobCompactionRequest { - - protected long selectionTime; - protected CompactionType type = CompactionType.PART_FILES; - - public void setCompactionType(CompactionType type) { - this.type = type; - } - - /** - * Gets the selection time. - * @return The selection time. - */ - public long getSelectionTime() { - return this.selectionTime; - } - - /** - * Gets the compaction type. - * @return The compaction type. - */ - public CompactionType getCompactionType() { - return type; - } - - protected enum CompactionType { - - /** - * Part of mob files are selected. - */ - PART_FILES, - - /** - * All of mob files are selected. - */ - ALL_FILES - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java deleted file mode 100644 index c5d93ea6e064..000000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java +++ /dev/null @@ -1,90 +0,0 @@ -/** - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.mob.compactions; - -import java.io.IOException; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.ExecutorService; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.TableName; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.mob.MobUtils; -import org.apache.hadoop.hbase.util.FSUtils; - -/** - * A mob compactor to directly compact the mob files. - */ -@InterfaceAudience.Private -public abstract class MobCompactor { - - protected FileSystem fs; - protected Configuration conf; - protected TableName tableName; - protected ColumnFamilyDescriptor column; - - protected Path mobTableDir; - protected Path mobFamilyDir; - protected ExecutorService pool; - - public MobCompactor(Configuration conf, FileSystem fs, TableName tableName, - ColumnFamilyDescriptor column, ExecutorService pool) { - this.conf = conf; - this.fs = fs; - this.tableName = tableName; - this.column = column; - this.pool = pool; - mobTableDir = FSUtils.getTableDir(MobUtils.getMobHome(conf), tableName); - mobFamilyDir = MobUtils.getMobFamilyPath(conf, tableName, column.getNameAsString()); - } - - /** - * Compacts the mob files for the current column family. - * @return The paths of new mob files generated in the compaction. - * @throws IOException - */ - public List compact() throws IOException { - return compact(false); - } - - /** - * Compacts the mob files by compaction type for the current column family. - * @param allFiles Whether add all mob files into the compaction. - * @return The paths of new mob files generated in the compaction. - * @throws IOException - */ - public List compact(boolean allFiles) throws IOException { - return compact(Arrays.asList(fs.listStatus(mobFamilyDir)), allFiles); - } - - /** - * Compacts the candidate mob files. - * @param files The candidate mob files. - * @param allFiles Whether add all mob files into the compaction. - * @return The paths of new mob files generated in the compaction. - * @throws IOException - */ - public abstract List compact(List files, boolean allFiles) - throws IOException; -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactionRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactionRequest.java deleted file mode 100644 index ab917a2d9228..000000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactionRequest.java +++ /dev/null @@ -1,333 +0,0 @@ -/** - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.mob.compactions; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; - -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.regionserver.HStoreFile; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.yetus.audience.InterfaceAudience; - -/** - * An implementation of {@link MobCompactionRequest} that is used in - * {@link PartitionedMobCompactor}. - * The mob files that have the same start key and date in their names belong to - * the same partition. - */ -@InterfaceAudience.Private -public class PartitionedMobCompactionRequest extends MobCompactionRequest { - - protected List delPartitions; - protected Collection compactionPartitions; - - public PartitionedMobCompactionRequest(Collection compactionPartitions, - List delPartitions) { - this.selectionTime = EnvironmentEdgeManager.currentTime(); - this.compactionPartitions = compactionPartitions; - this.delPartitions = delPartitions; - } - - /** - * Gets the compaction partitions. - * @return The compaction partitions. - */ - public Collection getCompactionPartitions() { - return this.compactionPartitions; - } - - /** - * Gets the del files. - * @return The del files. - */ - public List getDelPartitions() { - return this.delPartitions; - } - - /** - * The partition in the mob compaction. - * The mob files that have the same start key and date in their names belong to - * the same partition. - */ - protected static class CompactionPartition { - private List files = new ArrayList<>(); - private CompactionPartitionId partitionId; - - // The startKey and endKey of this partition, both are inclusive. - private byte[] startKey; - private byte[] endKey; - - public CompactionPartition(CompactionPartitionId partitionId) { - this.partitionId = partitionId; - } - - public CompactionPartitionId getPartitionId() { - return this.partitionId; - } - - public void addFile(FileStatus file) { - files.add(file); - } - - public List listFiles() { - return Collections.unmodifiableList(files); - } - - public int getFileCount () { - return files.size(); - } - - public byte[] getStartKey() { - return startKey; - } - - /** - * Set start key of this partition, only if the input startKey is less than - * the current start key. - */ - public void setStartKey(final byte[] startKey) { - if ((this.startKey == null) || (Bytes.compareTo(startKey, this.startKey) < 0)) { - this.startKey = startKey; - } - } - - public byte[] getEndKey() { - return endKey; - } - - /** - * Set end key of this partition, only if the input endKey is greater than - * the current end key. - */ - public void setEndKey(final byte[] endKey) { - if ((this.endKey == null) || (Bytes.compareTo(endKey, this.endKey) > 0)) { - this.endKey = endKey; - } - } - } - - /** - * The partition id that consists of start key and date of the mob file name. - */ - public static class CompactionPartitionId { - private String startKey; - private String date; - private String latestDate; - private long threshold; - - public CompactionPartitionId() { - // initialize these fields to empty string - this.startKey = ""; - this.date = ""; - this.latestDate = ""; - this.threshold = 0; - } - - public CompactionPartitionId(String startKey, String date) { - if (startKey == null || date == null) { - throw new IllegalArgumentException("Neither of start key and date could be null"); - } - this.startKey = startKey; - this.date = date; - this.latestDate = ""; - this.threshold = 0; - } - - public void setThreshold (final long threshold) { - this.threshold = threshold; - } - - public long getThreshold () { - return this.threshold; - } - - public String getStartKey() { - return this.startKey; - } - - public void setStartKey(final String startKey) { - this.startKey = startKey; - } - - public String getDate() { - return this.date; - } - - public void setDate(final String date) { - this.date = date; - } - - public String getLatestDate () { return this.latestDate; } - - public void updateLatestDate(final String latestDate) { - if (this.latestDate.compareTo(latestDate) < 0) { - this.latestDate = latestDate; - } - } - - @Override - public int hashCode() { - int result = 17; - result = 31 * result + startKey.hashCode(); - result = 31 * result + date.hashCode(); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (!(obj instanceof CompactionPartitionId)) { - return false; - } - CompactionPartitionId another = (CompactionPartitionId) obj; - if (!this.startKey.equals(another.startKey)) { - return false; - } - if (!this.date.equals(another.date)) { - return false; - } - return true; - } - - @Override - public String toString() { - return new StringBuilder(startKey).append(date).toString(); - } - } - - /** - * The delete file partition in the mob compaction. - * The delete partition is defined as [startKey, endKey] pair. - * The mob delete files that have the same start key and end key belong to - * the same partition. - */ - protected static class CompactionDelPartition { - private List delFiles = new ArrayList(); - private List storeFiles = new ArrayList<>(); - private CompactionDelPartitionId id; - - public CompactionDelPartition(CompactionDelPartitionId id) { - this.id = id; - } - - public CompactionDelPartitionId getId() { - return this.id; - } - - void addDelFile(FileStatus file) { - delFiles.add(file.getPath()); - } - public void addStoreFile(HStoreFile file) { - storeFiles.add(file); - } - - public List getStoreFiles() { - return storeFiles; - } - - List listDelFiles() { - return Collections.unmodifiableList(delFiles); - } - - void addDelFileList(final Collection list) { - delFiles.addAll(list); - } - - int getDelFileCount () { - return delFiles.size(); - } - - void cleanDelFiles() { - delFiles.clear(); - } - } - - /** - * The delete partition id that consists of start key and end key - */ - public static class CompactionDelPartitionId implements Comparable { - private byte[] startKey; - private byte[] endKey; - - public CompactionDelPartitionId() { - } - - public CompactionDelPartitionId(final byte[] startKey, final byte[] endKey) { - this.startKey = startKey; - this.endKey = endKey; - } - - public byte[] getStartKey() { - return this.startKey; - } - public void setStartKey(final byte[] startKey) { - this.startKey = startKey; - } - - public byte[] getEndKey() { - return this.endKey; - } - public void setEndKey(final byte[] endKey) { - this.endKey = endKey; - } - - @Override - public int compareTo(CompactionDelPartitionId o) { - /* - * 1). Compare the start key, if the k1 < k2, then k1 is less - * 2). If start Key is same, check endKey, k1 < k2, k1 is less - * If both are same, then they are equal. - */ - int result = Bytes.compareTo(this.startKey, o.getStartKey()); - if (result != 0) { - return result; - } - - return Bytes.compareTo(this.endKey, o.getEndKey()); - } - - @Override - public int hashCode() { - int result = 17; - result = 31 * result + java.util.Arrays.hashCode(startKey); - result = 31 * result + java.util.Arrays.hashCode(endKey); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (!(obj instanceof CompactionDelPartitionId)) { - return false; - } - CompactionDelPartitionId another = (CompactionDelPartitionId) obj; - - return (this.compareTo(another) == 0); - } - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java deleted file mode 100644 index a5823ec7d7bb..000000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java +++ /dev/null @@ -1,929 +0,0 @@ -/** - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.mob.compactions; - -import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TIME_KEY; -import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_CELLS_COUNT; -import static org.apache.hadoop.hbase.regionserver.HStoreFile.SKIP_RESET_SEQ_ID; - -import java.io.FileNotFoundException; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Calendar; -import java.util.Collection; -import java.util.Collections; -import java.util.Comparator; -import java.util.Date; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.NavigableMap; -import java.util.Objects; -import java.util.TreeMap; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.ArrayBackedTag; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellComparator; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.Tag; -import org.apache.hadoop.hbase.TagType; -import org.apache.hadoop.hbase.TagUtil; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.io.HFileLink; -import org.apache.hadoop.hbase.io.crypto.Encryption; -import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hadoop.hbase.io.hfile.HFile; -import org.apache.hadoop.hbase.io.hfile.HFile.Reader; -import org.apache.hadoop.hbase.mob.MobConstants; -import org.apache.hadoop.hbase.mob.MobFileName; -import org.apache.hadoop.hbase.mob.MobUtils; -import org.apache.hadoop.hbase.mob.compactions.MobCompactionRequest.CompactionType; -import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionDelPartition; -import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionDelPartitionId; -import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartition; -import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId; -import org.apache.hadoop.hbase.regionserver.BloomType; -import org.apache.hadoop.hbase.regionserver.HStore; -import org.apache.hadoop.hbase.regionserver.HStoreFile; -import org.apache.hadoop.hbase.regionserver.ScanInfo; -import org.apache.hadoop.hbase.regionserver.ScanType; -import org.apache.hadoop.hbase.regionserver.ScannerContext; -import org.apache.hadoop.hbase.regionserver.StoreFileInfo; -import org.apache.hadoop.hbase.regionserver.StoreFileScanner; -import org.apache.hadoop.hbase.regionserver.StoreFileWriter; -import org.apache.hadoop.hbase.regionserver.StoreScanner; -import org.apache.hadoop.hbase.security.EncryptionUtil; -import org.apache.hadoop.hbase.tool.BulkLoadHFiles; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - -/** - * An implementation of {@link MobCompactor} that compacts the mob files in partitions. - */ -@InterfaceAudience.Private -public class PartitionedMobCompactor extends MobCompactor { - - private static final Logger LOG = LoggerFactory.getLogger(PartitionedMobCompactor.class); - protected long mergeableSize; - protected int delFileMaxCount; - /** The number of files compacted in a batch */ - protected int compactionBatchSize; - protected int compactionKVMax; - - private final Path tempPath; - private final Path bulkloadPath; - private final CacheConfig compactionCacheConfig; - private final byte[] refCellTags; - private Encryption.Context cryptoContext = Encryption.Context.NONE; - - public PartitionedMobCompactor(Configuration conf, FileSystem fs, TableName tableName, - ColumnFamilyDescriptor column, ExecutorService pool) throws IOException { - super(conf, fs, tableName, column, pool); - mergeableSize = conf.getLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, - MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD); - delFileMaxCount = conf.getInt(MobConstants.MOB_DELFILE_MAX_COUNT, - MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT); - // default is 100 - compactionBatchSize = conf.getInt(MobConstants.MOB_COMPACTION_BATCH_SIZE, - MobConstants.DEFAULT_MOB_COMPACTION_BATCH_SIZE); - tempPath = new Path(MobUtils.getMobHome(conf), MobConstants.TEMP_DIR_NAME); - bulkloadPath = new Path(tempPath, new Path(MobConstants.BULKLOAD_DIR_NAME, new Path( - tableName.getNamespaceAsString(), tableName.getQualifierAsString()))); - compactionKVMax = this.conf.getInt(HConstants.COMPACTION_KV_MAX, - HConstants.COMPACTION_KV_MAX_DEFAULT); - Configuration copyOfConf = new Configuration(conf); - copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f); - compactionCacheConfig = new CacheConfig(copyOfConf); - List tags = new ArrayList<>(2); - tags.add(MobConstants.MOB_REF_TAG); - Tag tableNameTag = new ArrayBackedTag(TagType.MOB_TABLE_NAME_TAG_TYPE, tableName.getName()); - tags.add(tableNameTag); - this.refCellTags = TagUtil.fromList(tags); - cryptoContext = EncryptionUtil.createEncryptionContext(copyOfConf, column); - } - - @Override - public List compact(List files, boolean allFiles) throws IOException { - if (files == null || files.isEmpty()) { - LOG.info("No candidate mob files"); - return null; - } - LOG.info("is allFiles: " + allFiles); - - // find the files to compact. - PartitionedMobCompactionRequest request = select(files, allFiles); - // compact the files. - return performCompaction(request); - } - - /** - * Selects the compacted mob/del files. - * Iterates the candidates to find out all the del files and small mob files. - * @param candidates All the candidates. - * @param allFiles Whether add all mob files into the compaction. - * @return A compaction request. - * @throws IOException if IO failure is encountered - */ - protected PartitionedMobCompactionRequest select(List candidates, - boolean allFiles) throws IOException { - final Map filesToCompact = new HashMap<>(); - final CompactionPartitionId id = new CompactionPartitionId(); - final NavigableMap delFilesToCompact = new TreeMap<>(); - final CompactionDelPartitionId delId = new CompactionDelPartitionId(); - final ArrayList allDelPartitions = new ArrayList<>(); - int selectedFileCount = 0; - int irrelevantFileCount = 0; - int totalDelFiles = 0; - MobCompactPartitionPolicy policy = column.getMobCompactPartitionPolicy(); - - Calendar calendar = Calendar.getInstance(); - Date currentDate = new Date(); - Date firstDayOfCurrentMonth = null; - Date firstDayOfCurrentWeek = null; - - if (policy == MobCompactPartitionPolicy.MONTHLY) { - firstDayOfCurrentMonth = MobUtils.getFirstDayOfMonth(calendar, currentDate); - firstDayOfCurrentWeek = MobUtils.getFirstDayOfWeek(calendar, currentDate); - } else if (policy == MobCompactPartitionPolicy.WEEKLY) { - firstDayOfCurrentWeek = MobUtils.getFirstDayOfWeek(calendar, currentDate); - } - - // We check if there is any del files so the logic can be optimized for the following processing - // First step is to check if there is any delete files. If there is any delete files, - // For each Partition, it needs to read its startKey and endKey from files. - // If there is no delete file, there is no need to read startKey and endKey from files, this - // is an optimization. - boolean withDelFiles = false; - for (FileStatus file : candidates) { - if (!file.isFile()) { - continue; - } - // group the del files and small files. - FileStatus linkedFile = file; - if (HFileLink.isHFileLink(file.getPath())) { - HFileLink link = HFileLink.buildFromHFileLinkPattern(conf, file.getPath()); - linkedFile = getLinkedFileStatus(link); - if (linkedFile == null) { - continue; - } - } - if (StoreFileInfo.isDelFile(linkedFile.getPath())) { - withDelFiles = true; - break; - } - } - - for (FileStatus file : candidates) { - if (!file.isFile()) { - irrelevantFileCount++; - continue; - } - // group the del files and small files. - FileStatus linkedFile = file; - if (HFileLink.isHFileLink(file.getPath())) { - HFileLink link = HFileLink.buildFromHFileLinkPattern(conf, file.getPath()); - linkedFile = getLinkedFileStatus(link); - if (linkedFile == null) { - // If the linked file cannot be found, regard it as an irrelevantFileCount file - irrelevantFileCount++; - continue; - } - } - if (withDelFiles && StoreFileInfo.isDelFile(linkedFile.getPath())) { - // File in the Del Partition List - - // Get delId from the file - try (Reader reader = HFile.createReader(fs, linkedFile.getPath(), conf)) { - delId.setStartKey(reader.getFirstRowKey().get()); - delId.setEndKey(reader.getLastRowKey().get()); - } - CompactionDelPartition delPartition = delFilesToCompact.get(delId); - if (delPartition == null) { - CompactionDelPartitionId newDelId = - new CompactionDelPartitionId(delId.getStartKey(), delId.getEndKey()); - delPartition = new CompactionDelPartition(newDelId); - delFilesToCompact.put(newDelId, delPartition); - } - delPartition.addDelFile(file); - totalDelFiles ++; - } else { - String fileName = linkedFile.getPath().getName(); - String date = MobFileName.getDateFromName(fileName); - boolean skipCompaction = MobUtils - .fillPartitionId(id, firstDayOfCurrentMonth, firstDayOfCurrentWeek, date, policy, - calendar, mergeableSize); - if (allFiles || (!skipCompaction && (linkedFile.getLen() < id.getThreshold()))) { - // add all files if allFiles is true, - // otherwise add the small files to the merge pool - // filter out files which are not supposed to be compacted with the - // current policy - - id.setStartKey(MobFileName.getStartKeyFromName(fileName)); - CompactionPartition compactionPartition = filesToCompact.get(id); - if (compactionPartition == null) { - CompactionPartitionId newId = new CompactionPartitionId(id.getStartKey(), id.getDate()); - compactionPartition = new CompactionPartition(newId); - compactionPartition.addFile(file); - filesToCompact.put(newId, compactionPartition); - newId.updateLatestDate(date); - } else { - compactionPartition.addFile(file); - compactionPartition.getPartitionId().updateLatestDate(date); - } - - if (withDelFiles) { - // get startKey and endKey from the file and update partition - // TODO: is it possible to skip read of most hfiles? - try (Reader reader = HFile.createReader(fs, linkedFile.getPath(), conf)) { - compactionPartition.setStartKey(reader.getFirstRowKey().get()); - compactionPartition.setEndKey(reader.getLastRowKey().get()); - } - } - - selectedFileCount++; - } - } - } - - /* - * Merge del files so there are only non-overlapped del file lists - */ - for(Map.Entry entry : delFilesToCompact.entrySet()) { - if (allDelPartitions.size() > 0) { - // check if the current key range overlaps the previous one - CompactionDelPartition prev = allDelPartitions.get(allDelPartitions.size() - 1); - if (Bytes.compareTo(prev.getId().getEndKey(), entry.getKey().getStartKey()) >= 0) { - // merge them together - prev.getId().setEndKey(entry.getValue().getId().getEndKey()); - prev.addDelFileList(entry.getValue().listDelFiles()); - - } else { - allDelPartitions.add(entry.getValue()); - } - } else { - allDelPartitions.add(entry.getValue()); - } - } - - PartitionedMobCompactionRequest request = new PartitionedMobCompactionRequest( - filesToCompact.values(), allDelPartitions); - if (candidates.size() == (totalDelFiles + selectedFileCount + irrelevantFileCount)) { - // all the files are selected - request.setCompactionType(CompactionType.ALL_FILES); - } - LOG.info("The compaction type is " + request.getCompactionType() + ", the request has " - + totalDelFiles + " del files, " + selectedFileCount + " selected files, and " - + irrelevantFileCount + " irrelevant files"); - return request; - } - - /** - * Performs the compaction on the selected files. - *
    - *
  1. Compacts the del files.
  2. - *
  3. Compacts the selected small mob files and all the del files.
  4. - *
  5. If all the candidates are selected, delete the del files.
  6. - *
- * @param request The compaction request. - * @return The paths of new mob files generated in the compaction. - * @throws IOException if IO failure is encountered - */ - protected List performCompaction(PartitionedMobCompactionRequest request) - throws IOException { - - // merge the del files, it is per del partition - for (CompactionDelPartition delPartition : request.getDelPartitions()) { - if (delPartition.getDelFileCount() <= 1) continue; - List newDelPaths = compactDelFiles(request, delPartition.listDelFiles()); - delPartition.cleanDelFiles(); - delPartition.addDelFileList(newDelPaths); - } - - List paths = null; - int totalDelFileCount = 0; - try { - for (CompactionDelPartition delPartition : request.getDelPartitions()) { - for (Path newDelPath : delPartition.listDelFiles()) { - HStoreFile sf = - new HStoreFile(fs, newDelPath, conf, compactionCacheConfig, BloomType.NONE, true); - // pre-create reader of a del file to avoid race condition when opening the reader in each - // partition. - sf.initReader(); - delPartition.addStoreFile(sf); - totalDelFileCount++; - } - } - LOG.info("After merging, there are " + totalDelFileCount + " del files"); - // compact the mob files by partitions. - paths = compactMobFiles(request); - LOG.info("After compaction, there are " + paths.size() + " mob files"); - } finally { - for (CompactionDelPartition delPartition : request.getDelPartitions()) { - closeStoreFileReaders(delPartition.getStoreFiles()); - } - } - - // archive the del files if all the mob files are selected. - if (request.type == CompactionType.ALL_FILES && !request.getDelPartitions().isEmpty()) { - LOG.info( - "After a mob compaction with all files selected, archiving the del files "); - for (CompactionDelPartition delPartition : request.getDelPartitions()) { - LOG.info(Objects.toString(delPartition.listDelFiles())); - try { - MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), - delPartition.getStoreFiles()); - } catch (IOException e) { - LOG.error("Failed to archive the del files " + delPartition.getStoreFiles(), e); - } - } - } - return paths; - } - - static class DelPartitionComparator implements Comparator { - private boolean compareStartKey; - - DelPartitionComparator(boolean compareStartKey) { - this.compareStartKey = compareStartKey; - } - - public boolean getCompareStartKey() { - return this.compareStartKey; - } - - public void setCompareStartKey(final boolean compareStartKey) { - this.compareStartKey = compareStartKey; - } - - @Override - public int compare(CompactionDelPartition o1, CompactionDelPartition o2) { - - if (compareStartKey) { - return Bytes.compareTo(o1.getId().getStartKey(), o2.getId().getStartKey()); - } else { - return Bytes.compareTo(o1.getId().getEndKey(), o2.getId().getEndKey()); - } - } - } - - @VisibleForTesting - List getListOfDelFilesForPartition(final CompactionPartition partition, - final List delPartitions) { - // Binary search for startKey and endKey - - List result = new ArrayList<>(); - - DelPartitionComparator comparator = new DelPartitionComparator(false); - CompactionDelPartitionId id = new CompactionDelPartitionId(null, partition.getStartKey()); - CompactionDelPartition target = new CompactionDelPartition(id); - int start = Collections.binarySearch(delPartitions, target, comparator); - - // Get the start index for partition - if (start < 0) { - // Calculate the insert point - start = (start + 1) * (-1); - if (start == delPartitions.size()) { - // no overlap - return result; - } else { - // Check another case which has no overlap - if (Bytes.compareTo(partition.getEndKey(), delPartitions.get(start).getId().getStartKey()) < 0) { - return result; - } - } - } - - // Search for end index for the partition - comparator.setCompareStartKey(true); - id.setStartKey(partition.getEndKey()); - int end = Collections.binarySearch(delPartitions, target, comparator); - - if (end < 0) { - end = (end + 1) * (-1); - if (end == 0) { - return result; - } else { - --end; - if (Bytes.compareTo(partition.getStartKey(), delPartitions.get(end).getId().getEndKey()) > 0) { - return result; - } - } - } - - for (int i = start; i <= end; ++i) { - result.addAll(delPartitions.get(i).getStoreFiles()); - } - - return result; - } - - /** - * Compacts the selected small mob files and all the del files. - * @param request The compaction request. - * @return The paths of new mob files after compactions. - * @throws IOException if IO failure is encountered - */ - protected List compactMobFiles(final PartitionedMobCompactionRequest request) - throws IOException { - Collection partitions = request.compactionPartitions; - if (partitions == null || partitions.isEmpty()) { - LOG.info("No partitions of mob files"); - return Collections.emptyList(); - } - List paths = new ArrayList<>(); - final Connection c = ConnectionFactory.createConnection(conf); - final Table table = c.getTable(tableName); - - try { - Map>> results = new HashMap<>(); - // compact the mob files by partitions in parallel. - for (final CompactionPartition partition : partitions) { - - // How to efficiently come up a list of delFiles for one partition? - // Search the delPartitions and collect all the delFiles for the partition - // One optimization can do is that if there is no del file, we do not need to - // come up with startKey/endKey. - List delFiles = getListOfDelFilesForPartition(partition, - request.getDelPartitions()); - - results.put(partition.getPartitionId(), pool.submit(new Callable>() { - @Override - public List call() throws Exception { - LOG.info("Compacting mob files for partition " + partition.getPartitionId()); - return compactMobFilePartition(request, partition, delFiles, c, table); - } - })); - } - // compact the partitions in parallel. - List failedPartitions = new ArrayList<>(); - for (Entry>> result : results.entrySet()) { - try { - paths.addAll(result.getValue().get()); - } catch (Exception e) { - // just log the error - LOG.error("Failed to compact the partition " + result.getKey(), e); - failedPartitions.add(result.getKey()); - } - } - if (!failedPartitions.isEmpty()) { - // if any partition fails in the compaction, directly throw an exception. - throw new IOException("Failed to compact the partitions " + failedPartitions); - } - } finally { - try { - table.close(); - } catch (IOException e) { - LOG.error("Failed to close the Table", e); - } - } - return paths; - } - - /** - * Compacts a partition of selected small mob files and all the del files. - * @param request The compaction request. - * @param partition A compaction partition. - * @param delFiles The del files. - * @param connection The connection to use. - * @param table The current table. - * @return The paths of new mob files after compactions. - * @throws IOException if IO failure is encountered - */ - private List compactMobFilePartition(PartitionedMobCompactionRequest request, - CompactionPartition partition, - List delFiles, - Connection connection, - Table table) throws IOException { - if (MobUtils.isMobFileExpired(column, EnvironmentEdgeManager.currentTime(), - partition.getPartitionId().getDate())) { - // If the files in the partition are expired, do not compact them and directly - // return an empty list. - return Collections.emptyList(); - } - List newFiles = new ArrayList<>(); - List files = partition.listFiles(); - int offset = 0; - Path bulkloadPathOfPartition = new Path(bulkloadPath, partition.getPartitionId().toString()); - Path bulkloadColumnPath = new Path(bulkloadPathOfPartition, column.getNameAsString()); - while (offset < files.size()) { - int batch = compactionBatchSize; - if (files.size() - offset < compactionBatchSize) { - batch = files.size() - offset; - } - if (batch == 1 && delFiles.isEmpty()) { - // only one file left and no del files, do not compact it, - // and directly add it to the new files. - newFiles.add(files.get(offset).getPath()); - offset++; - continue; - } - // clean the bulkload directory to avoid loading old files. - fs.delete(bulkloadPathOfPartition, true); - // add the selected mob files and del files into filesToCompact - List filesToCompact = new ArrayList<>(); - for (int i = offset; i < batch + offset; i++) { - HStoreFile sf = new HStoreFile(fs, files.get(i).getPath(), conf, compactionCacheConfig, - BloomType.NONE, true); - filesToCompact.add(sf); - } - filesToCompact.addAll(delFiles); - // compact the mob files in a batch. - compactMobFilesInBatch(request, partition, connection, table, filesToCompact, batch, - bulkloadPathOfPartition, bulkloadColumnPath, newFiles); - // move to the next batch. - offset += batch; - } - LOG.info("Compaction is finished. The number of mob files is changed from " + files.size() - + " to " + newFiles.size()); - return newFiles; - } - - /** - * Closes the readers of store files. - * @param storeFiles The store files to be closed. - */ - private void closeStoreFileReaders(List storeFiles) { - for (HStoreFile storeFile : storeFiles) { - try { - storeFile.closeStoreFile(true); - } catch (IOException e) { - LOG.warn("Failed to close the reader on store file " + storeFile.getPath(), e); - } - } - } - - /** - * Compacts a partition of selected small mob files and all the del files in a batch. - * @param request The compaction request. - * @param partition A compaction partition. - * @param connection To use for transport - * @param table The current table. - * @param filesToCompact The files to be compacted. - * @param batch The number of mob files to be compacted in a batch. - * @param bulkloadPathOfPartition The directory where the bulkload column of the current - * partition is saved. - * @param bulkloadColumnPath The directory where the bulkload files of current partition - * are saved. - * @param newFiles The paths of new mob files after compactions. - * @throws IOException if IO failure is encountered - */ - private void compactMobFilesInBatch(PartitionedMobCompactionRequest request, - CompactionPartition partition, - Connection connection, Table table, - List filesToCompact, int batch, - Path bulkloadPathOfPartition, Path bulkloadColumnPath, - List newFiles) - throws IOException { - // open scanner to the selected mob files and del files. - StoreScanner scanner = createScanner(filesToCompact, ScanType.COMPACT_DROP_DELETES); - // the mob files to be compacted, not include the del files. - List mobFilesToCompact = filesToCompact.subList(0, batch); - // Pair(maxSeqId, cellsCount) - Pair fileInfo = getFileInfo(mobFilesToCompact); - // open writers for the mob files and new ref store files. - StoreFileWriter writer = null; - StoreFileWriter refFileWriter = null; - Path filePath = null; - long mobCells = 0; - boolean cleanupTmpMobFile = false; - boolean cleanupBulkloadDirOfPartition = false; - boolean cleanupCommittedMobFile = false; - boolean closeReaders= true; - - try { - try { - writer = MobUtils - .createWriter(conf, fs, column, partition.getPartitionId().getLatestDate(), tempPath, - Long.MAX_VALUE, column.getCompactionCompressionType(), - partition.getPartitionId().getStartKey(), compactionCacheConfig, cryptoContext, - true); - cleanupTmpMobFile = true; - filePath = writer.getPath(); - byte[] fileName = Bytes.toBytes(filePath.getName()); - // create a temp file and open a writer for it in the bulkloadPath - refFileWriter = MobUtils.createRefFileWriter(conf, fs, column, bulkloadColumnPath, - fileInfo.getSecond().longValue(), compactionCacheConfig, cryptoContext, true); - cleanupBulkloadDirOfPartition = true; - List cells = new ArrayList<>(); - boolean hasMore; - ScannerContext scannerContext = - ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build(); - do { - hasMore = scanner.next(cells, scannerContext); - for (Cell cell : cells) { - // write the mob cell to the mob file. - writer.append(cell); - // write the new reference cell to the store file. - Cell reference = MobUtils.createMobRefCell(cell, fileName, this.refCellTags); - refFileWriter.append(reference); - mobCells++; - } - cells.clear(); - } while (hasMore); - } finally { - // close the scanner. - scanner.close(); - - if (cleanupTmpMobFile) { - // append metadata to the mob file, and close the mob file writer. - closeMobFileWriter(writer, fileInfo.getFirst(), mobCells); - } - - if (cleanupBulkloadDirOfPartition) { - // append metadata and bulkload info to the ref mob file, and close the writer. - closeRefFileWriter(refFileWriter, fileInfo.getFirst(), request.selectionTime); - } - } - - if (mobCells > 0) { - // commit mob file - MobUtils.commitFile(conf, fs, filePath, mobFamilyDir, compactionCacheConfig); - cleanupTmpMobFile = false; - cleanupCommittedMobFile = true; - // bulkload the ref file - bulkloadRefFile(table.getName(), bulkloadPathOfPartition, filePath.getName()); - cleanupCommittedMobFile = false; - newFiles.add(new Path(mobFamilyDir, filePath.getName())); - } - - // archive the old mob files, do not archive the del files. - try { - closeStoreFileReaders(mobFilesToCompact); - closeReaders = false; - MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), mobFilesToCompact); - } catch (IOException e) { - LOG.error("Failed to archive the files " + mobFilesToCompact, e); - } - } finally { - if (closeReaders) { - closeStoreFileReaders(mobFilesToCompact); - } - - if (cleanupTmpMobFile) { - deletePath(filePath); - } - - if (cleanupBulkloadDirOfPartition) { - // delete the bulkload files in bulkloadPath - deletePath(bulkloadPathOfPartition); - } - - if (cleanupCommittedMobFile) { - deletePath(new Path(mobFamilyDir, filePath.getName())); - } - } - } - - /** - * Compacts the del files in batches which avoids opening too many files. - * @param request The compaction request. - * @param delFilePaths Del file paths to compact - * @return The paths of new del files after merging or the original files if no merging - * is necessary. - * @throws IOException if IO failure is encountered - */ - protected List compactDelFiles(PartitionedMobCompactionRequest request, - List delFilePaths) throws IOException { - if (delFilePaths.size() <= delFileMaxCount) { - return delFilePaths; - } - // when there are more del files than the number that is allowed, merge it firstly. - int offset = 0; - List paths = new ArrayList<>(); - while (offset < delFilePaths.size()) { - // get the batch - int batch = compactionBatchSize; - if (delFilePaths.size() - offset < compactionBatchSize) { - batch = delFilePaths.size() - offset; - } - List batchedDelFiles = new ArrayList<>(); - if (batch == 1) { - // only one file left, do not compact it, directly add it to the new files. - paths.add(delFilePaths.get(offset)); - offset++; - continue; - } - for (int i = offset; i < batch + offset; i++) { - batchedDelFiles.add(new HStoreFile(fs, delFilePaths.get(i), conf, compactionCacheConfig, - BloomType.NONE, true)); - } - // compact the del files in a batch. - paths.add(compactDelFilesInBatch(request, batchedDelFiles)); - // move to the next batch. - offset += batch; - } - return compactDelFiles(request, paths); - } - - /** - * Compacts the del file in a batch. - * @param request The compaction request. - * @param delFiles The del files. - * @return The path of new del file after merging. - * @throws IOException if IO failure is encountered - */ - private Path compactDelFilesInBatch(PartitionedMobCompactionRequest request, - List delFiles) throws IOException { - // create a scanner for the del files. - StoreScanner scanner = createScanner(delFiles, ScanType.COMPACT_RETAIN_DELETES); - StoreFileWriter writer = null; - Path filePath = null; - try { - writer = MobUtils.createDelFileWriter(conf, fs, column, - MobUtils.formatDate(new Date(request.selectionTime)), tempPath, Long.MAX_VALUE, - column.getCompactionCompressionType(), HConstants.EMPTY_START_ROW, compactionCacheConfig, - cryptoContext); - filePath = writer.getPath(); - List cells = new ArrayList<>(); - boolean hasMore; - ScannerContext scannerContext = - ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build(); - do { - hasMore = scanner.next(cells, scannerContext); - for (Cell cell : cells) { - writer.append(cell); - } - cells.clear(); - } while (hasMore); - } finally { - scanner.close(); - if (writer != null) { - try { - writer.close(); - } catch (IOException e) { - LOG.error("Failed to close the writer of the file " + filePath, e); - } - } - } - // commit the new del file - Path path = MobUtils.commitFile(conf, fs, filePath, mobFamilyDir, compactionCacheConfig); - // archive the old del files - try { - MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), delFiles); - } catch (IOException e) { - LOG.error("Failed to archive the old del files " + delFiles, e); - } - return path; - } - - /** - * Creates a store scanner. - * @param filesToCompact The files to be compacted. - * @param scanType The scan type. - * @return The store scanner. - * @throws IOException if IO failure is encountered - */ - private StoreScanner createScanner(List filesToCompact, ScanType scanType) - throws IOException { - List scanners = StoreFileScanner.getScannersForStoreFiles(filesToCompact, - false, true, false, false, HConstants.LATEST_TIMESTAMP); - long ttl = HStore.determineTTLFromFamily(column); - ScanInfo scanInfo = new ScanInfo(conf, column, ttl, 0, CellComparator.getInstance()); - return new StoreScanner(scanInfo, scanType, scanners); - } - - /** - * Bulkloads the current file. - * @param tableName The table to load into. - * @param bulkloadDirectory The path of bulkload directory. - * @param fileName The current file name. - * @throws IOException if IO failure is encountered - */ - private void bulkloadRefFile(TableName tableName, Path bulkloadDirectory, String fileName) - throws IOException { - // bulkload the ref file - try { - BulkLoadHFiles.create(conf).bulkLoad(tableName, bulkloadDirectory); - } catch (Exception e) { - throw new IOException(e); - } - } - - /** - * Closes the mob file writer. - * @param writer The mob file writer. - * @param maxSeqId Maximum sequence id. - * @param mobCellsCount The number of mob cells. - * @throws IOException if IO failure is encountered - */ - private void closeMobFileWriter(StoreFileWriter writer, long maxSeqId, long mobCellsCount) - throws IOException { - if (writer != null) { - writer.appendMetadata(maxSeqId, false, mobCellsCount); - try { - writer.close(); - } catch (IOException e) { - LOG.error("Failed to close the writer of the file " + writer.getPath(), e); - } - } - } - - /** - * Closes the ref file writer. - * @param writer The ref file writer. - * @param maxSeqId Maximum sequence id. - * @param bulkloadTime The timestamp at which the bulk load file is created. - * @throws IOException if IO failure is encountered - */ - private void closeRefFileWriter(StoreFileWriter writer, long maxSeqId, long bulkloadTime) - throws IOException { - if (writer != null) { - writer.appendMetadata(maxSeqId, false); - writer.appendFileInfo(BULKLOAD_TIME_KEY, Bytes.toBytes(bulkloadTime)); - writer.appendFileInfo(SKIP_RESET_SEQ_ID, Bytes.toBytes(true)); - try { - writer.close(); - } catch (IOException e) { - LOG.error("Failed to close the writer of the ref file " + writer.getPath(), e); - } - } - } - - /** - * Gets the max seqId and number of cells of the store files. - * @param storeFiles The store files. - * @return The pair of the max seqId and number of cells of the store files. - * @throws IOException if IO failure is encountered - */ - private Pair getFileInfo(List storeFiles) throws IOException { - long maxSeqId = 0; - long maxKeyCount = 0; - for (HStoreFile sf : storeFiles) { - // the readers will be closed later after the merge. - maxSeqId = Math.max(maxSeqId, sf.getMaxSequenceId()); - sf.initReader(); - byte[] count = sf.getReader().loadFileInfo().get(MOB_CELLS_COUNT); - if (count != null) { - maxKeyCount += Bytes.toLong(count); - } - } - return new Pair<>(maxSeqId, maxKeyCount); - } - - /** - * Deletes a file. - * @param path The path of the file to be deleted. - */ - private void deletePath(Path path) { - try { - if (path != null) { - fs.delete(path, true); - } - } catch (IOException e) { - LOG.error("Failed to delete the file " + path, e); - } - } - - private FileStatus getLinkedFileStatus(HFileLink link) throws IOException { - Path[] locations = link.getLocations(); - FileStatus file; - for (Path location : locations) { - - if (location != null) { - try { - file = fs.getFileStatus(location); - if (file != null) { - return file; - } - } catch (FileNotFoundException e) { - } - } - } - LOG.warn("The file " + link + " links to can not be found"); - return null; - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java index b8ea9609e2bc..ee5aa6cb2580 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java @@ -193,28 +193,6 @@ public StoreFileWriter createWriterInTmp(Date date, long maxKeyCount, isCompaction); } - /** - * Creates the writer for the del file in temp directory. - * The del file keeps tracking the delete markers. Its name has a suffix _del, - * the format is [0-9a-f]+(_del)?. - * @param date The latest date of written cells. - * @param maxKeyCount The key count. - * @param compression The compression algorithm. - * @param startKey The start key. - * @return The writer for the del file. - * @throws IOException - */ - public StoreFileWriter createDelFileWriterInTmp(Date date, long maxKeyCount, - Compression.Algorithm compression, byte[] startKey) throws IOException { - if (startKey == null) { - startKey = HConstants.EMPTY_START_ROW; - } - Path path = getTempDir(); - String suffix = UUID - .randomUUID().toString().replaceAll("-", "") + "_del"; - MobFileName mobFileName = MobFileName.create(startKey, MobUtils.formatDate(date), suffix); - return createWriterInTmp(mobFileName, path, maxKeyCount, compression, true); - } /** * Creates the writer for the mob file in temp directory. @@ -231,7 +209,7 @@ public StoreFileWriter createWriterInTmp(String date, Path basePath, long maxKey Compression.Algorithm compression, byte[] startKey, boolean isCompaction) throws IOException { MobFileName mobFileName = MobFileName.create(startKey, date, UUID.randomUUID() - .toString().replaceAll("-", "")); + .toString().replaceAll("-", ""), region.getRegionInfo().getEncodedName()); return createWriterInTmp(mobFileName, basePath, maxKeyCount, compression, isCompaction); } @@ -265,7 +243,7 @@ public void commitFile(final Path sourceFile, Path targetPath) throws IOExceptio } Path dstPath = new Path(targetPath, sourceFile.getName()); validateMobFile(sourceFile); - String msg = "Renaming flushed file from " + sourceFile + " to " + dstPath; + String msg = " FLUSH Renaming flushed file from " + sourceFile + " to " + dstPath; LOG.info(msg); Path parent = dstPath.getParent(); if (!region.getFilesystem().exists(parent)) { @@ -309,6 +287,19 @@ public MobCell resolve(Cell reference, boolean cacheBlocks) throws IOException { return resolve(reference, cacheBlocks, -1, true); } + /** + * Reads the cell from the mob file with readEmptyValueOnMobCellMiss + * @param reference + * @param cacheBlocks + * @param readEmptyValueOnMobCellMiss + * @return The cell found in the mob file. + * @throws IOException + */ + public MobCell resolve(Cell reference, boolean cacheBlocks, boolean readEmptyValueOnMobCellMiss) + throws IOException { + return resolve(reference, cacheBlocks, -1, readEmptyValueOnMobCellMiss); + } + /** * Reads the cell from the mob file. * @param reference The cell found in the HBase, its value is a path to a mob file. @@ -520,4 +511,6 @@ public long getMobScanCellsSize() { public byte[] getRefCellTags() { return this.refCellTags; } + + } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index 447bc736789a..32d0f0bde277 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -1905,7 +1905,7 @@ public void cancelRequestedCompaction(CompactionContext compaction) { finishCompactionRequest(compaction.getRequest()); } - private void finishCompactionRequest(CompactionRequestImpl cr) { + protected void finishCompactionRequest(CompactionRequestImpl cr) { this.region.reportCompactionRequestEnd(cr.isMajor(), cr.getFiles().size(), cr.getSize()); if (cr.isOffPeak()) { offPeakCompactionTracker.set(false); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java index 682185a89a10..37c513ef77e2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java @@ -108,6 +108,9 @@ public class HStoreFile implements StoreFile { /** Key for the number of mob cells in metadata */ public static final byte[] MOB_CELLS_COUNT = Bytes.toBytes("MOB_CELLS_COUNT"); + /** Key for the list of MOB file references */ + public static final byte[] MOB_FILE_REFS = Bytes.toBytes("MOB_FILE_REFS"); + /** Meta key set when store file is a result of a bulk load */ public static final byte[] BULKLOAD_TASK_KEY = Bytes.toBytes("BULKLOAD_SOURCE_TASK"); public static final byte[] BULKLOAD_TIME_KEY = Bytes.toBytes("BULKLOAD_TIMESTAMP"); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java index 76144f08d5dc..61cd34c6d235 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java @@ -114,4 +114,5 @@ public void close() { LOG.warn("Failed to free referenced mob cells: ", e); } } + } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java index 54e278031959..ee0ed072f304 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.io.HalfStoreFileReader; import org.apache.hadoop.hbase.io.Reference; import org.apache.hadoop.hbase.io.hfile.CacheConfig; +import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.util.FSUtils; /** @@ -134,7 +135,7 @@ public StoreFileInfo(final Configuration conf, final FileSystem fs, final Path i } if (LOG.isTraceEnabled()) LOG.trace(p + " is a " + reference.getFileRegion() + " reference to " + referencePath); - } else if (isHFile(p)) { + } else if (isHFile(p) || isMobFile(p)) { // HFile this.createdTimestamp = fs.getFileStatus(initialPath).getModificationTime(); this.reference = null; @@ -405,6 +406,16 @@ public static boolean isHFile(final String fileName) { return m.matches() && m.groupCount() > 0; } + public static boolean isMobFile(final Path path) { + String fileName = path.getName(); + String[] parts = fileName.split(MobUtils.SEP); + if (parts.length != 2) { + return false; + } + Matcher m = HFILE_NAME_PATTERN.matcher(parts[0]); + Matcher mm = HFILE_NAME_PATTERN.matcher(parts[1]); + return m.matches() && mm.matches(); + } /** * @param path Path to check. * @return True if the path has format of a del file. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java index 7b4fb4a0c06f..522efb5ea18d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java @@ -25,6 +25,7 @@ import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY; import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAX_SEQ_ID_KEY; import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_CELLS_COUNT; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_FILE_REFS; import static org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY; import java.io.IOException; @@ -242,6 +243,28 @@ public void appendMetadata(final long maxSequenceId, final boolean majorCompacti appendTrackedTimestampsToMetadata(); } + /** + * Appends MOB - specific metadata + * @param mobRefSet - set of MOB file names + * @throws IOException problem writing to FS + */ + public void appendMobMetadata(Set mobRefSet) throws IOException { + if (mobRefSet.isEmpty()) { + return; + } + StringBuilder sb = new StringBuilder(2 * mobRefSet.size() - 1); + String[] arr = new String[mobRefSet.size()]; + arr = mobRefSet.toArray(arr); + for (int i = 0; i < arr.length; i++) { + sb.append(arr[i]); + if (i < arr.length - 1) { + sb.append(","); + } + } + byte[] bytes = sb.toString().getBytes(); + writer.appendFileInfo(MOB_FILE_REFS, bytes); + } + /** * Add TimestampRange and earliest put timestamp to Metadata */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java index 503325a1a374..3eab002dba5a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java @@ -115,7 +115,7 @@ public CompactionProgress getProgress() { } /** The sole reason this class exists is that java has no ref/out/pointer parameters. */ - protected static class FileDetails { + public static class FileDetails { /** Maximum key count after compaction (for blooms) */ public long maxKeyCount = 0; /** Earliest put timestamp if major compaction */ diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java new file mode 100644 index 000000000000..6527b11b306e --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java @@ -0,0 +1,355 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.mob; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InterruptedIOException; +import java.util.ArrayList; +import java.util.Date; +import java.util.List; +import java.util.Random; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.PrivateCellUtil; +import org.apache.hadoop.hbase.io.hfile.CorruptHFileException; +import org.apache.hadoop.hbase.regionserver.CellSink; +import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.regionserver.KeyValueScanner; +import org.apache.hadoop.hbase.regionserver.ScannerContext; +import org.apache.hadoop.hbase.regionserver.ShipperListener; +import org.apache.hadoop.hbase.regionserver.StoreFileWriter; +import org.apache.hadoop.hbase.regionserver.throttle.ThroughputControlUtil; +import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public class FaultyMobStoreCompactor extends DefaultMobStoreCompactor +{ + + public static AtomicLong mobCounter = new AtomicLong(); + public static AtomicLong totalFailures = new AtomicLong(); + public static AtomicLong totalCompactions = new AtomicLong(); + public static AtomicLong totalMajorCompactions = new AtomicLong(); + + static double failureProb = 0.1d; + static Random rnd = new Random(); + + + public FaultyMobStoreCompactor(Configuration conf, HStore store) { + super(conf, store); + failureProb = conf.getDouble("injected.fault.probability", 0.1); + } + + @Override + protected boolean performCompaction(FileDetails fd, InternalScanner scanner, CellSink writer, + long smallestReadPoint, boolean cleanSeqId, ThroughputController throughputController, + boolean major, int numofFilesToCompact) throws IOException { + + totalCompactions.incrementAndGet(); + if (major) { + totalMajorCompactions.incrementAndGet(); + } + long bytesWrittenProgressForCloseCheck = 0; + long bytesWrittenProgressForLog = 0; + long bytesWrittenProgressForShippedCall = 0; + // Clear old mob references + mobRefSet.get().clear(); + boolean isUserRequest = userRequest.get(); + boolean compactMOBs = major && isUserRequest; + boolean discardMobMiss = conf.getBoolean(MobConstants.MOB_DISCARD_MISS_KEY, + MobConstants.DEFAULT_MOB_DISCARD_MISS); + + boolean mustFail = false; + if (compactMOBs) { + mobCounter.incrementAndGet(); + double dv = rnd.nextDouble(); + if (dv < failureProb) { + mustFail = true; + totalFailures.incrementAndGet(); + } + } + + FileSystem fs = FileSystem.get(conf); + + // Since scanner.next() can return 'false' but still be delivering data, + // we have to use a do/while loop. + List cells = new ArrayList<>(); + // Limit to "hbase.hstore.compaction.kv.max" (default 10) to avoid OOME + int closeCheckSizeLimit = HStore.getCloseCheckInterval(); + long lastMillis = 0; + if (LOG.isDebugEnabled()) { + lastMillis = EnvironmentEdgeManager.currentTime(); + } + String compactionName = ThroughputControlUtil.getNameForThrottling(store, "compaction"); + long now = 0; + boolean hasMore; + Path path = MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName()); + byte[] fileName = null; + StoreFileWriter mobFileWriter = null; + long mobCells = 0; + long cellsCountCompactedToMob = 0, cellsCountCompactedFromMob = 0; + long cellsSizeCompactedToMob = 0, cellsSizeCompactedFromMob = 0; + boolean finished = false; + + ScannerContext scannerContext = + ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build(); + throughputController.start(compactionName); + KeyValueScanner kvs = (scanner instanceof KeyValueScanner)? (KeyValueScanner)scanner : null; + long shippedCallSizeLimit = + (long) numofFilesToCompact * this.store.getColumnFamilyDescriptor().getBlocksize(); + + MobCell mobCell = null; + + + long counter = 0; + long countFailAt = -1; + if (mustFail) { + countFailAt = rnd.nextInt(100); // randomly fail fast + } + + try { + try { + // If the mob file writer could not be created, directly write the cell to the store file. + mobFileWriter = mobStore.createWriterInTmp(new Date(fd.latestPutTs), fd.maxKeyCount, + compactionCompression, store.getRegionInfo().getStartKey(), true); + fileName = Bytes.toBytes(mobFileWriter.getPath().getName()); + } catch (IOException e) { + // Bailing out + LOG.error("Failed to create mob writer, ", e); + throw e; + } + if (compactMOBs) { + // Add the only reference we get for compact MOB case + // because new store file will have only one MOB reference + // in this case - of newly compacted MOB file + mobRefSet.get().add(mobFileWriter.getPath().getName()); + } + do { + hasMore = scanner.next(cells, scannerContext); + if (LOG.isDebugEnabled()) { + now = EnvironmentEdgeManager.currentTime(); + } + for (Cell c : cells) { + counter++; + if (compactMOBs) { + if (MobUtils.isMobReferenceCell(c)) { + if (counter == countFailAt) { + LOG.warn("\n\n INJECTED FAULT mobCounter="+mobCounter.get()+"\n\n"); + throw new CorruptHFileException("injected fault"); + } + String fName = MobUtils.getMobFileName(c); + Path pp = new Path(new Path(fs.getUri()), new Path(path, fName)); + + // Added to support migration + try { + mobCell = mobStore.resolve(c, true, false); + } catch (FileNotFoundException fnfe) { + if (discardMobMiss) { + LOG.error("Missing MOB cell: file=" + pp + " not found"); + continue; + } else { + throw fnfe; + } + } + + if (discardMobMiss && mobCell.getCell().getValueLength() == 0) { + LOG.error("Missing MOB cell value: file=" + pp +" cell=" + mobCell); + continue; + } + + if (mobCell.getCell().getValueLength() > mobSizeThreshold) { + // put the mob data back to the store file + PrivateCellUtil.setSequenceId(mobCell.getCell(), c.getSequenceId()); + mobFileWriter.append(mobCell.getCell()); + writer.append(MobUtils.createMobRefCell(mobCell.getCell(), fileName, + this.mobStore.getRefCellTags())); + cellsCountCompactedFromMob++; + cellsSizeCompactedFromMob += mobCell.getCell().getValueLength(); + mobCells++; + } else { + // If MOB value is less than threshold, append it directly to a store file + PrivateCellUtil.setSequenceId(mobCell.getCell(), c.getSequenceId()); + writer.append(mobCell.getCell()); + } + + } else { + // Not a MOB reference cell + int size = c.getValueLength(); + if (size > mobSizeThreshold) { + mobFileWriter.append(c); + writer.append(MobUtils.createMobRefCell(c, fileName, this.mobStore.getRefCellTags())); + mobCells++; + } else { + writer.append(c); + } + } + } else if (c.getTypeByte() != KeyValue.Type.Put.getCode()) { + // Not a major compaction or major with MOB disabled + // If the kv type is not put, directly write the cell + // to the store file. + writer.append(c); + } else if (MobUtils.isMobReferenceCell(c)) { + // Not a major MOB compaction, Put MOB reference + if (MobUtils.hasValidMobRefCellValue(c)) { + int size = MobUtils.getMobValueLength(c); + if (size > mobSizeThreshold) { + // If the value size is larger than the threshold, it's regarded as a mob. Since + // its value is already in the mob file, directly write this cell to the store file + writer.append(c); + // Add MOB reference to a set + mobRefSet.get().add(MobUtils.getMobFileName(c)); + } else { + // If the value is not larger than the threshold, it's not regarded a mob. Retrieve + // the mob cell from the mob file, and write it back to the store file. + mobCell = mobStore.resolve(c, true, false); + if (mobCell.getCell().getValueLength() != 0) { + // put the mob data back to the store file + PrivateCellUtil.setSequenceId(mobCell.getCell(), c.getSequenceId()); + writer.append(mobCell.getCell()); + cellsCountCompactedFromMob++; + cellsSizeCompactedFromMob += mobCell.getCell().getValueLength(); + } else { + // If the value of a file is empty, there might be issues when retrieving, + // directly write the cell to the store file, and leave it to be handled by the + // next compaction. + LOG.error("Empty value for: " + c); + writer.append(c); + // Add MOB reference to a set + mobRefSet.get().add(MobUtils.getMobFileName(c)); + } + } + } else { + // TODO ???? + LOG.error("Corrupted MOB reference: " + c); + writer.append(c); + } + } else if (c.getValueLength() <= mobSizeThreshold) { + // If the value size of a cell is not larger than the threshold, directly write it to + // the store file. + writer.append(c); + } else { + // If the value size of a cell is larger than the threshold, it's regarded as a mob, + // write this cell to a mob file, and write the path to the store file. + mobCells++; + // append the original keyValue in the mob file. + mobFileWriter.append(c); + Cell reference = MobUtils.createMobRefCell(c, fileName,this.mobStore.getRefCellTags()); + // write the cell whose value is the path of a mob file to the store file. + writer.append(reference); + cellsCountCompactedToMob++; + cellsSizeCompactedToMob += c.getValueLength(); + // Add ref we get for compact MOB case + mobRefSet.get().add(mobFileWriter.getPath().getName()); + } + + int len = c.getSerializedSize(); + ++progress.currentCompactedKVs; + progress.totalCompactedSize += len; + bytesWrittenProgressForShippedCall += len; + if (LOG.isDebugEnabled()) { + bytesWrittenProgressForLog += len; + } + throughputController.control(compactionName, len); + // check periodically to see if a system stop is requested + if (closeCheckSizeLimit > 0) { + bytesWrittenProgressForCloseCheck += len; + if (bytesWrittenProgressForCloseCheck > closeCheckSizeLimit) { + bytesWrittenProgressForCloseCheck = 0; + if (!store.areWritesEnabled()) { + progress.cancel(); + return false; + } + } + } + if (kvs != null && bytesWrittenProgressForShippedCall > shippedCallSizeLimit) { + ((ShipperListener)writer).beforeShipped(); + kvs.shipped(); + bytesWrittenProgressForShippedCall = 0; + } + } + // Log the progress of long running compactions every minute if + // logging at DEBUG level + if (LOG.isDebugEnabled()) { + if ((now - lastMillis) >= COMPACTION_PROGRESS_LOG_INTERVAL) { + String rate = String.format("%.2f", + (bytesWrittenProgressForLog / 1024.0) / ((now - lastMillis) / 1000.0)); + LOG.debug("Compaction progress: {} {}, rate={} KB/sec, throughputController is {}", + compactionName, progress, rate, throughputController); + lastMillis = now; + bytesWrittenProgressForLog = 0; + } + } + cells.clear(); + } while (hasMore); + finished = true; + } catch (InterruptedException e) { + progress.cancel(); + throw new InterruptedIOException( + "Interrupted while control throughput of compacting " + compactionName); + } catch (FileNotFoundException e) { + LOG.error("MOB Stress Test FAILED, region: "+store.getRegionInfo().getEncodedName(), e); + System.exit(-1); + } catch (IOException t) { + LOG.error("Mob compaction failed for region: "+ store.getRegionInfo().getEncodedName()); + throw t; + } finally { + // Clone last cell in the final because writer will append last cell when committing. If + // don't clone here and once the scanner get closed, then the memory of last cell will be + // released. (HBASE-22582) + ((ShipperListener) writer).beforeShipped(); + throughputController.finish(compactionName); + if (!finished && mobFileWriter != null) { + // Remove all MOB references because compaction failed + mobRefSet.get().clear(); + // Abort writer + abortWriter(mobFileWriter); + } + } + + if (mobFileWriter != null) { + if (mobCells > 0) { + // If the mob file is not empty, commit it. + mobFileWriter.appendMetadata(fd.maxSeqId, major, mobCells); + mobFileWriter.close(); + mobStore.commitFile(mobFileWriter.getPath(), path); + } else { + // If the mob file is empty, delete it instead of committing. + abortWriter(mobFileWriter); + } + } + mobStore.updateCellsCountCompactedFromMob(cellsCountCompactedFromMob); + mobStore.updateCellsCountCompactedToMob(cellsCountCompactedToMob); + mobStore.updateCellsSizeCompactedFromMob(cellsSizeCompactedFromMob); + mobStore.updateCellsSizeCompactedToMob(cellsSizeCompactedToMob); + progress.complete(); + return true; + + } + + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java new file mode 100644 index 000000000000..7222f6611e7b --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java @@ -0,0 +1,80 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.mob; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.util.AbstractHBaseTool; +import org.apache.hadoop.util.ToolRunner; +import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.yetus.audience.InterfaceStability; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class MobStressTool extends AbstractHBaseTool +{ + private static final Logger LOG = LoggerFactory.getLogger(MobStressTool.class); + private CommandLine cmd; + + public MobStressTool() throws IOException { + init(); + } + + protected void init() throws IOException { + // define supported options + addOptWithArg("n", "Number of MOB key-values to insert, default - 10000000"); + } + + @Override + protected void addOptions() { + } + + @Override + protected void processOptions(CommandLine cmd) { + this.cmd = cmd; + } + + @Override + protected int doWork() throws Exception { + long numRowsToInsert = 10000000; + if (cmd.hasOption("n")) { + numRowsToInsert = Long.parseLong(cmd.getOptionValue("n")); + if (numRowsToInsert < 0) { + LOG.warn("Ignore wrong option '-n'"); + numRowsToInsert = 10000000; + } + } + + TestMobCompaction test = new TestMobCompaction(); + test.init(getConf(), numRowsToInsert); + test.testMobCompaction(); + return 0; + } + + public static void main(String[] args) throws Exception { + Configuration conf = HBaseConfiguration.create(); + int ret = ToolRunner.run(conf, new MobStressTool(), args); + System.exit(ret); + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java index bffb062b4b0e..075117e597e5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java @@ -153,7 +153,7 @@ public void testCleaner() throws Exception { String[] args = new String[2]; args[0] = tableName.getNameAsString(); args[1] = family; - ToolRunner.run(TEST_UTIL.getConfiguration(), new ExpiredMobFileCleaner(), args); + ToolRunner.run(TEST_UTIL.getConfiguration(), new MobFileCleanerTool(), args); FileStatus[] filesAfterClean = TEST_UTIL.getTestFileSystem().listStatus(mobDirPath); String lastFile = filesAfterClean[0].getPath().getName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java new file mode 100644 index 000000000000..6c20bf416180 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java @@ -0,0 +1,344 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.mob; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.Arrays; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.KeepDeletedCells; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.BufferedMutator; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner; +import org.apache.hadoop.hbase.testclassification.IntegrationTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +/** + Reproduction for MOB data loss + + 1. Settings: Region Size 200 MB, Flush threshold 800 KB. + 2. Insert 10 Million records + 3. MOB Compaction and Archiver + a) Trigger MOB Compaction (every 2 minutes) + b) Trigger major compaction (every 2 minutes) + c) Trigger archive cleaner (every 3 minutes) + 4. Validate MOB data after complete data load. + + */ +@Category(IntegrationTests.class) +public class TestMobCompaction { + private static final Logger LOG = LoggerFactory.getLogger(TestMobCompaction.class); + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobCompaction.class); + @Rule + public TestName testName = new TestName(); + + private HBaseTestingUtility HTU; + + private final static String famStr = "f1"; + private final static byte[] fam = Bytes.toBytes(famStr); + private final static byte[] qualifier = Bytes.toBytes("q1"); + private final static long mobLen = 10; + private final static byte[] mobVal = Bytes + .toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789"); + + private Configuration conf; + private HTableDescriptor hdt; + private HColumnDescriptor hcd; + private Admin admin; + private long count = 1000000; + private double failureProb = 0.1; + private Table table = null; + + private static volatile boolean run = true; + + public TestMobCompaction() { + + } + + public void init(Configuration conf, long numRows) throws IOException { + this.conf = conf; + this.count = numRows; + printConf(); + hdt = createTableDescriptor("testMobCompactTable"); + Connection conn = ConnectionFactory.createConnection(this.conf); + this.admin = conn.getAdmin(); + this.hcd = new HColumnDescriptor(fam); + this.hcd.setMobEnabled(true); + this.hcd.setMobThreshold(mobLen); + this.hcd.setMaxVersions(1); + this.hdt.addFamily(hcd); + if (admin.tableExists(hdt.getTableName())) { + admin.disableTable(hdt.getTableName()); + admin.deleteTable(hdt.getTableName()); + } + admin.createTable(hdt); + table = conn.getTable(hdt.getTableName()); + } + + private void printConf() { + LOG.info("To run stress test, please change HBase configuration as following:"); + LOG.info("hfile.format.version=3"); + LOG.info("hbase.master.hfilecleaner.ttl=0"); + LOG.info("hbase.hregion.max.filesize=200000000"); + LOG.info("hbase.client.retries.number=100"); + LOG.info("hbase.hregion.memstore.flush.size=800000"); + LOG.info("hbase.hstore.blockingStoreFiles=150"); + LOG.info("hbase.hstore.compaction.throughput.lower.bound=50000000"); + LOG.info("hbase.hstore.compaction.throughput.higher.bound=100000000"); + LOG.info("hbase.master.mob.cleaner.period=0"); + LOG.info("hbase.mob.default.compactor=org.apache.hadoop.hbase.mob.FaultyMobStoreCompactor"); + LOG.warn("injected.fault.probability=x, where x is between 0. and 1."); + + + } + + + private HTableDescriptor createTableDescriptor(final String name, + final int minVersions, final int versions, final int ttl, KeepDeletedCells keepDeleted) { + HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name)); + return htd; + } + + + private HTableDescriptor createTableDescriptor(final String name) { + return createTableDescriptor(name, HColumnDescriptor.DEFAULT_MIN_VERSIONS, + 1, HConstants.FOREVER, HColumnDescriptor.DEFAULT_KEEP_DELETED); + } + + @Before + public void setUp() throws Exception { + HTU = new HBaseTestingUtility(); + hdt = HTU.createTableDescriptor("testMobCompactTable"); + conf = HTU.getConfiguration(); + + initConf(); + + // HTU.getConfiguration().setInt("hbase.mob.compaction.chore.period", 0); + HTU.startMiniCluster(); + admin = HTU.getAdmin(); + + hcd = new HColumnDescriptor(fam); + hcd.setMobEnabled(true); + hcd.setMobThreshold(mobLen); + hcd.setMaxVersions(1); + hdt.addFamily(hcd); + table = HTU.createTable(hdt, null); + } + + private void initConf() { + + conf.setInt("hfile.format.version", 3); + conf.setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0); + conf.setInt("hbase.client.retries.number", 100); + conf.setInt("hbase.hregion.max.filesize", 200000000); + conf.setInt("hbase.hregion.memstore.flush.size", 800000); + conf.setInt("hbase.hstore.blockingStoreFiles", 150); + conf.setInt("hbase.hstore.compaction.throughput.lower.bound", 52428800); + conf.setInt("hbase.hstore.compaction.throughput.higher.bound", 2*52428800); + conf.setDouble("injected.fault.probability", failureProb); + conf.set(MobStoreEngine.DEFAULT_MOB_COMPACTOR_CLASS_KEY, + FaultyMobStoreCompactor.class.getName()); + + } + + @After + public void tearDown() throws Exception { + HTU.shutdownMiniCluster(); + } + + + class MajorCompaction implements Runnable { + + @Override + public void run() { + while (run) { + try { + admin.majorCompact(hdt.getTableName(), fam); + Thread.sleep(120000); + } catch (Exception e) { + LOG.error("MOB Stress Test FAILED", e); + System.exit(-1); + } + } + } + } + + class CleanMobAndArchive implements Runnable { + + @Override + public void run() { + while (run) { + try { + LOG.info("MOB cleanup chore started ..."); + MobUtils.cleanupObsoleteMobFiles(conf, table.getName()); + LOG.info("MOB cleanup chore finished"); + + Thread.sleep(130000); + } catch (Exception e) { + e.printStackTrace(); + } + } + } + } + + class WriteData implements Runnable { + + private long rows = -1; + + public WriteData(long rows) { + this.rows = rows; + } + + @Override + public void run() { + try { + + BufferedMutator bm = admin.getConnection().getBufferedMutator(table.getName()); + // Put Operation + for (int i = 0; i < rows; i++) { + Put p = new Put(Bytes.toBytes(i)); + p.addColumn(fam, qualifier, mobVal); +// table.put(p); + + bm.mutate(p); + if (i % 10000 == 0) { + LOG.info("LOADED="+ i); +// try { +// Thread.sleep(500); +// } catch (InterruptedException ee) { +// +// } + } + if (i % 100000 == 0) { + printStats(i); + } + } + bm.flush(); + admin.flush(table.getName()); + run = false; + } catch (Exception e) { + LOG.error("MOB Stress Test FAILED", e); + System.exit(-1); + } + } + } + + @Test + public void testMobCompaction() throws InterruptedException, IOException { + + try { + + Thread writeData = new Thread(new WriteData(count)); + writeData.start(); + + + Thread majorcompact = new Thread(new MajorCompaction()); + majorcompact.start(); + + Thread cleaner = new Thread(new CleanMobAndArchive()); + cleaner.start(); + + while (run) { + Thread.sleep(1000); + } + + MobUtils.getNumberOfMobFiles(conf, table.getName(), new String(fam)); + LOG.info("Waiting for write thread to finish ..."); + writeData.join(); + // Cleanup again + MobUtils.cleanupObsoleteMobFiles(conf, table.getName()); + MobUtils.getNumberOfMobFiles(conf, table.getName(), new String(fam)); + + if (HTU != null) { + LOG.info("Archive cleaner started ..."); + // Call archive cleaner again + HTU.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting(); + LOG.info("Archive cleaner finished"); + } + + scanTable(); + + } finally { + + admin.disableTable(hdt.getTableName()); + admin.deleteTable(hdt.getTableName()); + } + LOG.info("MOB Stress Test finished OK"); + printStats(count); + + } + + public void printStats(long loaded) { + LOG.info("MOB Stress Test: loaded=" + loaded+ " compactions=" + + FaultyMobStoreCompactor.totalCompactions.get() + + " major="+ FaultyMobStoreCompactor.totalMajorCompactions.get() +" mob="+ + FaultyMobStoreCompactor.mobCounter.get() + " injected failures="+ + FaultyMobStoreCompactor.totalFailures.get() + ); + } + + private void scanTable() { + try { + + Result result; + ResultScanner scanner = table.getScanner(fam); + int counter = 0; + while((result = scanner.next()) != null) { + assertTrue(Arrays.equals(result.getValue(fam, qualifier), mobVal)); + if ( counter % 10000 == 0 ) { + LOG.info("GET="+ counter); + } + counter++; + } + assertEquals(count, counter); + } catch (Exception e) { + e.printStackTrace(); + LOG.error("MOB Stress Test FAILED"); + if (HTU != null) { + assertTrue(false); + } else { + System.exit(-1); + } + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java index ae53ff21feba..a6a2ee55009c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java @@ -47,6 +47,7 @@ public class TestMobFileName { private Date date; private String dateStr; private byte[] startKey; + private String regionName = "region"; @Before public void setUp() { @@ -59,32 +60,32 @@ public void setUp() { @Test public void testHashCode() { - assertEquals(MobFileName.create(startKey, dateStr, uuid).hashCode(), - MobFileName.create(startKey, dateStr, uuid).hashCode()); - assertNotSame(MobFileName.create(startKey, dateStr, uuid), - MobFileName.create(startKey, dateStr, uuid)); + assertEquals(MobFileName.create(startKey, dateStr, uuid, regionName).hashCode(), + MobFileName.create(startKey, dateStr, uuid, regionName).hashCode()); + assertNotSame(MobFileName.create(startKey, dateStr, uuid, regionName), + MobFileName.create(startKey, dateStr, uuid, regionName)); } @Test public void testCreate() { - MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid); + MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid, regionName); assertEquals(mobFileName, MobFileName.create(mobFileName.getFileName())); } @Test public void testGet() { - MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid); + MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid, regionName); assertEquals(MD5Hash.getMD5AsHex(startKey, 0, startKey.length), mobFileName.getStartKey()); assertEquals(dateStr, mobFileName.getDate()); assertEquals(mobFileName.getFileName(), MD5Hash.getMD5AsHex(startKey, 0, startKey.length) - + dateStr + uuid); + + dateStr + uuid+"_"+regionName); } @Test public void testEquals() { - MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid); + MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid, regionName); assertTrue(mobFileName.equals(mobFileName)); assertFalse(mobFileName.equals(this)); - assertTrue(mobFileName.equals(MobFileName.create(startKey, dateStr, uuid))); + assertTrue(mobFileName.equals(MobFileName.create(startKey, dateStr, uuid, regionName))); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java deleted file mode 100644 index 130a6d31cba6..000000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java +++ /dev/null @@ -1,1202 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.mob.compactions; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; -import java.security.Key; -import java.security.SecureRandom; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Calendar; -import java.util.Collections; -import java.util.List; -import java.util.Objects; -import java.util.Optional; -import java.util.Random; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.RejectedExecutionHandler; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import javax.crypto.spec.SecretKeySpec; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.NamespaceDescriptor; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.BufferedMutator; -import org.apache.hadoop.hbase.client.CompactType; -import org.apache.hadoop.hbase.client.CompactionState; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.Delete; -import org.apache.hadoop.hbase.client.Durability; -import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.ResultScanner; -import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.coprocessor.ObserverContext; -import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; -import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; -import org.apache.hadoop.hbase.coprocessor.RegionObserver; -import org.apache.hadoop.hbase.io.HFileLink; -import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; -import org.apache.hadoop.hbase.io.crypto.aes.AES; -import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hadoop.hbase.io.hfile.HFile; -import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner; -import org.apache.hadoop.hbase.mob.MobConstants; -import org.apache.hadoop.hbase.mob.MobFileName; -import org.apache.hadoop.hbase.mob.MobTestUtil; -import org.apache.hadoop.hbase.mob.MobUtils; -import org.apache.hadoop.hbase.regionserver.BloomType; -import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.HStoreFile; -import org.apache.hadoop.hbase.regionserver.Store; -import org.apache.hadoop.hbase.regionserver.StoreFile; -import org.apache.hadoop.hbase.regionserver.StoreFileInfo; -import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker; -import org.apache.hadoop.hbase.security.EncryptionUtil; -import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.Threads; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Category(LargeTests.class) -public class TestMobCompactor { - - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestMobCompactor.class); - - private static final Logger LOG = LoggerFactory.getLogger(TestMobCompactor.class); - private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private static Configuration conf = null; - private TableName tableName; - private static Connection conn; - private BufferedMutator bufMut; - private Table table; - private static Admin admin; - private HTableDescriptor desc; - private HColumnDescriptor hcd1; - private HColumnDescriptor hcd2; - private static FileSystem fs; - private static final String family1 = "family1"; - private static final String family2 = "family2"; - private static final String qf1 = "qualifier1"; - private static final String qf2 = "qualifier2"; - - private static long tsFor20150907Monday; - private static long tsFor20151120Sunday; - private static long tsFor20151128Saturday; - private static long tsFor20151130Monday; - private static long tsFor20151201Tuesday; - private static long tsFor20151205Saturday; - private static long tsFor20151228Monday; - private static long tsFor20151231Thursday; - private static long tsFor20160101Friday; - private static long tsFor20160103Sunday; - - private static final byte[] mobKey01 = Bytes.toBytes("r01"); - private static final byte[] mobKey02 = Bytes.toBytes("r02"); - private static final byte[] mobKey03 = Bytes.toBytes("r03"); - private static final byte[] mobKey04 = Bytes.toBytes("r04"); - private static final byte[] mobKey05 = Bytes.toBytes("r05"); - private static final byte[] mobKey06 = Bytes.toBytes("r05"); - private static final byte[] mobKey1 = Bytes.toBytes("r1"); - private static final byte[] mobKey2 = Bytes.toBytes("r2"); - private static final byte[] mobKey3 = Bytes.toBytes("r3"); - private static final byte[] mobKey4 = Bytes.toBytes("r4"); - private static final byte[] mobKey5 = Bytes.toBytes("r5"); - private static final byte[] mobKey6 = Bytes.toBytes("r6"); - private static final byte[] mobKey7 = Bytes.toBytes("r7"); - private static final byte[] mobKey8 = Bytes.toBytes("r8"); - private static final String mobValue0 = "mobValue00000000000000000000000000"; - private static final String mobValue1 = "mobValue00000111111111111111111111"; - private static final String mobValue2 = "mobValue00000222222222222222222222"; - private static final String mobValue3 = "mobValue00000333333333333333333333"; - private static final String mobValue4 = "mobValue00000444444444444444444444"; - private static final String mobValue5 = "mobValue00000666666666666666666666"; - private static final String mobValue6 = "mobValue00000777777777777777777777"; - private static final String mobValue7 = "mobValue00000888888888888888888888"; - private static final String mobValue8 = "mobValue00000888888888888888888899"; - - private static byte[] KEYS = Bytes.toBytes("012"); - private static int regionNum = KEYS.length; - private static int delRowNum = 1; - private static int delCellNum = 6; - private static int cellNumPerRow = 3; - private static int rowNumPerFile = 2; - private static ExecutorService pool; - - @Rule - public TestName name = new TestName(); - - @BeforeClass - public static void setUpBeforeClass() throws Exception { - TEST_UTIL.getConfiguration().setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, 5000); - TEST_UTIL.getConfiguration() - .set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, KeyProviderForTesting.class.getName()); - TEST_UTIL.getConfiguration().set(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, "hbase"); - TEST_UTIL.getConfiguration().setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0); - TEST_UTIL.getConfiguration().setInt("hbase.client.retries.number", 1); - TEST_UTIL.getConfiguration().setInt("hbase.hfile.compaction.discharger.interval", 100); - TEST_UTIL.startMiniCluster(1); - pool = createThreadPool(TEST_UTIL.getConfiguration()); - conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration(), pool); - fs = TEST_UTIL.getTestFileSystem(); - conf = TEST_UTIL.getConfiguration(); - admin = TEST_UTIL.getAdmin(); - - // Initialize timestamps for these days - Calendar calendar = Calendar.getInstance(); - calendar.set(2015, 8, 7, 10, 20); - tsFor20150907Monday = calendar.getTimeInMillis(); - - calendar.set(2015, 10, 20, 10, 20); - tsFor20151120Sunday = calendar.getTimeInMillis(); - - calendar.set(2015, 10, 28, 10, 20); - tsFor20151128Saturday = calendar.getTimeInMillis(); - - calendar.set(2015, 10, 30, 10, 20); - tsFor20151130Monday = calendar.getTimeInMillis(); - - calendar.set(2015, 11, 1, 10, 20); - tsFor20151201Tuesday = calendar.getTimeInMillis(); - - calendar.set(2015, 11, 5, 10, 20); - tsFor20151205Saturday = calendar.getTimeInMillis(); - - calendar.set(2015, 11, 28, 10, 20); - tsFor20151228Monday = calendar.getTimeInMillis(); - - calendar.set(2015, 11, 31, 10, 20); - tsFor20151231Thursday = calendar.getTimeInMillis(); - - calendar.set(2016, 0, 1, 10, 20); - tsFor20160101Friday = calendar.getTimeInMillis(); - - calendar.set(2016, 0, 3, 10, 20); - tsFor20160103Sunday = calendar.getTimeInMillis(); - } - - @AfterClass - public static void tearDownAfterClass() throws Exception { - pool.shutdown(); - conn.close(); - TEST_UTIL.shutdownMiniCluster(); - } - - public void setUp(String tableNameAsString) throws IOException { - tableName = TableName.valueOf(tableNameAsString); - hcd1 = new HColumnDescriptor(family1); - hcd1.setMobEnabled(true); - hcd1.setMobThreshold(5); - hcd2 = new HColumnDescriptor(family2); - hcd2.setMobEnabled(true); - hcd2.setMobThreshold(5); - desc = new HTableDescriptor(tableName); - desc.addFamily(hcd1); - desc.addFamily(hcd2); - admin.createTable(desc, getSplitKeys()); - table = conn.getTable(tableName); - bufMut = conn.getBufferedMutator(tableName); - } - - // Set up for mob compaction policy testing - private void setUpForPolicyTest(String tableNameAsString, MobCompactPartitionPolicy type) - throws IOException { - tableName = TableName.valueOf(tableNameAsString); - hcd1 = new HColumnDescriptor(family1); - hcd1.setMobEnabled(true); - hcd1.setMobThreshold(10); - hcd1.setMobCompactPartitionPolicy(type); - desc = new HTableDescriptor(tableName); - desc.addFamily(hcd1); - admin.createTable(desc); - table = conn.getTable(tableName); - bufMut = conn.getBufferedMutator(tableName); - } - - // alter mob compaction policy - private void alterForPolicyTest(final MobCompactPartitionPolicy type) - throws Exception { - - hcd1.setMobCompactPartitionPolicy(type); - desc.modifyFamily(hcd1); - admin.modifyTable(desc); - LOG.info("alter status finished"); - } - - @Test - public void testMinorCompaction() throws Exception { - resetConf(); - int mergeSize = 5000; - // change the mob compaction merge size - conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize); - - // create a table with namespace - NamespaceDescriptor namespaceDescriptor = NamespaceDescriptor.create("ns").build(); - String tableNameAsString = "ns:testMinorCompaction"; - admin.createNamespace(namespaceDescriptor); - setUp(tableNameAsString); - int count = 4; - // generate mob files - loadData(admin, bufMut, tableName, count, rowNumPerFile); - int rowNumPerRegion = count * rowNumPerFile; - - assertEquals("Before deleting: mob rows count", regionNum * rowNumPerRegion, - MobTestUtil.countMobRows(table)); - assertEquals("Before deleting: mob cells count", regionNum * cellNumPerRow * rowNumPerRegion, - countMobCells(table)); - assertEquals("Before deleting: mob file count", regionNum * count, - countFiles(tableName, true, family1)); - - int largeFilesCount = countLargeFiles(mergeSize, tableName, family1); - createDelFile(table, tableName, Bytes.toBytes(family1), Bytes.toBytes(qf1)); - - assertEquals("Before compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum), - MobTestUtil.countMobRows(table)); - assertEquals("Before compaction: mob cells count", regionNum - * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table)); - assertEquals("Before compaction: family1 mob file count", regionNum * count, - countFiles(tableName, true, family1)); - assertEquals("Before compaction: family2 mob file count", regionNum * count, - countFiles(tableName, true, family2)); - assertEquals("Before compaction: family1 del file count", regionNum, - countFiles(tableName, false, family1)); - assertEquals("Before compaction: family2 del file count", regionNum, - countFiles(tableName, false, family2)); - - // do the mob file compaction - MobCompactor compactor = new PartitionedMobCompactor(conf, fs, tableName, hcd1, pool); - compactor.compact(); - - assertEquals("After compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum), - MobTestUtil.countMobRows(table)); - assertEquals("After compaction: mob cells count", regionNum - * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table)); - // After the compaction, the files smaller than the mob compaction merge size - // is merge to one file - assertEquals("After compaction: family1 mob file count", largeFilesCount + regionNum, - countFiles(tableName, true, family1)); - assertEquals("After compaction: family2 mob file count", regionNum * count, - countFiles(tableName, true, family2)); - assertEquals("After compaction: family1 del file count", regionNum, - countFiles(tableName, false, family1)); - assertEquals("After compaction: family2 del file count", regionNum, - countFiles(tableName, false, family2)); - } - - @Test - public void testMinorCompactionWithWeeklyPolicy() throws Exception { - resetConf(); - int mergeSize = 5000; - // change the mob compaction merge size - conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize); - - commonPolicyTestLogic("testMinorCompactionWithWeeklyPolicy", - MobCompactPartitionPolicy.WEEKLY, false, 6, - new String[] { "20150907", "20151120", "20151128", "20151130", "20151205", "20160103" }, - true); - } - - @Test - public void testMajorCompactionWithWeeklyPolicy() throws Exception { - resetConf(); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyPolicy", - MobCompactPartitionPolicy.WEEKLY, true, 5, - new String[] { "20150907", "20151120", "20151128", "20151205", "20160103" }, true); - } - - @Test - public void testMinorCompactionWithMonthlyPolicy() throws Exception { - resetConf(); - int mergeSize = 5000; - // change the mob compaction merge size - conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize); - - commonPolicyTestLogic("testMinorCompactionWithMonthlyPolicy", - MobCompactPartitionPolicy.MONTHLY, false, 4, - new String[] { "20150907", "20151130", "20151231", "20160103" }, true); - } - - @Test - public void testMajorCompactionWithMonthlyPolicy() throws Exception { - resetConf(); - - commonPolicyTestLogic("testMajorCompactionWithMonthlyPolicy", - MobCompactPartitionPolicy.MONTHLY, true, 4, - new String[] {"20150907", "20151130", "20151231", "20160103"}, true); - } - - @Test - public void testMajorCompactionWithWeeklyFollowedByMonthly() throws Exception { - resetConf(); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthly", - MobCompactPartitionPolicy.WEEKLY, true, 5, - new String[] { "20150907", "20151120", "20151128", "20151205", "20160103" }, true); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthly", - MobCompactPartitionPolicy.MONTHLY, true, 4, - new String[] {"20150907", "20151128", "20151205", "20160103" }, false); - } - - @Test - public void testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByWeekly() throws Exception { - resetConf(); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByWeekly", - MobCompactPartitionPolicy.WEEKLY, true, 5, - new String[] { "20150907", "20151120", "20151128", "20151205", "20160103" }, true); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByWeekly", - MobCompactPartitionPolicy.MONTHLY, true, 4, - new String[] { "20150907", "20151128", "20151205", "20160103" }, false); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByWeekly", - MobCompactPartitionPolicy.WEEKLY, true, 4, - new String[] { "20150907", "20151128", "20151205", "20160103" }, false); - } - - @Test - public void testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByDaily() throws Exception { - resetConf(); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByDaily", - MobCompactPartitionPolicy.WEEKLY, true, 5, - new String[] { "20150907", "20151120", "20151128", "20151205", "20160103" }, true); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByDaily", - MobCompactPartitionPolicy.MONTHLY, true, 4, - new String[] { "20150907", "20151128", "20151205", "20160103" }, false); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByDaily", - MobCompactPartitionPolicy.DAILY, true, 4, - new String[] { "20150907", "20151128", "20151205", "20160103" }, false); - } - - @Test - public void testCompactionWithHFileLink() throws IOException, InterruptedException { - resetConf(); - String tableNameAsString = "testCompactionWithHFileLink"; - setUp(tableNameAsString); - int count = 4; - // generate mob files - loadData(admin, bufMut, tableName, count, rowNumPerFile); - int rowNumPerRegion = count * rowNumPerFile; - - long tid = System.currentTimeMillis(); - String snapshotName1 = "snaptb-" + tid; - // take a snapshot - admin.snapshot(snapshotName1, tableName); - - createDelFile(table, tableName, Bytes.toBytes(family1), Bytes.toBytes(qf1)); - - assertEquals("Before compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum), - MobTestUtil.countMobRows(table)); - assertEquals("Before compaction: mob cells count", regionNum - * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table)); - assertEquals("Before compaction: family1 mob file count", regionNum * count, - countFiles(tableName, true, family1)); - assertEquals("Before compaction: family2 mob file count", regionNum * count, - countFiles(tableName, true, family2)); - assertEquals("Before compaction: family1 del file count", regionNum, - countFiles(tableName, false, family1)); - assertEquals("Before compaction: family2 del file count", regionNum, - countFiles(tableName, false, family2)); - - // do the mob compaction - MobCompactor compactor = new PartitionedMobCompactor(conf, fs, tableName, hcd1, pool); - compactor.compact(); - - assertEquals("After first compaction: mob rows count", regionNum - * (rowNumPerRegion - delRowNum), MobTestUtil.countMobRows(table)); - assertEquals("After first compaction: mob cells count", regionNum - * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table)); - assertEquals("After first compaction: family1 mob file count", regionNum, - countFiles(tableName, true, family1)); - assertEquals("After first compaction: family2 mob file count", regionNum * count, - countFiles(tableName, true, family2)); - assertEquals("After first compaction: family1 del file count", 0, - countFiles(tableName, false, family1)); - assertEquals("After first compaction: family2 del file count", regionNum, - countFiles(tableName, false, family2)); - assertEquals("After first compaction: family1 hfilelink count", 0, countHFileLinks(family1)); - assertEquals("After first compaction: family2 hfilelink count", 0, countHFileLinks(family2)); - - admin.disableTable(tableName); - // Restore from snapshot, the hfilelink will exist in mob dir - admin.restoreSnapshot(snapshotName1); - admin.enableTable(tableName); - - assertEquals("After restoring snapshot: mob rows count", regionNum * rowNumPerRegion, - MobTestUtil.countMobRows(table)); - assertEquals("After restoring snapshot: mob cells count", regionNum * cellNumPerRow - * rowNumPerRegion, countMobCells(table)); - assertEquals("After restoring snapshot: family1 mob file count", regionNum * count, - countFiles(tableName, true, family1)); - assertEquals("After restoring snapshot: family2 mob file count", regionNum * count, - countFiles(tableName, true, family2)); - assertEquals("After restoring snapshot: family1 del file count", 0, - countFiles(tableName, false, family1)); - assertEquals("After restoring snapshot: family2 del file count", 0, - countFiles(tableName, false, family2)); - assertEquals("After restoring snapshot: family1 hfilelink count", regionNum * count, - countHFileLinks(family1)); - assertEquals("After restoring snapshot: family2 hfilelink count", 0, countHFileLinks(family2)); - - compactor.compact(); - - assertEquals("After second compaction: mob rows count", regionNum * rowNumPerRegion, - MobTestUtil.countMobRows(table)); - assertEquals("After second compaction: mob cells count", regionNum * cellNumPerRow - * rowNumPerRegion, countMobCells(table)); - assertEquals("After second compaction: family1 mob file count", regionNum, - countFiles(tableName, true, family1)); - assertEquals("After second compaction: family2 mob file count", regionNum * count, - countFiles(tableName, true, family2)); - assertEquals("After second compaction: family1 del file count", 0, - countFiles(tableName, false, family1)); - assertEquals("After second compaction: family2 del file count", 0, - countFiles(tableName, false, family2)); - assertEquals("After second compaction: family1 hfilelink count", 0, countHFileLinks(family1)); - assertEquals("After second compaction: family2 hfilelink count", 0, countHFileLinks(family2)); - assertRefFileNameEqual(family1); - } - - @Test - public void testMajorCompactionFromAdmin() throws Exception { - resetConf(); - int mergeSize = 5000; - // change the mob compaction merge size - conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize); - SecureRandom rng = new SecureRandom(); - byte[] keyBytes = new byte[AES.KEY_LENGTH]; - rng.nextBytes(keyBytes); - String algorithm = conf.get(HConstants.CRYPTO_KEY_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES); - Key cfKey = new SecretKeySpec(keyBytes, algorithm); - byte[] encryptionKey = EncryptionUtil.wrapKey(conf, - conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, User.getCurrent().getShortName()), cfKey); - final TableName tableName = TableName.valueOf(name.getMethodName()); - HTableDescriptor desc = new HTableDescriptor(tableName); - HColumnDescriptor hcd1 = new HColumnDescriptor(family1); - hcd1.setMobEnabled(true); - hcd1.setMobThreshold(0); - hcd1.setEncryptionType(algorithm); - hcd1.setEncryptionKey(encryptionKey); - HColumnDescriptor hcd2 = new HColumnDescriptor(family2); - hcd2.setMobEnabled(true); - hcd2.setMobThreshold(0); - desc.addFamily(hcd1); - desc.addFamily(hcd2); - admin.createTable(desc, getSplitKeys()); - Table table = conn.getTable(tableName); - BufferedMutator bufMut = conn.getBufferedMutator(tableName); - int count = 4; - // generate mob files - loadData(admin, bufMut, tableName, count, rowNumPerFile); - int rowNumPerRegion = count * rowNumPerFile; - - assertEquals("Before deleting: mob rows count", regionNum * rowNumPerRegion, - MobTestUtil.countMobRows(table)); - assertEquals("Before deleting: mob cells count", regionNum * cellNumPerRow * rowNumPerRegion, - countMobCells(table)); - assertEquals("Before deleting: mob file count", regionNum * count, - countFiles(tableName, true, family1)); - - createDelFile(table, tableName, Bytes.toBytes(family1), Bytes.toBytes(qf1)); - - assertEquals("Before compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum), - MobTestUtil.countMobRows(table)); - assertEquals("Before compaction: mob cells count", regionNum - * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table)); - assertEquals("Before compaction: family1 mob file count", regionNum * count, - countFiles(tableName, true, family1)); - assertEquals("Before compaction: family2 mob file count", regionNum * count, - countFiles(tableName, true, family2)); - assertEquals("Before compaction: family1 del file count", regionNum, - countFiles(tableName, false, family1)); - assertEquals("Before compaction: family2 del file count", regionNum, - countFiles(tableName, false, family2)); - - // do the major mob compaction, it will force all files to compaction - admin.majorCompact(tableName, hcd1.getName(), CompactType.MOB); - - waitUntilMobCompactionFinished(tableName); - assertEquals("After compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum), - MobTestUtil.countMobRows(table)); - assertEquals("After compaction: mob cells count", regionNum - * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table)); - assertEquals("After compaction: family1 mob file count", regionNum, - countFiles(tableName, true, family1)); - assertEquals("After compaction: family2 mob file count", regionNum * count, - countFiles(tableName, true, family2)); - assertEquals("After compaction: family1 del file count", 0, - countFiles(tableName, false, family1)); - assertEquals("After compaction: family2 del file count", regionNum, - countFiles(tableName, false, family2)); - Assert.assertTrue(verifyEncryption(tableName, family1)); - table.close(); - } - - @Test - public void testScannerOnBulkLoadRefHFiles() throws Exception { - resetConf(); - setUp("testScannerOnBulkLoadRefHFiles"); - long ts = EnvironmentEdgeManager.currentTime(); - byte[] key0 = Bytes.toBytes("k0"); - byte[] key1 = Bytes.toBytes("k1"); - String value0 = "mobValue0"; - String value1 = "mobValue1"; - String newValue0 = "new"; - Put put0 = new Put(key0); - put0.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), ts, Bytes.toBytes(value0)); - loadData(admin, bufMut, tableName, new Put[] { put0 }); - put0 = new Put(key0); - put0.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), ts, Bytes.toBytes(newValue0)); - Put put1 = new Put(key1); - put1.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), ts, Bytes.toBytes(value1)); - loadData(admin, bufMut, tableName, new Put[] { put0, put1 }); - // read the latest cell of key0. - Get get = new Get(key0); - Result result = table.get(get); - Cell cell = result.getColumnLatestCell(hcd1.getName(), Bytes.toBytes(qf1)); - assertEquals("Before compaction: mob value of k0", newValue0, - Bytes.toString(CellUtil.cloneValue(cell))); - admin.majorCompact(tableName, hcd1.getName(), CompactType.MOB); - waitUntilMobCompactionFinished(tableName); - // read the latest cell of key0, the cell seqId in bulk loaded file is not reset in the - // scanner. The cell that has "new" value is still visible. - result = table.get(get); - cell = result.getColumnLatestCell(hcd1.getName(), Bytes.toBytes(qf1)); - assertEquals("After compaction: mob value of k0", newValue0, - Bytes.toString(CellUtil.cloneValue(cell))); - // read the ref cell, not read further to the mob cell. - get = new Get(key1); - get.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(true)); - result = table.get(get); - cell = result.getColumnLatestCell(hcd1.getName(), Bytes.toBytes(qf1)); - // the ref name is the new file - Path mobFamilyPath = - MobUtils.getMobFamilyPath(TEST_UTIL.getConfiguration(), tableName, hcd1.getNameAsString()); - List paths = new ArrayList<>(); - if (fs.exists(mobFamilyPath)) { - FileStatus[] files = fs.listStatus(mobFamilyPath); - for (FileStatus file : files) { - if (!StoreFileInfo.isDelFile(file.getPath())) { - paths.add(file.getPath()); - } - } - } - assertEquals("After compaction: number of mob files:", 1, paths.size()); - assertEquals("After compaction: mob file name:", MobUtils.getMobFileName(cell), paths.get(0) - .getName()); - } - - /** - * This case tests the following mob compaction and normal compaction scenario, - * after mob compaction, the mob reference in new bulkloaded hfile will win even after it - * is compacted with some other normal hfiles. This is to make sure the mvcc is included - * after compaction for mob enabled store files. - */ - @Test - public void testGetAfterCompaction() throws Exception { - resetConf(); - conf.setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0); - String famStr = "f1"; - byte[] fam = Bytes.toBytes(famStr); - byte[] qualifier = Bytes.toBytes("q1"); - byte[] mobVal = Bytes.toBytes("01234567890"); - HTableDescriptor hdt = new HTableDescriptor(TableName.valueOf(name.getMethodName())); - hdt.addCoprocessor(CompactTwoLatestHfilesCopro.class.getName()); - HColumnDescriptor hcd = new HColumnDescriptor(fam); - hcd.setMobEnabled(true); - hcd.setMobThreshold(10); - hcd.setMaxVersions(1); - hdt.addFamily(hcd); - try { - Table table = TEST_UTIL.createTable(hdt, null); - HRegion r = TEST_UTIL.getMiniHBaseCluster().getRegions(hdt.getTableName()).get(0); - Put p = new Put(Bytes.toBytes("r1")); - p.addColumn(fam, qualifier, mobVal); - table.put(p); - // Create mob file mob1 and reference file ref1 - TEST_UTIL.flush(table.getName()); - // Make sure that it is flushed. - FileSystem fs = r.getRegionFileSystem().getFileSystem(); - Path path = r.getRegionFileSystem().getStoreDir(famStr); - waitUntilFilesShowup(fs, path, 1); - - p = new Put(Bytes.toBytes("r2")); - p.addColumn(fam, qualifier, mobVal); - table.put(p); - // Create mob file mob2 and reference file ref2 - TEST_UTIL.flush(table.getName()); - waitUntilFilesShowup(fs, path, 2); - // Do mob compaction to create mob3 and ref3 - TEST_UTIL.getAdmin().compact(hdt.getTableName(), fam, CompactType.MOB); - waitUntilFilesShowup(fs, path, 3); - - // Compact ref3 and ref2 into ref4 - TEST_UTIL.getAdmin().compact(hdt.getTableName(), fam); - waitUntilFilesShowup(fs, path, 2); - - // Sleep for some time, since TimeToLiveHFileCleaner is 0, the next run of - // clean chore is guaranteed to clean up files in archive - Thread.sleep(100); - // Run cleaner to make sure that files in archive directory are cleaned up - TEST_UTIL.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting(); - - // Get "r2" - Get get = new Get(Bytes.toBytes("r2")); - try { - Result result = table.get(get); - assertTrue(Arrays.equals(result.getValue(fam, qualifier), mobVal)); - } catch (IOException e) { - assertTrue("The MOB file doesn't exist", false); - } - } finally { - TEST_UTIL.deleteTable(hdt.getTableName()); - } - } - - private void waitUntilFilesShowup(final FileSystem fs, final Path path, final int num) - throws InterruptedException, IOException { - FileStatus[] fileList = fs.listStatus(path); - while (fileList.length != num) { - Thread.sleep(50); - fileList = fs.listStatus(path); - for (FileStatus fileStatus: fileList) { - LOG.info(Objects.toString(fileStatus)); - } - } - } - - /** - * This copro overwrites the default compaction policy. It always chooses two latest hfiles and - * compacts them into a new one. - */ - public static class CompactTwoLatestHfilesCopro implements RegionCoprocessor, RegionObserver { - - @Override - public Optional getRegionObserver() { - return Optional.of(this); - } - - @Override - public void preCompactSelection(ObserverContext c, Store store, - List candidates, CompactionLifeCycleTracker tracker) - throws IOException { - int count = candidates.size(); - if (count >= 2) { - for (int i = 0; i < count - 2; i++) { - candidates.remove(0); - } - c.bypass(); - } - } - } - - private void waitUntilMobCompactionFinished(TableName tableName) throws IOException, - InterruptedException { - long finished = EnvironmentEdgeManager.currentTime() + 60000; - CompactionState state = admin.getCompactionState(tableName, CompactType.MOB); - while (EnvironmentEdgeManager.currentTime() < finished) { - if (state == CompactionState.NONE) { - break; - } - state = admin.getCompactionState(tableName, CompactType.MOB); - Thread.sleep(10); - } - assertEquals(CompactionState.NONE, state); - } - - /** - * Gets the number of cells in the given table. - * @param table to get the scanner - * @return the number of cells - */ - private int countMobCells(final Table table) throws IOException { - Scan scan = new Scan(); - // Do not retrieve the mob data when scanning - scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE)); - ResultScanner results = table.getScanner(scan); - int count = 0; - for (Result res : results) { - count += res.size(); - } - results.close(); - return count; - } - - /** - * Gets the number of files in the mob path. - * @param isMobFile gets number of the mob files or del files - * @param familyName the family name - * @return the number of the files - */ - private int countFiles(TableName tableName, boolean isMobFile, String familyName) - throws IOException { - Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, familyName); - int count = 0; - if (fs.exists(mobDirPath)) { - FileStatus[] files = fs.listStatus(mobDirPath); - for (FileStatus file : files) { - if (isMobFile == true) { - if (!StoreFileInfo.isDelFile(file.getPath())) { - count++; - } - } else { - if (StoreFileInfo.isDelFile(file.getPath())) { - count++; - } - } - } - } - return count; - } - - private boolean verifyEncryption(TableName tableName, String familyName) throws IOException { - Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, familyName); - boolean hasFiles = false; - if (fs.exists(mobDirPath)) { - FileStatus[] files = fs.listStatus(mobDirPath); - hasFiles = files != null && files.length > 0; - Assert.assertTrue(hasFiles); - Path path = files[0].getPath(); - CacheConfig cacheConf = new CacheConfig(conf); - HStoreFile sf = new HStoreFile(TEST_UTIL.getTestFileSystem(), path, conf, cacheConf, - BloomType.NONE, true); - sf.initReader(); - HFile.Reader reader = sf.getReader().getHFileReader(); - byte[] encryptionKey = reader.getTrailer().getEncryptionKey(); - Assert.assertTrue(null != encryptionKey); - Assert.assertTrue(reader.getFileContext().getEncryptionContext().getCipher().getName() - .equals(HConstants.CIPHER_AES)); - } - return hasFiles; - } - - /** - * Gets the number of HFileLink in the mob path. - * @param familyName the family name - * @return the number of the HFileLink - */ - private int countHFileLinks(String familyName) throws IOException { - Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, familyName); - int count = 0; - if (fs.exists(mobDirPath)) { - FileStatus[] files = fs.listStatus(mobDirPath); - for (FileStatus file : files) { - if (HFileLink.isHFileLink(file.getPath())) { - count++; - } - } - } - return count; - } - - /** - * Gets the number of files. - * @param size the size of the file - * @param tableName the current table name - * @param familyName the family name - * @return the number of files large than the size - */ - private int countLargeFiles(int size, TableName tableName, String familyName) throws IOException { - Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, familyName); - int count = 0; - if (fs.exists(mobDirPath)) { - FileStatus[] files = fs.listStatus(mobDirPath); - for (FileStatus file : files) { - // ignore the del files in the mob path - if ((!StoreFileInfo.isDelFile(file.getPath())) && (file.getLen() > size)) { - count++; - } - } - } - return count; - } - - /** - * loads some data to the table. - */ - private void loadData(Admin admin, BufferedMutator table, TableName tableName, int fileNum, - int rowNumPerFile) throws IOException, InterruptedException { - if (fileNum <= 0) { - throw new IllegalArgumentException(); - } - for (int i = 0; i < fileNum * rowNumPerFile; i++) { - for (byte k0 : KEYS) { - byte[] k = new byte[] { k0 }; - byte[] key = Bytes.add(k, Bytes.toBytes(i)); - byte[] mobVal = makeDummyData(10 * (i + 1)); - Put put = new Put(key); - put.setDurability(Durability.SKIP_WAL); - put.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), mobVal); - put.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf2), mobVal); - put.addColumn(Bytes.toBytes(family2), Bytes.toBytes(qf1), mobVal); - table.mutate(put); - } - if ((i + 1) % rowNumPerFile == 0) { - table.flush(); - admin.flush(tableName); - } - } - } - - private void loadData(Admin admin, BufferedMutator table, TableName tableName, Put[] puts) - throws IOException { - table.mutate(Arrays.asList(puts)); - table.flush(); - admin.flush(tableName); - } - - private void loadDataForPartitionPolicy(Admin admin, BufferedMutator table, TableName tableName) - throws IOException { - - Put[] pArray = new Put[1000]; - - for (int i = 0; i < 1000; i ++) { - Put put0 = new Put(Bytes.toBytes("r0" + i)); - put0.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), - tsFor20151130Monday, Bytes.toBytes(mobValue0)); - pArray[i] = put0; - } - loadData(admin, bufMut, tableName, pArray); - - Put put06 = new Put(mobKey06); - put06.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), - tsFor20151128Saturday, Bytes.toBytes(mobValue0)); - - loadData(admin, bufMut, tableName, new Put[] { put06 }); - - Put put1 = new Put(mobKey1); - put1.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151201Tuesday, - Bytes.toBytes(mobValue1)); - loadData(admin, bufMut, tableName, new Put[] { put1 }); - - Put put2 = new Put(mobKey2); - put2.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151205Saturday, - Bytes.toBytes(mobValue2)); - loadData(admin, bufMut, tableName, new Put[] { put2 }); - - Put put3 = new Put(mobKey3); - put3.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151228Monday, - Bytes.toBytes(mobValue3)); - loadData(admin, bufMut, tableName, new Put[] { put3 }); - - Put put4 = new Put(mobKey4); - put4.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151231Thursday, - Bytes.toBytes(mobValue4)); - loadData(admin, bufMut, tableName, new Put[] { put4 }); - - Put put5 = new Put(mobKey5); - put5.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20160101Friday, - Bytes.toBytes(mobValue5)); - loadData(admin, bufMut, tableName, new Put[] { put5 }); - - Put put6 = new Put(mobKey6); - put6.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20160103Sunday, - Bytes.toBytes(mobValue6)); - loadData(admin, bufMut, tableName, new Put[] { put6 }); - - Put put7 = new Put(mobKey7); - put7.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20150907Monday, - Bytes.toBytes(mobValue7)); - loadData(admin, bufMut, tableName, new Put[] { put7 }); - - Put put8 = new Put(mobKey8); - put8.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151120Sunday, - Bytes.toBytes(mobValue8)); - loadData(admin, bufMut, tableName, new Put[] { put8 }); - } - - - /** - * delete the row, family and cell to create the del file - */ - private void createDelFile(Table table, TableName tableName, byte[] family, byte[] qf) - throws IOException, InterruptedException { - for (byte k0 : KEYS) { - byte[] k = new byte[] { k0 }; - // delete a family - byte[] key1 = Bytes.add(k, Bytes.toBytes(0)); - Delete delete1 = new Delete(key1); - delete1.addFamily(family); - table.delete(delete1); - // delete one row - byte[] key2 = Bytes.add(k, Bytes.toBytes(2)); - Delete delete2 = new Delete(key2); - table.delete(delete2); - // delete one cell - byte[] key3 = Bytes.add(k, Bytes.toBytes(4)); - Delete delete3 = new Delete(key3); - delete3.addColumn(family, qf); - table.delete(delete3); - } - admin.flush(tableName); - List regions = TEST_UTIL.getHBaseCluster().getRegions(tableName); - for (HRegion region : regions) { - region.waitForFlushesAndCompactions(); - region.compact(true); - } - } - /** - * Creates the dummy data with a specific size. - * @param size the size of value - * @return the dummy data - */ - private byte[] makeDummyData(int size) { - byte[] dummyData = new byte[size]; - new Random().nextBytes(dummyData); - return dummyData; - } - - /** - * Gets the split keys - */ - private byte[][] getSplitKeys() { - byte[][] splitKeys = new byte[KEYS.length - 1][]; - for (int i = 0; i < splitKeys.length; ++i) { - splitKeys[i] = new byte[] { KEYS[i + 1] }; - } - return splitKeys; - } - - private static ExecutorService createThreadPool(Configuration conf) { - int maxThreads = 10; - long keepAliveTime = 60; - final SynchronousQueue queue = new SynchronousQueue<>(); - ThreadPoolExecutor pool = new ThreadPoolExecutor(1, maxThreads, - keepAliveTime, TimeUnit.SECONDS, queue, - Threads.newDaemonThreadFactory("MobFileCompactionChore"), - new RejectedExecutionHandler() { - @Override - public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) { - try { - // waiting for a thread to pick up instead of throwing exceptions. - queue.put(r); - } catch (InterruptedException e) { - throw new RejectedExecutionException(e); - } - } - }); - pool.allowCoreThreadTimeOut(true); - return pool; - } - - private void assertRefFileNameEqual(String familyName) throws IOException { - Scan scan = new Scan(); - scan.addFamily(Bytes.toBytes(familyName)); - // Do not retrieve the mob data when scanning - scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE)); - ResultScanner results = table.getScanner(scan); - Path mobFamilyPath = MobUtils.getMobFamilyPath(TEST_UTIL.getConfiguration(), - tableName, familyName); - List actualFilePaths = new ArrayList<>(); - List expectFilePaths = new ArrayList<>(); - for (Result res : results) { - for (Cell cell : res.listCells()) { - byte[] referenceValue = CellUtil.cloneValue(cell); - String fileName = Bytes.toString(referenceValue, Bytes.SIZEOF_INT, - referenceValue.length - Bytes.SIZEOF_INT); - Path targetPath = new Path(mobFamilyPath, fileName); - if(!actualFilePaths.contains(targetPath)) { - actualFilePaths.add(targetPath); - } - } - } - results.close(); - if (fs.exists(mobFamilyPath)) { - FileStatus[] files = fs.listStatus(mobFamilyPath); - for (FileStatus file : files) { - if (!StoreFileInfo.isDelFile(file.getPath())) { - expectFilePaths.add(file.getPath()); - } - } - } - Collections.sort(actualFilePaths); - Collections.sort(expectFilePaths); - assertEquals(expectFilePaths, actualFilePaths); - } - - /** - * Resets the configuration. - */ - private void resetConf() { - conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, - MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD); - conf.setInt(MobConstants.MOB_COMPACTION_BATCH_SIZE, - MobConstants.DEFAULT_MOB_COMPACTION_BATCH_SIZE); - } - - /** - * Verify mob partition policy compaction values. - */ - private void verifyPolicyValues() throws Exception { - Get get = new Get(mobKey01); - Result result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue0))); - - get = new Get(mobKey02); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue0))); - - get = new Get(mobKey03); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue0))); - - get = new Get(mobKey04); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue0))); - - get = new Get(mobKey05); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue0))); - - get = new Get(mobKey06); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue0))); - - get = new Get(mobKey1); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue1))); - - get = new Get(mobKey2); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue2))); - - get = new Get(mobKey3); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue3))); - - get = new Get(mobKey4); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue4))); - - get = new Get(mobKey5); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue5))); - - get = new Get(mobKey6); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue6))); - - get = new Get(mobKey7); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue7))); - - get = new Get(mobKey8); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue8))); - } - - private void commonPolicyTestLogic (final String tableNameAsString, - final MobCompactPartitionPolicy pType, final boolean majorCompact, - final int expectedFileNumbers, final String[] expectedFileNames, - final boolean setupAndLoadData - ) throws Exception { - if (setupAndLoadData) { - setUpForPolicyTest(tableNameAsString, pType); - - loadDataForPartitionPolicy(admin, bufMut, tableName); - } else { - alterForPolicyTest(pType); - } - - if (majorCompact) { - admin.majorCompact(tableName, hcd1.getName(), CompactType.MOB); - } else { - admin.compact(tableName, hcd1.getName(), CompactType.MOB); - } - - waitUntilMobCompactionFinished(tableName); - - // Run cleaner to make sure that files in archive directory are cleaned up - TEST_UTIL.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting(); - - //check the number of files - Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, family1); - FileStatus[] fileList = fs.listStatus(mobDirPath); - - assertTrue(fileList.length == expectedFileNumbers); - - // the file names are expected - ArrayList fileNames = new ArrayList<>(expectedFileNumbers); - for (FileStatus file : fileList) { - fileNames.add(MobFileName.getDateFromName(file.getPath().getName())); - } - int index = 0; - for (String fileName : expectedFileNames) { - index = fileNames.indexOf(fileName); - assertTrue(index >= 0); - fileNames.remove(index); - } - - // Check daily mob files are removed from the mobdir, and only weekly mob files are there. - // Also check that there is no data loss. - - verifyPolicyValues(); - } - } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactionRequest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactionRequest.java deleted file mode 100644 index 27801533dc5a..000000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactionRequest.java +++ /dev/null @@ -1,65 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.mob.compactions; - -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartition; -import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -@Category(SmallTests.class) -public class TestPartitionedMobCompactionRequest { - - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestPartitionedMobCompactionRequest.class); - - @Test - public void testCompactedPartitionId() { - String startKey1 = "startKey1"; - String startKey2 = "startKey2"; - String date1 = "date1"; - String date2 = "date2"; - CompactionPartitionId partitionId1 = new CompactionPartitionId(startKey1, date1); - CompactionPartitionId partitionId2 = new CompactionPartitionId(startKey2, date2); - CompactionPartitionId partitionId3 = new CompactionPartitionId(startKey1, date2); - - Assert.assertTrue(partitionId1.equals(partitionId1)); - Assert.assertFalse(partitionId1.equals(partitionId2)); - Assert.assertFalse(partitionId1.equals(partitionId3)); - Assert.assertFalse(partitionId2.equals(partitionId3)); - - Assert.assertEquals(startKey1, partitionId1.getStartKey()); - Assert.assertEquals(date1, partitionId1.getDate()); - } - - @Test - public void testCompactedPartition() { - CompactionPartitionId partitionId = new CompactionPartitionId("startKey1", "date1"); - CompactionPartition partition = new CompactionPartition(partitionId); - FileStatus file = new FileStatus(1, false, 1, 1024, 1, new Path("/test")); - partition.addFile(file); - Assert.assertEquals(file, partition.listFiles().get(0)); - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java deleted file mode 100644 index 4e6461aaf800..000000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java +++ /dev/null @@ -1,958 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.mob.compactions; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; -import java.text.ParseException; -import java.util.ArrayList; -import java.util.Calendar; -import java.util.Collection; -import java.util.Collections; -import java.util.Date; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.RejectedExecutionHandler; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellComparatorImpl; -import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValue.Type; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy; -import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hadoop.hbase.io.hfile.HFileContext; -import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; -import org.apache.hadoop.hbase.mob.MobConstants; -import org.apache.hadoop.hbase.mob.MobFileName; -import org.apache.hadoop.hbase.mob.MobUtils; -import org.apache.hadoop.hbase.mob.compactions.MobCompactionRequest.CompactionType; -import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionDelPartition; -import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartition; -import org.apache.hadoop.hbase.regionserver.BloomType; -import org.apache.hadoop.hbase.regionserver.HStore; -import org.apache.hadoop.hbase.regionserver.HStoreFile; -import org.apache.hadoop.hbase.regionserver.KeyValueScanner; -import org.apache.hadoop.hbase.regionserver.ScanInfo; -import org.apache.hadoop.hbase.regionserver.ScanType; -import org.apache.hadoop.hbase.regionserver.StoreFileScanner; -import org.apache.hadoop.hbase.regionserver.StoreFileWriter; -import org.apache.hadoop.hbase.regionserver.StoreScanner; -import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.hbase.util.Threads; -import org.apache.hadoop.hdfs.DistributedFileSystem; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Category(LargeTests.class) -public class TestPartitionedMobCompactor { - - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestPartitionedMobCompactor.class); - - private static final Logger LOG = LoggerFactory.getLogger(TestPartitionedMobCompactor.class); - private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private final static String family = "family"; - private final static String qf = "qf"; - private final long DAY_IN_MS = 1000 * 60 * 60 * 24; - private static byte[] KEYS = Bytes.toBytes("012"); - private HColumnDescriptor hcd = new HColumnDescriptor(family); - private Configuration conf = TEST_UTIL.getConfiguration(); - private CacheConfig cacheConf = new CacheConfig(conf); - private FileSystem fs; - private List mobFiles = new ArrayList<>(); - private List delFiles = new ArrayList<>(); - private List allFiles = new ArrayList<>(); - private Path basePath; - private String mobSuffix; - private String delSuffix; - private static ExecutorService pool; - - @Rule - public TestName name = new TestName(); - - @BeforeClass - public static void setUpBeforeClass() throws Exception { - TEST_UTIL.getConfiguration().setInt("hfile.format.version", 3); - // Inject our customized DistributedFileSystem - TEST_UTIL.getConfiguration().setClass("fs.hdfs.impl", FaultyDistributedFileSystem.class, - DistributedFileSystem.class); - TEST_UTIL.startMiniCluster(1); - pool = createThreadPool(); - } - - @AfterClass - public static void tearDownAfterClass() throws Exception { - pool.shutdown(); - TEST_UTIL.shutdownMiniCluster(); - } - - private void init(String tableName) throws Exception { - fs = FileSystem.get(conf); - Path testDir = FSUtils.getRootDir(conf); - Path mobTestDir = new Path(testDir, MobConstants.MOB_DIR_NAME); - basePath = new Path(new Path(mobTestDir, tableName), family); - mobSuffix = TEST_UTIL.getRandomUUID().toString().replaceAll("-", ""); - delSuffix = TEST_UTIL.getRandomUUID().toString().replaceAll("-", "") + "_del"; - allFiles.clear(); - mobFiles.clear(); - delFiles.clear(); - } - - @Test - public void testCompactionSelectAllFilesWeeklyPolicy() throws Exception { - String tableName = "testCompactionSelectAllFilesWeeklyPolicy"; - testCompactionAtMergeSize(tableName, MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD, - CompactionType.ALL_FILES, false, false, new Date(), MobCompactPartitionPolicy.WEEKLY, 1); - } - - @Test - public void testCompactionSelectPartFilesWeeklyPolicy() throws Exception { - String tableName = "testCompactionSelectPartFilesWeeklyPolicy"; - testCompactionAtMergeSize(tableName, 4000, CompactionType.PART_FILES, false, false, - new Date(), MobCompactPartitionPolicy.WEEKLY, 1); - } - - @Test - public void testCompactionSelectPartFilesWeeklyPolicyWithPastWeek() throws Exception { - String tableName = "testCompactionSelectPartFilesWeeklyPolicyWithPastWeek"; - Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS)); - testCompactionAtMergeSize(tableName, 700, CompactionType.PART_FILES, false, false, dateLastWeek, - MobCompactPartitionPolicy.WEEKLY, 7); - } - - @Test - public void testCompactionSelectAllFilesWeeklyPolicyWithPastWeek() throws Exception { - String tableName = "testCompactionSelectAllFilesWeeklyPolicyWithPastWeek"; - Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS)); - testCompactionAtMergeSize(tableName, 3000, CompactionType.ALL_FILES, - false, false, dateLastWeek, MobCompactPartitionPolicy.WEEKLY, 7); - } - - @Test - public void testCompactionSelectAllFilesMonthlyPolicy() throws Exception { - String tableName = "testCompactionSelectAllFilesMonthlyPolicy"; - Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS)); - testCompactionAtMergeSize(tableName, MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD, - CompactionType.ALL_FILES, false, false, dateLastWeek, - MobCompactPartitionPolicy.MONTHLY, 7); - } - - @Test - public void testCompactionSelectNoFilesWithinCurrentWeekMonthlyPolicy() throws Exception { - String tableName = "testCompactionSelectNoFilesWithinCurrentWeekMonthlyPolicy"; - testCompactionAtMergeSize(tableName, MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD, - CompactionType.PART_FILES, false, false, new Date(), MobCompactPartitionPolicy.MONTHLY, 1); - } - - @Test - public void testCompactionSelectPartFilesMonthlyPolicy() throws Exception { - String tableName = "testCompactionSelectPartFilesMonthlyPolicy"; - testCompactionAtMergeSize(tableName, 4000, CompactionType.PART_FILES, false, false, - new Date(), MobCompactPartitionPolicy.MONTHLY, 1); - } - - @Test - public void testCompactionSelectPartFilesMonthlyPolicyWithPastWeek() throws Exception { - String tableName = "testCompactionSelectPartFilesMonthlyPolicyWithPastWeek"; - Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS)); - Calendar calendar = Calendar.getInstance(); - Date firstDayOfCurrentMonth = MobUtils.getFirstDayOfMonth(calendar, new Date()); - CompactionType type = CompactionType.PART_FILES; - long mergeSizeMultiFactor = 7; - - - // The dateLastWeek may not really be last week, suppose that it runs at 2/1/2017, it is going - // to be last month and the monthly policy is going to be applied here. - if (dateLastWeek.before(firstDayOfCurrentMonth)) { - type = CompactionType.ALL_FILES; - mergeSizeMultiFactor *= 4; - } - - testCompactionAtMergeSize(tableName, 700, type, false, false, dateLastWeek, - MobCompactPartitionPolicy.MONTHLY, mergeSizeMultiFactor); - } - - @Test - public void testCompactionSelectAllFilesMonthlyPolicyWithPastWeek() throws Exception { - String tableName = "testCompactionSelectAllFilesMonthlyPolicyWithPastWeek"; - Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS)); - - testCompactionAtMergeSize(tableName, 3000, CompactionType.ALL_FILES, - false, false, dateLastWeek, MobCompactPartitionPolicy.MONTHLY, 7); - } - - @Test - public void testCompactionSelectPartFilesMonthlyPolicyWithPastMonth() throws Exception { - String tableName = "testCompactionSelectPartFilesMonthlyPolicyWithPastMonth"; - - // back 5 weeks, it is going to be a past month - Date dateLastMonth = new Date(System.currentTimeMillis() - (7 * 5 * DAY_IN_MS)); - testCompactionAtMergeSize(tableName, 200, CompactionType.PART_FILES, false, false, dateLastMonth, - MobCompactPartitionPolicy.MONTHLY, 28); - } - - @Test - public void testCompactionSelectAllFilesMonthlyPolicyWithPastMonth() throws Exception { - String tableName = "testCompactionSelectAllFilesMonthlyPolicyWithPastMonth"; - - // back 5 weeks, it is going to be a past month - Date dateLastMonth = new Date(System.currentTimeMillis() - (7 * 5 * DAY_IN_MS)); - testCompactionAtMergeSize(tableName, 750, CompactionType.ALL_FILES, - false, false, dateLastMonth, MobCompactPartitionPolicy.MONTHLY, 28); - } - - @Test - public void testCompactionSelectWithAllFiles() throws Exception { - String tableName = "testCompactionSelectWithAllFiles"; - // If there is only 1 file, it will not be compacted with _del files, so - // It wont be CompactionType.ALL_FILES in this case, do not create with _del files. - testCompactionAtMergeSize(tableName, MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD, - CompactionType.ALL_FILES, false, false); - } - - @Test - public void testCompactionSelectWithPartFiles() throws Exception { - String tableName = "testCompactionSelectWithPartFiles"; - testCompactionAtMergeSize(tableName, 4000, CompactionType.PART_FILES, false); - } - - @Test - public void testCompactionSelectWithForceAllFiles() throws Exception { - String tableName = "testCompactionSelectWithForceAllFiles"; - testCompactionAtMergeSize(tableName, Long.MAX_VALUE, CompactionType.ALL_FILES, true); - } - - private void testCompactionAtMergeSize(final String tableName, - final long mergeSize, final CompactionType type, final boolean isForceAllFiles) - throws Exception { - testCompactionAtMergeSize(tableName, mergeSize, type, isForceAllFiles, true); - } - - private void testCompactionAtMergeSize(final String tableName, - final long mergeSize, final CompactionType type, final boolean isForceAllFiles, - final boolean createDelFiles) - throws Exception { - Date date = new Date(); - testCompactionAtMergeSize(tableName, mergeSize, type, isForceAllFiles, createDelFiles, date); - } - - private void testCompactionAtMergeSize(final String tableName, - final long mergeSize, final CompactionType type, final boolean isForceAllFiles, - final boolean createDelFiles, final Date date) - throws Exception { - testCompactionAtMergeSize(tableName, mergeSize, type, isForceAllFiles, createDelFiles, date, - MobCompactPartitionPolicy.DAILY, 1); - } - - private void testCompactionAtMergeSize(final String tableName, - final long mergeSize, final CompactionType type, final boolean isForceAllFiles, - final boolean createDelFiles, final Date date, final MobCompactPartitionPolicy policy, - final long mergeSizeMultiFactor) - throws Exception { - resetConf(); - init(tableName); - int count = 10; - // create 10 mob files. - createStoreFiles(basePath, family, qf, count, Type.Put, date); - - if (createDelFiles) { - // create 10 del files - createStoreFiles(basePath, family, qf, count, Type.Delete, date); - } - - Calendar calendar = Calendar.getInstance(); - Date firstDayOfCurrentWeek = MobUtils.getFirstDayOfWeek(calendar, new Date()); - - listFiles(); - List expectedStartKeys = new ArrayList<>(); - for(FileStatus file : mobFiles) { - if(file.getLen() < mergeSize * mergeSizeMultiFactor) { - String fileName = file.getPath().getName(); - String startKey = fileName.substring(0, 32); - - // If the policy is monthly and files are in current week, they will be skipped - // in minor compcation. - boolean skipCompaction = false; - if (policy == MobCompactPartitionPolicy.MONTHLY) { - String fileDateStr = MobFileName.getDateFromName(fileName); - Date fileDate; - try { - fileDate = MobUtils.parseDate(fileDateStr); - } catch (ParseException e) { - LOG.warn("Failed to parse date " + fileDateStr, e); - fileDate = new Date(); - } - if (!fileDate.before(firstDayOfCurrentWeek)) { - skipCompaction = true; - } - } - - // If it is not an major mob compaction and del files are there, - // these mob files wont be compacted. - if (isForceAllFiles || (!createDelFiles && !skipCompaction)) { - expectedStartKeys.add(startKey); - } - } - } - - // Set the policy - this.hcd.setMobCompactPartitionPolicy(policy); - // set the mob compaction mergeable threshold - conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize); - testSelectFiles(tableName, type, isForceAllFiles, expectedStartKeys); - // go back to the default daily policy - this.hcd.setMobCompactPartitionPolicy(MobCompactPartitionPolicy.DAILY); - } - - @Test - public void testCompactDelFilesWithDefaultBatchSize() throws Exception { - testCompactDelFilesAtBatchSize(name.getMethodName(), MobConstants.DEFAULT_MOB_COMPACTION_BATCH_SIZE, - MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT); - } - - @Test - public void testCompactDelFilesWithSmallBatchSize() throws Exception { - testCompactDelFilesAtBatchSize(name.getMethodName(), 4, MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT); - } - - @Test - public void testCompactDelFilesChangeMaxDelFileCount() throws Exception { - testCompactDelFilesAtBatchSize(name.getMethodName(), 4, 2); - } - - @Test - public void testCompactFilesWithDstDirFull() throws Exception { - String tableName = name.getMethodName(); - fs = FileSystem.get(conf); - FaultyDistributedFileSystem faultyFs = (FaultyDistributedFileSystem)fs; - Path testDir = FSUtils.getRootDir(conf); - Path mobTestDir = new Path(testDir, MobConstants.MOB_DIR_NAME); - basePath = new Path(new Path(mobTestDir, tableName), family); - - try { - int count = 2; - // create 2 mob files. - createStoreFiles(basePath, family, qf, count, Type.Put, true, new Date()); - listFiles(); - - TableName tName = TableName.valueOf(tableName); - MobCompactor compactor = new PartitionedMobCompactor(conf, faultyFs, tName, hcd, pool); - faultyFs.setThrowException(true); - try { - compactor.compact(allFiles, true); - } catch (IOException e) { - System.out.println("Expected exception, ignore"); - } - - // Verify that all the files in tmp directory are cleaned up - Path tempPath = new Path(MobUtils.getMobHome(conf), MobConstants.TEMP_DIR_NAME); - FileStatus[] ls = faultyFs.listStatus(tempPath); - - // Only .bulkload under this directory - assertTrue(ls.length == 1); - assertTrue(MobConstants.BULKLOAD_DIR_NAME.equalsIgnoreCase(ls[0].getPath().getName())); - - Path bulkloadPath = new Path(tempPath, new Path(MobConstants.BULKLOAD_DIR_NAME, new Path( - tName.getNamespaceAsString(), tName.getQualifierAsString()))); - - // Nothing in bulkLoad directory - FileStatus[] lsBulkload = faultyFs.listStatus(bulkloadPath); - assertTrue(lsBulkload.length == 0); - - } finally { - faultyFs.setThrowException(false); - } - } - - /** - * Create mulitple partition files - */ - private void createMobFile(Path basePath) throws IOException { - HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build(); - MobFileName mobFileName = null; - int ii = 0; - Date today = new Date(); - for (byte k0 : KEYS) { - byte[] startRow = Bytes.toBytes(ii++); - - mobFileName = MobFileName.create(startRow, MobUtils.formatDate(today), mobSuffix); - - StoreFileWriter mobFileWriter = - new StoreFileWriter.Builder(conf, cacheConf, fs).withFileContext(meta) - .withFilePath(new Path(basePath, mobFileName.getFileName())).build(); - - long now = System.currentTimeMillis(); - try { - for (int i = 0; i < 10; i++) { - byte[] key = Bytes.add(Bytes.toBytes(k0), Bytes.toBytes(i)); - byte[] dummyData = new byte[5000]; - new Random().nextBytes(dummyData); - mobFileWriter.append( - new KeyValue(key, Bytes.toBytes(family), Bytes.toBytes(qf), now, Type.Put, dummyData)); - } - } finally { - mobFileWriter.close(); - } - } - } - - /** - * Create mulitple partition delete files - */ - private void createMobDelFile(Path basePath, int startKey) throws IOException { - HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build(); - MobFileName mobFileName = null; - Date today = new Date(); - - byte[] startRow = Bytes.toBytes(startKey); - - mobFileName = MobFileName.create(startRow, MobUtils.formatDate(today), delSuffix); - - StoreFileWriter mobFileWriter = - new StoreFileWriter.Builder(conf, cacheConf, fs).withFileContext(meta) - .withFilePath(new Path(basePath, mobFileName.getFileName())).build(); - - long now = System.currentTimeMillis(); - try { - byte[] key = Bytes.add(Bytes.toBytes(KEYS[startKey]), Bytes.toBytes(0)); - byte[] dummyData = new byte[5000]; - new Random().nextBytes(dummyData); - mobFileWriter.append( - new KeyValue(key, Bytes.toBytes(family), Bytes.toBytes(qf), now, Type.Delete, dummyData)); - key = Bytes.add(Bytes.toBytes(KEYS[startKey]), Bytes.toBytes(2)); - mobFileWriter.append( - new KeyValue(key, Bytes.toBytes(family), Bytes.toBytes(qf), now, Type.Delete, dummyData)); - key = Bytes.add(Bytes.toBytes(KEYS[startKey]), Bytes.toBytes(4)); - mobFileWriter.append( - new KeyValue(key, Bytes.toBytes(family), Bytes.toBytes(qf), now, Type.Delete, dummyData)); - - } finally { - mobFileWriter.close(); - } - } - - @Test - public void testCompactFilesWithoutDelFile() throws Exception { - String tableName = "testCompactFilesWithoutDelFile"; - resetConf(); - init(tableName); - - createMobFile(basePath); - - listFiles(); - - PartitionedMobCompactor compactor = new PartitionedMobCompactor(conf, fs, - TableName.valueOf(tableName), hcd, pool) { - @Override - public List compact(List files, boolean isForceAllFiles) - throws IOException { - if (files == null || files.isEmpty()) { - return null; - } - - PartitionedMobCompactionRequest request = select(files, isForceAllFiles); - - // Make sure that there is no del Partitions - assertTrue(request.getDelPartitions().size() == 0); - - // Make sure that when there is no startKey/endKey for partition. - for (CompactionPartition p : request.getCompactionPartitions()) { - assertTrue(p.getStartKey() == null); - assertTrue(p.getEndKey() == null); - } - return null; - } - }; - - compactor.compact(allFiles, true); - } - - static class MyPartitionedMobCompactor extends PartitionedMobCompactor { - int delPartitionSize = 0; - int PartitionsIncludeDelFiles = 0; - CacheConfig cacheConfig = null; - - MyPartitionedMobCompactor(Configuration conf, FileSystem fs, TableName tableName, - ColumnFamilyDescriptor column, ExecutorService pool, final int delPartitionSize, - final CacheConfig cacheConf, final int PartitionsIncludeDelFiles) - throws IOException { - super(conf, fs, tableName, column, pool); - this.delPartitionSize = delPartitionSize; - this.cacheConfig = cacheConf; - this.PartitionsIncludeDelFiles = PartitionsIncludeDelFiles; - } - - @Override public List compact(List files, boolean isForceAllFiles) - throws IOException { - if (files == null || files.isEmpty()) { - return null; - } - PartitionedMobCompactionRequest request = select(files, isForceAllFiles); - - assertTrue(request.getDelPartitions().size() == delPartitionSize); - if (request.getDelPartitions().size() > 0) { - for (CompactionPartition p : request.getCompactionPartitions()) { - assertTrue(p.getStartKey() != null); - assertTrue(p.getEndKey() != null); - } - } - - try { - for (CompactionDelPartition delPartition : request.getDelPartitions()) { - for (Path newDelPath : delPartition.listDelFiles()) { - HStoreFile sf = - new HStoreFile(fs, newDelPath, conf, this.cacheConfig, BloomType.NONE, true); - // pre-create reader of a del file to avoid race condition when opening the reader in - // each partition. - sf.initReader(); - delPartition.addStoreFile(sf); - } - } - - // Make sure that CompactionDelPartitions does not overlap - CompactionDelPartition prevDelP = null; - for (CompactionDelPartition delP : request.getDelPartitions()) { - assertTrue( - Bytes.compareTo(delP.getId().getStartKey(), delP.getId().getEndKey()) <= 0); - - if (prevDelP != null) { - assertTrue( - Bytes.compareTo(prevDelP.getId().getEndKey(), delP.getId().getStartKey()) < 0); - } - } - - int affectedPartitions = 0; - - // Make sure that only del files within key range for a partition is included in compaction. - // compact the mob files by partitions in parallel. - for (CompactionPartition partition : request.getCompactionPartitions()) { - List delFiles = getListOfDelFilesForPartition(partition, request.getDelPartitions()); - if (!request.getDelPartitions().isEmpty()) { - if (!((Bytes.compareTo(request.getDelPartitions().get(0).getId().getStartKey(), - partition.getEndKey()) > 0) || (Bytes.compareTo( - request.getDelPartitions().get(request.getDelPartitions().size() - 1).getId() - .getEndKey(), partition.getStartKey()) < 0))) { - - if (delFiles.size() > 0) { - assertTrue(delFiles.size() == 1); - affectedPartitions += delFiles.size(); - assertTrue(Bytes.compareTo(partition.getStartKey(), - CellUtil.cloneRow(delFiles.get(0).getLastKey().get())) <= 0); - assertTrue(Bytes.compareTo(partition.getEndKey(), - CellUtil.cloneRow(delFiles.get(delFiles.size() - 1).getFirstKey().get())) >= 0); - } - } - } - } - // The del file is only included in one partition - assertTrue(affectedPartitions == PartitionsIncludeDelFiles); - } finally { - for (CompactionDelPartition delPartition : request.getDelPartitions()) { - for (HStoreFile storeFile : delPartition.getStoreFiles()) { - try { - storeFile.closeStoreFile(true); - } catch (IOException e) { - LOG.warn("Failed to close the reader on store file " + storeFile.getPath(), e); - } - } - } - } - - return null; - } - } - - @Test - public void testCompactFilesWithOneDelFile() throws Exception { - String tableName = "testCompactFilesWithOneDelFile"; - resetConf(); - init(tableName); - - // Create only del file. - createMobFile(basePath); - createMobDelFile(basePath, 2); - - listFiles(); - - MyPartitionedMobCompactor compactor = new MyPartitionedMobCompactor(conf, fs, - TableName.valueOf(tableName), hcd, pool, 1, cacheConf, 1); - - compactor.compact(allFiles, true); - } - - @Test - public void testCompactFilesWithMultiDelFiles() throws Exception { - String tableName = "testCompactFilesWithMultiDelFiles"; - resetConf(); - init(tableName); - - // Create only del file. - createMobFile(basePath); - createMobDelFile(basePath, 0); - createMobDelFile(basePath, 1); - createMobDelFile(basePath, 2); - - listFiles(); - - MyPartitionedMobCompactor compactor = new MyPartitionedMobCompactor(conf, fs, - TableName.valueOf(tableName), hcd, pool, 3, cacheConf, 3); - compactor.compact(allFiles, true); - } - - private void testCompactDelFilesAtBatchSize(String tableName, int batchSize, - int delfileMaxCount) throws Exception { - resetConf(); - init(tableName); - // create 20 mob files. - createStoreFiles(basePath, family, qf, 20, Type.Put, new Date()); - // create 13 del files - createStoreFiles(basePath, family, qf, 13, Type.Delete, new Date()); - listFiles(); - - // set the max del file count - conf.setInt(MobConstants.MOB_DELFILE_MAX_COUNT, delfileMaxCount); - // set the mob compaction batch size - conf.setInt(MobConstants.MOB_COMPACTION_BATCH_SIZE, batchSize); - testCompactDelFiles(tableName, 1, 13, false); - } - - /** - * Tests the selectFiles - * @param tableName the table name - * @param type the expected compaction type - * @param isForceAllFiles whether all the mob files are selected - * @param expected the expected start keys - */ - private void testSelectFiles(String tableName, final CompactionType type, - final boolean isForceAllFiles, final List expected) throws IOException { - PartitionedMobCompactor compactor = new PartitionedMobCompactor(conf, fs, - TableName.valueOf(tableName), hcd, pool) { - @Override - public List compact(List files, boolean isForceAllFiles) - throws IOException { - if (files == null || files.isEmpty()) { - return null; - } - PartitionedMobCompactionRequest request = select(files, isForceAllFiles); - - // Make sure that when there is no del files, there will be no startKey/endKey for partition. - if (request.getDelPartitions().size() == 0) { - for (CompactionPartition p : request.getCompactionPartitions()) { - assertTrue(p.getStartKey() == null); - assertTrue(p.getEndKey() == null); - } - } - - // Make sure that CompactionDelPartitions does not overlap - CompactionDelPartition prevDelP = null; - for (CompactionDelPartition delP : request.getDelPartitions()) { - assertTrue(Bytes.compareTo(delP.getId().getStartKey(), - delP.getId().getEndKey()) <= 0); - - if (prevDelP != null) { - assertTrue(Bytes.compareTo(prevDelP.getId().getEndKey(), - delP.getId().getStartKey()) < 0); - } - } - - // Make sure that only del files within key range for a partition is included in compaction. - // compact the mob files by partitions in parallel. - for (CompactionPartition partition : request.getCompactionPartitions()) { - List delFiles = getListOfDelFilesForPartition(partition, request.getDelPartitions()); - if (!request.getDelPartitions().isEmpty()) { - if (!((Bytes.compareTo(request.getDelPartitions().get(0).getId().getStartKey(), - partition.getEndKey()) > 0) || (Bytes.compareTo( - request.getDelPartitions().get(request.getDelPartitions().size() - 1).getId() - .getEndKey(), partition.getStartKey()) < 0))) { - if (delFiles.size() > 0) { - assertTrue(Bytes.compareTo(partition.getStartKey(), - delFiles.get(0).getFirstKey().get().getRowArray()) >= 0); - assertTrue(Bytes.compareTo(partition.getEndKey(), - delFiles.get(delFiles.size() - 1).getLastKey().get().getRowArray()) <= 0); - } - } - } - } - - // assert the compaction type - assertEquals(type, request.type); - // assert get the right partitions - compareCompactedPartitions(expected, request.compactionPartitions); - // assert get the right del files - compareDelFiles(request.getDelPartitions()); - return null; - } - }; - compactor.compact(allFiles, isForceAllFiles); - } - - /** - * Tests the compacteDelFile - * @param tableName the table name - * @param expectedFileCount the expected file count - * @param expectedCellCount the expected cell count - * @param isForceAllFiles whether all the mob files are selected - */ - private void testCompactDelFiles(String tableName, final int expectedFileCount, - final int expectedCellCount, boolean isForceAllFiles) throws IOException { - PartitionedMobCompactor compactor = new PartitionedMobCompactor(conf, fs, - TableName.valueOf(tableName), hcd, pool) { - @Override - protected List performCompaction(PartitionedMobCompactionRequest request) - throws IOException { - List delFilePaths = new ArrayList<>(); - for (CompactionDelPartition delPartition: request.getDelPartitions()) { - for (Path p : delPartition.listDelFiles()) { - delFilePaths.add(p); - } - } - List newDelPaths = compactDelFiles(request, delFilePaths); - // assert the del files are merged. - assertEquals(expectedFileCount, newDelPaths.size()); - assertEquals(expectedCellCount, countDelCellsInDelFiles(newDelPaths)); - return null; - } - }; - compactor.compact(allFiles, isForceAllFiles); - } - - /** - * Lists the files in the path - */ - private void listFiles() throws IOException { - for (FileStatus file : fs.listStatus(basePath)) { - allFiles.add(file); - if (file.getPath().getName().endsWith("_del")) { - delFiles.add(file.getPath()); - } else { - mobFiles.add(file); - } - } - } - - /** - * Compares the compacted partitions. - * @param partitions the collection of CompactedPartitions - */ - private void compareCompactedPartitions(List expected, - Collection partitions) { - List actualKeys = new ArrayList<>(); - for (CompactionPartition partition : partitions) { - actualKeys.add(partition.getPartitionId().getStartKey()); - } - Collections.sort(expected); - Collections.sort(actualKeys); - assertEquals(expected.size(), actualKeys.size()); - for (int i = 0; i < expected.size(); i++) { - assertEquals(expected.get(i), actualKeys.get(i)); - } - } - - /** - * Compares the del files. - * @param delPartitions all del partitions - */ - private void compareDelFiles(List delPartitions) { - Map delMap = new HashMap<>(); - for (CompactionDelPartition delPartition : delPartitions) { - for (Path f : delPartition.listDelFiles()) { - delMap.put(f, f); - } - } - for (Path f : delFiles) { - assertTrue(delMap.containsKey(f)); - } - } - - /** - * Creates store files. - * @param basePath the path to create file - * @param family the column family name - * @param qualifier the column qualifier assigned to data values - * @param count the store file number - * @param type the row key type - * @param date the latest timestamp when an instance of MobFileName is created - */ - private void createStoreFiles(Path basePath, String family, String qualifier, int count, - Type type, final Date date) throws IOException { - createStoreFiles(basePath, family, qualifier, count, type, false, date); - } - - private void createStoreFiles(Path basePath, String family, String qualifier, int count, - Type type, boolean sameStartKey, final Date date) throws IOException { - HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build(); - String startKey = "row_"; - MobFileName mobFileName = null; - for (int i = 0; i < count; i++) { - byte[] startRow; - if (sameStartKey) { - // When creating multiple files under one partition, suffix needs to be different. - startRow = Bytes.toBytes(startKey); - mobSuffix = TEST_UTIL.getRandomUUID().toString().replaceAll("-", ""); - delSuffix = TEST_UTIL.getRandomUUID().toString().replaceAll("-", "") + "_del"; - } else { - startRow = Bytes.toBytes(startKey + i); - } - if(type.equals(Type.Delete)) { - mobFileName = MobFileName.create(startRow, MobUtils.formatDate(date), delSuffix); - } - if(type.equals(Type.Put)){ - mobFileName = MobFileName.create(startRow, MobUtils.formatDate(date), mobSuffix); - } - StoreFileWriter mobFileWriter = new StoreFileWriter.Builder(conf, cacheConf, fs) - .withFileContext(meta).withFilePath(new Path(basePath, mobFileName.getFileName())).build(); - writeStoreFile(mobFileWriter, startRow, Bytes.toBytes(family), Bytes.toBytes(qualifier), - type, (i+1)*1000); - } - } - - /** - * Writes data to store file. - * @param writer the store file writer - * @param row the row key - * @param family the family name - * @param qualifier the column qualifier - * @param type the key type - * @param size the size of value - */ - private static void writeStoreFile(final StoreFileWriter writer, byte[]row, byte[] family, - byte[] qualifier, Type type, int size) throws IOException { - long now = System.currentTimeMillis(); - try { - byte[] dummyData = new byte[size]; - new Random().nextBytes(dummyData); - writer.append(new KeyValue(row, family, qualifier, now, type, dummyData)); - } finally { - writer.close(); - } - } - - /** - * Gets the number of del cell in the del files - * @param paths the del file paths - * @return the cell size - */ - private int countDelCellsInDelFiles(List paths) throws IOException { - List sfs = new ArrayList<>(); - int size = 0; - for (Path path : paths) { - HStoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true); - sfs.add(sf); - } - List scanners = new ArrayList<>(StoreFileScanner.getScannersForStoreFiles(sfs, - false, true, false, false, HConstants.LATEST_TIMESTAMP)); - long timeToPurgeDeletes = Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0); - long ttl = HStore.determineTTLFromFamily(hcd); - ScanInfo scanInfo = new ScanInfo(conf, hcd, ttl, timeToPurgeDeletes, CellComparatorImpl.COMPARATOR); - StoreScanner scanner = new StoreScanner(scanInfo, ScanType.COMPACT_RETAIN_DELETES, scanners); - List results = new ArrayList<>(); - boolean hasMore = true; - - while (hasMore) { - hasMore = scanner.next(results); - size += results.size(); - results.clear(); - } - scanner.close(); - return size; - } - - private static ExecutorService createThreadPool() { - int maxThreads = 10; - long keepAliveTime = 60; - final SynchronousQueue queue = new SynchronousQueue<>(); - ThreadPoolExecutor pool = new ThreadPoolExecutor(1, maxThreads, keepAliveTime, - TimeUnit.SECONDS, queue, Threads.newDaemonThreadFactory("MobFileCompactionChore"), - new RejectedExecutionHandler() { - @Override - public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) { - try { - // waiting for a thread to pick up instead of throwing exceptions. - queue.put(r); - } catch (InterruptedException e) { - throw new RejectedExecutionException(e); - } - } - }); - ((ThreadPoolExecutor) pool).allowCoreThreadTimeOut(true); - return pool; - } - - /** - * Resets the configuration. - */ - private void resetConf() { - conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, - MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD); - conf.setInt(MobConstants.MOB_DELFILE_MAX_COUNT, MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT); - conf.setInt(MobConstants.MOB_COMPACTION_BATCH_SIZE, - MobConstants.DEFAULT_MOB_COMPACTION_BATCH_SIZE); - } - - /** - * The customized Distributed File System Implementation - */ - static class FaultyDistributedFileSystem extends DistributedFileSystem { - private volatile boolean throwException = false; - - public FaultyDistributedFileSystem() { - super(); - } - - public void setThrowException(boolean throwException) { - this.throwException = throwException; - } - - @Override - public boolean rename(Path src, Path dst) throws IOException { - if (throwException) { - throw new IOException("No more files allowed"); - } - return super.rename(src, dst); - } - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java index e8096fb95106..727f6febc366 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java @@ -402,7 +402,7 @@ String createMobFileName(String oldFileName) { String startKey = mobFileName.getStartKey(); String date = mobFileName.getDate(); return MobFileName.create(startKey, date, - TEST_UTIL.getRandomUUID().toString().replaceAll("-", "")) + TEST_UTIL.getRandomUUID().toString().replaceAll("-", ""), "fakeregion") .getFileName(); } From d8e70189a8ff583ef4d908e0e9f9c4ade30ab685 Mon Sep 17 00:00:00 2001 From: Vladimir Rodionov Date: Mon, 14 Oct 2019 12:23:41 -0700 Subject: [PATCH 02/15] Revert "HBASE-22749: Distributed MOB compactions" This reverts commit b1b3bbe140c44ef6417ef845fc8e0fb852c01c36. --- ...e.java => ExpiredMobFileCleanerChore.java} | 24 +- .../apache/hadoop/hbase/master/HMaster.java | 33 +- .../master/MasterMobCompactionThread.java | 181 +++ .../hbase/master/MasterRpcServices.java | 90 +- .../hbase/master/MobCompactionChore.java | 96 ++ .../hbase/master/MobFileCompactionChore.java | 179 --- .../hbase/mob/DefaultMobStoreCompactor.java | 654 +-------- .../hbase/mob/DefaultMobStoreFlusher.java | 27 - ...erTool.java => ExpiredMobFileCleaner.java} | 8 +- .../apache/hadoop/hbase/mob/MobConstants.java | 99 +- .../apache/hadoop/hbase/mob/MobFileName.java | 33 +- .../hadoop/hbase/mob/MobStoreEngine.java | 12 +- .../org/apache/hadoop/hbase/mob/MobUtils.java | 610 ++++----- .../mob/compactions/MobCompactionRequest.java | 64 + .../hbase/mob/compactions/MobCompactor.java | 90 ++ .../PartitionedMobCompactionRequest.java | 333 +++++ .../compactions/PartitionedMobCompactor.java | 929 +++++++++++++ .../hadoop/hbase/regionserver/HMobStore.java | 41 +- .../hadoop/hbase/regionserver/HStore.java | 2 +- .../hadoop/hbase/regionserver/HStoreFile.java | 3 - .../hbase/regionserver/MobStoreScanner.java | 1 - .../hbase/regionserver/StoreFileInfo.java | 13 +- .../hbase/regionserver/StoreFileWriter.java | 23 - .../regionserver/compactions/Compactor.java | 2 +- .../hbase/mob/FaultyMobStoreCompactor.java | 355 ----- .../hadoop/hbase/mob/MobStressTool.java | 80 -- .../hbase/mob/TestExpiredMobFileCleaner.java | 2 +- .../hadoop/hbase/mob/TestMobCompaction.java | 344 ----- .../hadoop/hbase/mob/TestMobFileName.java | 19 +- .../mob/compactions/TestMobCompactor.java | 1202 +++++++++++++++++ .../TestPartitionedMobCompactionRequest.java | 65 + .../TestPartitionedMobCompactor.java | 958 +++++++++++++ .../hadoop/hbase/util/BaseTestHBaseFsck.java | 2 +- 33 files changed, 4468 insertions(+), 2106 deletions(-) rename hbase-server/src/main/java/org/apache/hadoop/hbase/master/{MobFileCleanerChore.java => ExpiredMobFileCleanerChore.java} (80%) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobCompactionChore.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java rename hbase-server/src/main/java/org/apache/hadoop/hbase/mob/{MobFileCleanerTool.java => ExpiredMobFileCleaner.java} (94%) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactionRequest.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactionRequest.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactionRequest.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java similarity index 80% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java index f6207e024e2f..d37a80a1c3db 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java @@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.master.locking.LockManager; -import org.apache.hadoop.hbase.mob.MobFileCleanerTool; +import org.apache.hadoop.hbase.mob.ExpiredMobFileCleaner; import org.apache.hadoop.hbase.mob.MobConstants; import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.procedure2.LockType; @@ -39,33 +39,27 @@ * mob files. */ @InterfaceAudience.Private -public class MobFileCleanerChore extends ScheduledChore { +public class ExpiredMobFileCleanerChore extends ScheduledChore { - private static final Logger LOG = LoggerFactory.getLogger(MobFileCleanerChore.class); + private static final Logger LOG = LoggerFactory.getLogger(ExpiredMobFileCleanerChore.class); private final HMaster master; - private MobFileCleanerTool cleaner; - private volatile boolean running = false; + private ExpiredMobFileCleaner cleaner; - public MobFileCleanerChore(HMaster master) { + public ExpiredMobFileCleanerChore(HMaster master) { super(master.getServerName() + "-ExpiredMobFileCleanerChore", master, master.getConfiguration() .getInt(MobConstants.MOB_CLEANER_PERIOD, MobConstants.DEFAULT_MOB_CLEANER_PERIOD), master .getConfiguration().getInt(MobConstants.MOB_CLEANER_PERIOD, MobConstants.DEFAULT_MOB_CLEANER_PERIOD), TimeUnit.SECONDS); this.master = master; - cleaner = new MobFileCleanerTool(); + cleaner = new ExpiredMobFileCleaner(); cleaner.setConf(master.getConfiguration()); } @Override @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION", justification="Intentional") - protected void chore() { try { - if (running) { - LOG.warn(getName() +" is running already, skipping this attempt."); - return; - } TableDescriptors htds = master.getTableDescriptors(); Map map = htds.getAll(); for (TableDescriptor htd : map.values()) { @@ -84,15 +78,9 @@ protected void chore() { } } } - // Now clean obsolete files for a table - LOG.info("Cleaning obsolete MOB files ..."); - MobUtils.cleanupObsoleteMobFiles(master.getConfiguration(), htd.getTableName()); - LOG.info("Cleaning obsolete MOB files finished"); } } catch (Exception e) { LOG.error("Fail to clean the expired mob files", e); - } finally { - running = false; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index ee41974b6f88..5939053aec84 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -394,8 +394,9 @@ public void run() { private LogCleaner logCleaner; private HFileCleaner hfileCleaner; private ReplicationBarrierCleaner replicationBarrierCleaner; - private MobFileCleanerChore mobFileCleanerChore; - private MobFileCompactionChore mobFileCompactionChore; + private ExpiredMobFileCleanerChore expiredMobFileCleanerChore; + private MobCompactionChore mobCompactChore; + private MasterMobCompactionThread mobCompactThread; // used to synchronize the mobCompactionStates private final IdLock mobCompactionLock = new IdLock(); // save the information of mob compactions in tables. @@ -1298,18 +1299,19 @@ public void updateConfigurationForQuotasObserver(Configuration conf) { } private void initMobCleaner() { - this.mobFileCleanerChore = new MobFileCleanerChore(this); - getChoreService().scheduleChore(mobFileCleanerChore); + this.expiredMobFileCleanerChore = new ExpiredMobFileCleanerChore(this); + getChoreService().scheduleChore(expiredMobFileCleanerChore); int mobCompactionPeriod = conf.getInt(MobConstants.MOB_COMPACTION_CHORE_PERIOD, MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD); if (mobCompactionPeriod > 0) { - this.mobFileCompactionChore = new MobFileCompactionChore(this); - getChoreService().scheduleChore(mobFileCompactionChore); + this.mobCompactChore = new MobCompactionChore(this, mobCompactionPeriod); + getChoreService().scheduleChore(mobCompactChore); } else { LOG .info("The period is " + mobCompactionPeriod + " seconds, MobCompactionChore is disabled"); } + this.mobCompactThread = new MasterMobCompactionThread(this); } /** @@ -1497,7 +1499,9 @@ protected void stopServiceThreads() { } } stopChores(); - + if (this.mobCompactThread != null) { + this.mobCompactThread.close(); + } super.stopServiceThreads(); if (cleanerPool != null) { cleanerPool.shutdownNow(); @@ -1618,8 +1622,8 @@ private void stopProcedureExecutor() { private void stopChores() { ChoreService choreService = getChoreService(); if (choreService != null) { - choreService.cancelChore(this.mobFileCleanerChore); - choreService.cancelChore(this.mobFileCompactionChore); + choreService.cancelChore(this.expiredMobFileCleanerChore); + choreService.cancelChore(this.mobCompactChore); choreService.cancelChore(this.balancerChore); choreService.cancelChore(this.normalizerChore); choreService.cancelChore(this.clusterStatusChore); @@ -3426,6 +3430,17 @@ public void reportMobCompactionEnd(TableName tableName) throws IOException { } } + /** + * Requests mob compaction. + * @param tableName The table the compact. + * @param columns The compacted columns. + * @param allFiles Whether add all mob files into the compaction. + */ + public void requestMobCompaction(TableName tableName, + List columns, boolean allFiles) throws IOException { + mobCompactThread.requestMobCompaction(conf, fs, tableName, columns, allFiles); + } + /** * Queries the state of the {@link LoadBalancerTracker}. If the balancer is not initialized, * false is returned. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java new file mode 100644 index 000000000000..0779eeafe8b0 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java @@ -0,0 +1,181 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.master; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.master.locking.LockManager; +import org.apache.hadoop.hbase.mob.MobUtils; +import org.apache.hadoop.hbase.procedure2.LockType; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; + +/** + * The mob compaction thread used in {@link MasterRpcServices} + */ +@InterfaceAudience.Private +public class MasterMobCompactionThread { + static final Logger LOG = LoggerFactory.getLogger(MasterMobCompactionThread.class); + private final HMaster master; + private final Configuration conf; + private final ExecutorService mobCompactorPool; + private final ExecutorService masterMobPool; + + public MasterMobCompactionThread(HMaster master) { + this.master = master; + this.conf = master.getConfiguration(); + final String n = Thread.currentThread().getName(); + // this pool is used to run the mob compaction + this.masterMobPool = new ThreadPoolExecutor(1, 2, 60, + TimeUnit.SECONDS, new SynchronousQueue<>(), + new ThreadFactoryBuilder().setDaemon(true) + .setNameFormat(n + "-MasterMobCompaction-" + EnvironmentEdgeManager.currentTime()) + .build()); + ((ThreadPoolExecutor) this.masterMobPool).allowCoreThreadTimeOut(true); + // this pool is used in the mob compaction to compact the mob files by partitions + // in parallel + this.mobCompactorPool = MobUtils + .createMobCompactorThreadPool(master.getConfiguration()); + } + + /** + * Requests mob compaction + * @param conf The Configuration + * @param fs The file system + * @param tableName The table the compact + * @param columns The column descriptors + * @param allFiles Whether add all mob files into the compaction. + */ + public void requestMobCompaction(Configuration conf, FileSystem fs, TableName tableName, + List columns, boolean allFiles) throws IOException { + master.reportMobCompactionStart(tableName); + try { + masterMobPool.execute(new CompactionRunner(fs, tableName, columns, + allFiles, mobCompactorPool)); + } catch (RejectedExecutionException e) { + // in case the request is rejected by the pool + try { + master.reportMobCompactionEnd(tableName); + } catch (IOException e1) { + LOG.error("Failed to mark end of mob compaction", e1); + } + throw e; + } + if (LOG.isDebugEnabled()) { + LOG.debug("The mob compaction is requested for the columns " + columns + + " of the table " + tableName.getNameAsString()); + } + } + + private class CompactionRunner implements Runnable { + private FileSystem fs; + private TableName tableName; + private List hcds; + private boolean allFiles; + private ExecutorService pool; + + public CompactionRunner(FileSystem fs, TableName tableName, List hcds, + boolean allFiles, ExecutorService pool) { + super(); + this.fs = fs; + this.tableName = tableName; + this.hcds = hcds; + this.allFiles = allFiles; + this.pool = pool; + } + + @Override + public void run() { + // These locks are on dummy table names, and only used for compaction/mob file cleaning. + final LockManager.MasterLock lock = master.getLockManager().createMasterLock( + MobUtils.getTableLockName(tableName), LockType.EXCLUSIVE, + this.getClass().getName() + ": mob compaction"); + try { + for (ColumnFamilyDescriptor hcd : hcds) { + MobUtils.doMobCompaction(conf, fs, tableName, hcd, pool, allFiles, lock); + } + } catch (IOException e) { + LOG.error("Failed to perform the mob compaction", e); + } finally { + try { + master.reportMobCompactionEnd(tableName); + } catch (IOException e) { + LOG.error("Failed to mark end of mob compaction", e); + } + } + } + } + + /** + * Only interrupt once it's done with a run through the work loop. + */ + private void interruptIfNecessary() { + mobCompactorPool.shutdown(); + masterMobPool.shutdown(); + } + + /** + * Wait for all the threads finish. + */ + private void join() { + waitFor(mobCompactorPool, "Mob Compaction Thread"); + waitFor(masterMobPool, "Region Server Mob Compaction Thread"); + } + + /** + * Closes the MasterMobCompactionThread. + */ + public void close() { + interruptIfNecessary(); + join(); + } + + /** + * Wait for thread finish. + * @param t the thread to wait + * @param name the thread name. + */ + private void waitFor(ExecutorService t, String name) { + boolean done = false; + while (!done) { + try { + done = t.awaitTermination(60, TimeUnit.SECONDS); + LOG.info("Waiting for " + name + " to finish..."); + if (!done) { + t.shutdownNow(); + } + } catch (InterruptedException ie) { + LOG.warn("Interrupted waiting for " + name + " to finish..."); + } + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java index a411e85c47e7..06a99fa5432d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java @@ -32,7 +32,6 @@ import java.util.Map.Entry; import java.util.Set; import java.util.stream.Collectors; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ClusterMetricsBuilder; @@ -46,6 +45,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.UnknownRegionException; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.MasterSwitchType; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; @@ -103,6 +103,23 @@ import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil; import org.apache.hadoop.hbase.security.access.UserPermission; import org.apache.hadoop.hbase.security.visibility.VisibilityController; +import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; +import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.ForeignExceptionUtil; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; +import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.zookeeper.KeeperException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; +import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter; import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos; @@ -325,21 +342,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription; -import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; -import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.ForeignExceptionUtil; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; -import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; -import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; -import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; -import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.zookeeper.KeeperException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Implements the master RPC services. @@ -1742,13 +1744,10 @@ public CompactRegionResponse compactRegion(final RpcController controller, master.checkInitialized(); byte[] regionName = request.getRegion().getValue().toByteArray(); TableName tableName = RegionInfo.getTable(regionName); - // TODO: support CompactType.MOB // if the region is a mob region, do the mob file compaction. if (MobUtils.isMobRegionName(tableName, regionName)) { checkHFileFormatVersionForMob(); - //return compactMob(request, tableName); - //TODO: support CompactType.MOB - return super.compactRegion(controller, request); + return compactMob(request, tableName); } else { return super.compactRegion(controller, request); } @@ -1791,6 +1790,57 @@ public GetRegionInfoResponse getRegionInfo(final RpcController controller, } } + /** + * Compacts the mob files in the current table. + * @param request the request. + * @param tableName the current table name. + * @return The response of the mob file compaction. + * @throws IOException + */ + private CompactRegionResponse compactMob(final CompactRegionRequest request, + TableName tableName) throws IOException { + if (!master.getTableStateManager().isTableState(tableName, TableState.State.ENABLED)) { + throw new DoNotRetryIOException("Table " + tableName + " is not enabled"); + } + boolean allFiles = false; + List compactedColumns = new ArrayList<>(); + ColumnFamilyDescriptor[] hcds = master.getTableDescriptors().get(tableName).getColumnFamilies(); + byte[] family = null; + if (request.hasFamily()) { + family = request.getFamily().toByteArray(); + for (ColumnFamilyDescriptor hcd : hcds) { + if (Bytes.equals(family, hcd.getName())) { + if (!hcd.isMobEnabled()) { + LOG.error("Column family " + hcd.getNameAsString() + " is not a mob column family"); + throw new DoNotRetryIOException("Column family " + hcd.getNameAsString() + + " is not a mob column family"); + } + compactedColumns.add(hcd); + } + } + } else { + for (ColumnFamilyDescriptor hcd : hcds) { + if (hcd.isMobEnabled()) { + compactedColumns.add(hcd); + } + } + } + if (compactedColumns.isEmpty()) { + LOG.error("No mob column families are assigned in the mob compaction"); + throw new DoNotRetryIOException( + "No mob column families are assigned in the mob compaction"); + } + if (request.hasMajor() && request.getMajor()) { + allFiles = true; + } + String familyLogMsg = (family != null) ? Bytes.toString(family) : ""; + if (LOG.isTraceEnabled()) { + LOG.trace("User-triggered mob compaction requested for table: " + + tableName.getNameAsString() + " for column family: " + familyLogMsg); + } + master.requestMobCompaction(tableName, compactedColumns, allFiles); + return CompactRegionResponse.newBuilder().build(); + } @Override public IsBalancerEnabledResponse isBalancerEnabled(RpcController controller, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobCompactionChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobCompactionChore.java new file mode 100644 index 000000000000..6c5d677a86e5 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobCompactionChore.java @@ -0,0 +1,96 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.master; + +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.hbase.ScheduledChore; +import org.apache.hadoop.hbase.TableDescriptors; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableState; +import org.apache.hadoop.hbase.master.locking.LockManager; +import org.apache.hadoop.hbase.mob.MobUtils; +import org.apache.hadoop.hbase.procedure2.LockType; + +/** + * The Class MobCompactChore for running compaction regularly to merge small mob files. + */ +@InterfaceAudience.Private +public class MobCompactionChore extends ScheduledChore { + + private static final Logger LOG = LoggerFactory.getLogger(MobCompactionChore.class); + private HMaster master; + private ExecutorService pool; + + public MobCompactionChore(HMaster master, int period) { + // use the period as initial delay. + super(master.getServerName() + "-MobCompactionChore", master, period, period, TimeUnit.SECONDS); + this.master = master; + this.pool = MobUtils.createMobCompactorThreadPool(master.getConfiguration()); + } + + @Override + protected void chore() { + try { + TableDescriptors htds = master.getTableDescriptors(); + Map map = htds.getAll(); + for (TableDescriptor htd : map.values()) { + if (!master.getTableStateManager().isTableState(htd.getTableName(), + TableState.State.ENABLED)) { + continue; + } + boolean reported = false; + try { + final LockManager.MasterLock lock = master.getLockManager().createMasterLock( + MobUtils.getTableLockName(htd.getTableName()), LockType.EXCLUSIVE, + this.getClass().getName() + ": mob compaction"); + for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) { + if (!hcd.isMobEnabled()) { + continue; + } + if (!reported) { + master.reportMobCompactionStart(htd.getTableName()); + reported = true; + } + MobUtils.doMobCompaction(master.getConfiguration(), master.getFileSystem(), + htd.getTableName(), hcd, pool, false, lock); + } + } finally { + if (reported) { + master.reportMobCompactionEnd(htd.getTableName()); + } + } + } + } catch (Exception e) { + LOG.error("Failed to compact mob files", e); + } + } + + @Override + protected synchronized void cleanup() { + super.cleanup(); + pool.shutdown(); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java deleted file mode 100644 index b2c2206382b8..000000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java +++ /dev/null @@ -1,179 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.master; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.concurrent.TimeUnit; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.ScheduledChore; -import org.apache.hadoop.hbase.TableDescriptors; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.client.CompactionState; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.RegionInfo; -import org.apache.hadoop.hbase.client.TableDescriptor; -import org.apache.hadoop.hbase.client.TableState; -import org.apache.hadoop.hbase.mob.MobConstants; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@InterfaceAudience.Private -public class MobFileCompactionChore extends ScheduledChore { - - private static final Logger LOG = LoggerFactory.getLogger(MobFileCompactionChore.class); - private final Configuration conf; - private final HMaster master; - private volatile boolean running = false; - private int regionBatchSize = 0;// not set - compact all - - public MobFileCompactionChore(HMaster master) { - super(master.getServerName() + "-MobFileCompactionChore", master, master.getConfiguration() - .getInt(MobConstants.MOB_COMPACTION_CHORE_PERIOD, - MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD), master - .getConfiguration().getInt(MobConstants.MOB_COMPACTION_CHORE_PERIOD, - MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD), TimeUnit.SECONDS); - this.master = master; - this.conf = master.getConfiguration(); - this.regionBatchSize = - master.getConfiguration().getInt(MobConstants.MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE, - MobConstants.DEFAULT_MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE); - - } - - @Override - protected void chore() { - - boolean reported = false; - - try (Connection conn = ConnectionFactory.createConnection(conf); - Admin admin = conn.getAdmin(); ) { - - if (running) { - LOG.warn(getName() +" is running already, skipping this attempt."); - return; - } - running = true; - TableDescriptors htds = master.getTableDescriptors(); - Map map = htds.getAll(); - for (TableDescriptor htd : map.values()) { - if (!master.getTableStateManager().isTableState(htd.getTableName(), - TableState.State.ENABLED)) { - continue; - } - for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) { - if (hcd.isMobEnabled()) { - if (!reported) { - master.reportMobCompactionStart(htd.getTableName()); - reported = true; - } - LOG.info(" Major compacting "+ htd.getTableName() + " cf=" + hcd.getNameAsString()); - if (regionBatchSize == MobConstants.DEFAULT_MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE) { - admin.majorCompact(htd.getTableName(), hcd.getName()); - } else { - performMajorCompactionInBatches(admin, htd, hcd); - } - } - } - if (reported) { - master.reportMobCompactionEnd(htd.getTableName()); - reported = false; - } - } - } catch (Exception e) { - LOG.error("Failed to compact", e); - } finally { - running = false; - } - } - - private void performMajorCompactionInBatches(Admin admin, TableDescriptor htd, - ColumnFamilyDescriptor hcd) throws IOException { - - List regions = admin.getRegions(htd.getTableName()); - if (regions.size() <= this.regionBatchSize) { - admin.majorCompact(htd.getTableName(), hcd.getName()); - return; - } - // Shuffle list of regions in case if they come ordered by region server - Collections.shuffle(regions); - // Create first batch - List toCompact = new ArrayList(); - for (int i=0; i < this.regionBatchSize; i++) { - toCompact.add(regions.remove(0)); - } - - // Start compaction now - for(RegionInfo ri: toCompact) { - startCompaction(admin, htd.getTableName(), ri); - } - - List compacted = new ArrayList(); - while(!toCompact.isEmpty()) { - // Check status of active compactions - for (RegionInfo ri: toCompact) { - if (admin.getCompactionStateForRegion(ri.getRegionName()) == CompactionState.NONE) { - LOG.info("Finished major compaction: table={} region={}", htd.getTableName(), - ri.getRegionNameAsString()); - compacted.add(ri); - } - } - // Update batch: remove compacted regions and add new ones - for (RegionInfo ri: compacted) { - toCompact.remove(ri); - if (regions.size() > 0) { - RegionInfo region = regions.remove(0); - startCompaction(admin, htd.getTableName(),region); - toCompact.add(region); - } - } - compacted.clear(); - try { - Thread.sleep(1000); - } catch(InterruptedException e) { - // swallow - } - } - LOG.info(" Finished major compacting "+ htd.getTableName() + " cf=" + hcd.getNameAsString()); - - } - - private void startCompaction(Admin admin, TableName table, RegionInfo region) - throws IOException { - - LOG.info("Started major compaction: table={} region={}", table, - region.getRegionNameAsString()); - admin.majorCompactRegion(region.getRegionName()); - // Wait until it really starts - while(admin.getCompactionStateForRegion(region.getRegionName()) == CompactionState.NONE) { - //TODO: what if we stuck here? - try { - Thread.sleep(10); - } catch (InterruptedException e) { - } - } - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java index 408ca4d68d44..e73a7d2226d7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java @@ -17,36 +17,21 @@ */ package org.apache.hadoop.hbase.mob; -import static org.apache.hadoop.hbase.regionserver.ScanType.COMPACT_DROP_DELETES; -import static org.apache.hadoop.hbase.regionserver.ScanType.COMPACT_RETAIN_DELETES; - -import java.io.FileNotFoundException; import java.io.IOException; import java.io.InterruptedIOException; import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; import java.util.Date; -import java.util.HashMap; -import java.util.HashSet; import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.PrivateCellUtil; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.regionserver.CellSink; import org.apache.hadoop.hbase.regionserver.HMobStore; import org.apache.hadoop.hbase.regionserver.HStore; -import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.KeyValueScanner; import org.apache.hadoop.hbase.regionserver.ScanInfo; @@ -63,7 +48,6 @@ import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -74,33 +58,16 @@ @InterfaceAudience.Private public class DefaultMobStoreCompactor extends DefaultCompactor { - protected static final Logger LOG = LoggerFactory.getLogger(DefaultMobStoreCompactor.class); - protected long mobSizeThreshold; - protected HMobStore mobStore; - - // MOB file reference set - static ThreadLocal> mobRefSet = new ThreadLocal>() { - @Override - protected Set initialValue() { - return new HashSet(); - } - }; - - static ThreadLocal userRequest = new ThreadLocal () { - @Override - protected Boolean initialValue() { - return Boolean.FALSE; - } - }; - - static ThreadLocal> compSelections = - new ThreadLocal>(); + private static final Logger LOG = LoggerFactory.getLogger(DefaultMobStoreCompactor.class); + private long mobSizeThreshold; + private HMobStore mobStore; private final InternalScannerFactory scannerFactory = new InternalScannerFactory() { @Override public ScanType getScanType(CompactionRequestImpl request) { - return request.isAllFiles() ? COMPACT_DROP_DELETES : COMPACT_RETAIN_DELETES; + // retain the delete markers until they are expired. + return ScanType.COMPACT_RETAIN_DELETES; } @Override @@ -136,34 +103,9 @@ public DefaultMobStoreCompactor(Configuration conf, HStore store) { mobSizeThreshold = store.getColumnFamilyDescriptor().getMobThreshold(); } - @Override - public List compact(CompactionRequestImpl request, - ThroughputController throughputController, User user) throws IOException { - LOG.debug("Mob compaction: major=" + request.isMajor() + " isAll=" + request.isAllFiles() - + " priority=" + request.getPriority()); - if (request.getPriority() == HStore.PRIORITY_USER) { - userRequest.set(Boolean.TRUE); - } else { - userRequest.set(Boolean.FALSE); - } - LOG.debug("Mob compaction files: " + request.getFiles()); - // Check if generational MOB compaction - compSelections.set(null); - if (conf.get(MobConstants.MOB_COMPACTION_TYPE_KEY, MobConstants.DEFAULT_MOB_COMPACTION_TYPE) - .equals(MobConstants.GENERATIONAL_MOB_COMPACTION_TYPE)) { - if (request.isMajor() && request.getPriority() == HStore.PRIORITY_USER) { - // Compact MOBs - List mobFiles = getReferencedMobFiles(request.getFiles()); - if (mobFiles.size() > 0) { - Generations gens = Generations.build(mobFiles, conf); - List list = gens.getCompactionSelections(); - if (list.size() > 0) { - compSelections.set(list); - } - } - } - } + public List compact(CompactionRequestImpl request, ThroughputController throughputController, + User user) throws IOException { return compact(request, scannerFactory, writerFactory, throughputController, user); } @@ -227,35 +169,6 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel long bytesWrittenProgressForCloseCheck = 0; long bytesWrittenProgressForLog = 0; long bytesWrittenProgressForShippedCall = 0; - // Clear old mob references - mobRefSet.get().clear(); - boolean isUserRequest = userRequest.get(); - boolean compactMOBs = major && isUserRequest; - boolean generationalMob = conf.get(MobConstants.MOB_COMPACTION_TYPE_KEY, - MobConstants.DEFAULT_MOB_COMPACTION_TYPE) - .equals(MobConstants.GENERATIONAL_MOB_COMPACTION_TYPE); - OutputMobWriters mobWriters = null; - - if (compactMOBs && generationalMob) { - List sel = compSelections.get(); - if (sel != null && sel.size() > 0) { - // Create output writers for compaction selections - mobWriters = new OutputMobWriters(sel); - int numWriters = mobWriters.getNumberOfWriters(); - List writers = new ArrayList(); - for (int i=0; i < numWriters; i++) { - StoreFileWriter sfw = mobStore.createWriterInTmp(new Date(fd.latestPutTs), fd.maxKeyCount, - compactionCompression, store.getRegionInfo().getStartKey(), true); - writers.add(sfw); - } - mobWriters.initOutputWriters(writers); - } - } - - boolean discardMobMiss = - conf.getBoolean(MobConstants.MOB_DISCARD_MISS_KEY, MobConstants.DEFAULT_MOB_DISCARD_MISS); - FileSystem fs = FileSystem.get(conf); - // Since scanner.next() can return 'false' but still be delivering data, // we have to use a do/while loop. List cells = new ArrayList<>(); @@ -270,20 +183,16 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel boolean hasMore; Path path = MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName()); byte[] fileName = null; - StoreFileWriter mobFileWriter = null; - long mobCells = 0; + StoreFileWriter mobFileWriter = null, delFileWriter = null; + long mobCells = 0, deleteMarkersCount = 0; long cellsCountCompactedToMob = 0, cellsCountCompactedFromMob = 0; long cellsSizeCompactedToMob = 0, cellsSizeCompactedFromMob = 0; boolean finished = false; - ScannerContext scannerContext = ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build(); throughputController.start(compactionName); - KeyValueScanner kvs = (scanner instanceof KeyValueScanner) ? (KeyValueScanner) scanner : null; - long shippedCallSizeLimit = - (long) numofFilesToCompact * this.store.getColumnFamilyDescriptor().getBlocksize(); - - MobCell mobCell = null; + KeyValueScanner kvs = (scanner instanceof KeyValueScanner)? (KeyValueScanner)scanner : null; + long shippedCallSizeLimit = (long) numofFilesToCompact * this.store.getColumnFamilyDescriptor().getBlocksize(); try { try { // If the mob file writer could not be created, directly write the cell to the store file. @@ -291,15 +200,19 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel compactionCompression, store.getRegionInfo().getStartKey(), true); fileName = Bytes.toBytes(mobFileWriter.getPath().getName()); } catch (IOException e) { - // Bailing out - LOG.error("Failed to create mob writer, ", e); - throw e; + LOG.warn("Failed to create mob writer, " + + "we will continue the compaction by writing MOB cells directly in store files", e); } - if (compactMOBs) { - // Add the only reference we get for compact MOB case - // because new store file will have only one MOB reference - // in this case - of newly compacted MOB file - mobRefSet.get().add(mobFileWriter.getPath().getName()); + if (major) { + try { + delFileWriter = mobStore.createDelFileWriterInTmp(new Date(fd.latestPutTs), + fd.maxKeyCount, compactionCompression, store.getRegionInfo().getStartKey()); + } catch (IOException e) { + LOG.warn( + "Failed to create del writer, " + + "we will continue the compaction by writing delete markers directly in store files", + e); + } } do { hasMore = scanner.next(cells, scannerContext); @@ -307,113 +220,54 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel now = EnvironmentEdgeManager.currentTime(); } for (Cell c : cells) { - - if (compactMOBs) { - if (MobUtils.isMobReferenceCell(c)) { - String fName = MobUtils.getMobFileName(c); - Path pp = new Path(new Path(fs.getUri()), new Path(path, fName)); - - // Added to support migration - try { - mobCell = mobStore.resolve(c, true, false); - } catch (FileNotFoundException fnfe) { - if (discardMobMiss) { - LOG.error("Missing MOB cell: file=" + pp + " not found"); - continue; - } else { - throw fnfe; - } - } - - if (discardMobMiss && mobCell.getCell().getValueLength() == 0) { - LOG.error("Missing MOB cell value: file=" + pp + " cell=" + mobCell); - continue; - } - - if (mobCell.getCell().getValueLength() > mobSizeThreshold) { - // put the mob data back to the store file - PrivateCellUtil.setSequenceId(mobCell.getCell(), c.getSequenceId()); - if (generationalMob) { - //TODO: verify fName - StoreFileWriter stw = mobWriters.getOutputWriterForInputFile(fName); - if (stw != null) { - stw.append(mobCell.getCell()); - mobWriters.incrementMobCountForOutputWriter(stw, 1); - } else { - // leave mob cell in a MOB file which is not in compaction selections - } - } else { - mobFileWriter.append(mobCell.getCell()); - mobCells++; - } - writer.append(MobUtils.createMobRefCell(mobCell.getCell(), fileName, - this.mobStore.getRefCellTags())); - cellsCountCompactedFromMob++; - cellsSizeCompactedFromMob += mobCell.getCell().getValueLength(); - } else { - - // If MOB value is less than threshold, append it directly to a store file - PrivateCellUtil.setSequenceId(mobCell.getCell(), c.getSequenceId()); - writer.append(mobCell.getCell()); - } - + if (major && CellUtil.isDelete(c)) { + if (MobUtils.isMobReferenceCell(c) || delFileWriter == null) { + // Directly write it to a store file + writer.append(c); } else { - // Not a MOB reference cell - int size = c.getValueLength(); - if (size > mobSizeThreshold) { - // This MOB cell comes from a regular store file - // therefore we store it in original mob output - mobFileWriter.append(c); - writer - .append(MobUtils.createMobRefCell(c, fileName, this.mobStore.getRefCellTags())); - mobCells++; - } else { - writer.append(c); - } + // Add a ref tag to this cell and write it to a store file. + writer.append(MobUtils.createMobRefDeleteMarker(c)); + // Write the cell to a del file + delFileWriter.append(c); + deleteMarkersCount++; } - } else if (c.getTypeByte() != KeyValue.Type.Put.getCode()) { - // Not a major compaction or major with MOB disabled - // If the kv type is not put, directly write the cell + } else if (mobFileWriter == null || c.getTypeByte() != KeyValue.Type.Put.getCode()) { + // If the mob file writer is null or the kv type is not put, directly write the cell // to the store file. writer.append(c); } else if (MobUtils.isMobReferenceCell(c)) { - // Not a major MOB compaction, Put MOB reference if (MobUtils.hasValidMobRefCellValue(c)) { int size = MobUtils.getMobValueLength(c); if (size > mobSizeThreshold) { // If the value size is larger than the threshold, it's regarded as a mob. Since // its value is already in the mob file, directly write this cell to the store file writer.append(c); - // Add MOB reference to a set - mobRefSet.get().add(MobUtils.getMobFileName(c)); } else { // If the value is not larger than the threshold, it's not regarded a mob. Retrieve - // the mob cell from the mob file, and write it back to the store file. - mobCell = mobStore.resolve(c, true, false); - if (mobCell.getCell().getValueLength() != 0) { - // put the mob data back to the store file - PrivateCellUtil.setSequenceId(mobCell.getCell(), c.getSequenceId()); - writer.append(mobCell.getCell()); - cellsCountCompactedFromMob++; - cellsSizeCompactedFromMob += mobCell.getCell().getValueLength(); - } else { - // If the value of a file is empty, there might be issues when retrieving, - // directly write the cell to the store file, and leave it to be handled by the - // next compaction. - LOG.error("Empty value for: " + c); - writer.append(c); - // Add MOB reference to a set - mobRefSet.get().add(MobUtils.getMobFileName(c)); + // the mob cell from the mob file, and write it back to the store file. Must + // close the mob scanner once the life cycle finished. + try (MobCell mobCell = mobStore.resolve(c, false)) { + if (mobCell.getCell().getValueLength() != 0) { + // put the mob data back to the store file + PrivateCellUtil.setSequenceId(mobCell.getCell(), c.getSequenceId()); + writer.append(mobCell.getCell()); + cellsCountCompactedFromMob++; + cellsSizeCompactedFromMob += mobCell.getCell().getValueLength(); + } else { + // If the value of a file is empty, there might be issues when retrieving, + // directly write the cell to the store file, and leave it to be handled by the + // next compaction. + writer.append(c); + } } } } else { - // TODO ???? - LOG.error("Corrupted MOB reference: " + c); + LOG.warn("The value format of the KeyValue " + c + + " is wrong, its length is less than " + Bytes.SIZEOF_INT); writer.append(c); } } else if (c.getValueLength() <= mobSizeThreshold) { - // If the value size of a cell is not larger than the threshold, directly write it to - // the store file. + //If value size of a cell is not larger than the threshold, directly write to store file writer.append(c); } else { // If the value size of a cell is larger than the threshold, it's regarded as a mob, @@ -421,15 +275,13 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel mobCells++; // append the original keyValue in the mob file. mobFileWriter.append(c); - Cell reference = MobUtils.createMobRefCell(c, fileName, this.mobStore.getRefCellTags()); + Cell reference = MobUtils.createMobRefCell(c, fileName, + this.mobStore.getRefCellTags()); // write the cell whose value is the path of a mob file to the store file. writer.append(reference); cellsCountCompactedToMob++; cellsSizeCompactedToMob += c.getValueLength(); - // Add ref we get for compact MOB case - mobRefSet.get().add(mobFileWriter.getPath().getName()); } - int len = c.getSerializedSize(); ++progress.currentCompactedKVs; progress.totalCompactedSize += len; @@ -450,7 +302,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel } } if (kvs != null && bytesWrittenProgressForShippedCall > shippedCallSizeLimit) { - ((ShipperListener) writer).beforeShipped(); + ((ShipperListener)writer).beforeShipped(); kvs.shipped(); bytesWrittenProgressForShippedCall = 0; } @@ -474,9 +326,6 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel progress.cancel(); throw new InterruptedIOException( "Interrupted while control throughput of compacting " + compactionName); - } catch (IOException t) { - LOG.error("Mob compaction failed for region: " + store.getRegionInfo().getEncodedName()); - throw t; } finally { // Clone last cell in the final because writer will append last cell when committing. If // don't clone here and once the scanner get closed, then the memory of last cell will be @@ -484,19 +333,24 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel ((ShipperListener) writer).beforeShipped(); throughputController.finish(compactionName); if (!finished && mobFileWriter != null) { - // Remove all MOB references because compaction failed - mobRefSet.get().clear(); - // Abort writer abortWriter(mobFileWriter); - //Check if other writers exist - if (mobWriters != null) { - for(StoreFileWriter w: mobWriters.getOutputWriters()) { - abortWriter(w); - } - } + } + if (!finished && delFileWriter != null) { + abortWriter(delFileWriter); + } + } + if (delFileWriter != null) { + if (deleteMarkersCount > 0) { + // If the del file is not empty, commit it. + // If the commit fails, the compaction is re-performed again. + delFileWriter.appendMetadata(fd.maxSeqId, major, deleteMarkersCount); + delFileWriter.close(); + mobStore.commitFile(delFileWriter.getPath(), path); + } else { + // If the del file is empty, delete it instead of committing. + abortWriter(delFileWriter); } } - // Commit or abort major mob writer if (mobFileWriter != null) { if (mobCells > 0) { // If the mob file is not empty, commit it. @@ -508,20 +362,6 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel abortWriter(mobFileWriter); } } - // Commit or abort generational writers - if (mobWriters != null) { - for (StoreFileWriter w: mobWriters.getOutputWriters()) { - Long mobs = mobWriters.getMobCountForOutputWriter(w); - if (mobs != null && mobs > 0) { - mobRefSet.get().add(w.getPath().getName()); - w.appendMetadata(fd.maxSeqId, major, mobs); - w.close(); - mobStore.commitFile(w.getPath(), path); - } else { - abortWriter(w); - } - } - } mobStore.updateCellsCountCompactedFromMob(cellsCountCompactedFromMob); mobStore.updateCellsCountCompactedToMob(cellsCountCompactedToMob); mobStore.updateCellsSizeCompactedFromMob(cellsSizeCompactedFromMob); @@ -529,354 +369,4 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel progress.complete(); return true; } - - protected static String createKey(TableName tableName, String encodedName, - String columnFamilyName) { - return tableName.getNameAsString()+ "_" + encodedName + "_"+ columnFamilyName; - } - - @Override - protected List commitWriter(StoreFileWriter writer, FileDetails fd, - CompactionRequestImpl request) throws IOException { - List newFiles = Lists.newArrayList(writer.getPath()); - writer.appendMetadata(fd.maxSeqId, request.isAllFiles(), request.getFiles()); - // Append MOB references - Set refSet = mobRefSet.get(); - writer.appendMobMetadata(refSet); - writer.close(); - return newFiles; - } - - private List getReferencedMobFiles(Collection storeFiles) { - Path mobDir = MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName()); - Set mobSet = new HashSet(); - for (HStoreFile sf: storeFiles) { - byte[] value = sf.getMetadataValue(HStoreFile.MOB_FILE_REFS); - if (value != null) { - String s = new String(value); - String[] all = s.split(","); - Collections.addAll(mobSet, all); - } - } - List retList = new ArrayList(); - for(String name: mobSet) { - retList.add(new Path(mobDir, name)); - } - return retList; - } -} - -class FileSelection implements Comparable { - - public final static String NULL_REGION = ""; - private Path path; - private long earliestTs; - private Configuration conf; - - public FileSelection(Path path, Configuration conf) throws IOException { - this.path = path; - this.conf = conf; - readEarliestTimestamp(); - } - - public String getEncodedRegionName() { - String fileName = path.getName(); - String[] parts = fileName.split("_"); - if (parts.length == 2) { - return parts[1]; - } else { - return NULL_REGION; - } - } - - public Path getPath() { - return path; - } - - public long getEarliestTimestamp() { - return earliestTs; - } - - private void readEarliestTimestamp() throws IOException { - FileSystem fs = path.getFileSystem(conf); - HStoreFile sf = new HStoreFile(fs, path, conf, CacheConfig.DISABLED, - BloomType.NONE, true); - sf.initReader(); - byte[] tsData = sf.getMetadataValue(HStoreFile.EARLIEST_PUT_TS); - if (tsData != null) { - this.earliestTs = Bytes.toLong(tsData); - } - sf.closeStoreFile(true); - } - - @Override - public int compareTo(FileSelection o) { - if (this.earliestTs > o.earliestTs) { - return +1; - } else if (this.earliestTs == o.earliestTs) { - return 0; - } else { - return -1; - } - } - -} - -class Generations { - - private List generations; - private Configuration conf; - - private Generations(List gens, Configuration conf) { - this.generations = gens; - this.conf = conf; - } - - List getCompactionSelections() throws IOException { - int maxTotalFiles = this.conf.getInt(MobConstants.MOB_COMPACTION_MAX_TOTAL_FILES_KEY, - MobConstants.DEFAULT_MOB_COMPACTION_MAX_TOTAL_FILES); - int currentTotal = 0; - List list = new ArrayList(); - - for (Generation g: generations) { - List sel = g.getCompactionSelections(conf); - int size = getSize(sel); - if ((currentTotal + size > maxTotalFiles) && currentTotal > 0) { - break; - } else { - currentTotal += size; - list.addAll(sel); - } - } - return list; - } - - private int getSize(List sel) { - int size = 0; - for(CompactionSelection cs: sel) { - size += cs.size(); - } - return size; - } - - static Generations build(List files, Configuration conf) throws IOException { - Map > map = new HashMap>(); - for(Path p: files) { - String key = getRegionNameFromFileName(p.getName()); - ArrayList list = map.get(key); - if (list == null) { - list = new ArrayList(); - map.put(key, list); - } - list.add(new FileSelection(p, conf)); - } - - List gens = new ArrayList(); - for (String key: map.keySet()) { - Generation g = new Generation(key); - List selFiles = map.get(key); - for(FileSelection fs: selFiles) { - g.addFile(fs); - } - gens.add(g); - } - // Sort all generation files one-by-one - for(Generation gg: gens) { - gg.sortFiles(); - } - // Sort generations - Collections.sort(gens); - return new Generations(gens, conf); - } - - static String getRegionNameFromFileName(String name) { - int index = name.lastIndexOf("_"); - if (index < 0) return Generation.GEN0; - return name.substring(index+1); - } -} - -class Generation implements Comparable { - - static final String GEN0 ="GEN0"; - private String regionName; - private long earliestTs = Long.MAX_VALUE; - private List files = new ArrayList<>(); - List compSelections; - - public Generation(String name) { - this.regionName = name; - } - - @SuppressWarnings("deprecation") - public List getCompactionSelections(Configuration conf) throws IOException { - - - int minFiles = conf.getInt(MobConstants.MOB_COMPACTION_MIN_FILES_KEY, - MobConstants.DEFAULT_MOB_COMPACTION_MIN_FILES); - int maxFiles = conf.getInt(MobConstants.MOB_COMPACTION_MAX_FILES_KEY, - MobConstants.DEFAULT_MOB_COMPACTION_MAX_FILES); - long maxSelectionSize = conf.getLong(MobConstants.MOB_COMPACTION_MAX_SELECTION_SIZE_KEY, - MobConstants.DEFAULT_MOB_COMPACTION_MAX_SELECTION_SIZE); - // Now it is ordered from oldest to newest ones - List rfiles = Lists.reverse(files); - List retList = new ArrayList(); - FileSystem fs = rfiles.get(0).getPath().getFileSystem(conf); - int off = 0; - while (off < rfiles.size()) { - if (fs.getLength(rfiles.get(off).getPath()) >= maxSelectionSize) { - off++; continue; - } - long selSize = 0; - int limit = Math.min(off + maxFiles, rfiles.size()); - int start = off; - List sel = new ArrayList(); - for (; off < limit; off++) { - Path p = rfiles.get(off).getPath(); - long fSize = fs.getLength(p); - if (selSize + fSize < maxSelectionSize) { - selSize+= fSize; - sel.add(new FileSelection(p, conf)); - } else { - if (sel.size() < minFiles) { - // discard - sel.clear(); - // advance by 1 - off = start +1; - } else { - // we have new selection - CompactionSelection cs = new CompactionSelection(sel); - retList.add(cs); - off++; - } - break; // continue outer loop - } - } - } - return retList; - } - - public boolean addFile(FileSelection f) { - if (f.getEncodedRegionName().equals(regionName)) { - files.add(f); - if (f.getEarliestTimestamp() < earliestTs) { - earliestTs = f.getEarliestTimestamp(); - } - return true; - } else { - return false; - } - } - - public void sortFiles() { - Collections.sort(files); - } - - public List getFiles() { - return files; - } - - public String getEncodedRegionName() { - return regionName; - } - - public long getEarliestTimestamp() { - return earliestTs; - } - - @Override - public int compareTo(Generation o) { - if (this.earliestTs > o.earliestTs) { - return +1; - } else if (this.earliestTs == o.earliestTs) { - return 0; - } else { - return -1; - } - } -} - -class CompactionSelection { - private static AtomicLong idGen = new AtomicLong(); - private List files; - private long id; - - public CompactionSelection(List files) { - this.files = files; - this.id = idGen.getAndIncrement(); - } - - public List getFiles() { - return files; - } - - public long getId() { - return id; - } - - int size() { - return files.size(); - } - -} - -class OutputMobWriters { - - /* - * Input MOB file name -> output file writer - */ - private Map writerMap = new HashMap(); - /* - * Output file name -> MOB counter - */ - private Map mapMobCounts = new HashMap(); - /* - * List of compaction selections - */ - private List compSelections; - - public OutputMobWriters(List compSelections) - { - this.compSelections = compSelections; - } - - int getNumberOfWriters() { - return compSelections.size(); - } - - StoreFileWriter getWriterForFile(String fileName) { - return writerMap.get(fileName); - } - - void initOutputWriters(List writers) { - for (int i = 0; i < writers.size(); i++) { - StoreFileWriter sw = writers.get(i); - mapMobCounts.put(sw.getPath().getName(), 0L); - CompactionSelection cs = compSelections.get(i); - for (FileSelection fs: cs.getFiles()) { - writerMap.put(fs.getPath().getName(), sw); - } - } - } - - Collection getOutputWriters() { - return writerMap.values(); - } - - StoreFileWriter getOutputWriterForInputFile(String name) { - return writerMap.get(name); - } - - long getMobCountForOutputWriter(StoreFileWriter writer) { - return mapMobCounts.get(writer.getPath().getName()); - } - - void incrementMobCountForOutputWriter(StoreFileWriter writer, int val) { - String key = writer.getPath().getName(); - Long count = mapMobCounts.get(key); - if (count == null) { - count = 0L; - } - mapMobCounts.put(key, count + val); - } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java index 2c0d9ff1c0ba..3de7992cb121 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java @@ -22,9 +22,7 @@ import java.io.InterruptedIOException; import java.util.ArrayList; import java.util.Date; -import java.util.HashSet; import java.util.List; -import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -39,7 +37,6 @@ import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.MemStoreSnapshot; import org.apache.hadoop.hbase.regionserver.ScannerContext; -import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFileWriter; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputControlUtil; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; @@ -72,13 +69,6 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher { private long mobCellValueSizeThreshold = 0; private Path targetPath; private HMobStore mobStore; - // MOB file reference set - static ThreadLocal> mobRefSet = new ThreadLocal>() { - @Override - protected Set initialValue() { - return new HashSet(); - } - }; public DefaultMobStoreFlusher(Configuration conf, HStore store) throws IOException { super(conf, store); @@ -198,8 +188,6 @@ protected void performMobFlush(MemStoreSnapshot snapshot, long cacheFlushId, throughputController.start(flushName); } IOException ioe = null; - // Clear all past MOB references - mobRefSet.get().clear(); try { do { hasMore = scanner.next(cells, scannerContext); @@ -256,12 +244,9 @@ protected void performMobFlush(MemStoreSnapshot snapshot, long cacheFlushId, status.setStatus("Flushing mob file " + store + ": closing flushed file"); mobFileWriter.close(); mobStore.commitFile(mobFileWriter.getPath(), targetPath); - LOG.debug(" FLUSH store file: " + writer.getPath()); mobStore.updateMobFlushCount(); mobStore.updateMobFlushedCellsCount(mobCount); mobStore.updateMobFlushedCellsSize(mobSize); - // Add mob reference to store file metadata - mobRefSet.get().add(mobFileWriter.getPath().getName()); } else { try { status.setStatus("Flushing mob file " + store + ": no mob cells, closing flushed file"); @@ -273,16 +258,4 @@ protected void performMobFlush(MemStoreSnapshot snapshot, long cacheFlushId, } } } - - protected void finalizeWriter(StoreFileWriter writer, long cacheFlushSeqNum, - MonitoredTask status) throws IOException { - // Write out the log sequence number that corresponds to this output - // hfile. Also write current time in metadata as minFlushTime. - // The hfile is current up to and including cacheFlushSeqNum. - status.setStatus("Flushing " + store + ": appending metadata"); - writer.appendMetadata(cacheFlushSeqNum, false); - writer.appendMobMetadata(mobRefSet.get()); - status.setStatus("Flushing " + store + ": closing flushed file"); - writer.close(); - } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java similarity index 94% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerTool.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java index 07c89355caf1..42e78bbd007d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerTool.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java @@ -43,9 +43,9 @@ * The cleaner to delete the expired MOB files. */ @InterfaceAudience.Private -public class MobFileCleanerTool extends Configured implements Tool { +public class ExpiredMobFileCleaner extends Configured implements Tool { - private static final Logger LOG = LoggerFactory.getLogger(MobFileCleanerTool.class); + private static final Logger LOG = LoggerFactory.getLogger(ExpiredMobFileCleaner.class); /** * Cleans the MOB files when they're expired and their min versions are 0. * If the latest timestamp of Cells in a MOB file is older than the TTL in the column family, @@ -72,12 +72,12 @@ public void cleanExpiredMobFiles(String tableName, ColumnFamilyDescriptor family public static void main(String[] args) throws Exception { Configuration conf = HBaseConfiguration.create(); - ToolRunner.run(conf, new MobFileCleanerTool(), args); + ToolRunner.run(conf, new ExpiredMobFileCleaner(), args); } private void printUsage() { System.err.println("Usage:\n" + "--------------------------\n" - + MobFileCleanerTool.class.getName() + " tableName familyName"); + + ExpiredMobFileCleaner.class.getName() + " tableName familyName"); System.err.println(" tableName The table name"); System.err.println(" familyName The column family name"); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java index 59777e780b43..4afd75b9c680 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java @@ -43,7 +43,7 @@ public final class MobConstants { public static final String MOB_REGION_NAME = ".mob"; public static final byte[] MOB_REGION_NAME_BYTES = Bytes.toBytes(MOB_REGION_NAME); - public static final String MOB_CLEANER_PERIOD = "hbase.master.mob.cleaner.period"; + public static final String MOB_CLEANER_PERIOD = "hbase.master.mob.ttl.cleaner.period"; public static final int DEFAULT_MOB_CLEANER_PERIOD = 24 * 60 * 60; // one day public static final String MOB_CACHE_EVICT_PERIOD = "hbase.mob.cache.evict.period"; @@ -55,23 +55,33 @@ public final class MobConstants { public static final long DEFAULT_MOB_CACHE_EVICT_PERIOD = 3600L; public final static String TEMP_DIR_NAME = ".tmp"; + public final static String BULKLOAD_DIR_NAME = ".bulkload"; public final static byte[] MOB_TABLE_LOCK_SUFFIX = Bytes.toBytes(".mobLock"); - + public final static String EMPTY_STRING = ""; /** - * The max number of a MOB table regions that is allowed in a batch of the mob compaction. - * By setting this number to a custom value, users can control the overall effect - * of a major compaction of a large MOB-enabled table. + * If the size of a mob file is less than this value, it's regarded as a small file and needs to + * be merged in mob compaction. The default value is 1280MB. */ - - public static final String MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE = - "hbase.mob.compaction.batch.size"; - + public static final String MOB_COMPACTION_MERGEABLE_THRESHOLD = + "hbase.mob.compaction.mergeable.threshold"; + public static final long DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD = 10 * 128 * 1024 * 1024; /** - * Default is 0 - means no limit - all regions of a MOB table will be compacted at once + * The max number of del files that is allowed in the mob file compaction. In the mob + * compaction, when the number of existing del files is larger than this value, they are merged + * until number of del files is not larger this value. The default value is 3. */ - - public static final int DEFAULT_MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE = 0; - + public static final String MOB_DELFILE_MAX_COUNT = "hbase.mob.delfile.max.count"; + public static final int DEFAULT_MOB_DELFILE_MAX_COUNT = 3; + /** + * The max number of the mob files that is allowed in a batch of the mob compaction. + * The mob compaction merges the small mob files to bigger ones. If the number of the + * small files is very large, it could lead to a "too many opened file handlers" in the merge. + * And the merge has to be split into batches. This value limits the number of mob files + * that are selected in a batch of the mob compaction. The default value is 100. + */ + public static final String MOB_COMPACTION_BATCH_SIZE = + "hbase.mob.compaction.batch.size"; + public static final int DEFAULT_MOB_COMPACTION_BATCH_SIZE = 100; /** * The period that MobCompactionChore runs. The unit is second. * The default value is one week. @@ -81,67 +91,12 @@ public final class MobConstants { public static final int DEFAULT_MOB_COMPACTION_CHORE_PERIOD = 24 * 60 * 60 * 7; // a week public static final String MOB_COMPACTOR_CLASS_KEY = "hbase.mob.compactor.class"; - /** - * Mob compaction type: "full", "generational" - * "full" - run full major compaction (during migration) - * "generational" - optimized version + * The max number of threads used in MobCompactor. */ - public final static String MOB_COMPACTION_TYPE_KEY = "hbase.mob.compaction.type"; - - public final static String DEFAULT_MOB_COMPACTION_TYPE = "full"; - - public final static String GENERATIONAL_MOB_COMPACTION_TYPE = "generational"; - - public final static String FULL_MOB_COMPACTION_TYPE = "full"; - - - /** - * Maximum size of a MOB compaction selection - */ - public static final String MOB_COMPACTION_MAX_SELECTION_SIZE_KEY = - "hbase.mob.compactions.max.selection.size"; - /** - * Default maximum selection size = 1GB - */ - public static final long DEFAULT_MOB_COMPACTION_MAX_SELECTION_SIZE = 1024 * 1024 * 1024; - - - /** - * Minimum number of MOB files eligible for compaction - */ - public static final String MOB_COMPACTION_MIN_FILES_KEY = "hbase.mob.compactions.min.files"; - - public static final int DEFAULT_MOB_COMPACTION_MIN_FILES = 3; - - /** - * Maximum number of MOB files (in one selection) eligible for compaction - */ - - public static final String MOB_COMPACTION_MAX_FILES_KEY = "hbase.mob.compactions.max.files"; - - public static final int DEFAULT_MOB_COMPACTION_MAX_FILES = 100; - - /** - * Maximum number of MOB files allowed in MOB compaction (per region) - */ - - public static final String MOB_COMPACTION_MAX_TOTAL_FILES_KEY = - "hbase.mob.compactions.max.total.files"; - - public static final int DEFAULT_MOB_COMPACTION_MAX_TOTAL_FILES = 1000; - - public static final String MOB_DISCARD_MISS_KEY = "hbase.mob.discard.miss"; - - public static final boolean DEFAULT_MOB_DISCARD_MISS = false; - - /** - * Minimum age required for MOB file to be archived - */ - public static final String MOB_MINIMUM_FILE_AGE_TO_ARCHIVE_KEY = "mob.minimum.file.age.to.archive"; - - public static final long DEFAULT_MOB_MINIMUM_FILE_AGE_TO_ARCHIVE = 3600000; // 1 hour - + public static final String MOB_COMPACTION_THREADS_MAX = + "hbase.mob.compaction.threads.max"; + public static final int DEFAULT_MOB_COMPACTION_THREADS_MAX = 1; private MobConstants() { } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileName.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileName.java index d8b3544e6c7a..3a29274285b6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileName.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileName.java @@ -45,13 +45,10 @@ public final class MobFileName { private final String startKey; private final String uuid; private final String fileName; - // Encoded region name - private final String regionName; private static final int STARTKEY_END_INDEX = 32; private static final int DATE_END_INDEX = 40; private static final int UUID_END_INDEX = 72; - public static final String REGION_SEP = "_"; /** * @param startKey @@ -60,14 +57,12 @@ public final class MobFileName { * The string of the latest timestamp of cells in this file, the format is yyyymmdd. * @param uuid * The uuid - * @param regionName region's name */ - private MobFileName(byte[] startKey, String date, String uuid, String regionName) { + private MobFileName(byte[] startKey, String date, String uuid) { this.startKey = MD5Hash.getMD5AsHex(startKey, 0, startKey.length); this.uuid = uuid; this.date = date; - this.regionName = regionName; - this.fileName = this.startKey + this.date + this.uuid + REGION_SEP + this.regionName; + this.fileName = this.startKey + this.date + this.uuid; } /** @@ -77,14 +72,12 @@ private MobFileName(byte[] startKey, String date, String uuid, String regionName * The string of the latest timestamp of cells in this file, the format is yyyymmdd. * @param uuid * The uuid - * @param regionName region's name */ - private MobFileName(String startKey, String date, String uuid, String regionName) { + private MobFileName(String startKey, String date, String uuid) { this.startKey = startKey; this.uuid = uuid; this.date = date; - this.regionName = regionName; - this.fileName = this.startKey + this.date + this.uuid + REGION_SEP + this.regionName; + this.fileName = this.startKey + this.date + this.uuid; } /** @@ -97,8 +90,8 @@ private MobFileName(String startKey, String date, String uuid, String regionName * @param uuid The uuid. * @return An instance of a MobFileName. */ - public static MobFileName create(byte[] startKey, String date, String uuid, String regionName) { - return new MobFileName(startKey, date, uuid, regionName); + public static MobFileName create(byte[] startKey, String date, String uuid) { + return new MobFileName(startKey, date, uuid); } /** @@ -111,8 +104,8 @@ public static MobFileName create(byte[] startKey, String date, String uuid, Stri * @param uuid The uuid. * @return An instance of a MobFileName. */ - public static MobFileName create(String startKey, String date, String uuid, String regionName) { - return new MobFileName(startKey, date, uuid, regionName); + public static MobFileName create(String startKey, String date, String uuid) { + return new MobFileName(startKey, date, uuid); } /** @@ -126,8 +119,7 @@ public static MobFileName create(String fileName) { String startKey = fileName.substring(0, STARTKEY_END_INDEX); String date = fileName.substring(STARTKEY_END_INDEX, DATE_END_INDEX); String uuid = fileName.substring(DATE_END_INDEX, UUID_END_INDEX); - String regionName = fileName.substring(UUID_END_INDEX+1); - return new MobFileName(startKey, date, uuid, regionName); + return new MobFileName(startKey, date, uuid); } /** @@ -156,13 +148,6 @@ public String getStartKey() { return startKey; } - /** - * Gets region name - * @return region name - */ - public String getRegionName() { - return regionName; - } /** * Gets the date string. Its format is yyyymmdd. * @return The date string. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java index 6adb4b58ded5..ee1fe7db14fc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java @@ -22,7 +22,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine; import org.apache.hadoop.hbase.regionserver.HStore; -import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.yetus.audience.InterfaceAudience; /** @@ -30,7 +29,7 @@ */ @InterfaceAudience.Private public class MobStoreEngine extends DefaultStoreEngine { - public final static String DEFAULT_MOB_COMPACTOR_CLASS_KEY = "hbase.mob.default.compactor"; + @Override protected void createStoreFlusher(Configuration conf, HStore store) throws IOException { // When using MOB, we use DefaultMobStoreFlusher always @@ -44,13 +43,6 @@ protected void createStoreFlusher(Configuration conf, HStore store) throws IOExc */ @Override protected void createCompactor(Configuration conf, HStore store) throws IOException { - String className = - conf.get(DEFAULT_MOB_COMPACTOR_CLASS_KEY, DefaultMobStoreCompactor.class.getName()); - try { - compactor = ReflectionUtils.instantiateWithCustomCtor(className, - new Class[] { Configuration.class, HStore.class }, new Object[] { conf, store }); - } catch (Exception e) { - throw new IOException("Unable to load configured compactor '" + className + "'", e); - } + compactor = new DefaultMobStoreCompactor(conf, store); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java index 73c6f25548dd..304a62e8901b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java @@ -23,22 +23,23 @@ import java.text.ParseException; import java.text.SimpleDateFormat; import java.util.ArrayList; -import java.util.Arrays; import java.util.Calendar; import java.util.Collection; import java.util.Date; -import java.util.HashSet; import java.util.List; import java.util.Optional; -import java.util.Set; import java.util.UUID; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.RejectedExecutionHandler; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.HConstants; @@ -48,10 +49,8 @@ import org.apache.hadoop.hbase.TagType; import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.backup.HFileArchiver; -import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.Scan; @@ -63,6 +62,10 @@ import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; +import org.apache.hadoop.hbase.master.locking.LockManager; +import org.apache.hadoop.hbase.mob.compactions.MobCompactor; +import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId; +import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactor; import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.HStoreFile; @@ -71,7 +74,8 @@ import org.apache.hadoop.hbase.util.ChecksumType; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.hbase.util.HFileArchiveUtil; +import org.apache.hadoop.hbase.util.ReflectionUtils; +import org.apache.hadoop.hbase.util.Threads; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -83,7 +87,8 @@ public final class MobUtils { private static final Logger LOG = LoggerFactory.getLogger(MobUtils.class); - public static final String SEP = "_"; + private final static long WEEKLY_THRESHOLD_MULTIPLIER = 7; + private final static long MONTHLY_THRESHOLD_MULTIPLIER = 4 * WEEKLY_THRESHOLD_MULTIPLIER; private static final ThreadLocal LOCAL_FORMAT = new ThreadLocal() { @@ -93,6 +98,13 @@ protected SimpleDateFormat initialValue() { } }; + private static final byte[] REF_DELETE_MARKER_TAG_BYTES; + static { + List tags = new ArrayList<>(); + tags.add(MobConstants.MOB_REF_TAG); + REF_DELETE_MARKER_TAG_BYTES = TagUtil.fromList(tags); + } + /** * Private constructor to keep this class from being instantiated. */ @@ -118,6 +130,44 @@ public static Date parseDate(String dateString) throws ParseException { return LOCAL_FORMAT.get().parse(dateString); } + /** + * Get the first day of the input date's month + * @param calendar Calendar object + * @param date The date to find out its first day of that month + * @return The first day in the month + */ + public static Date getFirstDayOfMonth(final Calendar calendar, final Date date) { + + calendar.setTime(date); + calendar.set(Calendar.HOUR_OF_DAY, 0); + calendar.set(Calendar.MINUTE, 0); + calendar.set(Calendar.SECOND, 0); + calendar.set(Calendar.MILLISECOND, 0); + calendar.set(Calendar.DAY_OF_MONTH, 1); + + Date firstDayInMonth = calendar.getTime(); + return firstDayInMonth; + } + + /** + * Get the first day of the input date's week + * @param calendar Calendar object + * @param date The date to find out its first day of that week + * @return The first day in the week + */ + public static Date getFirstDayOfWeek(final Calendar calendar, final Date date) { + + calendar.setTime(date); + calendar.set(Calendar.HOUR_OF_DAY, 0); + calendar.set(Calendar.MINUTE, 0); + calendar.set(Calendar.SECOND, 0); + calendar.set(Calendar.MILLISECOND, 0); + calendar.setFirstDayOfWeek(Calendar.MONDAY); + calendar.set(Calendar.DAY_OF_WEEK, Calendar.MONDAY); + + Date firstDayInWeek = calendar.getTime(); + return firstDayInWeek; + } /** * Whether the current cell is a mob reference cell. @@ -320,23 +370,6 @@ public static Path getMobHome(Configuration conf) { return getMobHome(hbaseDir); } - /** - * Gets region encoded name from a MOB file name - * @param name name of a MOB file - * @return encoded region name - * - */ - public static String getEncodedRegionNameFromMobFileName(String mobFileName) - { - int index = mobFileName.lastIndexOf(MobFileName.REGION_SEP); - if (index > 0) { - return mobFileName.substring(index+1); - } else { - return null; - } - } - - /** * Gets the root dir of the mob files under the qualified HBase root dir. * It's {rootDir}/mobdir. @@ -454,6 +487,15 @@ public static boolean isMobRegionName(TableName tableName, byte[] regionName) { return Bytes.equals(regionName, getMobRegionInfo(tableName).getRegionName()); } + /** + * Gets the working directory of the mob compaction. + * @param root The root directory of the mob compaction. + * @param jobName The current job name. + * @return The directory of the mob compaction for the current job. + */ + public static Path getCompactionWorkingPath(Path root, String jobName) { + return new Path(root, jobName); + } /** * Archives the mob files. @@ -471,40 +513,6 @@ public static void removeMobFiles(Configuration conf, FileSystem fs, TableName t storeFiles); } - /** - * Archives the mob files. - * @param conf The current configuration. - * @param tableName The table name. - * @param family The name of the column family. - * @param storeFiles The files to be archived. - * @throws IOException - */ - public static void removeMobFiles(Configuration conf, TableName tableName, - byte[] family, List storeFiles) throws IOException { - - if (storeFiles.size() == 0) { - // nothing to remove - LOG.debug("Skipping archiving old MOB file: collection is empty"); - return; - } - Path mobTableDir = FSUtils.getTableDir(MobUtils.getMobHome(conf), tableName); - FileSystem fs = storeFiles.get(0).getFileSystem(conf); - Path storeArchiveDir = HFileArchiveUtil.getStoreArchivePath(conf, getMobRegionInfo(tableName), - mobTableDir, family); - - for (Path p: storeFiles) { - Path archiveFilePath = new Path(storeArchiveDir, p.getName()); - if (fs.exists(archiveFilePath)) { - LOG.info(" MOB Cleaner skip archiving: " + p); - continue; - } - LOG.info(" MOB Cleaner archiving: " + p); - HFileArchiver.archiveStoreFile(conf, fs, getMobRegionInfo(tableName), mobTableDir, family, p); - } - } - - - /** * Creates a mob reference KeyValue. * The value of the mob reference KeyValue is mobCellValueSize + mobFileName. @@ -553,43 +561,91 @@ public static Cell createMobRefCell(Cell cell, byte[] fileName, byte[] refCellTa public static StoreFileWriter createWriter(Configuration conf, FileSystem fs, ColumnFamilyDescriptor family, String date, Path basePath, long maxKeyCount, Compression.Algorithm compression, String startKey, CacheConfig cacheConfig, - Encryption.Context cryptoContext, boolean isCompaction, String regionName) + Encryption.Context cryptoContext, boolean isCompaction) throws IOException { MobFileName mobFileName = MobFileName.create(startKey, date, - UUID.randomUUID().toString().replaceAll("-", ""), regionName); + UUID.randomUUID().toString().replaceAll("-", "")); return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression, cacheConfig, cryptoContext, isCompaction); } + /** + * Creates a writer for the ref file in temp directory. + * @param conf The current configuration. + * @param fs The current file system. + * @param family The descriptor of the current column family. + * @param basePath The basic path for a temp directory. + * @param maxKeyCount The key count. + * @param cacheConfig The current cache config. + * @param cryptoContext The encryption context. + * @param isCompaction If the writer is used in compaction. + * @return The writer for the mob file. + * @throws IOException + */ + public static StoreFileWriter createRefFileWriter(Configuration conf, FileSystem fs, + ColumnFamilyDescriptor family, Path basePath, long maxKeyCount, CacheConfig cacheConfig, + Encryption.Context cryptoContext, boolean isCompaction) + throws IOException { + return createWriter(conf, fs, family, + new Path(basePath, UUID.randomUUID().toString().replaceAll("-", "")), maxKeyCount, + family.getCompactionCompressionType(), cacheConfig, cryptoContext, + HStore.getChecksumType(conf), HStore.getBytesPerChecksum(conf), family.getBlocksize(), + family.getBloomFilterType(), isCompaction); + } -// /** -// * Creates a writer for the mob file in temp directory. -// * @param conf The current configuration. -// * @param fs The current file system. -// * @param family The descriptor of the current column family. -// * @param date The date string, its format is yyyymmmdd. -// * @param basePath The basic path for a temp directory. -// * @param maxKeyCount The key count. -// * @param compression The compression algorithm. -// * @param startKey The start key. -// * @param cacheConfig The current cache config. -// * @param cryptoContext The encryption context. -// * @param isCompaction If the writer is used in compaction. -// * @return The writer for the mob file. -// * @throws IOException -// */ -// public static StoreFileWriter createWriter(Configuration conf, FileSystem fs, -// ColumnFamilyDescriptor family, String date, Path basePath, long maxKeyCount, -// Compression.Algorithm compression, byte[] startKey, CacheConfig cacheConfig, -// Encryption.Context cryptoContext, boolean isCompaction) -// throws IOException { -// MobFileName mobFileName = MobFileName.create(startKey, date, -// UUID.randomUUID().toString().replaceAll("-", "")); -// return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression, -// cacheConfig, cryptoContext, isCompaction); -// } - + /** + * Creates a writer for the mob file in temp directory. + * @param conf The current configuration. + * @param fs The current file system. + * @param family The descriptor of the current column family. + * @param date The date string, its format is yyyymmmdd. + * @param basePath The basic path for a temp directory. + * @param maxKeyCount The key count. + * @param compression The compression algorithm. + * @param startKey The start key. + * @param cacheConfig The current cache config. + * @param cryptoContext The encryption context. + * @param isCompaction If the writer is used in compaction. + * @return The writer for the mob file. + * @throws IOException + */ + public static StoreFileWriter createWriter(Configuration conf, FileSystem fs, + ColumnFamilyDescriptor family, String date, Path basePath, long maxKeyCount, + Compression.Algorithm compression, byte[] startKey, CacheConfig cacheConfig, + Encryption.Context cryptoContext, boolean isCompaction) + throws IOException { + MobFileName mobFileName = MobFileName.create(startKey, date, + UUID.randomUUID().toString().replaceAll("-", "")); + return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression, + cacheConfig, cryptoContext, isCompaction); + } + /** + * Creates a writer for the del file in temp directory. + * @param conf The current configuration. + * @param fs The current file system. + * @param family The descriptor of the current column family. + * @param date The date string, its format is yyyymmmdd. + * @param basePath The basic path for a temp directory. + * @param maxKeyCount The key count. + * @param compression The compression algorithm. + * @param startKey The start key. + * @param cacheConfig The current cache config. + * @param cryptoContext The encryption context. + * @return The writer for the del file. + * @throws IOException + */ + public static StoreFileWriter createDelFileWriter(Configuration conf, FileSystem fs, + ColumnFamilyDescriptor family, String date, Path basePath, long maxKeyCount, + Compression.Algorithm compression, byte[] startKey, CacheConfig cacheConfig, + Encryption.Context cryptoContext) + throws IOException { + String suffix = UUID + .randomUUID().toString().replaceAll("-", "") + "_del"; + MobFileName mobFileName = MobFileName.create(startKey, date, suffix); + return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression, + cacheConfig, cryptoContext, true); + } /** * Creates a writer for the mob file in temp directory. @@ -771,8 +827,70 @@ public static TableName getTableLockName(TableName tn) { return TableName.valueOf(Bytes.add(tableName, MobConstants.MOB_TABLE_LOCK_SUFFIX)); } + /** + * Performs the mob compaction. + * @param conf the Configuration + * @param fs the file system + * @param tableName the table the compact + * @param hcd the column descriptor + * @param pool the thread pool + * @param allFiles Whether add all mob files into the compaction. + */ + public static void doMobCompaction(Configuration conf, FileSystem fs, TableName tableName, + ColumnFamilyDescriptor hcd, ExecutorService pool, boolean allFiles, LockManager.MasterLock lock) + throws IOException { + String className = conf.get(MobConstants.MOB_COMPACTOR_CLASS_KEY, + PartitionedMobCompactor.class.getName()); + // instantiate the mob compactor. + MobCompactor compactor = null; + try { + compactor = ReflectionUtils.instantiateWithCustomCtor(className, new Class[] { + Configuration.class, FileSystem.class, TableName.class, ColumnFamilyDescriptor.class, + ExecutorService.class }, new Object[] { conf, fs, tableName, hcd, pool }); + } catch (Exception e) { + throw new IOException("Unable to load configured mob file compactor '" + className + "'", e); + } + // compact only for mob-enabled column. + // obtain a write table lock before performing compaction to avoid race condition + // with major compaction in mob-enabled column. + try { + lock.acquire(); + compactor.compact(allFiles); + } catch (Exception e) { + LOG.error("Failed to compact the mob files for the column " + hcd.getNameAsString() + + " in the table " + tableName.getNameAsString(), e); + } finally { + lock.release(); + } + } - + /** + * Creates a thread pool. + * @param conf the Configuration + * @return A thread pool. + */ + public static ExecutorService createMobCompactorThreadPool(Configuration conf) { + int maxThreads = conf.getInt(MobConstants.MOB_COMPACTION_THREADS_MAX, + MobConstants.DEFAULT_MOB_COMPACTION_THREADS_MAX); + if (maxThreads == 0) { + maxThreads = 1; + } + final SynchronousQueue queue = new SynchronousQueue<>(); + ThreadPoolExecutor pool = new ThreadPoolExecutor(1, maxThreads, 60, TimeUnit.SECONDS, queue, + Threads.newDaemonThreadFactory("MobCompactor"), new RejectedExecutionHandler() { + @Override + public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) { + try { + // waiting for a thread to pick up instead of throwing exceptions. + queue.put(r); + } catch (InterruptedException e) { + throw new RejectedExecutionException(e); + } + } + }); + ((ThreadPoolExecutor) pool).allowCoreThreadTimeOut(true); + return pool; + } /** * Checks whether this table has mob-enabled columns. @@ -789,143 +907,6 @@ public static boolean hasMobColumns(TableDescriptor htd) { return false; } - /** - * Get list of Mob column families (if any exists) - * @param htd table descriptor - * @return list of Mob column families - */ - public static List getMobColumnFamilies(TableDescriptor htd){ - - List fams = new ArrayList(); - ColumnFamilyDescriptor[] hcds = htd.getColumnFamilies(); - for (ColumnFamilyDescriptor hcd : hcds) { - if (hcd.isMobEnabled()) { - fams.add(hcd); - } - } - return fams; - } - - /** - * Performs housekeeping file cleaning (called by MOB Cleaner chore) - * @param conf configuration - * @param table table name - * @throws IOException - */ - public static void cleanupObsoleteMobFiles(Configuration conf, TableName table) - throws IOException { - - try (final Connection conn = ConnectionFactory.createConnection(conf); - final Admin admin = conn.getAdmin();) { - TableDescriptor htd = admin.getDescriptor(table); - List list = getMobColumnFamilies(htd); - if (list.size() == 0) { - LOG.info("Skipping non-MOB table [" + table + "]"); - return; - } - Path rootDir = FSUtils.getRootDir(conf); - Path tableDir = FSUtils.getTableDir(rootDir, table); - // How safe is this call? - List regionDirs = FSUtils.getRegionDirs(FileSystem.get(conf), tableDir); - - Set allActiveMobFileName = new HashSet(); - FileSystem fs = FileSystem.get(conf); - for (Path regionPath: regionDirs) { - for (ColumnFamilyDescriptor hcd: list) { - String family = hcd.getNameAsString(); - Path storePath = new Path(regionPath, family); - boolean succeed = false; - Set regionMobs = new HashSet(); - while(!succeed) { - //TODO handle FNFE - RemoteIterator rit = fs.listLocatedStatus(storePath); - List storeFiles = new ArrayList(); - // Load list of store files first - while(rit.hasNext()) { - Path p = rit.next().getPath(); - if (fs.isFile(p)) { - storeFiles.add(p); - } - } - try { - for(Path pp: storeFiles) { - HStoreFile sf = new HStoreFile(fs, pp, conf, CacheConfig.DISABLED, - BloomType.NONE, true); - sf.initReader(); - byte[] mobRefData = sf.getMetadataValue(HStoreFile.MOB_FILE_REFS); - byte[] mobCellCountData = sf.getMetadataValue(HStoreFile.MOB_CELLS_COUNT); - byte[] bulkloadMarkerData = sf.getMetadataValue(HStoreFile.BULKLOAD_TASK_KEY); - if (mobRefData == null && (mobCellCountData != null || - bulkloadMarkerData == null)) { - LOG.info("Found old store file with no MOB_FILE_REFS: " + pp - +" - can not proceed until all old files will be MOB-compacted"); - return; - } else if (mobRefData == null) { - LOG.info("Skipping file without MOB references (can be bulkloaded file):"+ pp); - continue; - } - String[] mobs = new String(mobRefData).split(","); - regionMobs.addAll(Arrays.asList(mobs)); - } - } catch (FileNotFoundException e) { - //TODO - LOG.warn(e.getMessage()); - continue; - } - succeed = true; - } - // Add MOB refs for current region/family - allActiveMobFileName.addAll(regionMobs); - } // END column families - }//END regions - - // Now scan MOB directories and find MOB files with no references to them - long now = System.currentTimeMillis(); - long minAgeToArchive = conf.getLong(MobConstants.MOB_MINIMUM_FILE_AGE_TO_ARCHIVE_KEY, - MobConstants.DEFAULT_MOB_MINIMUM_FILE_AGE_TO_ARCHIVE); - for (ColumnFamilyDescriptor hcd: list) { - List toArchive = new ArrayList(); - String family = hcd.getNameAsString(); - Path dir = getMobFamilyPath(conf, table, family); - RemoteIterator rit = fs.listLocatedStatus(dir); - while(rit.hasNext()) { - LocatedFileStatus lfs = rit.next(); - Path p = lfs.getPath(); - if (!allActiveMobFileName.contains(p.getName())) { - // MOB is not in a list of active references, but it can be too - // fresh, skip it in this case - /*DEBUG*/ LOG.debug(" Age=" + (now - fs.getFileStatus(p).getModificationTime()) + - " MOB file="+ p); - if (now - fs.getFileStatus(p).getModificationTime() > minAgeToArchive) { - toArchive.add(p); - } else { - LOG.debug(" Skipping fresh file: " + p); - } - } - } - LOG.info(" MOB Cleaner found "+ toArchive.size()+" files for family="+family); - removeMobFiles(conf, table, family.getBytes(), toArchive); - LOG.info(" MOB Cleaner archived "+ toArchive.size()+" files"); - } - } - } - - - - - public static long getNumberOfMobFiles(Configuration conf, TableName tableName, String family) - throws IOException { - FileSystem fs = FileSystem.get(conf); - Path dir = getMobFamilyPath(conf, tableName, family); - FileStatus[] stat = fs.listStatus(dir); - for (FileStatus st: stat) { - LOG.info(" MOB Directory content: "+ st.getPath()); - } - LOG.info(" MOB Directory content total files: "+ stat.length); - - return stat.length; - } - /** * Indicates whether return null value when the mob file is missing or corrupt. * The information is set in the attribute "empty.value.on.mobcell.miss" of scan. @@ -942,6 +923,14 @@ public static boolean isReadEmptyValueOnMobCellMiss(Scan scan) { } } + /** + * Creates a mob ref delete marker. + * @param cell The current delete marker. + * @return A delete marker with the ref tag. + */ + public static Cell createMobRefDeleteMarker(Cell cell) { + return PrivateCellUtil.createCell(cell, TagUtil.concatTags(REF_DELETE_MARKER_TAG_BYTES, cell)); + } /** * Checks if the mob file is expired. @@ -973,89 +962,86 @@ public static boolean isMobFileExpired(ColumnFamilyDescriptor column, long curre return false; } + /** + * fill out partition id based on compaction policy and date, threshold... + * @param id Partition id to be filled out + * @param firstDayOfCurrentMonth The first day in the current month + * @param firstDayOfCurrentWeek The first day in the current week + * @param dateStr Date string from the mob file + * @param policy Mob compaction policy + * @param calendar Calendar object + * @param threshold Mob compaciton threshold configured + * @return true if the file needs to be excluded from compaction + */ + public static boolean fillPartitionId(final CompactionPartitionId id, + final Date firstDayOfCurrentMonth, final Date firstDayOfCurrentWeek, final String dateStr, + final MobCompactPartitionPolicy policy, final Calendar calendar, final long threshold) { + + boolean skipCompcation = false; + id.setThreshold(threshold); + if (threshold <= 0) { + id.setDate(dateStr); + return skipCompcation; + } + + long finalThreshold; + Date date; + try { + date = MobUtils.parseDate(dateStr); + } catch (ParseException e) { + LOG.warn("Failed to parse date " + dateStr, e); + id.setDate(dateStr); + return true; + } + + /* The algorithm works as follows: + * For monthly policy: + * 1). If the file's date is in past months, apply 4 * 7 * threshold + * 2). If the file's date is in past weeks, apply 7 * threshold + * 3). If the file's date is in current week, exclude it from the compaction + * For weekly policy: + * 1). If the file's date is in past weeks, apply 7 * threshold + * 2). If the file's date in currently, apply threshold + * For daily policy: + * 1). apply threshold + */ + if (policy == MobCompactPartitionPolicy.MONTHLY) { + if (date.before(firstDayOfCurrentMonth)) { + // Check overflow + if (threshold < (Long.MAX_VALUE / MONTHLY_THRESHOLD_MULTIPLIER)) { + finalThreshold = MONTHLY_THRESHOLD_MULTIPLIER * threshold; + } else { + finalThreshold = Long.MAX_VALUE; + } + id.setThreshold(finalThreshold); + + // set to the date for the first day of that month + id.setDate(MobUtils.formatDate(MobUtils.getFirstDayOfMonth(calendar, date))); + return skipCompcation; + } + } + + if ((policy == MobCompactPartitionPolicy.MONTHLY) || + (policy == MobCompactPartitionPolicy.WEEKLY)) { + // Check if it needs to apply weekly multiplier + if (date.before(firstDayOfCurrentWeek)) { + // Check overflow + if (threshold < (Long.MAX_VALUE / WEEKLY_THRESHOLD_MULTIPLIER)) { + finalThreshold = WEEKLY_THRESHOLD_MULTIPLIER * threshold; + } else { + finalThreshold = Long.MAX_VALUE; + } + id.setThreshold(finalThreshold); -//TODO : remove below after analysis - -// /** -// * fill out partition id based on compaction policy and date, threshold... -// * @param id Partition id to be filled out -// * @param firstDayOfCurrentMonth The first day in the current month -// * @param firstDayOfCurrentWeek The first day in the current week -// * @param dateStr Date string from the mob file -// * @param policy Mob compaction policy -// * @param calendar Calendar object -// * @param threshold Mob compaciton threshold configured -// * @return true if the file needs to be excluded from compaction -// */ -// public static boolean fillPartitionId(final CompactionPartitionId id, -// final Date firstDayOfCurrentMonth, final Date firstDayOfCurrentWeek, final String dateStr, -// final MobCompactPartitionPolicy policy, final Calendar calendar, final long threshold) { -// -// boolean skipCompcation = false; -// id.setThreshold(threshold); -// if (threshold <= 0) { -// id.setDate(dateStr); -// return skipCompcation; -// } -// -// long finalThreshold; -// Date date; -// try { -// date = MobUtils.parseDate(dateStr); -// } catch (ParseException e) { -// LOG.warn("Failed to parse date " + dateStr, e); -// id.setDate(dateStr); -// return true; -// } -// -// /* The algorithm works as follows: -// * For monthly policy: -// * 1). If the file's date is in past months, apply 4 * 7 * threshold -// * 2). If the file's date is in past weeks, apply 7 * threshold -// * 3). If the file's date is in current week, exclude it from the compaction -// * For weekly policy: -// * 1). If the file's date is in past weeks, apply 7 * threshold -// * 2). If the file's date in currently, apply threshold -// * For daily policy: -// * 1). apply threshold -// */ -// if (policy == MobCompactPartitionPolicy.MONTHLY) { -// if (date.before(firstDayOfCurrentMonth)) { -// // Check overflow -// if (threshold < (Long.MAX_VALUE / MONTHLY_THRESHOLD_MULTIPLIER)) { -// finalThreshold = MONTHLY_THRESHOLD_MULTIPLIER * threshold; -// } else { -// finalThreshold = Long.MAX_VALUE; -// } -// id.setThreshold(finalThreshold); -// -// // set to the date for the first day of that month -// id.setDate(MobUtils.formatDate(MobUtils.getFirstDayOfMonth(calendar, date))); -// return skipCompcation; -// } -// } -// -// if ((policy == MobCompactPartitionPolicy.MONTHLY) || -// (policy == MobCompactPartitionPolicy.WEEKLY)) { -// // Check if it needs to apply weekly multiplier -// if (date.before(firstDayOfCurrentWeek)) { -// // Check overflow -// if (threshold < (Long.MAX_VALUE / WEEKLY_THRESHOLD_MULTIPLIER)) { -// finalThreshold = WEEKLY_THRESHOLD_MULTIPLIER * threshold; -// } else { -// finalThreshold = Long.MAX_VALUE; -// } -// id.setThreshold(finalThreshold); -// -// id.setDate(MobUtils.formatDate(MobUtils.getFirstDayOfWeek(calendar, date))); -// return skipCompcation; -// } else if (policy == MobCompactPartitionPolicy.MONTHLY) { -// skipCompcation = true; -// } -// } -// -// // Rest is daily -// id.setDate(dateStr); -// return skipCompcation; -// } + id.setDate(MobUtils.formatDate(MobUtils.getFirstDayOfWeek(calendar, date))); + return skipCompcation; + } else if (policy == MobCompactPartitionPolicy.MONTHLY) { + skipCompcation = true; + } + } + + // Rest is daily + id.setDate(dateStr); + return skipCompcation; + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactionRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactionRequest.java new file mode 100644 index 000000000000..5025ed261229 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactionRequest.java @@ -0,0 +1,64 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.mob.compactions; + +import org.apache.yetus.audience.InterfaceAudience; + +/** + * The compaction request for mob files. + */ +@InterfaceAudience.Private +public abstract class MobCompactionRequest { + + protected long selectionTime; + protected CompactionType type = CompactionType.PART_FILES; + + public void setCompactionType(CompactionType type) { + this.type = type; + } + + /** + * Gets the selection time. + * @return The selection time. + */ + public long getSelectionTime() { + return this.selectionTime; + } + + /** + * Gets the compaction type. + * @return The compaction type. + */ + public CompactionType getCompactionType() { + return type; + } + + protected enum CompactionType { + + /** + * Part of mob files are selected. + */ + PART_FILES, + + /** + * All of mob files are selected. + */ + ALL_FILES + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java new file mode 100644 index 000000000000..c5d93ea6e064 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java @@ -0,0 +1,90 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.mob.compactions; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.ExecutorService; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.TableName; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.mob.MobUtils; +import org.apache.hadoop.hbase.util.FSUtils; + +/** + * A mob compactor to directly compact the mob files. + */ +@InterfaceAudience.Private +public abstract class MobCompactor { + + protected FileSystem fs; + protected Configuration conf; + protected TableName tableName; + protected ColumnFamilyDescriptor column; + + protected Path mobTableDir; + protected Path mobFamilyDir; + protected ExecutorService pool; + + public MobCompactor(Configuration conf, FileSystem fs, TableName tableName, + ColumnFamilyDescriptor column, ExecutorService pool) { + this.conf = conf; + this.fs = fs; + this.tableName = tableName; + this.column = column; + this.pool = pool; + mobTableDir = FSUtils.getTableDir(MobUtils.getMobHome(conf), tableName); + mobFamilyDir = MobUtils.getMobFamilyPath(conf, tableName, column.getNameAsString()); + } + + /** + * Compacts the mob files for the current column family. + * @return The paths of new mob files generated in the compaction. + * @throws IOException + */ + public List compact() throws IOException { + return compact(false); + } + + /** + * Compacts the mob files by compaction type for the current column family. + * @param allFiles Whether add all mob files into the compaction. + * @return The paths of new mob files generated in the compaction. + * @throws IOException + */ + public List compact(boolean allFiles) throws IOException { + return compact(Arrays.asList(fs.listStatus(mobFamilyDir)), allFiles); + } + + /** + * Compacts the candidate mob files. + * @param files The candidate mob files. + * @param allFiles Whether add all mob files into the compaction. + * @return The paths of new mob files generated in the compaction. + * @throws IOException + */ + public abstract List compact(List files, boolean allFiles) + throws IOException; +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactionRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactionRequest.java new file mode 100644 index 000000000000..ab917a2d9228 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactionRequest.java @@ -0,0 +1,333 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.mob.compactions; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.regionserver.HStoreFile; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * An implementation of {@link MobCompactionRequest} that is used in + * {@link PartitionedMobCompactor}. + * The mob files that have the same start key and date in their names belong to + * the same partition. + */ +@InterfaceAudience.Private +public class PartitionedMobCompactionRequest extends MobCompactionRequest { + + protected List delPartitions; + protected Collection compactionPartitions; + + public PartitionedMobCompactionRequest(Collection compactionPartitions, + List delPartitions) { + this.selectionTime = EnvironmentEdgeManager.currentTime(); + this.compactionPartitions = compactionPartitions; + this.delPartitions = delPartitions; + } + + /** + * Gets the compaction partitions. + * @return The compaction partitions. + */ + public Collection getCompactionPartitions() { + return this.compactionPartitions; + } + + /** + * Gets the del files. + * @return The del files. + */ + public List getDelPartitions() { + return this.delPartitions; + } + + /** + * The partition in the mob compaction. + * The mob files that have the same start key and date in their names belong to + * the same partition. + */ + protected static class CompactionPartition { + private List files = new ArrayList<>(); + private CompactionPartitionId partitionId; + + // The startKey and endKey of this partition, both are inclusive. + private byte[] startKey; + private byte[] endKey; + + public CompactionPartition(CompactionPartitionId partitionId) { + this.partitionId = partitionId; + } + + public CompactionPartitionId getPartitionId() { + return this.partitionId; + } + + public void addFile(FileStatus file) { + files.add(file); + } + + public List listFiles() { + return Collections.unmodifiableList(files); + } + + public int getFileCount () { + return files.size(); + } + + public byte[] getStartKey() { + return startKey; + } + + /** + * Set start key of this partition, only if the input startKey is less than + * the current start key. + */ + public void setStartKey(final byte[] startKey) { + if ((this.startKey == null) || (Bytes.compareTo(startKey, this.startKey) < 0)) { + this.startKey = startKey; + } + } + + public byte[] getEndKey() { + return endKey; + } + + /** + * Set end key of this partition, only if the input endKey is greater than + * the current end key. + */ + public void setEndKey(final byte[] endKey) { + if ((this.endKey == null) || (Bytes.compareTo(endKey, this.endKey) > 0)) { + this.endKey = endKey; + } + } + } + + /** + * The partition id that consists of start key and date of the mob file name. + */ + public static class CompactionPartitionId { + private String startKey; + private String date; + private String latestDate; + private long threshold; + + public CompactionPartitionId() { + // initialize these fields to empty string + this.startKey = ""; + this.date = ""; + this.latestDate = ""; + this.threshold = 0; + } + + public CompactionPartitionId(String startKey, String date) { + if (startKey == null || date == null) { + throw new IllegalArgumentException("Neither of start key and date could be null"); + } + this.startKey = startKey; + this.date = date; + this.latestDate = ""; + this.threshold = 0; + } + + public void setThreshold (final long threshold) { + this.threshold = threshold; + } + + public long getThreshold () { + return this.threshold; + } + + public String getStartKey() { + return this.startKey; + } + + public void setStartKey(final String startKey) { + this.startKey = startKey; + } + + public String getDate() { + return this.date; + } + + public void setDate(final String date) { + this.date = date; + } + + public String getLatestDate () { return this.latestDate; } + + public void updateLatestDate(final String latestDate) { + if (this.latestDate.compareTo(latestDate) < 0) { + this.latestDate = latestDate; + } + } + + @Override + public int hashCode() { + int result = 17; + result = 31 * result + startKey.hashCode(); + result = 31 * result + date.hashCode(); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (!(obj instanceof CompactionPartitionId)) { + return false; + } + CompactionPartitionId another = (CompactionPartitionId) obj; + if (!this.startKey.equals(another.startKey)) { + return false; + } + if (!this.date.equals(another.date)) { + return false; + } + return true; + } + + @Override + public String toString() { + return new StringBuilder(startKey).append(date).toString(); + } + } + + /** + * The delete file partition in the mob compaction. + * The delete partition is defined as [startKey, endKey] pair. + * The mob delete files that have the same start key and end key belong to + * the same partition. + */ + protected static class CompactionDelPartition { + private List delFiles = new ArrayList(); + private List storeFiles = new ArrayList<>(); + private CompactionDelPartitionId id; + + public CompactionDelPartition(CompactionDelPartitionId id) { + this.id = id; + } + + public CompactionDelPartitionId getId() { + return this.id; + } + + void addDelFile(FileStatus file) { + delFiles.add(file.getPath()); + } + public void addStoreFile(HStoreFile file) { + storeFiles.add(file); + } + + public List getStoreFiles() { + return storeFiles; + } + + List listDelFiles() { + return Collections.unmodifiableList(delFiles); + } + + void addDelFileList(final Collection list) { + delFiles.addAll(list); + } + + int getDelFileCount () { + return delFiles.size(); + } + + void cleanDelFiles() { + delFiles.clear(); + } + } + + /** + * The delete partition id that consists of start key and end key + */ + public static class CompactionDelPartitionId implements Comparable { + private byte[] startKey; + private byte[] endKey; + + public CompactionDelPartitionId() { + } + + public CompactionDelPartitionId(final byte[] startKey, final byte[] endKey) { + this.startKey = startKey; + this.endKey = endKey; + } + + public byte[] getStartKey() { + return this.startKey; + } + public void setStartKey(final byte[] startKey) { + this.startKey = startKey; + } + + public byte[] getEndKey() { + return this.endKey; + } + public void setEndKey(final byte[] endKey) { + this.endKey = endKey; + } + + @Override + public int compareTo(CompactionDelPartitionId o) { + /* + * 1). Compare the start key, if the k1 < k2, then k1 is less + * 2). If start Key is same, check endKey, k1 < k2, k1 is less + * If both are same, then they are equal. + */ + int result = Bytes.compareTo(this.startKey, o.getStartKey()); + if (result != 0) { + return result; + } + + return Bytes.compareTo(this.endKey, o.getEndKey()); + } + + @Override + public int hashCode() { + int result = 17; + result = 31 * result + java.util.Arrays.hashCode(startKey); + result = 31 * result + java.util.Arrays.hashCode(endKey); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (!(obj instanceof CompactionDelPartitionId)) { + return false; + } + CompactionDelPartitionId another = (CompactionDelPartitionId) obj; + + return (this.compareTo(another) == 0); + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java new file mode 100644 index 000000000000..a5823ec7d7bb --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java @@ -0,0 +1,929 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.mob.compactions; + +import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TIME_KEY; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_CELLS_COUNT; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.SKIP_RESET_SEQ_ID; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Calendar; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.NavigableMap; +import java.util.Objects; +import java.util.TreeMap; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.ArrayBackedTag; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.TagType; +import org.apache.hadoop.hbase.TagUtil; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.io.HFileLink; +import org.apache.hadoop.hbase.io.crypto.Encryption; +import org.apache.hadoop.hbase.io.hfile.CacheConfig; +import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.hadoop.hbase.io.hfile.HFile.Reader; +import org.apache.hadoop.hbase.mob.MobConstants; +import org.apache.hadoop.hbase.mob.MobFileName; +import org.apache.hadoop.hbase.mob.MobUtils; +import org.apache.hadoop.hbase.mob.compactions.MobCompactionRequest.CompactionType; +import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionDelPartition; +import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionDelPartitionId; +import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartition; +import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.HStoreFile; +import org.apache.hadoop.hbase.regionserver.ScanInfo; +import org.apache.hadoop.hbase.regionserver.ScanType; +import org.apache.hadoop.hbase.regionserver.ScannerContext; +import org.apache.hadoop.hbase.regionserver.StoreFileInfo; +import org.apache.hadoop.hbase.regionserver.StoreFileScanner; +import org.apache.hadoop.hbase.regionserver.StoreFileWriter; +import org.apache.hadoop.hbase.regionserver.StoreScanner; +import org.apache.hadoop.hbase.security.EncryptionUtil; +import org.apache.hadoop.hbase.tool.BulkLoadHFiles; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + +/** + * An implementation of {@link MobCompactor} that compacts the mob files in partitions. + */ +@InterfaceAudience.Private +public class PartitionedMobCompactor extends MobCompactor { + + private static final Logger LOG = LoggerFactory.getLogger(PartitionedMobCompactor.class); + protected long mergeableSize; + protected int delFileMaxCount; + /** The number of files compacted in a batch */ + protected int compactionBatchSize; + protected int compactionKVMax; + + private final Path tempPath; + private final Path bulkloadPath; + private final CacheConfig compactionCacheConfig; + private final byte[] refCellTags; + private Encryption.Context cryptoContext = Encryption.Context.NONE; + + public PartitionedMobCompactor(Configuration conf, FileSystem fs, TableName tableName, + ColumnFamilyDescriptor column, ExecutorService pool) throws IOException { + super(conf, fs, tableName, column, pool); + mergeableSize = conf.getLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, + MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD); + delFileMaxCount = conf.getInt(MobConstants.MOB_DELFILE_MAX_COUNT, + MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT); + // default is 100 + compactionBatchSize = conf.getInt(MobConstants.MOB_COMPACTION_BATCH_SIZE, + MobConstants.DEFAULT_MOB_COMPACTION_BATCH_SIZE); + tempPath = new Path(MobUtils.getMobHome(conf), MobConstants.TEMP_DIR_NAME); + bulkloadPath = new Path(tempPath, new Path(MobConstants.BULKLOAD_DIR_NAME, new Path( + tableName.getNamespaceAsString(), tableName.getQualifierAsString()))); + compactionKVMax = this.conf.getInt(HConstants.COMPACTION_KV_MAX, + HConstants.COMPACTION_KV_MAX_DEFAULT); + Configuration copyOfConf = new Configuration(conf); + copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f); + compactionCacheConfig = new CacheConfig(copyOfConf); + List tags = new ArrayList<>(2); + tags.add(MobConstants.MOB_REF_TAG); + Tag tableNameTag = new ArrayBackedTag(TagType.MOB_TABLE_NAME_TAG_TYPE, tableName.getName()); + tags.add(tableNameTag); + this.refCellTags = TagUtil.fromList(tags); + cryptoContext = EncryptionUtil.createEncryptionContext(copyOfConf, column); + } + + @Override + public List compact(List files, boolean allFiles) throws IOException { + if (files == null || files.isEmpty()) { + LOG.info("No candidate mob files"); + return null; + } + LOG.info("is allFiles: " + allFiles); + + // find the files to compact. + PartitionedMobCompactionRequest request = select(files, allFiles); + // compact the files. + return performCompaction(request); + } + + /** + * Selects the compacted mob/del files. + * Iterates the candidates to find out all the del files and small mob files. + * @param candidates All the candidates. + * @param allFiles Whether add all mob files into the compaction. + * @return A compaction request. + * @throws IOException if IO failure is encountered + */ + protected PartitionedMobCompactionRequest select(List candidates, + boolean allFiles) throws IOException { + final Map filesToCompact = new HashMap<>(); + final CompactionPartitionId id = new CompactionPartitionId(); + final NavigableMap delFilesToCompact = new TreeMap<>(); + final CompactionDelPartitionId delId = new CompactionDelPartitionId(); + final ArrayList allDelPartitions = new ArrayList<>(); + int selectedFileCount = 0; + int irrelevantFileCount = 0; + int totalDelFiles = 0; + MobCompactPartitionPolicy policy = column.getMobCompactPartitionPolicy(); + + Calendar calendar = Calendar.getInstance(); + Date currentDate = new Date(); + Date firstDayOfCurrentMonth = null; + Date firstDayOfCurrentWeek = null; + + if (policy == MobCompactPartitionPolicy.MONTHLY) { + firstDayOfCurrentMonth = MobUtils.getFirstDayOfMonth(calendar, currentDate); + firstDayOfCurrentWeek = MobUtils.getFirstDayOfWeek(calendar, currentDate); + } else if (policy == MobCompactPartitionPolicy.WEEKLY) { + firstDayOfCurrentWeek = MobUtils.getFirstDayOfWeek(calendar, currentDate); + } + + // We check if there is any del files so the logic can be optimized for the following processing + // First step is to check if there is any delete files. If there is any delete files, + // For each Partition, it needs to read its startKey and endKey from files. + // If there is no delete file, there is no need to read startKey and endKey from files, this + // is an optimization. + boolean withDelFiles = false; + for (FileStatus file : candidates) { + if (!file.isFile()) { + continue; + } + // group the del files and small files. + FileStatus linkedFile = file; + if (HFileLink.isHFileLink(file.getPath())) { + HFileLink link = HFileLink.buildFromHFileLinkPattern(conf, file.getPath()); + linkedFile = getLinkedFileStatus(link); + if (linkedFile == null) { + continue; + } + } + if (StoreFileInfo.isDelFile(linkedFile.getPath())) { + withDelFiles = true; + break; + } + } + + for (FileStatus file : candidates) { + if (!file.isFile()) { + irrelevantFileCount++; + continue; + } + // group the del files and small files. + FileStatus linkedFile = file; + if (HFileLink.isHFileLink(file.getPath())) { + HFileLink link = HFileLink.buildFromHFileLinkPattern(conf, file.getPath()); + linkedFile = getLinkedFileStatus(link); + if (linkedFile == null) { + // If the linked file cannot be found, regard it as an irrelevantFileCount file + irrelevantFileCount++; + continue; + } + } + if (withDelFiles && StoreFileInfo.isDelFile(linkedFile.getPath())) { + // File in the Del Partition List + + // Get delId from the file + try (Reader reader = HFile.createReader(fs, linkedFile.getPath(), conf)) { + delId.setStartKey(reader.getFirstRowKey().get()); + delId.setEndKey(reader.getLastRowKey().get()); + } + CompactionDelPartition delPartition = delFilesToCompact.get(delId); + if (delPartition == null) { + CompactionDelPartitionId newDelId = + new CompactionDelPartitionId(delId.getStartKey(), delId.getEndKey()); + delPartition = new CompactionDelPartition(newDelId); + delFilesToCompact.put(newDelId, delPartition); + } + delPartition.addDelFile(file); + totalDelFiles ++; + } else { + String fileName = linkedFile.getPath().getName(); + String date = MobFileName.getDateFromName(fileName); + boolean skipCompaction = MobUtils + .fillPartitionId(id, firstDayOfCurrentMonth, firstDayOfCurrentWeek, date, policy, + calendar, mergeableSize); + if (allFiles || (!skipCompaction && (linkedFile.getLen() < id.getThreshold()))) { + // add all files if allFiles is true, + // otherwise add the small files to the merge pool + // filter out files which are not supposed to be compacted with the + // current policy + + id.setStartKey(MobFileName.getStartKeyFromName(fileName)); + CompactionPartition compactionPartition = filesToCompact.get(id); + if (compactionPartition == null) { + CompactionPartitionId newId = new CompactionPartitionId(id.getStartKey(), id.getDate()); + compactionPartition = new CompactionPartition(newId); + compactionPartition.addFile(file); + filesToCompact.put(newId, compactionPartition); + newId.updateLatestDate(date); + } else { + compactionPartition.addFile(file); + compactionPartition.getPartitionId().updateLatestDate(date); + } + + if (withDelFiles) { + // get startKey and endKey from the file and update partition + // TODO: is it possible to skip read of most hfiles? + try (Reader reader = HFile.createReader(fs, linkedFile.getPath(), conf)) { + compactionPartition.setStartKey(reader.getFirstRowKey().get()); + compactionPartition.setEndKey(reader.getLastRowKey().get()); + } + } + + selectedFileCount++; + } + } + } + + /* + * Merge del files so there are only non-overlapped del file lists + */ + for(Map.Entry entry : delFilesToCompact.entrySet()) { + if (allDelPartitions.size() > 0) { + // check if the current key range overlaps the previous one + CompactionDelPartition prev = allDelPartitions.get(allDelPartitions.size() - 1); + if (Bytes.compareTo(prev.getId().getEndKey(), entry.getKey().getStartKey()) >= 0) { + // merge them together + prev.getId().setEndKey(entry.getValue().getId().getEndKey()); + prev.addDelFileList(entry.getValue().listDelFiles()); + + } else { + allDelPartitions.add(entry.getValue()); + } + } else { + allDelPartitions.add(entry.getValue()); + } + } + + PartitionedMobCompactionRequest request = new PartitionedMobCompactionRequest( + filesToCompact.values(), allDelPartitions); + if (candidates.size() == (totalDelFiles + selectedFileCount + irrelevantFileCount)) { + // all the files are selected + request.setCompactionType(CompactionType.ALL_FILES); + } + LOG.info("The compaction type is " + request.getCompactionType() + ", the request has " + + totalDelFiles + " del files, " + selectedFileCount + " selected files, and " + + irrelevantFileCount + " irrelevant files"); + return request; + } + + /** + * Performs the compaction on the selected files. + *
    + *
  1. Compacts the del files.
  2. + *
  3. Compacts the selected small mob files and all the del files.
  4. + *
  5. If all the candidates are selected, delete the del files.
  6. + *
+ * @param request The compaction request. + * @return The paths of new mob files generated in the compaction. + * @throws IOException if IO failure is encountered + */ + protected List performCompaction(PartitionedMobCompactionRequest request) + throws IOException { + + // merge the del files, it is per del partition + for (CompactionDelPartition delPartition : request.getDelPartitions()) { + if (delPartition.getDelFileCount() <= 1) continue; + List newDelPaths = compactDelFiles(request, delPartition.listDelFiles()); + delPartition.cleanDelFiles(); + delPartition.addDelFileList(newDelPaths); + } + + List paths = null; + int totalDelFileCount = 0; + try { + for (CompactionDelPartition delPartition : request.getDelPartitions()) { + for (Path newDelPath : delPartition.listDelFiles()) { + HStoreFile sf = + new HStoreFile(fs, newDelPath, conf, compactionCacheConfig, BloomType.NONE, true); + // pre-create reader of a del file to avoid race condition when opening the reader in each + // partition. + sf.initReader(); + delPartition.addStoreFile(sf); + totalDelFileCount++; + } + } + LOG.info("After merging, there are " + totalDelFileCount + " del files"); + // compact the mob files by partitions. + paths = compactMobFiles(request); + LOG.info("After compaction, there are " + paths.size() + " mob files"); + } finally { + for (CompactionDelPartition delPartition : request.getDelPartitions()) { + closeStoreFileReaders(delPartition.getStoreFiles()); + } + } + + // archive the del files if all the mob files are selected. + if (request.type == CompactionType.ALL_FILES && !request.getDelPartitions().isEmpty()) { + LOG.info( + "After a mob compaction with all files selected, archiving the del files "); + for (CompactionDelPartition delPartition : request.getDelPartitions()) { + LOG.info(Objects.toString(delPartition.listDelFiles())); + try { + MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), + delPartition.getStoreFiles()); + } catch (IOException e) { + LOG.error("Failed to archive the del files " + delPartition.getStoreFiles(), e); + } + } + } + return paths; + } + + static class DelPartitionComparator implements Comparator { + private boolean compareStartKey; + + DelPartitionComparator(boolean compareStartKey) { + this.compareStartKey = compareStartKey; + } + + public boolean getCompareStartKey() { + return this.compareStartKey; + } + + public void setCompareStartKey(final boolean compareStartKey) { + this.compareStartKey = compareStartKey; + } + + @Override + public int compare(CompactionDelPartition o1, CompactionDelPartition o2) { + + if (compareStartKey) { + return Bytes.compareTo(o1.getId().getStartKey(), o2.getId().getStartKey()); + } else { + return Bytes.compareTo(o1.getId().getEndKey(), o2.getId().getEndKey()); + } + } + } + + @VisibleForTesting + List getListOfDelFilesForPartition(final CompactionPartition partition, + final List delPartitions) { + // Binary search for startKey and endKey + + List result = new ArrayList<>(); + + DelPartitionComparator comparator = new DelPartitionComparator(false); + CompactionDelPartitionId id = new CompactionDelPartitionId(null, partition.getStartKey()); + CompactionDelPartition target = new CompactionDelPartition(id); + int start = Collections.binarySearch(delPartitions, target, comparator); + + // Get the start index for partition + if (start < 0) { + // Calculate the insert point + start = (start + 1) * (-1); + if (start == delPartitions.size()) { + // no overlap + return result; + } else { + // Check another case which has no overlap + if (Bytes.compareTo(partition.getEndKey(), delPartitions.get(start).getId().getStartKey()) < 0) { + return result; + } + } + } + + // Search for end index for the partition + comparator.setCompareStartKey(true); + id.setStartKey(partition.getEndKey()); + int end = Collections.binarySearch(delPartitions, target, comparator); + + if (end < 0) { + end = (end + 1) * (-1); + if (end == 0) { + return result; + } else { + --end; + if (Bytes.compareTo(partition.getStartKey(), delPartitions.get(end).getId().getEndKey()) > 0) { + return result; + } + } + } + + for (int i = start; i <= end; ++i) { + result.addAll(delPartitions.get(i).getStoreFiles()); + } + + return result; + } + + /** + * Compacts the selected small mob files and all the del files. + * @param request The compaction request. + * @return The paths of new mob files after compactions. + * @throws IOException if IO failure is encountered + */ + protected List compactMobFiles(final PartitionedMobCompactionRequest request) + throws IOException { + Collection partitions = request.compactionPartitions; + if (partitions == null || partitions.isEmpty()) { + LOG.info("No partitions of mob files"); + return Collections.emptyList(); + } + List paths = new ArrayList<>(); + final Connection c = ConnectionFactory.createConnection(conf); + final Table table = c.getTable(tableName); + + try { + Map>> results = new HashMap<>(); + // compact the mob files by partitions in parallel. + for (final CompactionPartition partition : partitions) { + + // How to efficiently come up a list of delFiles for one partition? + // Search the delPartitions and collect all the delFiles for the partition + // One optimization can do is that if there is no del file, we do not need to + // come up with startKey/endKey. + List delFiles = getListOfDelFilesForPartition(partition, + request.getDelPartitions()); + + results.put(partition.getPartitionId(), pool.submit(new Callable>() { + @Override + public List call() throws Exception { + LOG.info("Compacting mob files for partition " + partition.getPartitionId()); + return compactMobFilePartition(request, partition, delFiles, c, table); + } + })); + } + // compact the partitions in parallel. + List failedPartitions = new ArrayList<>(); + for (Entry>> result : results.entrySet()) { + try { + paths.addAll(result.getValue().get()); + } catch (Exception e) { + // just log the error + LOG.error("Failed to compact the partition " + result.getKey(), e); + failedPartitions.add(result.getKey()); + } + } + if (!failedPartitions.isEmpty()) { + // if any partition fails in the compaction, directly throw an exception. + throw new IOException("Failed to compact the partitions " + failedPartitions); + } + } finally { + try { + table.close(); + } catch (IOException e) { + LOG.error("Failed to close the Table", e); + } + } + return paths; + } + + /** + * Compacts a partition of selected small mob files and all the del files. + * @param request The compaction request. + * @param partition A compaction partition. + * @param delFiles The del files. + * @param connection The connection to use. + * @param table The current table. + * @return The paths of new mob files after compactions. + * @throws IOException if IO failure is encountered + */ + private List compactMobFilePartition(PartitionedMobCompactionRequest request, + CompactionPartition partition, + List delFiles, + Connection connection, + Table table) throws IOException { + if (MobUtils.isMobFileExpired(column, EnvironmentEdgeManager.currentTime(), + partition.getPartitionId().getDate())) { + // If the files in the partition are expired, do not compact them and directly + // return an empty list. + return Collections.emptyList(); + } + List newFiles = new ArrayList<>(); + List files = partition.listFiles(); + int offset = 0; + Path bulkloadPathOfPartition = new Path(bulkloadPath, partition.getPartitionId().toString()); + Path bulkloadColumnPath = new Path(bulkloadPathOfPartition, column.getNameAsString()); + while (offset < files.size()) { + int batch = compactionBatchSize; + if (files.size() - offset < compactionBatchSize) { + batch = files.size() - offset; + } + if (batch == 1 && delFiles.isEmpty()) { + // only one file left and no del files, do not compact it, + // and directly add it to the new files. + newFiles.add(files.get(offset).getPath()); + offset++; + continue; + } + // clean the bulkload directory to avoid loading old files. + fs.delete(bulkloadPathOfPartition, true); + // add the selected mob files and del files into filesToCompact + List filesToCompact = new ArrayList<>(); + for (int i = offset; i < batch + offset; i++) { + HStoreFile sf = new HStoreFile(fs, files.get(i).getPath(), conf, compactionCacheConfig, + BloomType.NONE, true); + filesToCompact.add(sf); + } + filesToCompact.addAll(delFiles); + // compact the mob files in a batch. + compactMobFilesInBatch(request, partition, connection, table, filesToCompact, batch, + bulkloadPathOfPartition, bulkloadColumnPath, newFiles); + // move to the next batch. + offset += batch; + } + LOG.info("Compaction is finished. The number of mob files is changed from " + files.size() + + " to " + newFiles.size()); + return newFiles; + } + + /** + * Closes the readers of store files. + * @param storeFiles The store files to be closed. + */ + private void closeStoreFileReaders(List storeFiles) { + for (HStoreFile storeFile : storeFiles) { + try { + storeFile.closeStoreFile(true); + } catch (IOException e) { + LOG.warn("Failed to close the reader on store file " + storeFile.getPath(), e); + } + } + } + + /** + * Compacts a partition of selected small mob files and all the del files in a batch. + * @param request The compaction request. + * @param partition A compaction partition. + * @param connection To use for transport + * @param table The current table. + * @param filesToCompact The files to be compacted. + * @param batch The number of mob files to be compacted in a batch. + * @param bulkloadPathOfPartition The directory where the bulkload column of the current + * partition is saved. + * @param bulkloadColumnPath The directory where the bulkload files of current partition + * are saved. + * @param newFiles The paths of new mob files after compactions. + * @throws IOException if IO failure is encountered + */ + private void compactMobFilesInBatch(PartitionedMobCompactionRequest request, + CompactionPartition partition, + Connection connection, Table table, + List filesToCompact, int batch, + Path bulkloadPathOfPartition, Path bulkloadColumnPath, + List newFiles) + throws IOException { + // open scanner to the selected mob files and del files. + StoreScanner scanner = createScanner(filesToCompact, ScanType.COMPACT_DROP_DELETES); + // the mob files to be compacted, not include the del files. + List mobFilesToCompact = filesToCompact.subList(0, batch); + // Pair(maxSeqId, cellsCount) + Pair fileInfo = getFileInfo(mobFilesToCompact); + // open writers for the mob files and new ref store files. + StoreFileWriter writer = null; + StoreFileWriter refFileWriter = null; + Path filePath = null; + long mobCells = 0; + boolean cleanupTmpMobFile = false; + boolean cleanupBulkloadDirOfPartition = false; + boolean cleanupCommittedMobFile = false; + boolean closeReaders= true; + + try { + try { + writer = MobUtils + .createWriter(conf, fs, column, partition.getPartitionId().getLatestDate(), tempPath, + Long.MAX_VALUE, column.getCompactionCompressionType(), + partition.getPartitionId().getStartKey(), compactionCacheConfig, cryptoContext, + true); + cleanupTmpMobFile = true; + filePath = writer.getPath(); + byte[] fileName = Bytes.toBytes(filePath.getName()); + // create a temp file and open a writer for it in the bulkloadPath + refFileWriter = MobUtils.createRefFileWriter(conf, fs, column, bulkloadColumnPath, + fileInfo.getSecond().longValue(), compactionCacheConfig, cryptoContext, true); + cleanupBulkloadDirOfPartition = true; + List cells = new ArrayList<>(); + boolean hasMore; + ScannerContext scannerContext = + ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build(); + do { + hasMore = scanner.next(cells, scannerContext); + for (Cell cell : cells) { + // write the mob cell to the mob file. + writer.append(cell); + // write the new reference cell to the store file. + Cell reference = MobUtils.createMobRefCell(cell, fileName, this.refCellTags); + refFileWriter.append(reference); + mobCells++; + } + cells.clear(); + } while (hasMore); + } finally { + // close the scanner. + scanner.close(); + + if (cleanupTmpMobFile) { + // append metadata to the mob file, and close the mob file writer. + closeMobFileWriter(writer, fileInfo.getFirst(), mobCells); + } + + if (cleanupBulkloadDirOfPartition) { + // append metadata and bulkload info to the ref mob file, and close the writer. + closeRefFileWriter(refFileWriter, fileInfo.getFirst(), request.selectionTime); + } + } + + if (mobCells > 0) { + // commit mob file + MobUtils.commitFile(conf, fs, filePath, mobFamilyDir, compactionCacheConfig); + cleanupTmpMobFile = false; + cleanupCommittedMobFile = true; + // bulkload the ref file + bulkloadRefFile(table.getName(), bulkloadPathOfPartition, filePath.getName()); + cleanupCommittedMobFile = false; + newFiles.add(new Path(mobFamilyDir, filePath.getName())); + } + + // archive the old mob files, do not archive the del files. + try { + closeStoreFileReaders(mobFilesToCompact); + closeReaders = false; + MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), mobFilesToCompact); + } catch (IOException e) { + LOG.error("Failed to archive the files " + mobFilesToCompact, e); + } + } finally { + if (closeReaders) { + closeStoreFileReaders(mobFilesToCompact); + } + + if (cleanupTmpMobFile) { + deletePath(filePath); + } + + if (cleanupBulkloadDirOfPartition) { + // delete the bulkload files in bulkloadPath + deletePath(bulkloadPathOfPartition); + } + + if (cleanupCommittedMobFile) { + deletePath(new Path(mobFamilyDir, filePath.getName())); + } + } + } + + /** + * Compacts the del files in batches which avoids opening too many files. + * @param request The compaction request. + * @param delFilePaths Del file paths to compact + * @return The paths of new del files after merging or the original files if no merging + * is necessary. + * @throws IOException if IO failure is encountered + */ + protected List compactDelFiles(PartitionedMobCompactionRequest request, + List delFilePaths) throws IOException { + if (delFilePaths.size() <= delFileMaxCount) { + return delFilePaths; + } + // when there are more del files than the number that is allowed, merge it firstly. + int offset = 0; + List paths = new ArrayList<>(); + while (offset < delFilePaths.size()) { + // get the batch + int batch = compactionBatchSize; + if (delFilePaths.size() - offset < compactionBatchSize) { + batch = delFilePaths.size() - offset; + } + List batchedDelFiles = new ArrayList<>(); + if (batch == 1) { + // only one file left, do not compact it, directly add it to the new files. + paths.add(delFilePaths.get(offset)); + offset++; + continue; + } + for (int i = offset; i < batch + offset; i++) { + batchedDelFiles.add(new HStoreFile(fs, delFilePaths.get(i), conf, compactionCacheConfig, + BloomType.NONE, true)); + } + // compact the del files in a batch. + paths.add(compactDelFilesInBatch(request, batchedDelFiles)); + // move to the next batch. + offset += batch; + } + return compactDelFiles(request, paths); + } + + /** + * Compacts the del file in a batch. + * @param request The compaction request. + * @param delFiles The del files. + * @return The path of new del file after merging. + * @throws IOException if IO failure is encountered + */ + private Path compactDelFilesInBatch(PartitionedMobCompactionRequest request, + List delFiles) throws IOException { + // create a scanner for the del files. + StoreScanner scanner = createScanner(delFiles, ScanType.COMPACT_RETAIN_DELETES); + StoreFileWriter writer = null; + Path filePath = null; + try { + writer = MobUtils.createDelFileWriter(conf, fs, column, + MobUtils.formatDate(new Date(request.selectionTime)), tempPath, Long.MAX_VALUE, + column.getCompactionCompressionType(), HConstants.EMPTY_START_ROW, compactionCacheConfig, + cryptoContext); + filePath = writer.getPath(); + List cells = new ArrayList<>(); + boolean hasMore; + ScannerContext scannerContext = + ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build(); + do { + hasMore = scanner.next(cells, scannerContext); + for (Cell cell : cells) { + writer.append(cell); + } + cells.clear(); + } while (hasMore); + } finally { + scanner.close(); + if (writer != null) { + try { + writer.close(); + } catch (IOException e) { + LOG.error("Failed to close the writer of the file " + filePath, e); + } + } + } + // commit the new del file + Path path = MobUtils.commitFile(conf, fs, filePath, mobFamilyDir, compactionCacheConfig); + // archive the old del files + try { + MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), delFiles); + } catch (IOException e) { + LOG.error("Failed to archive the old del files " + delFiles, e); + } + return path; + } + + /** + * Creates a store scanner. + * @param filesToCompact The files to be compacted. + * @param scanType The scan type. + * @return The store scanner. + * @throws IOException if IO failure is encountered + */ + private StoreScanner createScanner(List filesToCompact, ScanType scanType) + throws IOException { + List scanners = StoreFileScanner.getScannersForStoreFiles(filesToCompact, + false, true, false, false, HConstants.LATEST_TIMESTAMP); + long ttl = HStore.determineTTLFromFamily(column); + ScanInfo scanInfo = new ScanInfo(conf, column, ttl, 0, CellComparator.getInstance()); + return new StoreScanner(scanInfo, scanType, scanners); + } + + /** + * Bulkloads the current file. + * @param tableName The table to load into. + * @param bulkloadDirectory The path of bulkload directory. + * @param fileName The current file name. + * @throws IOException if IO failure is encountered + */ + private void bulkloadRefFile(TableName tableName, Path bulkloadDirectory, String fileName) + throws IOException { + // bulkload the ref file + try { + BulkLoadHFiles.create(conf).bulkLoad(tableName, bulkloadDirectory); + } catch (Exception e) { + throw new IOException(e); + } + } + + /** + * Closes the mob file writer. + * @param writer The mob file writer. + * @param maxSeqId Maximum sequence id. + * @param mobCellsCount The number of mob cells. + * @throws IOException if IO failure is encountered + */ + private void closeMobFileWriter(StoreFileWriter writer, long maxSeqId, long mobCellsCount) + throws IOException { + if (writer != null) { + writer.appendMetadata(maxSeqId, false, mobCellsCount); + try { + writer.close(); + } catch (IOException e) { + LOG.error("Failed to close the writer of the file " + writer.getPath(), e); + } + } + } + + /** + * Closes the ref file writer. + * @param writer The ref file writer. + * @param maxSeqId Maximum sequence id. + * @param bulkloadTime The timestamp at which the bulk load file is created. + * @throws IOException if IO failure is encountered + */ + private void closeRefFileWriter(StoreFileWriter writer, long maxSeqId, long bulkloadTime) + throws IOException { + if (writer != null) { + writer.appendMetadata(maxSeqId, false); + writer.appendFileInfo(BULKLOAD_TIME_KEY, Bytes.toBytes(bulkloadTime)); + writer.appendFileInfo(SKIP_RESET_SEQ_ID, Bytes.toBytes(true)); + try { + writer.close(); + } catch (IOException e) { + LOG.error("Failed to close the writer of the ref file " + writer.getPath(), e); + } + } + } + + /** + * Gets the max seqId and number of cells of the store files. + * @param storeFiles The store files. + * @return The pair of the max seqId and number of cells of the store files. + * @throws IOException if IO failure is encountered + */ + private Pair getFileInfo(List storeFiles) throws IOException { + long maxSeqId = 0; + long maxKeyCount = 0; + for (HStoreFile sf : storeFiles) { + // the readers will be closed later after the merge. + maxSeqId = Math.max(maxSeqId, sf.getMaxSequenceId()); + sf.initReader(); + byte[] count = sf.getReader().loadFileInfo().get(MOB_CELLS_COUNT); + if (count != null) { + maxKeyCount += Bytes.toLong(count); + } + } + return new Pair<>(maxSeqId, maxKeyCount); + } + + /** + * Deletes a file. + * @param path The path of the file to be deleted. + */ + private void deletePath(Path path) { + try { + if (path != null) { + fs.delete(path, true); + } + } catch (IOException e) { + LOG.error("Failed to delete the file " + path, e); + } + } + + private FileStatus getLinkedFileStatus(HFileLink link) throws IOException { + Path[] locations = link.getLocations(); + FileStatus file; + for (Path location : locations) { + + if (location != null) { + try { + file = fs.getFileStatus(location); + if (file != null) { + return file; + } + } catch (FileNotFoundException e) { + } + } + } + LOG.warn("The file " + link + " links to can not be found"); + return null; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java index ee5aa6cb2580..b8ea9609e2bc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java @@ -193,6 +193,28 @@ public StoreFileWriter createWriterInTmp(Date date, long maxKeyCount, isCompaction); } + /** + * Creates the writer for the del file in temp directory. + * The del file keeps tracking the delete markers. Its name has a suffix _del, + * the format is [0-9a-f]+(_del)?. + * @param date The latest date of written cells. + * @param maxKeyCount The key count. + * @param compression The compression algorithm. + * @param startKey The start key. + * @return The writer for the del file. + * @throws IOException + */ + public StoreFileWriter createDelFileWriterInTmp(Date date, long maxKeyCount, + Compression.Algorithm compression, byte[] startKey) throws IOException { + if (startKey == null) { + startKey = HConstants.EMPTY_START_ROW; + } + Path path = getTempDir(); + String suffix = UUID + .randomUUID().toString().replaceAll("-", "") + "_del"; + MobFileName mobFileName = MobFileName.create(startKey, MobUtils.formatDate(date), suffix); + return createWriterInTmp(mobFileName, path, maxKeyCount, compression, true); + } /** * Creates the writer for the mob file in temp directory. @@ -209,7 +231,7 @@ public StoreFileWriter createWriterInTmp(String date, Path basePath, long maxKey Compression.Algorithm compression, byte[] startKey, boolean isCompaction) throws IOException { MobFileName mobFileName = MobFileName.create(startKey, date, UUID.randomUUID() - .toString().replaceAll("-", ""), region.getRegionInfo().getEncodedName()); + .toString().replaceAll("-", "")); return createWriterInTmp(mobFileName, basePath, maxKeyCount, compression, isCompaction); } @@ -243,7 +265,7 @@ public void commitFile(final Path sourceFile, Path targetPath) throws IOExceptio } Path dstPath = new Path(targetPath, sourceFile.getName()); validateMobFile(sourceFile); - String msg = " FLUSH Renaming flushed file from " + sourceFile + " to " + dstPath; + String msg = "Renaming flushed file from " + sourceFile + " to " + dstPath; LOG.info(msg); Path parent = dstPath.getParent(); if (!region.getFilesystem().exists(parent)) { @@ -287,19 +309,6 @@ public MobCell resolve(Cell reference, boolean cacheBlocks) throws IOException { return resolve(reference, cacheBlocks, -1, true); } - /** - * Reads the cell from the mob file with readEmptyValueOnMobCellMiss - * @param reference - * @param cacheBlocks - * @param readEmptyValueOnMobCellMiss - * @return The cell found in the mob file. - * @throws IOException - */ - public MobCell resolve(Cell reference, boolean cacheBlocks, boolean readEmptyValueOnMobCellMiss) - throws IOException { - return resolve(reference, cacheBlocks, -1, readEmptyValueOnMobCellMiss); - } - /** * Reads the cell from the mob file. * @param reference The cell found in the HBase, its value is a path to a mob file. @@ -511,6 +520,4 @@ public long getMobScanCellsSize() { public byte[] getRefCellTags() { return this.refCellTags; } - - } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index 32d0f0bde277..447bc736789a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -1905,7 +1905,7 @@ public void cancelRequestedCompaction(CompactionContext compaction) { finishCompactionRequest(compaction.getRequest()); } - protected void finishCompactionRequest(CompactionRequestImpl cr) { + private void finishCompactionRequest(CompactionRequestImpl cr) { this.region.reportCompactionRequestEnd(cr.isMajor(), cr.getFiles().size(), cr.getSize()); if (cr.isOffPeak()) { offPeakCompactionTracker.set(false); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java index 37c513ef77e2..682185a89a10 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java @@ -108,9 +108,6 @@ public class HStoreFile implements StoreFile { /** Key for the number of mob cells in metadata */ public static final byte[] MOB_CELLS_COUNT = Bytes.toBytes("MOB_CELLS_COUNT"); - /** Key for the list of MOB file references */ - public static final byte[] MOB_FILE_REFS = Bytes.toBytes("MOB_FILE_REFS"); - /** Meta key set when store file is a result of a bulk load */ public static final byte[] BULKLOAD_TASK_KEY = Bytes.toBytes("BULKLOAD_SOURCE_TASK"); public static final byte[] BULKLOAD_TIME_KEY = Bytes.toBytes("BULKLOAD_TIMESTAMP"); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java index 61cd34c6d235..76144f08d5dc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java @@ -114,5 +114,4 @@ public void close() { LOG.warn("Failed to free referenced mob cells: ", e); } } - } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java index ee0ed072f304..54e278031959 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java @@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.io.HalfStoreFileReader; import org.apache.hadoop.hbase.io.Reference; import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.util.FSUtils; /** @@ -135,7 +134,7 @@ public StoreFileInfo(final Configuration conf, final FileSystem fs, final Path i } if (LOG.isTraceEnabled()) LOG.trace(p + " is a " + reference.getFileRegion() + " reference to " + referencePath); - } else if (isHFile(p) || isMobFile(p)) { + } else if (isHFile(p)) { // HFile this.createdTimestamp = fs.getFileStatus(initialPath).getModificationTime(); this.reference = null; @@ -406,16 +405,6 @@ public static boolean isHFile(final String fileName) { return m.matches() && m.groupCount() > 0; } - public static boolean isMobFile(final Path path) { - String fileName = path.getName(); - String[] parts = fileName.split(MobUtils.SEP); - if (parts.length != 2) { - return false; - } - Matcher m = HFILE_NAME_PATTERN.matcher(parts[0]); - Matcher mm = HFILE_NAME_PATTERN.matcher(parts[1]); - return m.matches() && mm.matches(); - } /** * @param path Path to check. * @return True if the path has format of a del file. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java index 522efb5ea18d..7b4fb4a0c06f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java @@ -25,7 +25,6 @@ import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY; import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAX_SEQ_ID_KEY; import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_CELLS_COUNT; -import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_FILE_REFS; import static org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY; import java.io.IOException; @@ -243,28 +242,6 @@ public void appendMetadata(final long maxSequenceId, final boolean majorCompacti appendTrackedTimestampsToMetadata(); } - /** - * Appends MOB - specific metadata - * @param mobRefSet - set of MOB file names - * @throws IOException problem writing to FS - */ - public void appendMobMetadata(Set mobRefSet) throws IOException { - if (mobRefSet.isEmpty()) { - return; - } - StringBuilder sb = new StringBuilder(2 * mobRefSet.size() - 1); - String[] arr = new String[mobRefSet.size()]; - arr = mobRefSet.toArray(arr); - for (int i = 0; i < arr.length; i++) { - sb.append(arr[i]); - if (i < arr.length - 1) { - sb.append(","); - } - } - byte[] bytes = sb.toString().getBytes(); - writer.appendFileInfo(MOB_FILE_REFS, bytes); - } - /** * Add TimestampRange and earliest put timestamp to Metadata */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java index 3eab002dba5a..503325a1a374 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java @@ -115,7 +115,7 @@ public CompactionProgress getProgress() { } /** The sole reason this class exists is that java has no ref/out/pointer parameters. */ - public static class FileDetails { + protected static class FileDetails { /** Maximum key count after compaction (for blooms) */ public long maxKeyCount = 0; /** Earliest put timestamp if major compaction */ diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java deleted file mode 100644 index 6527b11b306e..000000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java +++ /dev/null @@ -1,355 +0,0 @@ -/** - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.mob; - -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.InterruptedIOException; -import java.util.ArrayList; -import java.util.Date; -import java.util.List; -import java.util.Random; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.PrivateCellUtil; -import org.apache.hadoop.hbase.io.hfile.CorruptHFileException; -import org.apache.hadoop.hbase.regionserver.CellSink; -import org.apache.hadoop.hbase.regionserver.HStore; -import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.KeyValueScanner; -import org.apache.hadoop.hbase.regionserver.ScannerContext; -import org.apache.hadoop.hbase.regionserver.ShipperListener; -import org.apache.hadoop.hbase.regionserver.StoreFileWriter; -import org.apache.hadoop.hbase.regionserver.throttle.ThroughputControlUtil; -import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.yetus.audience.InterfaceAudience; - -@InterfaceAudience.Private -public class FaultyMobStoreCompactor extends DefaultMobStoreCompactor -{ - - public static AtomicLong mobCounter = new AtomicLong(); - public static AtomicLong totalFailures = new AtomicLong(); - public static AtomicLong totalCompactions = new AtomicLong(); - public static AtomicLong totalMajorCompactions = new AtomicLong(); - - static double failureProb = 0.1d; - static Random rnd = new Random(); - - - public FaultyMobStoreCompactor(Configuration conf, HStore store) { - super(conf, store); - failureProb = conf.getDouble("injected.fault.probability", 0.1); - } - - @Override - protected boolean performCompaction(FileDetails fd, InternalScanner scanner, CellSink writer, - long smallestReadPoint, boolean cleanSeqId, ThroughputController throughputController, - boolean major, int numofFilesToCompact) throws IOException { - - totalCompactions.incrementAndGet(); - if (major) { - totalMajorCompactions.incrementAndGet(); - } - long bytesWrittenProgressForCloseCheck = 0; - long bytesWrittenProgressForLog = 0; - long bytesWrittenProgressForShippedCall = 0; - // Clear old mob references - mobRefSet.get().clear(); - boolean isUserRequest = userRequest.get(); - boolean compactMOBs = major && isUserRequest; - boolean discardMobMiss = conf.getBoolean(MobConstants.MOB_DISCARD_MISS_KEY, - MobConstants.DEFAULT_MOB_DISCARD_MISS); - - boolean mustFail = false; - if (compactMOBs) { - mobCounter.incrementAndGet(); - double dv = rnd.nextDouble(); - if (dv < failureProb) { - mustFail = true; - totalFailures.incrementAndGet(); - } - } - - FileSystem fs = FileSystem.get(conf); - - // Since scanner.next() can return 'false' but still be delivering data, - // we have to use a do/while loop. - List cells = new ArrayList<>(); - // Limit to "hbase.hstore.compaction.kv.max" (default 10) to avoid OOME - int closeCheckSizeLimit = HStore.getCloseCheckInterval(); - long lastMillis = 0; - if (LOG.isDebugEnabled()) { - lastMillis = EnvironmentEdgeManager.currentTime(); - } - String compactionName = ThroughputControlUtil.getNameForThrottling(store, "compaction"); - long now = 0; - boolean hasMore; - Path path = MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName()); - byte[] fileName = null; - StoreFileWriter mobFileWriter = null; - long mobCells = 0; - long cellsCountCompactedToMob = 0, cellsCountCompactedFromMob = 0; - long cellsSizeCompactedToMob = 0, cellsSizeCompactedFromMob = 0; - boolean finished = false; - - ScannerContext scannerContext = - ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build(); - throughputController.start(compactionName); - KeyValueScanner kvs = (scanner instanceof KeyValueScanner)? (KeyValueScanner)scanner : null; - long shippedCallSizeLimit = - (long) numofFilesToCompact * this.store.getColumnFamilyDescriptor().getBlocksize(); - - MobCell mobCell = null; - - - long counter = 0; - long countFailAt = -1; - if (mustFail) { - countFailAt = rnd.nextInt(100); // randomly fail fast - } - - try { - try { - // If the mob file writer could not be created, directly write the cell to the store file. - mobFileWriter = mobStore.createWriterInTmp(new Date(fd.latestPutTs), fd.maxKeyCount, - compactionCompression, store.getRegionInfo().getStartKey(), true); - fileName = Bytes.toBytes(mobFileWriter.getPath().getName()); - } catch (IOException e) { - // Bailing out - LOG.error("Failed to create mob writer, ", e); - throw e; - } - if (compactMOBs) { - // Add the only reference we get for compact MOB case - // because new store file will have only one MOB reference - // in this case - of newly compacted MOB file - mobRefSet.get().add(mobFileWriter.getPath().getName()); - } - do { - hasMore = scanner.next(cells, scannerContext); - if (LOG.isDebugEnabled()) { - now = EnvironmentEdgeManager.currentTime(); - } - for (Cell c : cells) { - counter++; - if (compactMOBs) { - if (MobUtils.isMobReferenceCell(c)) { - if (counter == countFailAt) { - LOG.warn("\n\n INJECTED FAULT mobCounter="+mobCounter.get()+"\n\n"); - throw new CorruptHFileException("injected fault"); - } - String fName = MobUtils.getMobFileName(c); - Path pp = new Path(new Path(fs.getUri()), new Path(path, fName)); - - // Added to support migration - try { - mobCell = mobStore.resolve(c, true, false); - } catch (FileNotFoundException fnfe) { - if (discardMobMiss) { - LOG.error("Missing MOB cell: file=" + pp + " not found"); - continue; - } else { - throw fnfe; - } - } - - if (discardMobMiss && mobCell.getCell().getValueLength() == 0) { - LOG.error("Missing MOB cell value: file=" + pp +" cell=" + mobCell); - continue; - } - - if (mobCell.getCell().getValueLength() > mobSizeThreshold) { - // put the mob data back to the store file - PrivateCellUtil.setSequenceId(mobCell.getCell(), c.getSequenceId()); - mobFileWriter.append(mobCell.getCell()); - writer.append(MobUtils.createMobRefCell(mobCell.getCell(), fileName, - this.mobStore.getRefCellTags())); - cellsCountCompactedFromMob++; - cellsSizeCompactedFromMob += mobCell.getCell().getValueLength(); - mobCells++; - } else { - // If MOB value is less than threshold, append it directly to a store file - PrivateCellUtil.setSequenceId(mobCell.getCell(), c.getSequenceId()); - writer.append(mobCell.getCell()); - } - - } else { - // Not a MOB reference cell - int size = c.getValueLength(); - if (size > mobSizeThreshold) { - mobFileWriter.append(c); - writer.append(MobUtils.createMobRefCell(c, fileName, this.mobStore.getRefCellTags())); - mobCells++; - } else { - writer.append(c); - } - } - } else if (c.getTypeByte() != KeyValue.Type.Put.getCode()) { - // Not a major compaction or major with MOB disabled - // If the kv type is not put, directly write the cell - // to the store file. - writer.append(c); - } else if (MobUtils.isMobReferenceCell(c)) { - // Not a major MOB compaction, Put MOB reference - if (MobUtils.hasValidMobRefCellValue(c)) { - int size = MobUtils.getMobValueLength(c); - if (size > mobSizeThreshold) { - // If the value size is larger than the threshold, it's regarded as a mob. Since - // its value is already in the mob file, directly write this cell to the store file - writer.append(c); - // Add MOB reference to a set - mobRefSet.get().add(MobUtils.getMobFileName(c)); - } else { - // If the value is not larger than the threshold, it's not regarded a mob. Retrieve - // the mob cell from the mob file, and write it back to the store file. - mobCell = mobStore.resolve(c, true, false); - if (mobCell.getCell().getValueLength() != 0) { - // put the mob data back to the store file - PrivateCellUtil.setSequenceId(mobCell.getCell(), c.getSequenceId()); - writer.append(mobCell.getCell()); - cellsCountCompactedFromMob++; - cellsSizeCompactedFromMob += mobCell.getCell().getValueLength(); - } else { - // If the value of a file is empty, there might be issues when retrieving, - // directly write the cell to the store file, and leave it to be handled by the - // next compaction. - LOG.error("Empty value for: " + c); - writer.append(c); - // Add MOB reference to a set - mobRefSet.get().add(MobUtils.getMobFileName(c)); - } - } - } else { - // TODO ???? - LOG.error("Corrupted MOB reference: " + c); - writer.append(c); - } - } else if (c.getValueLength() <= mobSizeThreshold) { - // If the value size of a cell is not larger than the threshold, directly write it to - // the store file. - writer.append(c); - } else { - // If the value size of a cell is larger than the threshold, it's regarded as a mob, - // write this cell to a mob file, and write the path to the store file. - mobCells++; - // append the original keyValue in the mob file. - mobFileWriter.append(c); - Cell reference = MobUtils.createMobRefCell(c, fileName,this.mobStore.getRefCellTags()); - // write the cell whose value is the path of a mob file to the store file. - writer.append(reference); - cellsCountCompactedToMob++; - cellsSizeCompactedToMob += c.getValueLength(); - // Add ref we get for compact MOB case - mobRefSet.get().add(mobFileWriter.getPath().getName()); - } - - int len = c.getSerializedSize(); - ++progress.currentCompactedKVs; - progress.totalCompactedSize += len; - bytesWrittenProgressForShippedCall += len; - if (LOG.isDebugEnabled()) { - bytesWrittenProgressForLog += len; - } - throughputController.control(compactionName, len); - // check periodically to see if a system stop is requested - if (closeCheckSizeLimit > 0) { - bytesWrittenProgressForCloseCheck += len; - if (bytesWrittenProgressForCloseCheck > closeCheckSizeLimit) { - bytesWrittenProgressForCloseCheck = 0; - if (!store.areWritesEnabled()) { - progress.cancel(); - return false; - } - } - } - if (kvs != null && bytesWrittenProgressForShippedCall > shippedCallSizeLimit) { - ((ShipperListener)writer).beforeShipped(); - kvs.shipped(); - bytesWrittenProgressForShippedCall = 0; - } - } - // Log the progress of long running compactions every minute if - // logging at DEBUG level - if (LOG.isDebugEnabled()) { - if ((now - lastMillis) >= COMPACTION_PROGRESS_LOG_INTERVAL) { - String rate = String.format("%.2f", - (bytesWrittenProgressForLog / 1024.0) / ((now - lastMillis) / 1000.0)); - LOG.debug("Compaction progress: {} {}, rate={} KB/sec, throughputController is {}", - compactionName, progress, rate, throughputController); - lastMillis = now; - bytesWrittenProgressForLog = 0; - } - } - cells.clear(); - } while (hasMore); - finished = true; - } catch (InterruptedException e) { - progress.cancel(); - throw new InterruptedIOException( - "Interrupted while control throughput of compacting " + compactionName); - } catch (FileNotFoundException e) { - LOG.error("MOB Stress Test FAILED, region: "+store.getRegionInfo().getEncodedName(), e); - System.exit(-1); - } catch (IOException t) { - LOG.error("Mob compaction failed for region: "+ store.getRegionInfo().getEncodedName()); - throw t; - } finally { - // Clone last cell in the final because writer will append last cell when committing. If - // don't clone here and once the scanner get closed, then the memory of last cell will be - // released. (HBASE-22582) - ((ShipperListener) writer).beforeShipped(); - throughputController.finish(compactionName); - if (!finished && mobFileWriter != null) { - // Remove all MOB references because compaction failed - mobRefSet.get().clear(); - // Abort writer - abortWriter(mobFileWriter); - } - } - - if (mobFileWriter != null) { - if (mobCells > 0) { - // If the mob file is not empty, commit it. - mobFileWriter.appendMetadata(fd.maxSeqId, major, mobCells); - mobFileWriter.close(); - mobStore.commitFile(mobFileWriter.getPath(), path); - } else { - // If the mob file is empty, delete it instead of committing. - abortWriter(mobFileWriter); - } - } - mobStore.updateCellsCountCompactedFromMob(cellsCountCompactedFromMob); - mobStore.updateCellsCountCompactedToMob(cellsCountCompactedToMob); - mobStore.updateCellsSizeCompactedFromMob(cellsSizeCompactedFromMob); - mobStore.updateCellsSizeCompactedToMob(cellsSizeCompactedToMob); - progress.complete(); - return true; - - } - - -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java deleted file mode 100644 index 7222f6611e7b..000000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java +++ /dev/null @@ -1,80 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.mob; - -import java.io.IOException; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.util.AbstractHBaseTool; -import org.apache.hadoop.util.ToolRunner; -import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.yetus.audience.InterfaceStability; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@InterfaceAudience.Public -@InterfaceStability.Evolving -public class MobStressTool extends AbstractHBaseTool -{ - private static final Logger LOG = LoggerFactory.getLogger(MobStressTool.class); - private CommandLine cmd; - - public MobStressTool() throws IOException { - init(); - } - - protected void init() throws IOException { - // define supported options - addOptWithArg("n", "Number of MOB key-values to insert, default - 10000000"); - } - - @Override - protected void addOptions() { - } - - @Override - protected void processOptions(CommandLine cmd) { - this.cmd = cmd; - } - - @Override - protected int doWork() throws Exception { - long numRowsToInsert = 10000000; - if (cmd.hasOption("n")) { - numRowsToInsert = Long.parseLong(cmd.getOptionValue("n")); - if (numRowsToInsert < 0) { - LOG.warn("Ignore wrong option '-n'"); - numRowsToInsert = 10000000; - } - } - - TestMobCompaction test = new TestMobCompaction(); - test.init(getConf(), numRowsToInsert); - test.testMobCompaction(); - return 0; - } - - public static void main(String[] args) throws Exception { - Configuration conf = HBaseConfiguration.create(); - int ret = ToolRunner.run(conf, new MobStressTool(), args); - System.exit(ret); - } - -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java index 075117e597e5..bffb062b4b0e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java @@ -153,7 +153,7 @@ public void testCleaner() throws Exception { String[] args = new String[2]; args[0] = tableName.getNameAsString(); args[1] = family; - ToolRunner.run(TEST_UTIL.getConfiguration(), new MobFileCleanerTool(), args); + ToolRunner.run(TEST_UTIL.getConfiguration(), new ExpiredMobFileCleaner(), args); FileStatus[] filesAfterClean = TEST_UTIL.getTestFileSystem().listStatus(mobDirPath); String lastFile = filesAfterClean[0].getPath().getName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java deleted file mode 100644 index 6c20bf416180..000000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java +++ /dev/null @@ -1,344 +0,0 @@ -/** - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.mob; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; -import java.util.Arrays; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.KeepDeletedCells; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.BufferedMutator; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.ResultScanner; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner; -import org.apache.hadoop.hbase.testclassification.IntegrationTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.junit.After; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -/** - Reproduction for MOB data loss - - 1. Settings: Region Size 200 MB, Flush threshold 800 KB. - 2. Insert 10 Million records - 3. MOB Compaction and Archiver - a) Trigger MOB Compaction (every 2 minutes) - b) Trigger major compaction (every 2 minutes) - c) Trigger archive cleaner (every 3 minutes) - 4. Validate MOB data after complete data load. - - */ -@Category(IntegrationTests.class) -public class TestMobCompaction { - private static final Logger LOG = LoggerFactory.getLogger(TestMobCompaction.class); - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestMobCompaction.class); - @Rule - public TestName testName = new TestName(); - - private HBaseTestingUtility HTU; - - private final static String famStr = "f1"; - private final static byte[] fam = Bytes.toBytes(famStr); - private final static byte[] qualifier = Bytes.toBytes("q1"); - private final static long mobLen = 10; - private final static byte[] mobVal = Bytes - .toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789"); - - private Configuration conf; - private HTableDescriptor hdt; - private HColumnDescriptor hcd; - private Admin admin; - private long count = 1000000; - private double failureProb = 0.1; - private Table table = null; - - private static volatile boolean run = true; - - public TestMobCompaction() { - - } - - public void init(Configuration conf, long numRows) throws IOException { - this.conf = conf; - this.count = numRows; - printConf(); - hdt = createTableDescriptor("testMobCompactTable"); - Connection conn = ConnectionFactory.createConnection(this.conf); - this.admin = conn.getAdmin(); - this.hcd = new HColumnDescriptor(fam); - this.hcd.setMobEnabled(true); - this.hcd.setMobThreshold(mobLen); - this.hcd.setMaxVersions(1); - this.hdt.addFamily(hcd); - if (admin.tableExists(hdt.getTableName())) { - admin.disableTable(hdt.getTableName()); - admin.deleteTable(hdt.getTableName()); - } - admin.createTable(hdt); - table = conn.getTable(hdt.getTableName()); - } - - private void printConf() { - LOG.info("To run stress test, please change HBase configuration as following:"); - LOG.info("hfile.format.version=3"); - LOG.info("hbase.master.hfilecleaner.ttl=0"); - LOG.info("hbase.hregion.max.filesize=200000000"); - LOG.info("hbase.client.retries.number=100"); - LOG.info("hbase.hregion.memstore.flush.size=800000"); - LOG.info("hbase.hstore.blockingStoreFiles=150"); - LOG.info("hbase.hstore.compaction.throughput.lower.bound=50000000"); - LOG.info("hbase.hstore.compaction.throughput.higher.bound=100000000"); - LOG.info("hbase.master.mob.cleaner.period=0"); - LOG.info("hbase.mob.default.compactor=org.apache.hadoop.hbase.mob.FaultyMobStoreCompactor"); - LOG.warn("injected.fault.probability=x, where x is between 0. and 1."); - - - } - - - private HTableDescriptor createTableDescriptor(final String name, - final int minVersions, final int versions, final int ttl, KeepDeletedCells keepDeleted) { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name)); - return htd; - } - - - private HTableDescriptor createTableDescriptor(final String name) { - return createTableDescriptor(name, HColumnDescriptor.DEFAULT_MIN_VERSIONS, - 1, HConstants.FOREVER, HColumnDescriptor.DEFAULT_KEEP_DELETED); - } - - @Before - public void setUp() throws Exception { - HTU = new HBaseTestingUtility(); - hdt = HTU.createTableDescriptor("testMobCompactTable"); - conf = HTU.getConfiguration(); - - initConf(); - - // HTU.getConfiguration().setInt("hbase.mob.compaction.chore.period", 0); - HTU.startMiniCluster(); - admin = HTU.getAdmin(); - - hcd = new HColumnDescriptor(fam); - hcd.setMobEnabled(true); - hcd.setMobThreshold(mobLen); - hcd.setMaxVersions(1); - hdt.addFamily(hcd); - table = HTU.createTable(hdt, null); - } - - private void initConf() { - - conf.setInt("hfile.format.version", 3); - conf.setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0); - conf.setInt("hbase.client.retries.number", 100); - conf.setInt("hbase.hregion.max.filesize", 200000000); - conf.setInt("hbase.hregion.memstore.flush.size", 800000); - conf.setInt("hbase.hstore.blockingStoreFiles", 150); - conf.setInt("hbase.hstore.compaction.throughput.lower.bound", 52428800); - conf.setInt("hbase.hstore.compaction.throughput.higher.bound", 2*52428800); - conf.setDouble("injected.fault.probability", failureProb); - conf.set(MobStoreEngine.DEFAULT_MOB_COMPACTOR_CLASS_KEY, - FaultyMobStoreCompactor.class.getName()); - - } - - @After - public void tearDown() throws Exception { - HTU.shutdownMiniCluster(); - } - - - class MajorCompaction implements Runnable { - - @Override - public void run() { - while (run) { - try { - admin.majorCompact(hdt.getTableName(), fam); - Thread.sleep(120000); - } catch (Exception e) { - LOG.error("MOB Stress Test FAILED", e); - System.exit(-1); - } - } - } - } - - class CleanMobAndArchive implements Runnable { - - @Override - public void run() { - while (run) { - try { - LOG.info("MOB cleanup chore started ..."); - MobUtils.cleanupObsoleteMobFiles(conf, table.getName()); - LOG.info("MOB cleanup chore finished"); - - Thread.sleep(130000); - } catch (Exception e) { - e.printStackTrace(); - } - } - } - } - - class WriteData implements Runnable { - - private long rows = -1; - - public WriteData(long rows) { - this.rows = rows; - } - - @Override - public void run() { - try { - - BufferedMutator bm = admin.getConnection().getBufferedMutator(table.getName()); - // Put Operation - for (int i = 0; i < rows; i++) { - Put p = new Put(Bytes.toBytes(i)); - p.addColumn(fam, qualifier, mobVal); -// table.put(p); - - bm.mutate(p); - if (i % 10000 == 0) { - LOG.info("LOADED="+ i); -// try { -// Thread.sleep(500); -// } catch (InterruptedException ee) { -// -// } - } - if (i % 100000 == 0) { - printStats(i); - } - } - bm.flush(); - admin.flush(table.getName()); - run = false; - } catch (Exception e) { - LOG.error("MOB Stress Test FAILED", e); - System.exit(-1); - } - } - } - - @Test - public void testMobCompaction() throws InterruptedException, IOException { - - try { - - Thread writeData = new Thread(new WriteData(count)); - writeData.start(); - - - Thread majorcompact = new Thread(new MajorCompaction()); - majorcompact.start(); - - Thread cleaner = new Thread(new CleanMobAndArchive()); - cleaner.start(); - - while (run) { - Thread.sleep(1000); - } - - MobUtils.getNumberOfMobFiles(conf, table.getName(), new String(fam)); - LOG.info("Waiting for write thread to finish ..."); - writeData.join(); - // Cleanup again - MobUtils.cleanupObsoleteMobFiles(conf, table.getName()); - MobUtils.getNumberOfMobFiles(conf, table.getName(), new String(fam)); - - if (HTU != null) { - LOG.info("Archive cleaner started ..."); - // Call archive cleaner again - HTU.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting(); - LOG.info("Archive cleaner finished"); - } - - scanTable(); - - } finally { - - admin.disableTable(hdt.getTableName()); - admin.deleteTable(hdt.getTableName()); - } - LOG.info("MOB Stress Test finished OK"); - printStats(count); - - } - - public void printStats(long loaded) { - LOG.info("MOB Stress Test: loaded=" + loaded+ " compactions=" + - FaultyMobStoreCompactor.totalCompactions.get() + - " major="+ FaultyMobStoreCompactor.totalMajorCompactions.get() +" mob="+ - FaultyMobStoreCompactor.mobCounter.get() + " injected failures="+ - FaultyMobStoreCompactor.totalFailures.get() - ); - } - - private void scanTable() { - try { - - Result result; - ResultScanner scanner = table.getScanner(fam); - int counter = 0; - while((result = scanner.next()) != null) { - assertTrue(Arrays.equals(result.getValue(fam, qualifier), mobVal)); - if ( counter % 10000 == 0 ) { - LOG.info("GET="+ counter); - } - counter++; - } - assertEquals(count, counter); - } catch (Exception e) { - e.printStackTrace(); - LOG.error("MOB Stress Test FAILED"); - if (HTU != null) { - assertTrue(false); - } else { - System.exit(-1); - } - } - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java index a6a2ee55009c..ae53ff21feba 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java @@ -47,7 +47,6 @@ public class TestMobFileName { private Date date; private String dateStr; private byte[] startKey; - private String regionName = "region"; @Before public void setUp() { @@ -60,32 +59,32 @@ public void setUp() { @Test public void testHashCode() { - assertEquals(MobFileName.create(startKey, dateStr, uuid, regionName).hashCode(), - MobFileName.create(startKey, dateStr, uuid, regionName).hashCode()); - assertNotSame(MobFileName.create(startKey, dateStr, uuid, regionName), - MobFileName.create(startKey, dateStr, uuid, regionName)); + assertEquals(MobFileName.create(startKey, dateStr, uuid).hashCode(), + MobFileName.create(startKey, dateStr, uuid).hashCode()); + assertNotSame(MobFileName.create(startKey, dateStr, uuid), + MobFileName.create(startKey, dateStr, uuid)); } @Test public void testCreate() { - MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid, regionName); + MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid); assertEquals(mobFileName, MobFileName.create(mobFileName.getFileName())); } @Test public void testGet() { - MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid, regionName); + MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid); assertEquals(MD5Hash.getMD5AsHex(startKey, 0, startKey.length), mobFileName.getStartKey()); assertEquals(dateStr, mobFileName.getDate()); assertEquals(mobFileName.getFileName(), MD5Hash.getMD5AsHex(startKey, 0, startKey.length) - + dateStr + uuid+"_"+regionName); + + dateStr + uuid); } @Test public void testEquals() { - MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid, regionName); + MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid); assertTrue(mobFileName.equals(mobFileName)); assertFalse(mobFileName.equals(this)); - assertTrue(mobFileName.equals(MobFileName.create(startKey, dateStr, uuid, regionName))); + assertTrue(mobFileName.equals(MobFileName.create(startKey, dateStr, uuid))); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java new file mode 100644 index 000000000000..130a6d31cba6 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java @@ -0,0 +1,1202 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.mob.compactions; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.security.Key; +import java.security.SecureRandom; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Calendar; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.Random; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.RejectedExecutionHandler; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import javax.crypto.spec.SecretKeySpec; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.NamespaceDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.BufferedMutator; +import org.apache.hadoop.hbase.client.CompactType; +import org.apache.hadoop.hbase.client.CompactionState; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.coprocessor.ObserverContext; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; +import org.apache.hadoop.hbase.coprocessor.RegionObserver; +import org.apache.hadoop.hbase.io.HFileLink; +import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; +import org.apache.hadoop.hbase.io.crypto.aes.AES; +import org.apache.hadoop.hbase.io.hfile.CacheConfig; +import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner; +import org.apache.hadoop.hbase.mob.MobConstants; +import org.apache.hadoop.hbase.mob.MobFileName; +import org.apache.hadoop.hbase.mob.MobTestUtil; +import org.apache.hadoop.hbase.mob.MobUtils; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.HStoreFile; +import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.StoreFileInfo; +import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker; +import org.apache.hadoop.hbase.security.EncryptionUtil; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.Threads; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category(LargeTests.class) +public class TestMobCompactor { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobCompactor.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestMobCompactor.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static Configuration conf = null; + private TableName tableName; + private static Connection conn; + private BufferedMutator bufMut; + private Table table; + private static Admin admin; + private HTableDescriptor desc; + private HColumnDescriptor hcd1; + private HColumnDescriptor hcd2; + private static FileSystem fs; + private static final String family1 = "family1"; + private static final String family2 = "family2"; + private static final String qf1 = "qualifier1"; + private static final String qf2 = "qualifier2"; + + private static long tsFor20150907Monday; + private static long tsFor20151120Sunday; + private static long tsFor20151128Saturday; + private static long tsFor20151130Monday; + private static long tsFor20151201Tuesday; + private static long tsFor20151205Saturday; + private static long tsFor20151228Monday; + private static long tsFor20151231Thursday; + private static long tsFor20160101Friday; + private static long tsFor20160103Sunday; + + private static final byte[] mobKey01 = Bytes.toBytes("r01"); + private static final byte[] mobKey02 = Bytes.toBytes("r02"); + private static final byte[] mobKey03 = Bytes.toBytes("r03"); + private static final byte[] mobKey04 = Bytes.toBytes("r04"); + private static final byte[] mobKey05 = Bytes.toBytes("r05"); + private static final byte[] mobKey06 = Bytes.toBytes("r05"); + private static final byte[] mobKey1 = Bytes.toBytes("r1"); + private static final byte[] mobKey2 = Bytes.toBytes("r2"); + private static final byte[] mobKey3 = Bytes.toBytes("r3"); + private static final byte[] mobKey4 = Bytes.toBytes("r4"); + private static final byte[] mobKey5 = Bytes.toBytes("r5"); + private static final byte[] mobKey6 = Bytes.toBytes("r6"); + private static final byte[] mobKey7 = Bytes.toBytes("r7"); + private static final byte[] mobKey8 = Bytes.toBytes("r8"); + private static final String mobValue0 = "mobValue00000000000000000000000000"; + private static final String mobValue1 = "mobValue00000111111111111111111111"; + private static final String mobValue2 = "mobValue00000222222222222222222222"; + private static final String mobValue3 = "mobValue00000333333333333333333333"; + private static final String mobValue4 = "mobValue00000444444444444444444444"; + private static final String mobValue5 = "mobValue00000666666666666666666666"; + private static final String mobValue6 = "mobValue00000777777777777777777777"; + private static final String mobValue7 = "mobValue00000888888888888888888888"; + private static final String mobValue8 = "mobValue00000888888888888888888899"; + + private static byte[] KEYS = Bytes.toBytes("012"); + private static int regionNum = KEYS.length; + private static int delRowNum = 1; + private static int delCellNum = 6; + private static int cellNumPerRow = 3; + private static int rowNumPerFile = 2; + private static ExecutorService pool; + + @Rule + public TestName name = new TestName(); + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + TEST_UTIL.getConfiguration().setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, 5000); + TEST_UTIL.getConfiguration() + .set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, KeyProviderForTesting.class.getName()); + TEST_UTIL.getConfiguration().set(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, "hbase"); + TEST_UTIL.getConfiguration().setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0); + TEST_UTIL.getConfiguration().setInt("hbase.client.retries.number", 1); + TEST_UTIL.getConfiguration().setInt("hbase.hfile.compaction.discharger.interval", 100); + TEST_UTIL.startMiniCluster(1); + pool = createThreadPool(TEST_UTIL.getConfiguration()); + conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration(), pool); + fs = TEST_UTIL.getTestFileSystem(); + conf = TEST_UTIL.getConfiguration(); + admin = TEST_UTIL.getAdmin(); + + // Initialize timestamps for these days + Calendar calendar = Calendar.getInstance(); + calendar.set(2015, 8, 7, 10, 20); + tsFor20150907Monday = calendar.getTimeInMillis(); + + calendar.set(2015, 10, 20, 10, 20); + tsFor20151120Sunday = calendar.getTimeInMillis(); + + calendar.set(2015, 10, 28, 10, 20); + tsFor20151128Saturday = calendar.getTimeInMillis(); + + calendar.set(2015, 10, 30, 10, 20); + tsFor20151130Monday = calendar.getTimeInMillis(); + + calendar.set(2015, 11, 1, 10, 20); + tsFor20151201Tuesday = calendar.getTimeInMillis(); + + calendar.set(2015, 11, 5, 10, 20); + tsFor20151205Saturday = calendar.getTimeInMillis(); + + calendar.set(2015, 11, 28, 10, 20); + tsFor20151228Monday = calendar.getTimeInMillis(); + + calendar.set(2015, 11, 31, 10, 20); + tsFor20151231Thursday = calendar.getTimeInMillis(); + + calendar.set(2016, 0, 1, 10, 20); + tsFor20160101Friday = calendar.getTimeInMillis(); + + calendar.set(2016, 0, 3, 10, 20); + tsFor20160103Sunday = calendar.getTimeInMillis(); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + pool.shutdown(); + conn.close(); + TEST_UTIL.shutdownMiniCluster(); + } + + public void setUp(String tableNameAsString) throws IOException { + tableName = TableName.valueOf(tableNameAsString); + hcd1 = new HColumnDescriptor(family1); + hcd1.setMobEnabled(true); + hcd1.setMobThreshold(5); + hcd2 = new HColumnDescriptor(family2); + hcd2.setMobEnabled(true); + hcd2.setMobThreshold(5); + desc = new HTableDescriptor(tableName); + desc.addFamily(hcd1); + desc.addFamily(hcd2); + admin.createTable(desc, getSplitKeys()); + table = conn.getTable(tableName); + bufMut = conn.getBufferedMutator(tableName); + } + + // Set up for mob compaction policy testing + private void setUpForPolicyTest(String tableNameAsString, MobCompactPartitionPolicy type) + throws IOException { + tableName = TableName.valueOf(tableNameAsString); + hcd1 = new HColumnDescriptor(family1); + hcd1.setMobEnabled(true); + hcd1.setMobThreshold(10); + hcd1.setMobCompactPartitionPolicy(type); + desc = new HTableDescriptor(tableName); + desc.addFamily(hcd1); + admin.createTable(desc); + table = conn.getTable(tableName); + bufMut = conn.getBufferedMutator(tableName); + } + + // alter mob compaction policy + private void alterForPolicyTest(final MobCompactPartitionPolicy type) + throws Exception { + + hcd1.setMobCompactPartitionPolicy(type); + desc.modifyFamily(hcd1); + admin.modifyTable(desc); + LOG.info("alter status finished"); + } + + @Test + public void testMinorCompaction() throws Exception { + resetConf(); + int mergeSize = 5000; + // change the mob compaction merge size + conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize); + + // create a table with namespace + NamespaceDescriptor namespaceDescriptor = NamespaceDescriptor.create("ns").build(); + String tableNameAsString = "ns:testMinorCompaction"; + admin.createNamespace(namespaceDescriptor); + setUp(tableNameAsString); + int count = 4; + // generate mob files + loadData(admin, bufMut, tableName, count, rowNumPerFile); + int rowNumPerRegion = count * rowNumPerFile; + + assertEquals("Before deleting: mob rows count", regionNum * rowNumPerRegion, + MobTestUtil.countMobRows(table)); + assertEquals("Before deleting: mob cells count", regionNum * cellNumPerRow * rowNumPerRegion, + countMobCells(table)); + assertEquals("Before deleting: mob file count", regionNum * count, + countFiles(tableName, true, family1)); + + int largeFilesCount = countLargeFiles(mergeSize, tableName, family1); + createDelFile(table, tableName, Bytes.toBytes(family1), Bytes.toBytes(qf1)); + + assertEquals("Before compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum), + MobTestUtil.countMobRows(table)); + assertEquals("Before compaction: mob cells count", regionNum + * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table)); + assertEquals("Before compaction: family1 mob file count", regionNum * count, + countFiles(tableName, true, family1)); + assertEquals("Before compaction: family2 mob file count", regionNum * count, + countFiles(tableName, true, family2)); + assertEquals("Before compaction: family1 del file count", regionNum, + countFiles(tableName, false, family1)); + assertEquals("Before compaction: family2 del file count", regionNum, + countFiles(tableName, false, family2)); + + // do the mob file compaction + MobCompactor compactor = new PartitionedMobCompactor(conf, fs, tableName, hcd1, pool); + compactor.compact(); + + assertEquals("After compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum), + MobTestUtil.countMobRows(table)); + assertEquals("After compaction: mob cells count", regionNum + * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table)); + // After the compaction, the files smaller than the mob compaction merge size + // is merge to one file + assertEquals("After compaction: family1 mob file count", largeFilesCount + regionNum, + countFiles(tableName, true, family1)); + assertEquals("After compaction: family2 mob file count", regionNum * count, + countFiles(tableName, true, family2)); + assertEquals("After compaction: family1 del file count", regionNum, + countFiles(tableName, false, family1)); + assertEquals("After compaction: family2 del file count", regionNum, + countFiles(tableName, false, family2)); + } + + @Test + public void testMinorCompactionWithWeeklyPolicy() throws Exception { + resetConf(); + int mergeSize = 5000; + // change the mob compaction merge size + conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize); + + commonPolicyTestLogic("testMinorCompactionWithWeeklyPolicy", + MobCompactPartitionPolicy.WEEKLY, false, 6, + new String[] { "20150907", "20151120", "20151128", "20151130", "20151205", "20160103" }, + true); + } + + @Test + public void testMajorCompactionWithWeeklyPolicy() throws Exception { + resetConf(); + + commonPolicyTestLogic("testMajorCompactionWithWeeklyPolicy", + MobCompactPartitionPolicy.WEEKLY, true, 5, + new String[] { "20150907", "20151120", "20151128", "20151205", "20160103" }, true); + } + + @Test + public void testMinorCompactionWithMonthlyPolicy() throws Exception { + resetConf(); + int mergeSize = 5000; + // change the mob compaction merge size + conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize); + + commonPolicyTestLogic("testMinorCompactionWithMonthlyPolicy", + MobCompactPartitionPolicy.MONTHLY, false, 4, + new String[] { "20150907", "20151130", "20151231", "20160103" }, true); + } + + @Test + public void testMajorCompactionWithMonthlyPolicy() throws Exception { + resetConf(); + + commonPolicyTestLogic("testMajorCompactionWithMonthlyPolicy", + MobCompactPartitionPolicy.MONTHLY, true, 4, + new String[] {"20150907", "20151130", "20151231", "20160103"}, true); + } + + @Test + public void testMajorCompactionWithWeeklyFollowedByMonthly() throws Exception { + resetConf(); + + commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthly", + MobCompactPartitionPolicy.WEEKLY, true, 5, + new String[] { "20150907", "20151120", "20151128", "20151205", "20160103" }, true); + + commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthly", + MobCompactPartitionPolicy.MONTHLY, true, 4, + new String[] {"20150907", "20151128", "20151205", "20160103" }, false); + } + + @Test + public void testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByWeekly() throws Exception { + resetConf(); + + commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByWeekly", + MobCompactPartitionPolicy.WEEKLY, true, 5, + new String[] { "20150907", "20151120", "20151128", "20151205", "20160103" }, true); + + commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByWeekly", + MobCompactPartitionPolicy.MONTHLY, true, 4, + new String[] { "20150907", "20151128", "20151205", "20160103" }, false); + + commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByWeekly", + MobCompactPartitionPolicy.WEEKLY, true, 4, + new String[] { "20150907", "20151128", "20151205", "20160103" }, false); + } + + @Test + public void testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByDaily() throws Exception { + resetConf(); + + commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByDaily", + MobCompactPartitionPolicy.WEEKLY, true, 5, + new String[] { "20150907", "20151120", "20151128", "20151205", "20160103" }, true); + + commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByDaily", + MobCompactPartitionPolicy.MONTHLY, true, 4, + new String[] { "20150907", "20151128", "20151205", "20160103" }, false); + + commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByDaily", + MobCompactPartitionPolicy.DAILY, true, 4, + new String[] { "20150907", "20151128", "20151205", "20160103" }, false); + } + + @Test + public void testCompactionWithHFileLink() throws IOException, InterruptedException { + resetConf(); + String tableNameAsString = "testCompactionWithHFileLink"; + setUp(tableNameAsString); + int count = 4; + // generate mob files + loadData(admin, bufMut, tableName, count, rowNumPerFile); + int rowNumPerRegion = count * rowNumPerFile; + + long tid = System.currentTimeMillis(); + String snapshotName1 = "snaptb-" + tid; + // take a snapshot + admin.snapshot(snapshotName1, tableName); + + createDelFile(table, tableName, Bytes.toBytes(family1), Bytes.toBytes(qf1)); + + assertEquals("Before compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum), + MobTestUtil.countMobRows(table)); + assertEquals("Before compaction: mob cells count", regionNum + * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table)); + assertEquals("Before compaction: family1 mob file count", regionNum * count, + countFiles(tableName, true, family1)); + assertEquals("Before compaction: family2 mob file count", regionNum * count, + countFiles(tableName, true, family2)); + assertEquals("Before compaction: family1 del file count", regionNum, + countFiles(tableName, false, family1)); + assertEquals("Before compaction: family2 del file count", regionNum, + countFiles(tableName, false, family2)); + + // do the mob compaction + MobCompactor compactor = new PartitionedMobCompactor(conf, fs, tableName, hcd1, pool); + compactor.compact(); + + assertEquals("After first compaction: mob rows count", regionNum + * (rowNumPerRegion - delRowNum), MobTestUtil.countMobRows(table)); + assertEquals("After first compaction: mob cells count", regionNum + * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table)); + assertEquals("After first compaction: family1 mob file count", regionNum, + countFiles(tableName, true, family1)); + assertEquals("After first compaction: family2 mob file count", regionNum * count, + countFiles(tableName, true, family2)); + assertEquals("After first compaction: family1 del file count", 0, + countFiles(tableName, false, family1)); + assertEquals("After first compaction: family2 del file count", regionNum, + countFiles(tableName, false, family2)); + assertEquals("After first compaction: family1 hfilelink count", 0, countHFileLinks(family1)); + assertEquals("After first compaction: family2 hfilelink count", 0, countHFileLinks(family2)); + + admin.disableTable(tableName); + // Restore from snapshot, the hfilelink will exist in mob dir + admin.restoreSnapshot(snapshotName1); + admin.enableTable(tableName); + + assertEquals("After restoring snapshot: mob rows count", regionNum * rowNumPerRegion, + MobTestUtil.countMobRows(table)); + assertEquals("After restoring snapshot: mob cells count", regionNum * cellNumPerRow + * rowNumPerRegion, countMobCells(table)); + assertEquals("After restoring snapshot: family1 mob file count", regionNum * count, + countFiles(tableName, true, family1)); + assertEquals("After restoring snapshot: family2 mob file count", regionNum * count, + countFiles(tableName, true, family2)); + assertEquals("After restoring snapshot: family1 del file count", 0, + countFiles(tableName, false, family1)); + assertEquals("After restoring snapshot: family2 del file count", 0, + countFiles(tableName, false, family2)); + assertEquals("After restoring snapshot: family1 hfilelink count", regionNum * count, + countHFileLinks(family1)); + assertEquals("After restoring snapshot: family2 hfilelink count", 0, countHFileLinks(family2)); + + compactor.compact(); + + assertEquals("After second compaction: mob rows count", regionNum * rowNumPerRegion, + MobTestUtil.countMobRows(table)); + assertEquals("After second compaction: mob cells count", regionNum * cellNumPerRow + * rowNumPerRegion, countMobCells(table)); + assertEquals("After second compaction: family1 mob file count", regionNum, + countFiles(tableName, true, family1)); + assertEquals("After second compaction: family2 mob file count", regionNum * count, + countFiles(tableName, true, family2)); + assertEquals("After second compaction: family1 del file count", 0, + countFiles(tableName, false, family1)); + assertEquals("After second compaction: family2 del file count", 0, + countFiles(tableName, false, family2)); + assertEquals("After second compaction: family1 hfilelink count", 0, countHFileLinks(family1)); + assertEquals("After second compaction: family2 hfilelink count", 0, countHFileLinks(family2)); + assertRefFileNameEqual(family1); + } + + @Test + public void testMajorCompactionFromAdmin() throws Exception { + resetConf(); + int mergeSize = 5000; + // change the mob compaction merge size + conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize); + SecureRandom rng = new SecureRandom(); + byte[] keyBytes = new byte[AES.KEY_LENGTH]; + rng.nextBytes(keyBytes); + String algorithm = conf.get(HConstants.CRYPTO_KEY_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES); + Key cfKey = new SecretKeySpec(keyBytes, algorithm); + byte[] encryptionKey = EncryptionUtil.wrapKey(conf, + conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, User.getCurrent().getShortName()), cfKey); + final TableName tableName = TableName.valueOf(name.getMethodName()); + HTableDescriptor desc = new HTableDescriptor(tableName); + HColumnDescriptor hcd1 = new HColumnDescriptor(family1); + hcd1.setMobEnabled(true); + hcd1.setMobThreshold(0); + hcd1.setEncryptionType(algorithm); + hcd1.setEncryptionKey(encryptionKey); + HColumnDescriptor hcd2 = new HColumnDescriptor(family2); + hcd2.setMobEnabled(true); + hcd2.setMobThreshold(0); + desc.addFamily(hcd1); + desc.addFamily(hcd2); + admin.createTable(desc, getSplitKeys()); + Table table = conn.getTable(tableName); + BufferedMutator bufMut = conn.getBufferedMutator(tableName); + int count = 4; + // generate mob files + loadData(admin, bufMut, tableName, count, rowNumPerFile); + int rowNumPerRegion = count * rowNumPerFile; + + assertEquals("Before deleting: mob rows count", regionNum * rowNumPerRegion, + MobTestUtil.countMobRows(table)); + assertEquals("Before deleting: mob cells count", regionNum * cellNumPerRow * rowNumPerRegion, + countMobCells(table)); + assertEquals("Before deleting: mob file count", regionNum * count, + countFiles(tableName, true, family1)); + + createDelFile(table, tableName, Bytes.toBytes(family1), Bytes.toBytes(qf1)); + + assertEquals("Before compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum), + MobTestUtil.countMobRows(table)); + assertEquals("Before compaction: mob cells count", regionNum + * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table)); + assertEquals("Before compaction: family1 mob file count", regionNum * count, + countFiles(tableName, true, family1)); + assertEquals("Before compaction: family2 mob file count", regionNum * count, + countFiles(tableName, true, family2)); + assertEquals("Before compaction: family1 del file count", regionNum, + countFiles(tableName, false, family1)); + assertEquals("Before compaction: family2 del file count", regionNum, + countFiles(tableName, false, family2)); + + // do the major mob compaction, it will force all files to compaction + admin.majorCompact(tableName, hcd1.getName(), CompactType.MOB); + + waitUntilMobCompactionFinished(tableName); + assertEquals("After compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum), + MobTestUtil.countMobRows(table)); + assertEquals("After compaction: mob cells count", regionNum + * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table)); + assertEquals("After compaction: family1 mob file count", regionNum, + countFiles(tableName, true, family1)); + assertEquals("After compaction: family2 mob file count", regionNum * count, + countFiles(tableName, true, family2)); + assertEquals("After compaction: family1 del file count", 0, + countFiles(tableName, false, family1)); + assertEquals("After compaction: family2 del file count", regionNum, + countFiles(tableName, false, family2)); + Assert.assertTrue(verifyEncryption(tableName, family1)); + table.close(); + } + + @Test + public void testScannerOnBulkLoadRefHFiles() throws Exception { + resetConf(); + setUp("testScannerOnBulkLoadRefHFiles"); + long ts = EnvironmentEdgeManager.currentTime(); + byte[] key0 = Bytes.toBytes("k0"); + byte[] key1 = Bytes.toBytes("k1"); + String value0 = "mobValue0"; + String value1 = "mobValue1"; + String newValue0 = "new"; + Put put0 = new Put(key0); + put0.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), ts, Bytes.toBytes(value0)); + loadData(admin, bufMut, tableName, new Put[] { put0 }); + put0 = new Put(key0); + put0.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), ts, Bytes.toBytes(newValue0)); + Put put1 = new Put(key1); + put1.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), ts, Bytes.toBytes(value1)); + loadData(admin, bufMut, tableName, new Put[] { put0, put1 }); + // read the latest cell of key0. + Get get = new Get(key0); + Result result = table.get(get); + Cell cell = result.getColumnLatestCell(hcd1.getName(), Bytes.toBytes(qf1)); + assertEquals("Before compaction: mob value of k0", newValue0, + Bytes.toString(CellUtil.cloneValue(cell))); + admin.majorCompact(tableName, hcd1.getName(), CompactType.MOB); + waitUntilMobCompactionFinished(tableName); + // read the latest cell of key0, the cell seqId in bulk loaded file is not reset in the + // scanner. The cell that has "new" value is still visible. + result = table.get(get); + cell = result.getColumnLatestCell(hcd1.getName(), Bytes.toBytes(qf1)); + assertEquals("After compaction: mob value of k0", newValue0, + Bytes.toString(CellUtil.cloneValue(cell))); + // read the ref cell, not read further to the mob cell. + get = new Get(key1); + get.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(true)); + result = table.get(get); + cell = result.getColumnLatestCell(hcd1.getName(), Bytes.toBytes(qf1)); + // the ref name is the new file + Path mobFamilyPath = + MobUtils.getMobFamilyPath(TEST_UTIL.getConfiguration(), tableName, hcd1.getNameAsString()); + List paths = new ArrayList<>(); + if (fs.exists(mobFamilyPath)) { + FileStatus[] files = fs.listStatus(mobFamilyPath); + for (FileStatus file : files) { + if (!StoreFileInfo.isDelFile(file.getPath())) { + paths.add(file.getPath()); + } + } + } + assertEquals("After compaction: number of mob files:", 1, paths.size()); + assertEquals("After compaction: mob file name:", MobUtils.getMobFileName(cell), paths.get(0) + .getName()); + } + + /** + * This case tests the following mob compaction and normal compaction scenario, + * after mob compaction, the mob reference in new bulkloaded hfile will win even after it + * is compacted with some other normal hfiles. This is to make sure the mvcc is included + * after compaction for mob enabled store files. + */ + @Test + public void testGetAfterCompaction() throws Exception { + resetConf(); + conf.setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0); + String famStr = "f1"; + byte[] fam = Bytes.toBytes(famStr); + byte[] qualifier = Bytes.toBytes("q1"); + byte[] mobVal = Bytes.toBytes("01234567890"); + HTableDescriptor hdt = new HTableDescriptor(TableName.valueOf(name.getMethodName())); + hdt.addCoprocessor(CompactTwoLatestHfilesCopro.class.getName()); + HColumnDescriptor hcd = new HColumnDescriptor(fam); + hcd.setMobEnabled(true); + hcd.setMobThreshold(10); + hcd.setMaxVersions(1); + hdt.addFamily(hcd); + try { + Table table = TEST_UTIL.createTable(hdt, null); + HRegion r = TEST_UTIL.getMiniHBaseCluster().getRegions(hdt.getTableName()).get(0); + Put p = new Put(Bytes.toBytes("r1")); + p.addColumn(fam, qualifier, mobVal); + table.put(p); + // Create mob file mob1 and reference file ref1 + TEST_UTIL.flush(table.getName()); + // Make sure that it is flushed. + FileSystem fs = r.getRegionFileSystem().getFileSystem(); + Path path = r.getRegionFileSystem().getStoreDir(famStr); + waitUntilFilesShowup(fs, path, 1); + + p = new Put(Bytes.toBytes("r2")); + p.addColumn(fam, qualifier, mobVal); + table.put(p); + // Create mob file mob2 and reference file ref2 + TEST_UTIL.flush(table.getName()); + waitUntilFilesShowup(fs, path, 2); + // Do mob compaction to create mob3 and ref3 + TEST_UTIL.getAdmin().compact(hdt.getTableName(), fam, CompactType.MOB); + waitUntilFilesShowup(fs, path, 3); + + // Compact ref3 and ref2 into ref4 + TEST_UTIL.getAdmin().compact(hdt.getTableName(), fam); + waitUntilFilesShowup(fs, path, 2); + + // Sleep for some time, since TimeToLiveHFileCleaner is 0, the next run of + // clean chore is guaranteed to clean up files in archive + Thread.sleep(100); + // Run cleaner to make sure that files in archive directory are cleaned up + TEST_UTIL.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting(); + + // Get "r2" + Get get = new Get(Bytes.toBytes("r2")); + try { + Result result = table.get(get); + assertTrue(Arrays.equals(result.getValue(fam, qualifier), mobVal)); + } catch (IOException e) { + assertTrue("The MOB file doesn't exist", false); + } + } finally { + TEST_UTIL.deleteTable(hdt.getTableName()); + } + } + + private void waitUntilFilesShowup(final FileSystem fs, final Path path, final int num) + throws InterruptedException, IOException { + FileStatus[] fileList = fs.listStatus(path); + while (fileList.length != num) { + Thread.sleep(50); + fileList = fs.listStatus(path); + for (FileStatus fileStatus: fileList) { + LOG.info(Objects.toString(fileStatus)); + } + } + } + + /** + * This copro overwrites the default compaction policy. It always chooses two latest hfiles and + * compacts them into a new one. + */ + public static class CompactTwoLatestHfilesCopro implements RegionCoprocessor, RegionObserver { + + @Override + public Optional getRegionObserver() { + return Optional.of(this); + } + + @Override + public void preCompactSelection(ObserverContext c, Store store, + List candidates, CompactionLifeCycleTracker tracker) + throws IOException { + int count = candidates.size(); + if (count >= 2) { + for (int i = 0; i < count - 2; i++) { + candidates.remove(0); + } + c.bypass(); + } + } + } + + private void waitUntilMobCompactionFinished(TableName tableName) throws IOException, + InterruptedException { + long finished = EnvironmentEdgeManager.currentTime() + 60000; + CompactionState state = admin.getCompactionState(tableName, CompactType.MOB); + while (EnvironmentEdgeManager.currentTime() < finished) { + if (state == CompactionState.NONE) { + break; + } + state = admin.getCompactionState(tableName, CompactType.MOB); + Thread.sleep(10); + } + assertEquals(CompactionState.NONE, state); + } + + /** + * Gets the number of cells in the given table. + * @param table to get the scanner + * @return the number of cells + */ + private int countMobCells(final Table table) throws IOException { + Scan scan = new Scan(); + // Do not retrieve the mob data when scanning + scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE)); + ResultScanner results = table.getScanner(scan); + int count = 0; + for (Result res : results) { + count += res.size(); + } + results.close(); + return count; + } + + /** + * Gets the number of files in the mob path. + * @param isMobFile gets number of the mob files or del files + * @param familyName the family name + * @return the number of the files + */ + private int countFiles(TableName tableName, boolean isMobFile, String familyName) + throws IOException { + Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, familyName); + int count = 0; + if (fs.exists(mobDirPath)) { + FileStatus[] files = fs.listStatus(mobDirPath); + for (FileStatus file : files) { + if (isMobFile == true) { + if (!StoreFileInfo.isDelFile(file.getPath())) { + count++; + } + } else { + if (StoreFileInfo.isDelFile(file.getPath())) { + count++; + } + } + } + } + return count; + } + + private boolean verifyEncryption(TableName tableName, String familyName) throws IOException { + Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, familyName); + boolean hasFiles = false; + if (fs.exists(mobDirPath)) { + FileStatus[] files = fs.listStatus(mobDirPath); + hasFiles = files != null && files.length > 0; + Assert.assertTrue(hasFiles); + Path path = files[0].getPath(); + CacheConfig cacheConf = new CacheConfig(conf); + HStoreFile sf = new HStoreFile(TEST_UTIL.getTestFileSystem(), path, conf, cacheConf, + BloomType.NONE, true); + sf.initReader(); + HFile.Reader reader = sf.getReader().getHFileReader(); + byte[] encryptionKey = reader.getTrailer().getEncryptionKey(); + Assert.assertTrue(null != encryptionKey); + Assert.assertTrue(reader.getFileContext().getEncryptionContext().getCipher().getName() + .equals(HConstants.CIPHER_AES)); + } + return hasFiles; + } + + /** + * Gets the number of HFileLink in the mob path. + * @param familyName the family name + * @return the number of the HFileLink + */ + private int countHFileLinks(String familyName) throws IOException { + Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, familyName); + int count = 0; + if (fs.exists(mobDirPath)) { + FileStatus[] files = fs.listStatus(mobDirPath); + for (FileStatus file : files) { + if (HFileLink.isHFileLink(file.getPath())) { + count++; + } + } + } + return count; + } + + /** + * Gets the number of files. + * @param size the size of the file + * @param tableName the current table name + * @param familyName the family name + * @return the number of files large than the size + */ + private int countLargeFiles(int size, TableName tableName, String familyName) throws IOException { + Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, familyName); + int count = 0; + if (fs.exists(mobDirPath)) { + FileStatus[] files = fs.listStatus(mobDirPath); + for (FileStatus file : files) { + // ignore the del files in the mob path + if ((!StoreFileInfo.isDelFile(file.getPath())) && (file.getLen() > size)) { + count++; + } + } + } + return count; + } + + /** + * loads some data to the table. + */ + private void loadData(Admin admin, BufferedMutator table, TableName tableName, int fileNum, + int rowNumPerFile) throws IOException, InterruptedException { + if (fileNum <= 0) { + throw new IllegalArgumentException(); + } + for (int i = 0; i < fileNum * rowNumPerFile; i++) { + for (byte k0 : KEYS) { + byte[] k = new byte[] { k0 }; + byte[] key = Bytes.add(k, Bytes.toBytes(i)); + byte[] mobVal = makeDummyData(10 * (i + 1)); + Put put = new Put(key); + put.setDurability(Durability.SKIP_WAL); + put.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), mobVal); + put.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf2), mobVal); + put.addColumn(Bytes.toBytes(family2), Bytes.toBytes(qf1), mobVal); + table.mutate(put); + } + if ((i + 1) % rowNumPerFile == 0) { + table.flush(); + admin.flush(tableName); + } + } + } + + private void loadData(Admin admin, BufferedMutator table, TableName tableName, Put[] puts) + throws IOException { + table.mutate(Arrays.asList(puts)); + table.flush(); + admin.flush(tableName); + } + + private void loadDataForPartitionPolicy(Admin admin, BufferedMutator table, TableName tableName) + throws IOException { + + Put[] pArray = new Put[1000]; + + for (int i = 0; i < 1000; i ++) { + Put put0 = new Put(Bytes.toBytes("r0" + i)); + put0.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), + tsFor20151130Monday, Bytes.toBytes(mobValue0)); + pArray[i] = put0; + } + loadData(admin, bufMut, tableName, pArray); + + Put put06 = new Put(mobKey06); + put06.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), + tsFor20151128Saturday, Bytes.toBytes(mobValue0)); + + loadData(admin, bufMut, tableName, new Put[] { put06 }); + + Put put1 = new Put(mobKey1); + put1.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151201Tuesday, + Bytes.toBytes(mobValue1)); + loadData(admin, bufMut, tableName, new Put[] { put1 }); + + Put put2 = new Put(mobKey2); + put2.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151205Saturday, + Bytes.toBytes(mobValue2)); + loadData(admin, bufMut, tableName, new Put[] { put2 }); + + Put put3 = new Put(mobKey3); + put3.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151228Monday, + Bytes.toBytes(mobValue3)); + loadData(admin, bufMut, tableName, new Put[] { put3 }); + + Put put4 = new Put(mobKey4); + put4.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151231Thursday, + Bytes.toBytes(mobValue4)); + loadData(admin, bufMut, tableName, new Put[] { put4 }); + + Put put5 = new Put(mobKey5); + put5.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20160101Friday, + Bytes.toBytes(mobValue5)); + loadData(admin, bufMut, tableName, new Put[] { put5 }); + + Put put6 = new Put(mobKey6); + put6.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20160103Sunday, + Bytes.toBytes(mobValue6)); + loadData(admin, bufMut, tableName, new Put[] { put6 }); + + Put put7 = new Put(mobKey7); + put7.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20150907Monday, + Bytes.toBytes(mobValue7)); + loadData(admin, bufMut, tableName, new Put[] { put7 }); + + Put put8 = new Put(mobKey8); + put8.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151120Sunday, + Bytes.toBytes(mobValue8)); + loadData(admin, bufMut, tableName, new Put[] { put8 }); + } + + + /** + * delete the row, family and cell to create the del file + */ + private void createDelFile(Table table, TableName tableName, byte[] family, byte[] qf) + throws IOException, InterruptedException { + for (byte k0 : KEYS) { + byte[] k = new byte[] { k0 }; + // delete a family + byte[] key1 = Bytes.add(k, Bytes.toBytes(0)); + Delete delete1 = new Delete(key1); + delete1.addFamily(family); + table.delete(delete1); + // delete one row + byte[] key2 = Bytes.add(k, Bytes.toBytes(2)); + Delete delete2 = new Delete(key2); + table.delete(delete2); + // delete one cell + byte[] key3 = Bytes.add(k, Bytes.toBytes(4)); + Delete delete3 = new Delete(key3); + delete3.addColumn(family, qf); + table.delete(delete3); + } + admin.flush(tableName); + List regions = TEST_UTIL.getHBaseCluster().getRegions(tableName); + for (HRegion region : regions) { + region.waitForFlushesAndCompactions(); + region.compact(true); + } + } + /** + * Creates the dummy data with a specific size. + * @param size the size of value + * @return the dummy data + */ + private byte[] makeDummyData(int size) { + byte[] dummyData = new byte[size]; + new Random().nextBytes(dummyData); + return dummyData; + } + + /** + * Gets the split keys + */ + private byte[][] getSplitKeys() { + byte[][] splitKeys = new byte[KEYS.length - 1][]; + for (int i = 0; i < splitKeys.length; ++i) { + splitKeys[i] = new byte[] { KEYS[i + 1] }; + } + return splitKeys; + } + + private static ExecutorService createThreadPool(Configuration conf) { + int maxThreads = 10; + long keepAliveTime = 60; + final SynchronousQueue queue = new SynchronousQueue<>(); + ThreadPoolExecutor pool = new ThreadPoolExecutor(1, maxThreads, + keepAliveTime, TimeUnit.SECONDS, queue, + Threads.newDaemonThreadFactory("MobFileCompactionChore"), + new RejectedExecutionHandler() { + @Override + public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) { + try { + // waiting for a thread to pick up instead of throwing exceptions. + queue.put(r); + } catch (InterruptedException e) { + throw new RejectedExecutionException(e); + } + } + }); + pool.allowCoreThreadTimeOut(true); + return pool; + } + + private void assertRefFileNameEqual(String familyName) throws IOException { + Scan scan = new Scan(); + scan.addFamily(Bytes.toBytes(familyName)); + // Do not retrieve the mob data when scanning + scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE)); + ResultScanner results = table.getScanner(scan); + Path mobFamilyPath = MobUtils.getMobFamilyPath(TEST_UTIL.getConfiguration(), + tableName, familyName); + List actualFilePaths = new ArrayList<>(); + List expectFilePaths = new ArrayList<>(); + for (Result res : results) { + for (Cell cell : res.listCells()) { + byte[] referenceValue = CellUtil.cloneValue(cell); + String fileName = Bytes.toString(referenceValue, Bytes.SIZEOF_INT, + referenceValue.length - Bytes.SIZEOF_INT); + Path targetPath = new Path(mobFamilyPath, fileName); + if(!actualFilePaths.contains(targetPath)) { + actualFilePaths.add(targetPath); + } + } + } + results.close(); + if (fs.exists(mobFamilyPath)) { + FileStatus[] files = fs.listStatus(mobFamilyPath); + for (FileStatus file : files) { + if (!StoreFileInfo.isDelFile(file.getPath())) { + expectFilePaths.add(file.getPath()); + } + } + } + Collections.sort(actualFilePaths); + Collections.sort(expectFilePaths); + assertEquals(expectFilePaths, actualFilePaths); + } + + /** + * Resets the configuration. + */ + private void resetConf() { + conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, + MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD); + conf.setInt(MobConstants.MOB_COMPACTION_BATCH_SIZE, + MobConstants.DEFAULT_MOB_COMPACTION_BATCH_SIZE); + } + + /** + * Verify mob partition policy compaction values. + */ + private void verifyPolicyValues() throws Exception { + Get get = new Get(mobKey01); + Result result = table.get(get); + assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), + Bytes.toBytes(mobValue0))); + + get = new Get(mobKey02); + result = table.get(get); + assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), + Bytes.toBytes(mobValue0))); + + get = new Get(mobKey03); + result = table.get(get); + assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), + Bytes.toBytes(mobValue0))); + + get = new Get(mobKey04); + result = table.get(get); + assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), + Bytes.toBytes(mobValue0))); + + get = new Get(mobKey05); + result = table.get(get); + assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), + Bytes.toBytes(mobValue0))); + + get = new Get(mobKey06); + result = table.get(get); + assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), + Bytes.toBytes(mobValue0))); + + get = new Get(mobKey1); + result = table.get(get); + assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), + Bytes.toBytes(mobValue1))); + + get = new Get(mobKey2); + result = table.get(get); + assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), + Bytes.toBytes(mobValue2))); + + get = new Get(mobKey3); + result = table.get(get); + assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), + Bytes.toBytes(mobValue3))); + + get = new Get(mobKey4); + result = table.get(get); + assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), + Bytes.toBytes(mobValue4))); + + get = new Get(mobKey5); + result = table.get(get); + assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), + Bytes.toBytes(mobValue5))); + + get = new Get(mobKey6); + result = table.get(get); + assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), + Bytes.toBytes(mobValue6))); + + get = new Get(mobKey7); + result = table.get(get); + assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), + Bytes.toBytes(mobValue7))); + + get = new Get(mobKey8); + result = table.get(get); + assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), + Bytes.toBytes(mobValue8))); + } + + private void commonPolicyTestLogic (final String tableNameAsString, + final MobCompactPartitionPolicy pType, final boolean majorCompact, + final int expectedFileNumbers, final String[] expectedFileNames, + final boolean setupAndLoadData + ) throws Exception { + if (setupAndLoadData) { + setUpForPolicyTest(tableNameAsString, pType); + + loadDataForPartitionPolicy(admin, bufMut, tableName); + } else { + alterForPolicyTest(pType); + } + + if (majorCompact) { + admin.majorCompact(tableName, hcd1.getName(), CompactType.MOB); + } else { + admin.compact(tableName, hcd1.getName(), CompactType.MOB); + } + + waitUntilMobCompactionFinished(tableName); + + // Run cleaner to make sure that files in archive directory are cleaned up + TEST_UTIL.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting(); + + //check the number of files + Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, family1); + FileStatus[] fileList = fs.listStatus(mobDirPath); + + assertTrue(fileList.length == expectedFileNumbers); + + // the file names are expected + ArrayList fileNames = new ArrayList<>(expectedFileNumbers); + for (FileStatus file : fileList) { + fileNames.add(MobFileName.getDateFromName(file.getPath().getName())); + } + int index = 0; + for (String fileName : expectedFileNames) { + index = fileNames.indexOf(fileName); + assertTrue(index >= 0); + fileNames.remove(index); + } + + // Check daily mob files are removed from the mobdir, and only weekly mob files are there. + // Also check that there is no data loss. + + verifyPolicyValues(); + } + } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactionRequest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactionRequest.java new file mode 100644 index 000000000000..27801533dc5a --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactionRequest.java @@ -0,0 +1,65 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.mob.compactions; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartition; +import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(SmallTests.class) +public class TestPartitionedMobCompactionRequest { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestPartitionedMobCompactionRequest.class); + + @Test + public void testCompactedPartitionId() { + String startKey1 = "startKey1"; + String startKey2 = "startKey2"; + String date1 = "date1"; + String date2 = "date2"; + CompactionPartitionId partitionId1 = new CompactionPartitionId(startKey1, date1); + CompactionPartitionId partitionId2 = new CompactionPartitionId(startKey2, date2); + CompactionPartitionId partitionId3 = new CompactionPartitionId(startKey1, date2); + + Assert.assertTrue(partitionId1.equals(partitionId1)); + Assert.assertFalse(partitionId1.equals(partitionId2)); + Assert.assertFalse(partitionId1.equals(partitionId3)); + Assert.assertFalse(partitionId2.equals(partitionId3)); + + Assert.assertEquals(startKey1, partitionId1.getStartKey()); + Assert.assertEquals(date1, partitionId1.getDate()); + } + + @Test + public void testCompactedPartition() { + CompactionPartitionId partitionId = new CompactionPartitionId("startKey1", "date1"); + CompactionPartition partition = new CompactionPartition(partitionId); + FileStatus file = new FileStatus(1, false, 1, 1024, 1, new Path("/test")); + partition.addFile(file); + Assert.assertEquals(file, partition.listFiles().get(0)); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java new file mode 100644 index 000000000000..4e6461aaf800 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java @@ -0,0 +1,958 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.mob.compactions; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.text.ParseException; +import java.util.ArrayList; +import java.util.Calendar; +import java.util.Collection; +import java.util.Collections; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.RejectedExecutionHandler; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparatorImpl; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValue.Type; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy; +import org.apache.hadoop.hbase.io.hfile.CacheConfig; +import org.apache.hadoop.hbase.io.hfile.HFileContext; +import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; +import org.apache.hadoop.hbase.mob.MobConstants; +import org.apache.hadoop.hbase.mob.MobFileName; +import org.apache.hadoop.hbase.mob.MobUtils; +import org.apache.hadoop.hbase.mob.compactions.MobCompactionRequest.CompactionType; +import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionDelPartition; +import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartition; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.HStoreFile; +import org.apache.hadoop.hbase.regionserver.KeyValueScanner; +import org.apache.hadoop.hbase.regionserver.ScanInfo; +import org.apache.hadoop.hbase.regionserver.ScanType; +import org.apache.hadoop.hbase.regionserver.StoreFileScanner; +import org.apache.hadoop.hbase.regionserver.StoreFileWriter; +import org.apache.hadoop.hbase.regionserver.StoreScanner; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.util.Threads; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category(LargeTests.class) +public class TestPartitionedMobCompactor { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestPartitionedMobCompactor.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestPartitionedMobCompactor.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private final static String family = "family"; + private final static String qf = "qf"; + private final long DAY_IN_MS = 1000 * 60 * 60 * 24; + private static byte[] KEYS = Bytes.toBytes("012"); + private HColumnDescriptor hcd = new HColumnDescriptor(family); + private Configuration conf = TEST_UTIL.getConfiguration(); + private CacheConfig cacheConf = new CacheConfig(conf); + private FileSystem fs; + private List mobFiles = new ArrayList<>(); + private List delFiles = new ArrayList<>(); + private List allFiles = new ArrayList<>(); + private Path basePath; + private String mobSuffix; + private String delSuffix; + private static ExecutorService pool; + + @Rule + public TestName name = new TestName(); + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + TEST_UTIL.getConfiguration().setInt("hfile.format.version", 3); + // Inject our customized DistributedFileSystem + TEST_UTIL.getConfiguration().setClass("fs.hdfs.impl", FaultyDistributedFileSystem.class, + DistributedFileSystem.class); + TEST_UTIL.startMiniCluster(1); + pool = createThreadPool(); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + pool.shutdown(); + TEST_UTIL.shutdownMiniCluster(); + } + + private void init(String tableName) throws Exception { + fs = FileSystem.get(conf); + Path testDir = FSUtils.getRootDir(conf); + Path mobTestDir = new Path(testDir, MobConstants.MOB_DIR_NAME); + basePath = new Path(new Path(mobTestDir, tableName), family); + mobSuffix = TEST_UTIL.getRandomUUID().toString().replaceAll("-", ""); + delSuffix = TEST_UTIL.getRandomUUID().toString().replaceAll("-", "") + "_del"; + allFiles.clear(); + mobFiles.clear(); + delFiles.clear(); + } + + @Test + public void testCompactionSelectAllFilesWeeklyPolicy() throws Exception { + String tableName = "testCompactionSelectAllFilesWeeklyPolicy"; + testCompactionAtMergeSize(tableName, MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD, + CompactionType.ALL_FILES, false, false, new Date(), MobCompactPartitionPolicy.WEEKLY, 1); + } + + @Test + public void testCompactionSelectPartFilesWeeklyPolicy() throws Exception { + String tableName = "testCompactionSelectPartFilesWeeklyPolicy"; + testCompactionAtMergeSize(tableName, 4000, CompactionType.PART_FILES, false, false, + new Date(), MobCompactPartitionPolicy.WEEKLY, 1); + } + + @Test + public void testCompactionSelectPartFilesWeeklyPolicyWithPastWeek() throws Exception { + String tableName = "testCompactionSelectPartFilesWeeklyPolicyWithPastWeek"; + Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS)); + testCompactionAtMergeSize(tableName, 700, CompactionType.PART_FILES, false, false, dateLastWeek, + MobCompactPartitionPolicy.WEEKLY, 7); + } + + @Test + public void testCompactionSelectAllFilesWeeklyPolicyWithPastWeek() throws Exception { + String tableName = "testCompactionSelectAllFilesWeeklyPolicyWithPastWeek"; + Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS)); + testCompactionAtMergeSize(tableName, 3000, CompactionType.ALL_FILES, + false, false, dateLastWeek, MobCompactPartitionPolicy.WEEKLY, 7); + } + + @Test + public void testCompactionSelectAllFilesMonthlyPolicy() throws Exception { + String tableName = "testCompactionSelectAllFilesMonthlyPolicy"; + Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS)); + testCompactionAtMergeSize(tableName, MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD, + CompactionType.ALL_FILES, false, false, dateLastWeek, + MobCompactPartitionPolicy.MONTHLY, 7); + } + + @Test + public void testCompactionSelectNoFilesWithinCurrentWeekMonthlyPolicy() throws Exception { + String tableName = "testCompactionSelectNoFilesWithinCurrentWeekMonthlyPolicy"; + testCompactionAtMergeSize(tableName, MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD, + CompactionType.PART_FILES, false, false, new Date(), MobCompactPartitionPolicy.MONTHLY, 1); + } + + @Test + public void testCompactionSelectPartFilesMonthlyPolicy() throws Exception { + String tableName = "testCompactionSelectPartFilesMonthlyPolicy"; + testCompactionAtMergeSize(tableName, 4000, CompactionType.PART_FILES, false, false, + new Date(), MobCompactPartitionPolicy.MONTHLY, 1); + } + + @Test + public void testCompactionSelectPartFilesMonthlyPolicyWithPastWeek() throws Exception { + String tableName = "testCompactionSelectPartFilesMonthlyPolicyWithPastWeek"; + Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS)); + Calendar calendar = Calendar.getInstance(); + Date firstDayOfCurrentMonth = MobUtils.getFirstDayOfMonth(calendar, new Date()); + CompactionType type = CompactionType.PART_FILES; + long mergeSizeMultiFactor = 7; + + + // The dateLastWeek may not really be last week, suppose that it runs at 2/1/2017, it is going + // to be last month and the monthly policy is going to be applied here. + if (dateLastWeek.before(firstDayOfCurrentMonth)) { + type = CompactionType.ALL_FILES; + mergeSizeMultiFactor *= 4; + } + + testCompactionAtMergeSize(tableName, 700, type, false, false, dateLastWeek, + MobCompactPartitionPolicy.MONTHLY, mergeSizeMultiFactor); + } + + @Test + public void testCompactionSelectAllFilesMonthlyPolicyWithPastWeek() throws Exception { + String tableName = "testCompactionSelectAllFilesMonthlyPolicyWithPastWeek"; + Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS)); + + testCompactionAtMergeSize(tableName, 3000, CompactionType.ALL_FILES, + false, false, dateLastWeek, MobCompactPartitionPolicy.MONTHLY, 7); + } + + @Test + public void testCompactionSelectPartFilesMonthlyPolicyWithPastMonth() throws Exception { + String tableName = "testCompactionSelectPartFilesMonthlyPolicyWithPastMonth"; + + // back 5 weeks, it is going to be a past month + Date dateLastMonth = new Date(System.currentTimeMillis() - (7 * 5 * DAY_IN_MS)); + testCompactionAtMergeSize(tableName, 200, CompactionType.PART_FILES, false, false, dateLastMonth, + MobCompactPartitionPolicy.MONTHLY, 28); + } + + @Test + public void testCompactionSelectAllFilesMonthlyPolicyWithPastMonth() throws Exception { + String tableName = "testCompactionSelectAllFilesMonthlyPolicyWithPastMonth"; + + // back 5 weeks, it is going to be a past month + Date dateLastMonth = new Date(System.currentTimeMillis() - (7 * 5 * DAY_IN_MS)); + testCompactionAtMergeSize(tableName, 750, CompactionType.ALL_FILES, + false, false, dateLastMonth, MobCompactPartitionPolicy.MONTHLY, 28); + } + + @Test + public void testCompactionSelectWithAllFiles() throws Exception { + String tableName = "testCompactionSelectWithAllFiles"; + // If there is only 1 file, it will not be compacted with _del files, so + // It wont be CompactionType.ALL_FILES in this case, do not create with _del files. + testCompactionAtMergeSize(tableName, MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD, + CompactionType.ALL_FILES, false, false); + } + + @Test + public void testCompactionSelectWithPartFiles() throws Exception { + String tableName = "testCompactionSelectWithPartFiles"; + testCompactionAtMergeSize(tableName, 4000, CompactionType.PART_FILES, false); + } + + @Test + public void testCompactionSelectWithForceAllFiles() throws Exception { + String tableName = "testCompactionSelectWithForceAllFiles"; + testCompactionAtMergeSize(tableName, Long.MAX_VALUE, CompactionType.ALL_FILES, true); + } + + private void testCompactionAtMergeSize(final String tableName, + final long mergeSize, final CompactionType type, final boolean isForceAllFiles) + throws Exception { + testCompactionAtMergeSize(tableName, mergeSize, type, isForceAllFiles, true); + } + + private void testCompactionAtMergeSize(final String tableName, + final long mergeSize, final CompactionType type, final boolean isForceAllFiles, + final boolean createDelFiles) + throws Exception { + Date date = new Date(); + testCompactionAtMergeSize(tableName, mergeSize, type, isForceAllFiles, createDelFiles, date); + } + + private void testCompactionAtMergeSize(final String tableName, + final long mergeSize, final CompactionType type, final boolean isForceAllFiles, + final boolean createDelFiles, final Date date) + throws Exception { + testCompactionAtMergeSize(tableName, mergeSize, type, isForceAllFiles, createDelFiles, date, + MobCompactPartitionPolicy.DAILY, 1); + } + + private void testCompactionAtMergeSize(final String tableName, + final long mergeSize, final CompactionType type, final boolean isForceAllFiles, + final boolean createDelFiles, final Date date, final MobCompactPartitionPolicy policy, + final long mergeSizeMultiFactor) + throws Exception { + resetConf(); + init(tableName); + int count = 10; + // create 10 mob files. + createStoreFiles(basePath, family, qf, count, Type.Put, date); + + if (createDelFiles) { + // create 10 del files + createStoreFiles(basePath, family, qf, count, Type.Delete, date); + } + + Calendar calendar = Calendar.getInstance(); + Date firstDayOfCurrentWeek = MobUtils.getFirstDayOfWeek(calendar, new Date()); + + listFiles(); + List expectedStartKeys = new ArrayList<>(); + for(FileStatus file : mobFiles) { + if(file.getLen() < mergeSize * mergeSizeMultiFactor) { + String fileName = file.getPath().getName(); + String startKey = fileName.substring(0, 32); + + // If the policy is monthly and files are in current week, they will be skipped + // in minor compcation. + boolean skipCompaction = false; + if (policy == MobCompactPartitionPolicy.MONTHLY) { + String fileDateStr = MobFileName.getDateFromName(fileName); + Date fileDate; + try { + fileDate = MobUtils.parseDate(fileDateStr); + } catch (ParseException e) { + LOG.warn("Failed to parse date " + fileDateStr, e); + fileDate = new Date(); + } + if (!fileDate.before(firstDayOfCurrentWeek)) { + skipCompaction = true; + } + } + + // If it is not an major mob compaction and del files are there, + // these mob files wont be compacted. + if (isForceAllFiles || (!createDelFiles && !skipCompaction)) { + expectedStartKeys.add(startKey); + } + } + } + + // Set the policy + this.hcd.setMobCompactPartitionPolicy(policy); + // set the mob compaction mergeable threshold + conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize); + testSelectFiles(tableName, type, isForceAllFiles, expectedStartKeys); + // go back to the default daily policy + this.hcd.setMobCompactPartitionPolicy(MobCompactPartitionPolicy.DAILY); + } + + @Test + public void testCompactDelFilesWithDefaultBatchSize() throws Exception { + testCompactDelFilesAtBatchSize(name.getMethodName(), MobConstants.DEFAULT_MOB_COMPACTION_BATCH_SIZE, + MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT); + } + + @Test + public void testCompactDelFilesWithSmallBatchSize() throws Exception { + testCompactDelFilesAtBatchSize(name.getMethodName(), 4, MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT); + } + + @Test + public void testCompactDelFilesChangeMaxDelFileCount() throws Exception { + testCompactDelFilesAtBatchSize(name.getMethodName(), 4, 2); + } + + @Test + public void testCompactFilesWithDstDirFull() throws Exception { + String tableName = name.getMethodName(); + fs = FileSystem.get(conf); + FaultyDistributedFileSystem faultyFs = (FaultyDistributedFileSystem)fs; + Path testDir = FSUtils.getRootDir(conf); + Path mobTestDir = new Path(testDir, MobConstants.MOB_DIR_NAME); + basePath = new Path(new Path(mobTestDir, tableName), family); + + try { + int count = 2; + // create 2 mob files. + createStoreFiles(basePath, family, qf, count, Type.Put, true, new Date()); + listFiles(); + + TableName tName = TableName.valueOf(tableName); + MobCompactor compactor = new PartitionedMobCompactor(conf, faultyFs, tName, hcd, pool); + faultyFs.setThrowException(true); + try { + compactor.compact(allFiles, true); + } catch (IOException e) { + System.out.println("Expected exception, ignore"); + } + + // Verify that all the files in tmp directory are cleaned up + Path tempPath = new Path(MobUtils.getMobHome(conf), MobConstants.TEMP_DIR_NAME); + FileStatus[] ls = faultyFs.listStatus(tempPath); + + // Only .bulkload under this directory + assertTrue(ls.length == 1); + assertTrue(MobConstants.BULKLOAD_DIR_NAME.equalsIgnoreCase(ls[0].getPath().getName())); + + Path bulkloadPath = new Path(tempPath, new Path(MobConstants.BULKLOAD_DIR_NAME, new Path( + tName.getNamespaceAsString(), tName.getQualifierAsString()))); + + // Nothing in bulkLoad directory + FileStatus[] lsBulkload = faultyFs.listStatus(bulkloadPath); + assertTrue(lsBulkload.length == 0); + + } finally { + faultyFs.setThrowException(false); + } + } + + /** + * Create mulitple partition files + */ + private void createMobFile(Path basePath) throws IOException { + HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build(); + MobFileName mobFileName = null; + int ii = 0; + Date today = new Date(); + for (byte k0 : KEYS) { + byte[] startRow = Bytes.toBytes(ii++); + + mobFileName = MobFileName.create(startRow, MobUtils.formatDate(today), mobSuffix); + + StoreFileWriter mobFileWriter = + new StoreFileWriter.Builder(conf, cacheConf, fs).withFileContext(meta) + .withFilePath(new Path(basePath, mobFileName.getFileName())).build(); + + long now = System.currentTimeMillis(); + try { + for (int i = 0; i < 10; i++) { + byte[] key = Bytes.add(Bytes.toBytes(k0), Bytes.toBytes(i)); + byte[] dummyData = new byte[5000]; + new Random().nextBytes(dummyData); + mobFileWriter.append( + new KeyValue(key, Bytes.toBytes(family), Bytes.toBytes(qf), now, Type.Put, dummyData)); + } + } finally { + mobFileWriter.close(); + } + } + } + + /** + * Create mulitple partition delete files + */ + private void createMobDelFile(Path basePath, int startKey) throws IOException { + HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build(); + MobFileName mobFileName = null; + Date today = new Date(); + + byte[] startRow = Bytes.toBytes(startKey); + + mobFileName = MobFileName.create(startRow, MobUtils.formatDate(today), delSuffix); + + StoreFileWriter mobFileWriter = + new StoreFileWriter.Builder(conf, cacheConf, fs).withFileContext(meta) + .withFilePath(new Path(basePath, mobFileName.getFileName())).build(); + + long now = System.currentTimeMillis(); + try { + byte[] key = Bytes.add(Bytes.toBytes(KEYS[startKey]), Bytes.toBytes(0)); + byte[] dummyData = new byte[5000]; + new Random().nextBytes(dummyData); + mobFileWriter.append( + new KeyValue(key, Bytes.toBytes(family), Bytes.toBytes(qf), now, Type.Delete, dummyData)); + key = Bytes.add(Bytes.toBytes(KEYS[startKey]), Bytes.toBytes(2)); + mobFileWriter.append( + new KeyValue(key, Bytes.toBytes(family), Bytes.toBytes(qf), now, Type.Delete, dummyData)); + key = Bytes.add(Bytes.toBytes(KEYS[startKey]), Bytes.toBytes(4)); + mobFileWriter.append( + new KeyValue(key, Bytes.toBytes(family), Bytes.toBytes(qf), now, Type.Delete, dummyData)); + + } finally { + mobFileWriter.close(); + } + } + + @Test + public void testCompactFilesWithoutDelFile() throws Exception { + String tableName = "testCompactFilesWithoutDelFile"; + resetConf(); + init(tableName); + + createMobFile(basePath); + + listFiles(); + + PartitionedMobCompactor compactor = new PartitionedMobCompactor(conf, fs, + TableName.valueOf(tableName), hcd, pool) { + @Override + public List compact(List files, boolean isForceAllFiles) + throws IOException { + if (files == null || files.isEmpty()) { + return null; + } + + PartitionedMobCompactionRequest request = select(files, isForceAllFiles); + + // Make sure that there is no del Partitions + assertTrue(request.getDelPartitions().size() == 0); + + // Make sure that when there is no startKey/endKey for partition. + for (CompactionPartition p : request.getCompactionPartitions()) { + assertTrue(p.getStartKey() == null); + assertTrue(p.getEndKey() == null); + } + return null; + } + }; + + compactor.compact(allFiles, true); + } + + static class MyPartitionedMobCompactor extends PartitionedMobCompactor { + int delPartitionSize = 0; + int PartitionsIncludeDelFiles = 0; + CacheConfig cacheConfig = null; + + MyPartitionedMobCompactor(Configuration conf, FileSystem fs, TableName tableName, + ColumnFamilyDescriptor column, ExecutorService pool, final int delPartitionSize, + final CacheConfig cacheConf, final int PartitionsIncludeDelFiles) + throws IOException { + super(conf, fs, tableName, column, pool); + this.delPartitionSize = delPartitionSize; + this.cacheConfig = cacheConf; + this.PartitionsIncludeDelFiles = PartitionsIncludeDelFiles; + } + + @Override public List compact(List files, boolean isForceAllFiles) + throws IOException { + if (files == null || files.isEmpty()) { + return null; + } + PartitionedMobCompactionRequest request = select(files, isForceAllFiles); + + assertTrue(request.getDelPartitions().size() == delPartitionSize); + if (request.getDelPartitions().size() > 0) { + for (CompactionPartition p : request.getCompactionPartitions()) { + assertTrue(p.getStartKey() != null); + assertTrue(p.getEndKey() != null); + } + } + + try { + for (CompactionDelPartition delPartition : request.getDelPartitions()) { + for (Path newDelPath : delPartition.listDelFiles()) { + HStoreFile sf = + new HStoreFile(fs, newDelPath, conf, this.cacheConfig, BloomType.NONE, true); + // pre-create reader of a del file to avoid race condition when opening the reader in + // each partition. + sf.initReader(); + delPartition.addStoreFile(sf); + } + } + + // Make sure that CompactionDelPartitions does not overlap + CompactionDelPartition prevDelP = null; + for (CompactionDelPartition delP : request.getDelPartitions()) { + assertTrue( + Bytes.compareTo(delP.getId().getStartKey(), delP.getId().getEndKey()) <= 0); + + if (prevDelP != null) { + assertTrue( + Bytes.compareTo(prevDelP.getId().getEndKey(), delP.getId().getStartKey()) < 0); + } + } + + int affectedPartitions = 0; + + // Make sure that only del files within key range for a partition is included in compaction. + // compact the mob files by partitions in parallel. + for (CompactionPartition partition : request.getCompactionPartitions()) { + List delFiles = getListOfDelFilesForPartition(partition, request.getDelPartitions()); + if (!request.getDelPartitions().isEmpty()) { + if (!((Bytes.compareTo(request.getDelPartitions().get(0).getId().getStartKey(), + partition.getEndKey()) > 0) || (Bytes.compareTo( + request.getDelPartitions().get(request.getDelPartitions().size() - 1).getId() + .getEndKey(), partition.getStartKey()) < 0))) { + + if (delFiles.size() > 0) { + assertTrue(delFiles.size() == 1); + affectedPartitions += delFiles.size(); + assertTrue(Bytes.compareTo(partition.getStartKey(), + CellUtil.cloneRow(delFiles.get(0).getLastKey().get())) <= 0); + assertTrue(Bytes.compareTo(partition.getEndKey(), + CellUtil.cloneRow(delFiles.get(delFiles.size() - 1).getFirstKey().get())) >= 0); + } + } + } + } + // The del file is only included in one partition + assertTrue(affectedPartitions == PartitionsIncludeDelFiles); + } finally { + for (CompactionDelPartition delPartition : request.getDelPartitions()) { + for (HStoreFile storeFile : delPartition.getStoreFiles()) { + try { + storeFile.closeStoreFile(true); + } catch (IOException e) { + LOG.warn("Failed to close the reader on store file " + storeFile.getPath(), e); + } + } + } + } + + return null; + } + } + + @Test + public void testCompactFilesWithOneDelFile() throws Exception { + String tableName = "testCompactFilesWithOneDelFile"; + resetConf(); + init(tableName); + + // Create only del file. + createMobFile(basePath); + createMobDelFile(basePath, 2); + + listFiles(); + + MyPartitionedMobCompactor compactor = new MyPartitionedMobCompactor(conf, fs, + TableName.valueOf(tableName), hcd, pool, 1, cacheConf, 1); + + compactor.compact(allFiles, true); + } + + @Test + public void testCompactFilesWithMultiDelFiles() throws Exception { + String tableName = "testCompactFilesWithMultiDelFiles"; + resetConf(); + init(tableName); + + // Create only del file. + createMobFile(basePath); + createMobDelFile(basePath, 0); + createMobDelFile(basePath, 1); + createMobDelFile(basePath, 2); + + listFiles(); + + MyPartitionedMobCompactor compactor = new MyPartitionedMobCompactor(conf, fs, + TableName.valueOf(tableName), hcd, pool, 3, cacheConf, 3); + compactor.compact(allFiles, true); + } + + private void testCompactDelFilesAtBatchSize(String tableName, int batchSize, + int delfileMaxCount) throws Exception { + resetConf(); + init(tableName); + // create 20 mob files. + createStoreFiles(basePath, family, qf, 20, Type.Put, new Date()); + // create 13 del files + createStoreFiles(basePath, family, qf, 13, Type.Delete, new Date()); + listFiles(); + + // set the max del file count + conf.setInt(MobConstants.MOB_DELFILE_MAX_COUNT, delfileMaxCount); + // set the mob compaction batch size + conf.setInt(MobConstants.MOB_COMPACTION_BATCH_SIZE, batchSize); + testCompactDelFiles(tableName, 1, 13, false); + } + + /** + * Tests the selectFiles + * @param tableName the table name + * @param type the expected compaction type + * @param isForceAllFiles whether all the mob files are selected + * @param expected the expected start keys + */ + private void testSelectFiles(String tableName, final CompactionType type, + final boolean isForceAllFiles, final List expected) throws IOException { + PartitionedMobCompactor compactor = new PartitionedMobCompactor(conf, fs, + TableName.valueOf(tableName), hcd, pool) { + @Override + public List compact(List files, boolean isForceAllFiles) + throws IOException { + if (files == null || files.isEmpty()) { + return null; + } + PartitionedMobCompactionRequest request = select(files, isForceAllFiles); + + // Make sure that when there is no del files, there will be no startKey/endKey for partition. + if (request.getDelPartitions().size() == 0) { + for (CompactionPartition p : request.getCompactionPartitions()) { + assertTrue(p.getStartKey() == null); + assertTrue(p.getEndKey() == null); + } + } + + // Make sure that CompactionDelPartitions does not overlap + CompactionDelPartition prevDelP = null; + for (CompactionDelPartition delP : request.getDelPartitions()) { + assertTrue(Bytes.compareTo(delP.getId().getStartKey(), + delP.getId().getEndKey()) <= 0); + + if (prevDelP != null) { + assertTrue(Bytes.compareTo(prevDelP.getId().getEndKey(), + delP.getId().getStartKey()) < 0); + } + } + + // Make sure that only del files within key range for a partition is included in compaction. + // compact the mob files by partitions in parallel. + for (CompactionPartition partition : request.getCompactionPartitions()) { + List delFiles = getListOfDelFilesForPartition(partition, request.getDelPartitions()); + if (!request.getDelPartitions().isEmpty()) { + if (!((Bytes.compareTo(request.getDelPartitions().get(0).getId().getStartKey(), + partition.getEndKey()) > 0) || (Bytes.compareTo( + request.getDelPartitions().get(request.getDelPartitions().size() - 1).getId() + .getEndKey(), partition.getStartKey()) < 0))) { + if (delFiles.size() > 0) { + assertTrue(Bytes.compareTo(partition.getStartKey(), + delFiles.get(0).getFirstKey().get().getRowArray()) >= 0); + assertTrue(Bytes.compareTo(partition.getEndKey(), + delFiles.get(delFiles.size() - 1).getLastKey().get().getRowArray()) <= 0); + } + } + } + } + + // assert the compaction type + assertEquals(type, request.type); + // assert get the right partitions + compareCompactedPartitions(expected, request.compactionPartitions); + // assert get the right del files + compareDelFiles(request.getDelPartitions()); + return null; + } + }; + compactor.compact(allFiles, isForceAllFiles); + } + + /** + * Tests the compacteDelFile + * @param tableName the table name + * @param expectedFileCount the expected file count + * @param expectedCellCount the expected cell count + * @param isForceAllFiles whether all the mob files are selected + */ + private void testCompactDelFiles(String tableName, final int expectedFileCount, + final int expectedCellCount, boolean isForceAllFiles) throws IOException { + PartitionedMobCompactor compactor = new PartitionedMobCompactor(conf, fs, + TableName.valueOf(tableName), hcd, pool) { + @Override + protected List performCompaction(PartitionedMobCompactionRequest request) + throws IOException { + List delFilePaths = new ArrayList<>(); + for (CompactionDelPartition delPartition: request.getDelPartitions()) { + for (Path p : delPartition.listDelFiles()) { + delFilePaths.add(p); + } + } + List newDelPaths = compactDelFiles(request, delFilePaths); + // assert the del files are merged. + assertEquals(expectedFileCount, newDelPaths.size()); + assertEquals(expectedCellCount, countDelCellsInDelFiles(newDelPaths)); + return null; + } + }; + compactor.compact(allFiles, isForceAllFiles); + } + + /** + * Lists the files in the path + */ + private void listFiles() throws IOException { + for (FileStatus file : fs.listStatus(basePath)) { + allFiles.add(file); + if (file.getPath().getName().endsWith("_del")) { + delFiles.add(file.getPath()); + } else { + mobFiles.add(file); + } + } + } + + /** + * Compares the compacted partitions. + * @param partitions the collection of CompactedPartitions + */ + private void compareCompactedPartitions(List expected, + Collection partitions) { + List actualKeys = new ArrayList<>(); + for (CompactionPartition partition : partitions) { + actualKeys.add(partition.getPartitionId().getStartKey()); + } + Collections.sort(expected); + Collections.sort(actualKeys); + assertEquals(expected.size(), actualKeys.size()); + for (int i = 0; i < expected.size(); i++) { + assertEquals(expected.get(i), actualKeys.get(i)); + } + } + + /** + * Compares the del files. + * @param delPartitions all del partitions + */ + private void compareDelFiles(List delPartitions) { + Map delMap = new HashMap<>(); + for (CompactionDelPartition delPartition : delPartitions) { + for (Path f : delPartition.listDelFiles()) { + delMap.put(f, f); + } + } + for (Path f : delFiles) { + assertTrue(delMap.containsKey(f)); + } + } + + /** + * Creates store files. + * @param basePath the path to create file + * @param family the column family name + * @param qualifier the column qualifier assigned to data values + * @param count the store file number + * @param type the row key type + * @param date the latest timestamp when an instance of MobFileName is created + */ + private void createStoreFiles(Path basePath, String family, String qualifier, int count, + Type type, final Date date) throws IOException { + createStoreFiles(basePath, family, qualifier, count, type, false, date); + } + + private void createStoreFiles(Path basePath, String family, String qualifier, int count, + Type type, boolean sameStartKey, final Date date) throws IOException { + HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build(); + String startKey = "row_"; + MobFileName mobFileName = null; + for (int i = 0; i < count; i++) { + byte[] startRow; + if (sameStartKey) { + // When creating multiple files under one partition, suffix needs to be different. + startRow = Bytes.toBytes(startKey); + mobSuffix = TEST_UTIL.getRandomUUID().toString().replaceAll("-", ""); + delSuffix = TEST_UTIL.getRandomUUID().toString().replaceAll("-", "") + "_del"; + } else { + startRow = Bytes.toBytes(startKey + i); + } + if(type.equals(Type.Delete)) { + mobFileName = MobFileName.create(startRow, MobUtils.formatDate(date), delSuffix); + } + if(type.equals(Type.Put)){ + mobFileName = MobFileName.create(startRow, MobUtils.formatDate(date), mobSuffix); + } + StoreFileWriter mobFileWriter = new StoreFileWriter.Builder(conf, cacheConf, fs) + .withFileContext(meta).withFilePath(new Path(basePath, mobFileName.getFileName())).build(); + writeStoreFile(mobFileWriter, startRow, Bytes.toBytes(family), Bytes.toBytes(qualifier), + type, (i+1)*1000); + } + } + + /** + * Writes data to store file. + * @param writer the store file writer + * @param row the row key + * @param family the family name + * @param qualifier the column qualifier + * @param type the key type + * @param size the size of value + */ + private static void writeStoreFile(final StoreFileWriter writer, byte[]row, byte[] family, + byte[] qualifier, Type type, int size) throws IOException { + long now = System.currentTimeMillis(); + try { + byte[] dummyData = new byte[size]; + new Random().nextBytes(dummyData); + writer.append(new KeyValue(row, family, qualifier, now, type, dummyData)); + } finally { + writer.close(); + } + } + + /** + * Gets the number of del cell in the del files + * @param paths the del file paths + * @return the cell size + */ + private int countDelCellsInDelFiles(List paths) throws IOException { + List sfs = new ArrayList<>(); + int size = 0; + for (Path path : paths) { + HStoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true); + sfs.add(sf); + } + List scanners = new ArrayList<>(StoreFileScanner.getScannersForStoreFiles(sfs, + false, true, false, false, HConstants.LATEST_TIMESTAMP)); + long timeToPurgeDeletes = Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0); + long ttl = HStore.determineTTLFromFamily(hcd); + ScanInfo scanInfo = new ScanInfo(conf, hcd, ttl, timeToPurgeDeletes, CellComparatorImpl.COMPARATOR); + StoreScanner scanner = new StoreScanner(scanInfo, ScanType.COMPACT_RETAIN_DELETES, scanners); + List results = new ArrayList<>(); + boolean hasMore = true; + + while (hasMore) { + hasMore = scanner.next(results); + size += results.size(); + results.clear(); + } + scanner.close(); + return size; + } + + private static ExecutorService createThreadPool() { + int maxThreads = 10; + long keepAliveTime = 60; + final SynchronousQueue queue = new SynchronousQueue<>(); + ThreadPoolExecutor pool = new ThreadPoolExecutor(1, maxThreads, keepAliveTime, + TimeUnit.SECONDS, queue, Threads.newDaemonThreadFactory("MobFileCompactionChore"), + new RejectedExecutionHandler() { + @Override + public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) { + try { + // waiting for a thread to pick up instead of throwing exceptions. + queue.put(r); + } catch (InterruptedException e) { + throw new RejectedExecutionException(e); + } + } + }); + ((ThreadPoolExecutor) pool).allowCoreThreadTimeOut(true); + return pool; + } + + /** + * Resets the configuration. + */ + private void resetConf() { + conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, + MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD); + conf.setInt(MobConstants.MOB_DELFILE_MAX_COUNT, MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT); + conf.setInt(MobConstants.MOB_COMPACTION_BATCH_SIZE, + MobConstants.DEFAULT_MOB_COMPACTION_BATCH_SIZE); + } + + /** + * The customized Distributed File System Implementation + */ + static class FaultyDistributedFileSystem extends DistributedFileSystem { + private volatile boolean throwException = false; + + public FaultyDistributedFileSystem() { + super(); + } + + public void setThrowException(boolean throwException) { + this.throwException = throwException; + } + + @Override + public boolean rename(Path src, Path dst) throws IOException { + if (throwException) { + throw new IOException("No more files allowed"); + } + return super.rename(src, dst); + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java index 727f6febc366..e8096fb95106 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java @@ -402,7 +402,7 @@ String createMobFileName(String oldFileName) { String startKey = mobFileName.getStartKey(); String date = mobFileName.getDate(); return MobFileName.create(startKey, date, - TEST_UTIL.getRandomUUID().toString().replaceAll("-", ""), "fakeregion") + TEST_UTIL.getRandomUUID().toString().replaceAll("-", "")) .getFileName(); } From 690544dfdcd4b24823357810d90822f3babcf590 Mon Sep 17 00:00:00 2001 From: Vladimir Rodionov Date: Thu, 17 Oct 2019 11:47:08 -0700 Subject: [PATCH 03/15] HBASE-22749: Distributed MOB compactions --- .../master/ExpiredMobFileCleanerChore.java | 87 -- .../apache/hadoop/hbase/master/HMaster.java | 40 +- .../master/MasterMobCompactionThread.java | 181 --- .../hbase/master/MasterRpcServices.java | 90 +- .../hbase/master/MobCompactionChore.java | 96 -- .../hbase/master/MobFileCleanerChore.java | 260 ++++ .../hbase/master/MobFileCompactionChore.java | 201 +++ .../hbase/mob/DefaultMobStoreCompactor.java | 690 ++++++++-- .../hbase/mob/DefaultMobStoreFlusher.java | 26 + .../hbase/mob/ExpiredMobFileCleaner.java | 5 +- .../apache/hadoop/hbase/mob/MobConstants.java | 100 +- .../apache/hadoop/hbase/mob/MobFileName.java | 39 +- .../hadoop/hbase/mob/MobStoreEngine.java | 12 +- .../org/apache/hadoop/hbase/mob/MobUtils.java | 518 ++----- .../mob/compactions/MobCompactionRequest.java | 64 - .../hbase/mob/compactions/MobCompactor.java | 90 -- .../PartitionedMobCompactionRequest.java | 333 ----- .../compactions/PartitionedMobCompactor.java | 929 ------------- .../hadoop/hbase/regionserver/HMobStore.java | 42 +- .../hadoop/hbase/regionserver/HStore.java | 2 +- .../hadoop/hbase/regionserver/HStoreFile.java | 3 + .../hbase/regionserver/MobStoreScanner.java | 1 + .../hbase/regionserver/StoreFileInfo.java | 33 +- .../hbase/regionserver/StoreFileWriter.java | 23 + .../regionserver/compactions/Compactor.java | 2 +- .../org/apache/hadoop/hbase/util/FSUtils.java | 2 +- .../hbase/mob/FaultyMobStoreCompactor.java | 356 +++++ .../hadoop/hbase/mob/MobStressTool.java | 79 ++ .../hadoop/hbase/mob/TestMobCompaction.java | 339 +++++ .../hadoop/hbase/mob/TestMobFileName.java | 19 +- .../mob/compactions/TestMobCompactor.java | 1202 ----------------- .../TestPartitionedMobCompactionRequest.java | 65 - .../TestPartitionedMobCompactor.java | 958 ------------- .../regionserver/TestMobStoreCompaction.java | 58 - .../hadoop/hbase/util/BaseTestHBaseFsck.java | 2 +- .../hadoop/hbase/util/TestHBaseFsckMOB.java | 1 + 36 files changed, 2211 insertions(+), 4737 deletions(-) delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobCompactionChore.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactionRequest.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactionRequest.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactionRequest.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java deleted file mode 100644 index d37a80a1c3db..000000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java +++ /dev/null @@ -1,87 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hbase.master; - -import java.util.Map; -import java.util.concurrent.TimeUnit; - -import org.apache.hadoop.hbase.ScheduledChore; -import org.apache.hadoop.hbase.TableDescriptors; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.client.TableDescriptor; -import org.apache.hadoop.hbase.master.locking.LockManager; -import org.apache.hadoop.hbase.mob.ExpiredMobFileCleaner; -import org.apache.hadoop.hbase.mob.MobConstants; -import org.apache.hadoop.hbase.mob.MobUtils; -import org.apache.hadoop.hbase.procedure2.LockType; - -/** - * The Class ExpiredMobFileCleanerChore for running cleaner regularly to remove the expired - * mob files. - */ -@InterfaceAudience.Private -public class ExpiredMobFileCleanerChore extends ScheduledChore { - - private static final Logger LOG = LoggerFactory.getLogger(ExpiredMobFileCleanerChore.class); - private final HMaster master; - private ExpiredMobFileCleaner cleaner; - - public ExpiredMobFileCleanerChore(HMaster master) { - super(master.getServerName() + "-ExpiredMobFileCleanerChore", master, master.getConfiguration() - .getInt(MobConstants.MOB_CLEANER_PERIOD, MobConstants.DEFAULT_MOB_CLEANER_PERIOD), master - .getConfiguration().getInt(MobConstants.MOB_CLEANER_PERIOD, - MobConstants.DEFAULT_MOB_CLEANER_PERIOD), TimeUnit.SECONDS); - this.master = master; - cleaner = new ExpiredMobFileCleaner(); - cleaner.setConf(master.getConfiguration()); - } - - @Override - @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION", - justification="Intentional") - protected void chore() { - try { - TableDescriptors htds = master.getTableDescriptors(); - Map map = htds.getAll(); - for (TableDescriptor htd : map.values()) { - for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) { - if (hcd.isMobEnabled() && hcd.getMinVersions() == 0) { - // clean only for mob-enabled column. - // obtain a read table lock before cleaning, synchronize with MobFileCompactionChore. - final LockManager.MasterLock lock = master.getLockManager().createMasterLock( - MobUtils.getTableLockName(htd.getTableName()), LockType.SHARED, - this.getClass().getSimpleName() + ": Cleaning expired mob files"); - try { - lock.acquire(); - cleaner.cleanExpiredMobFiles(htd.getTableName().getNameAsString(), hcd); - } finally { - lock.release(); - } - } - } - } - } catch (Exception e) { - LOG.error("Fail to clean the expired mob files", e); - } - } - -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 98841f95c838..9a1ae8e93911 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -395,9 +395,8 @@ public void run() { private LogCleaner logCleaner; private HFileCleaner hfileCleaner; private ReplicationBarrierCleaner replicationBarrierCleaner; - private ExpiredMobFileCleanerChore expiredMobFileCleanerChore; - private MobCompactionChore mobCompactChore; - private MasterMobCompactionThread mobCompactThread; + private MobFileCleanerChore mobFileCleanerChore; + private MobFileCompactionChore mobFileCompactionChore; // used to synchronize the mobCompactionStates private final IdLock mobCompactionLock = new IdLock(); // save the information of mob compactions in tables. @@ -1300,14 +1299,18 @@ public void updateConfigurationForQuotasObserver(Configuration conf) { } private void initMobCleaner() { - this.expiredMobFileCleanerChore = new ExpiredMobFileCleanerChore(this); - getChoreService().scheduleChore(expiredMobFileCleanerChore); + this.mobFileCleanerChore = new MobFileCleanerChore(this); + getChoreService().scheduleChore(mobFileCleanerChore); int mobCompactionPeriod = conf.getInt(MobConstants.MOB_COMPACTION_CHORE_PERIOD, - MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD); - this.mobCompactChore = new MobCompactionChore(this, mobCompactionPeriod); - getChoreService().scheduleChore(mobCompactChore); - this.mobCompactThread = new MasterMobCompactionThread(this); + MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD); + + if (mobCompactionPeriod > 0) { + this.mobFileCompactionChore = new MobFileCompactionChore(this); + getChoreService().scheduleChore(mobFileCompactionChore); + } else { + LOG.info("The period is " + mobCompactionPeriod + " seconds, MobCompactionChore is disabled"); + } } /** @@ -1495,9 +1498,7 @@ protected void stopServiceThreads() { } } stopChores(); - if (this.mobCompactThread != null) { - this.mobCompactThread.close(); - } + super.stopServiceThreads(); if (cleanerPool != null) { cleanerPool.shutdownNow(); @@ -1618,8 +1619,8 @@ private void stopProcedureExecutor() { private void stopChores() { ChoreService choreService = getChoreService(); if (choreService != null) { - choreService.cancelChore(this.expiredMobFileCleanerChore); - choreService.cancelChore(this.mobCompactChore); + choreService.cancelChore(this.mobFileCleanerChore); + choreService.cancelChore(this.mobFileCompactionChore); choreService.cancelChore(this.balancerChore); choreService.cancelChore(this.normalizerChore); choreService.cancelChore(this.clusterStatusChore); @@ -3444,17 +3445,6 @@ public void reportMobCompactionEnd(TableName tableName) throws IOException { } } - /** - * Requests mob compaction. - * @param tableName The table the compact. - * @param columns The compacted columns. - * @param allFiles Whether add all mob files into the compaction. - */ - public void requestMobCompaction(TableName tableName, - List columns, boolean allFiles) throws IOException { - mobCompactThread.requestMobCompaction(conf, fs, tableName, columns, allFiles); - } - /** * Queries the state of the {@link LoadBalancerTracker}. If the balancer is not initialized, * false is returned. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java deleted file mode 100644 index 0779eeafe8b0..000000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java +++ /dev/null @@ -1,181 +0,0 @@ -/** - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.master; - -import java.io.IOException; -import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.master.locking.LockManager; -import org.apache.hadoop.hbase.mob.MobUtils; -import org.apache.hadoop.hbase.procedure2.LockType; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; - -/** - * The mob compaction thread used in {@link MasterRpcServices} - */ -@InterfaceAudience.Private -public class MasterMobCompactionThread { - static final Logger LOG = LoggerFactory.getLogger(MasterMobCompactionThread.class); - private final HMaster master; - private final Configuration conf; - private final ExecutorService mobCompactorPool; - private final ExecutorService masterMobPool; - - public MasterMobCompactionThread(HMaster master) { - this.master = master; - this.conf = master.getConfiguration(); - final String n = Thread.currentThread().getName(); - // this pool is used to run the mob compaction - this.masterMobPool = new ThreadPoolExecutor(1, 2, 60, - TimeUnit.SECONDS, new SynchronousQueue<>(), - new ThreadFactoryBuilder().setDaemon(true) - .setNameFormat(n + "-MasterMobCompaction-" + EnvironmentEdgeManager.currentTime()) - .build()); - ((ThreadPoolExecutor) this.masterMobPool).allowCoreThreadTimeOut(true); - // this pool is used in the mob compaction to compact the mob files by partitions - // in parallel - this.mobCompactorPool = MobUtils - .createMobCompactorThreadPool(master.getConfiguration()); - } - - /** - * Requests mob compaction - * @param conf The Configuration - * @param fs The file system - * @param tableName The table the compact - * @param columns The column descriptors - * @param allFiles Whether add all mob files into the compaction. - */ - public void requestMobCompaction(Configuration conf, FileSystem fs, TableName tableName, - List columns, boolean allFiles) throws IOException { - master.reportMobCompactionStart(tableName); - try { - masterMobPool.execute(new CompactionRunner(fs, tableName, columns, - allFiles, mobCompactorPool)); - } catch (RejectedExecutionException e) { - // in case the request is rejected by the pool - try { - master.reportMobCompactionEnd(tableName); - } catch (IOException e1) { - LOG.error("Failed to mark end of mob compaction", e1); - } - throw e; - } - if (LOG.isDebugEnabled()) { - LOG.debug("The mob compaction is requested for the columns " + columns - + " of the table " + tableName.getNameAsString()); - } - } - - private class CompactionRunner implements Runnable { - private FileSystem fs; - private TableName tableName; - private List hcds; - private boolean allFiles; - private ExecutorService pool; - - public CompactionRunner(FileSystem fs, TableName tableName, List hcds, - boolean allFiles, ExecutorService pool) { - super(); - this.fs = fs; - this.tableName = tableName; - this.hcds = hcds; - this.allFiles = allFiles; - this.pool = pool; - } - - @Override - public void run() { - // These locks are on dummy table names, and only used for compaction/mob file cleaning. - final LockManager.MasterLock lock = master.getLockManager().createMasterLock( - MobUtils.getTableLockName(tableName), LockType.EXCLUSIVE, - this.getClass().getName() + ": mob compaction"); - try { - for (ColumnFamilyDescriptor hcd : hcds) { - MobUtils.doMobCompaction(conf, fs, tableName, hcd, pool, allFiles, lock); - } - } catch (IOException e) { - LOG.error("Failed to perform the mob compaction", e); - } finally { - try { - master.reportMobCompactionEnd(tableName); - } catch (IOException e) { - LOG.error("Failed to mark end of mob compaction", e); - } - } - } - } - - /** - * Only interrupt once it's done with a run through the work loop. - */ - private void interruptIfNecessary() { - mobCompactorPool.shutdown(); - masterMobPool.shutdown(); - } - - /** - * Wait for all the threads finish. - */ - private void join() { - waitFor(mobCompactorPool, "Mob Compaction Thread"); - waitFor(masterMobPool, "Region Server Mob Compaction Thread"); - } - - /** - * Closes the MasterMobCompactionThread. - */ - public void close() { - interruptIfNecessary(); - join(); - } - - /** - * Wait for thread finish. - * @param t the thread to wait - * @param name the thread name. - */ - private void waitFor(ExecutorService t, String name) { - boolean done = false; - while (!done) { - try { - done = t.awaitTermination(60, TimeUnit.SECONDS); - LOG.info("Waiting for " + name + " to finish..."); - if (!done) { - t.shutdownNow(); - } - } catch (InterruptedException ie) { - LOG.warn("Interrupted waiting for " + name + " to finish..."); - } - } - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java index 06a99fa5432d..a411e85c47e7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java @@ -32,6 +32,7 @@ import java.util.Map.Entry; import java.util.Set; import java.util.stream.Collectors; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ClusterMetricsBuilder; @@ -45,7 +46,6 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.UnknownRegionException; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.MasterSwitchType; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; @@ -103,23 +103,6 @@ import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil; import org.apache.hadoop.hbase.security.access.UserPermission; import org.apache.hadoop.hbase.security.visibility.VisibilityController; -import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; -import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.ForeignExceptionUtil; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; -import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.zookeeper.KeeperException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; -import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; -import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; - import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter; import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos; @@ -342,6 +325,21 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription; +import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; +import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.ForeignExceptionUtil; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; +import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; +import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.zookeeper.KeeperException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Implements the master RPC services. @@ -1744,10 +1742,13 @@ public CompactRegionResponse compactRegion(final RpcController controller, master.checkInitialized(); byte[] regionName = request.getRegion().getValue().toByteArray(); TableName tableName = RegionInfo.getTable(regionName); + // TODO: support CompactType.MOB // if the region is a mob region, do the mob file compaction. if (MobUtils.isMobRegionName(tableName, regionName)) { checkHFileFormatVersionForMob(); - return compactMob(request, tableName); + //return compactMob(request, tableName); + //TODO: support CompactType.MOB + return super.compactRegion(controller, request); } else { return super.compactRegion(controller, request); } @@ -1790,57 +1791,6 @@ public GetRegionInfoResponse getRegionInfo(final RpcController controller, } } - /** - * Compacts the mob files in the current table. - * @param request the request. - * @param tableName the current table name. - * @return The response of the mob file compaction. - * @throws IOException - */ - private CompactRegionResponse compactMob(final CompactRegionRequest request, - TableName tableName) throws IOException { - if (!master.getTableStateManager().isTableState(tableName, TableState.State.ENABLED)) { - throw new DoNotRetryIOException("Table " + tableName + " is not enabled"); - } - boolean allFiles = false; - List compactedColumns = new ArrayList<>(); - ColumnFamilyDescriptor[] hcds = master.getTableDescriptors().get(tableName).getColumnFamilies(); - byte[] family = null; - if (request.hasFamily()) { - family = request.getFamily().toByteArray(); - for (ColumnFamilyDescriptor hcd : hcds) { - if (Bytes.equals(family, hcd.getName())) { - if (!hcd.isMobEnabled()) { - LOG.error("Column family " + hcd.getNameAsString() + " is not a mob column family"); - throw new DoNotRetryIOException("Column family " + hcd.getNameAsString() - + " is not a mob column family"); - } - compactedColumns.add(hcd); - } - } - } else { - for (ColumnFamilyDescriptor hcd : hcds) { - if (hcd.isMobEnabled()) { - compactedColumns.add(hcd); - } - } - } - if (compactedColumns.isEmpty()) { - LOG.error("No mob column families are assigned in the mob compaction"); - throw new DoNotRetryIOException( - "No mob column families are assigned in the mob compaction"); - } - if (request.hasMajor() && request.getMajor()) { - allFiles = true; - } - String familyLogMsg = (family != null) ? Bytes.toString(family) : ""; - if (LOG.isTraceEnabled()) { - LOG.trace("User-triggered mob compaction requested for table: " - + tableName.getNameAsString() + " for column family: " + familyLogMsg); - } - master.requestMobCompaction(tableName, compactedColumns, allFiles); - return CompactRegionResponse.newBuilder().build(); - } @Override public IsBalancerEnabledResponse isBalancerEnabled(RpcController controller, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobCompactionChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobCompactionChore.java deleted file mode 100644 index 6c5d677a86e5..000000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobCompactionChore.java +++ /dev/null @@ -1,96 +0,0 @@ -/** - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.master; - -import java.util.Map; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.TimeUnit; - -import org.apache.hadoop.hbase.ScheduledChore; -import org.apache.hadoop.hbase.TableDescriptors; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.client.TableDescriptor; -import org.apache.hadoop.hbase.client.TableState; -import org.apache.hadoop.hbase.master.locking.LockManager; -import org.apache.hadoop.hbase.mob.MobUtils; -import org.apache.hadoop.hbase.procedure2.LockType; - -/** - * The Class MobCompactChore for running compaction regularly to merge small mob files. - */ -@InterfaceAudience.Private -public class MobCompactionChore extends ScheduledChore { - - private static final Logger LOG = LoggerFactory.getLogger(MobCompactionChore.class); - private HMaster master; - private ExecutorService pool; - - public MobCompactionChore(HMaster master, int period) { - // use the period as initial delay. - super(master.getServerName() + "-MobCompactionChore", master, period, period, TimeUnit.SECONDS); - this.master = master; - this.pool = MobUtils.createMobCompactorThreadPool(master.getConfiguration()); - } - - @Override - protected void chore() { - try { - TableDescriptors htds = master.getTableDescriptors(); - Map map = htds.getAll(); - for (TableDescriptor htd : map.values()) { - if (!master.getTableStateManager().isTableState(htd.getTableName(), - TableState.State.ENABLED)) { - continue; - } - boolean reported = false; - try { - final LockManager.MasterLock lock = master.getLockManager().createMasterLock( - MobUtils.getTableLockName(htd.getTableName()), LockType.EXCLUSIVE, - this.getClass().getName() + ": mob compaction"); - for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) { - if (!hcd.isMobEnabled()) { - continue; - } - if (!reported) { - master.reportMobCompactionStart(htd.getTableName()); - reported = true; - } - MobUtils.doMobCompaction(master.getConfiguration(), master.getFileSystem(), - htd.getTableName(), hcd, pool, false, lock); - } - } finally { - if (reported) { - master.reportMobCompactionEnd(htd.getTableName()); - } - } - } - } catch (Exception e) { - LOG.error("Failed to compact mob files", e); - } - } - - @Override - protected synchronized void cleanup() { - super.cleanup(); - pool.shutdown(); - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java new file mode 100644 index 000000000000..046c09c546bf --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java @@ -0,0 +1,260 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.master; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.hbase.ScheduledChore; +import org.apache.hadoop.hbase.TableDescriptors; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.HFileArchiver; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.io.hfile.CacheConfig; +import org.apache.hadoop.hbase.master.locking.LockManager; +import org.apache.hadoop.hbase.mob.MobConstants; +import org.apache.hadoop.hbase.mob.ExpiredMobFileCleaner; +import org.apache.hadoop.hbase.mob.MobUtils; +import org.apache.hadoop.hbase.procedure2.LockType; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.hbase.regionserver.HStoreFile; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.util.HFileArchiveUtil; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The Class ExpiredMobFileCleanerChore for running cleaner regularly to remove the expired + * mob files. + */ +@InterfaceAudience.Private +public class MobFileCleanerChore extends ScheduledChore { + + private static final Logger LOG = LoggerFactory.getLogger(MobFileCleanerChore.class); + private final HMaster master; + private ExpiredMobFileCleaner cleaner; + + public MobFileCleanerChore(HMaster master) { + super(master.getServerName() + "-ExpiredMobFileCleanerChore", master, master.getConfiguration() + .getInt(MobConstants.MOB_CLEANER_PERIOD, MobConstants.DEFAULT_MOB_CLEANER_PERIOD), master + .getConfiguration().getInt(MobConstants.MOB_CLEANER_PERIOD, + MobConstants.DEFAULT_MOB_CLEANER_PERIOD), TimeUnit.SECONDS); + this.master = master; + cleaner = new ExpiredMobFileCleaner(); + cleaner.setConf(master.getConfiguration()); + } + + @VisibleForTesting + public MobFileCleanerChore() { + this.master = null; + } + + @Override + @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION", + justification="Intentional") + + protected void chore() { + try { + + TableDescriptors htds = master.getTableDescriptors(); + Map map = htds.getAll(); + for (TableDescriptor htd : map.values()) { + for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) { + if (hcd.isMobEnabled() && hcd.getMinVersions() == 0) { + // clean only for mob-enabled column. + // obtain a read table lock before cleaning, synchronize with MobFileCompactionChore. + final LockManager.MasterLock lock = master.getLockManager().createMasterLock( + MobUtils.getTableLockName(htd.getTableName()), LockType.SHARED, + this.getClass().getSimpleName() + ": Cleaning expired mob files"); + try { + lock.acquire(); + cleaner.cleanExpiredMobFiles(htd.getTableName().getNameAsString(), hcd); + } finally { + lock.release(); + } + } + } + // Now clean obsolete files for a table + LOG.info("Cleaning obsolete MOB files ..."); + cleanupObsoleteMobFiles(master.getConfiguration(), htd.getTableName()); + LOG.info("Cleaning obsolete MOB files finished"); + } + } catch (Exception e) { + LOG.error("Fail to clean the expired mob files", e); + } + } + /** + * Performs housekeeping file cleaning (called by MOB Cleaner chore) + * @param conf configuration + * @param table table name + * @throws IOException + */ + public void cleanupObsoleteMobFiles(Configuration conf, TableName table) + throws IOException { + + try (final Connection conn = ConnectionFactory.createConnection(conf); + final Admin admin = conn.getAdmin();) { + TableDescriptor htd = admin.getDescriptor(table); + List list = MobUtils.getMobColumnFamilies(htd); + if (list.size() == 0) { + LOG.info("Skipping non-MOB table [{}]", table); + return; + } + Path rootDir = FSUtils.getRootDir(conf); + Path tableDir = FSUtils.getTableDir(rootDir, table); + // How safe is this call? + List regionDirs = FSUtils.getRegionDirs(FileSystem.get(conf), tableDir); + + Set allActiveMobFileName = new HashSet(); + FileSystem fs = FileSystem.get(conf); + for (Path regionPath : regionDirs) { + for (ColumnFamilyDescriptor hcd : list) { + String family = hcd.getNameAsString(); + Path storePath = new Path(regionPath, family); + boolean succeed = false; + Set regionMobs = new HashSet(); + while (!succeed) { + // TODO handle FNFE + RemoteIterator rit = fs.listLocatedStatus(storePath); + List storeFiles = new ArrayList(); + // Load list of store files first + while (rit.hasNext()) { + Path p = rit.next().getPath(); + if (fs.isFile(p)) { + storeFiles.add(p); + } + } + try { + for (Path pp : storeFiles) { + HStoreFile sf = + new HStoreFile(fs, pp, conf, CacheConfig.DISABLED, BloomType.NONE, true); + sf.initReader(); + byte[] mobRefData = sf.getMetadataValue(HStoreFile.MOB_FILE_REFS); + byte[] mobCellCountData = sf.getMetadataValue(HStoreFile.MOB_CELLS_COUNT); + byte[] bulkloadMarkerData = sf.getMetadataValue(HStoreFile.BULKLOAD_TASK_KEY); + if (mobRefData == null + && (mobCellCountData != null || bulkloadMarkerData == null)) { + LOG.info("Found old store file with no MOB_FILE_REFS: {} - " + + "can not proceed until all old files will be MOB-compacted.", pp); + return; + } else if (mobRefData == null) { + LOG.info("Skipping file without MOB references (can be bulkloaded file):{}", pp); + continue; + } + String[] mobs = new String(mobRefData).split(","); + regionMobs.addAll(Arrays.asList(mobs)); + } + } catch (FileNotFoundException e) { + // TODO + LOG.warn(e.getMessage()); + continue; + } + succeed = true; + } + // Add MOB refs for current region/family + allActiveMobFileName.addAll(regionMobs); + } // END column families + } // END regions + // Check if number of MOB files too big (over 1M) + if (allActiveMobFileName.size() > 1000000) { + LOG.warn("Found too many active MOB files: {}, this may result in high memory pressure.", + allActiveMobFileName.size()); + } + // Now scan MOB directories and find MOB files with no references to them + long now = System.currentTimeMillis(); + long minAgeToArchive = conf.getLong(MobConstants.MOB_MINIMUM_FILE_AGE_TO_ARCHIVE_KEY, + MobConstants.DEFAULT_MOB_MINIMUM_FILE_AGE_TO_ARCHIVE); + for (ColumnFamilyDescriptor hcd : list) { + List toArchive = new ArrayList(); + String family = hcd.getNameAsString(); + Path dir = MobUtils.getMobFamilyPath(conf, table, family); + RemoteIterator rit = fs.listLocatedStatus(dir); + while (rit.hasNext()) { + LocatedFileStatus lfs = rit.next(); + Path p = lfs.getPath(); + if (!allActiveMobFileName.contains(p.getName())) { + // MOB is not in a list of active references, but it can be too + // fresh, skip it in this case + /* DEBUG */ LOG.debug( + " Age=" + (now - fs.getFileStatus(p).getModificationTime()) + " MOB file=" + p); + if (now - fs.getFileStatus(p).getModificationTime() > minAgeToArchive) { + toArchive.add(p); + } else { + LOG.debug("Skipping fresh file: {}", p); + } + } + } + LOG.info(" MOB Cleaner found {} files for family={}", toArchive.size() , family); + removeMobFiles(conf, table, family.getBytes(), toArchive); + LOG.info(" MOB Cleaner archived {} files", toArchive.size()); + } + } + } + + + /** + * Archives the mob files. + * @param conf The current configuration. + * @param tableName The table name. + * @param family The name of the column family. + * @param storeFiles The files to be archived. + * @throws IOException + */ + public void removeMobFiles(Configuration conf, TableName tableName, byte[] family, + List storeFiles) throws IOException { + + if (storeFiles.size() == 0) { + // nothing to remove + LOG.debug("Skipping archiving old MOB file: collection is empty"); + return; + } + Path mobTableDir = FSUtils.getTableDir(MobUtils.getMobHome(conf), tableName); + FileSystem fs = storeFiles.get(0).getFileSystem(conf); + Path storeArchiveDir = HFileArchiveUtil.getStoreArchivePath(conf, MobUtils.getMobRegionInfo(tableName), + mobTableDir, family); + + for (Path p : storeFiles) { + Path archiveFilePath = new Path(storeArchiveDir, p.getName()); + if (fs.exists(archiveFilePath)) { + LOG.warn("MOB Cleaner skip archiving: {} because it has been archived already", p); + continue; + } + LOG.info("MOB Cleaner is archiving: {}", p); + HFileArchiver.archiveStoreFile(conf, fs, MobUtils.getMobRegionInfo(tableName), mobTableDir, family, p); + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java new file mode 100644 index 000000000000..91b9e8db9281 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java @@ -0,0 +1,201 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.master; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ScheduledChore; +import org.apache.hadoop.hbase.TableDescriptors; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.CompactionState; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableState; +import org.apache.hadoop.hbase.mob.MobConstants; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@InterfaceAudience.Private +public class MobFileCompactionChore extends ScheduledChore { + + private static final Logger LOG = LoggerFactory.getLogger(MobFileCompactionChore.class); + private final Configuration conf; + private final HMaster master; + private int regionBatchSize = 0;// not set - compact all + + public MobFileCompactionChore(HMaster master) { + super(master.getServerName() + "-MobFileCompactionChore", master, master.getConfiguration() + .getInt(MobConstants.MOB_COMPACTION_CHORE_PERIOD, + MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD), master + .getConfiguration().getInt(MobConstants.MOB_COMPACTION_CHORE_PERIOD, + MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD), TimeUnit.SECONDS); + this.master = master; + this.conf = master.getConfiguration(); + this.regionBatchSize = + master.getConfiguration().getInt(MobConstants.MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE, + MobConstants.DEFAULT_MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE); + + } + + @Override + protected void chore() { + + boolean reported = false; + + try (Connection conn = ConnectionFactory.createConnection(conf); + Admin admin = conn.getAdmin(); ) { + + TableDescriptors htds = master.getTableDescriptors(); + Map map = htds.getAll(); + for (TableDescriptor htd : map.values()) { + if (!master.getTableStateManager().isTableState(htd.getTableName(), + TableState.State.ENABLED)) { + LOG.debug("Skipping MOB compaction on table {} because it is not ENABLED", + htd.getTableName()); + continue; + } else { + LOG.debug("Starting MOB compaction on table {}", htd.getTableName()); + } + for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) { + try { + if (hcd.isMobEnabled()) { + if (!reported) { + master.reportMobCompactionStart(htd.getTableName()); + reported = true; + } + LOG.info(" Major compacting {} cf={}", htd.getTableName(), hcd.getNameAsString()); + if (regionBatchSize == MobConstants.DEFAULT_MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE) { + LOG.debug("Batch compaction is disabled, {}=0", "hbase.mob.compaction.batch.size"); + admin.majorCompact(htd.getTableName(), hcd.getName()); + } else { + LOG.debug("Performing compaction in batches, {}={}", + "hbase.mob.compaction.batch.size", regionBatchSize); + performMajorCompactionInBatches(admin, htd, hcd); + } + } else { + LOG.debug("Skipping column family {} because it is not MOB-enabled", + hcd.getNameAsString()); + } + } catch (IOException e) { + LOG.error("Failed to compact table="+ htd.getTableName() +" cf="+ hcd.getNameAsString(), + e); + } catch (InterruptedException ee) { + Thread.currentThread().interrupt(); + master.reportMobCompactionEnd(htd.getTableName()); + LOG.warn("Failed to compact table="+ htd.getTableName() +" cf="+ hcd.getNameAsString(), + ee); + // Quit the chore + return; + } + } + if (reported) { + master.reportMobCompactionEnd(htd.getTableName()); + reported = false; + } + } + } catch (IOException e) { + LOG.error("Failed to compact", e); + } + } + + private void performMajorCompactionInBatches(Admin admin, TableDescriptor htd, + ColumnFamilyDescriptor hcd) throws IOException, InterruptedException { + + List regions = admin.getRegions(htd.getTableName()); + if (regions.size() <= this.regionBatchSize) { + LOG.debug("Performing compaction in non-batched mode, regions={}, batch size={}", + regions.size(), regionBatchSize); + admin.majorCompact(htd.getTableName(), hcd.getName()); + return; + } + // Shuffle list of regions in case if they come ordered by region server + Collections.shuffle(regions); + // Create first batch + List toCompact = new ArrayList(); + for (int i=0; i < this.regionBatchSize; i++) { + toCompact.add(regions.remove(0)); + } + + // Start compaction now + for(RegionInfo ri: toCompact) { + startCompaction(admin, htd.getTableName(), ri, hcd.getName()); + } + + List compacted = new ArrayList(); + while(!toCompact.isEmpty()) { + // Check status of active compactions + for (RegionInfo ri: toCompact) { + try { + if (admin.getCompactionStateForRegion(ri.getRegionName()) == CompactionState.NONE) { + LOG.info("Finished major compaction: table={} region={}", htd.getTableName(), + ri.getRegionNameAsString()); + compacted.add(ri); + } + } catch (IOException e) { + LOG.warn("Could not get compaction state for region {}", ri.getEncodedName()); + } + } + // Update batch: remove compacted regions and add new ones + for (RegionInfo ri: compacted) { + toCompact.remove(ri); + if (regions.size() > 0) { + RegionInfo region = regions.remove(0); + startCompaction(admin, htd.getTableName(),region, hcd.getName()); + toCompact.add(region); + } + } + compacted.clear(); + Thread.sleep(60000); + } + LOG.info("Finished major compacting {}. cf={}", htd.getTableName(), hcd.getNameAsString()); + + } + + private void startCompaction(Admin admin, TableName table, RegionInfo region, byte[] cf) + throws IOException, InterruptedException { + + LOG.info("Started major compaction: table={} region={}", table, + region.getRegionNameAsString()); + admin.majorCompactRegion(region.getRegionName()); + // Wait until it really starts + // but with finite timeout + long waitTime = 300000; // 5 min + long startTime = EnvironmentEdgeManager.currentTime(); + while(admin.getCompactionStateForRegion(region.getRegionName()) == CompactionState.NONE) { + // Is 1 second too aggressive? + Thread.sleep(1000); + if (EnvironmentEdgeManager.currentTime() - startTime > waitTime) { + LOG.warn("Waited for {} ms to start major compaction on table: {} region: {}", waitTime, + table.getNameAsString(), region.getRegionNameAsString()); + break; + } + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java index e73a7d2226d7..b4468836ce89 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java @@ -17,21 +17,36 @@ */ package org.apache.hadoop.hbase.mob; +import static org.apache.hadoop.hbase.regionserver.ScanType.COMPACT_DROP_DELETES; +import static org.apache.hadoop.hbase.regionserver.ScanType.COMPACT_RETAIN_DELETES; + +import java.io.FileNotFoundException; import java.io.IOException; import java.io.InterruptedIOException; import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.Date; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.PrivateCellUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.io.hfile.CacheConfig; +import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.CellSink; import org.apache.hadoop.hbase.regionserver.HMobStore; import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.KeyValueScanner; import org.apache.hadoop.hbase.regionserver.ScanInfo; @@ -48,6 +63,7 @@ import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -59,15 +75,47 @@ public class DefaultMobStoreCompactor extends DefaultCompactor { private static final Logger LOG = LoggerFactory.getLogger(DefaultMobStoreCompactor.class); - private long mobSizeThreshold; - private HMobStore mobStore; + protected long mobSizeThreshold; + protected HMobStore mobStore; + + /* + * MOB file reference set thread local variable. It contains set of + * to a MOB file names, which newly compacted store file has references to. + * This variable is populated during compaction and the content of it is + * written into meta section of a newly created store file at the final step + * of compaction process. + */ + + static ThreadLocal> mobRefSet = new ThreadLocal>() { + @Override + protected Set initialValue() { + return new HashSet(); + } + }; + + /* + * Is it user or system-originated request. + */ + + static ThreadLocal userRequest = new ThreadLocal() { + @Override + protected Boolean initialValue() { + return Boolean.FALSE; + } + }; + + /* + * Contains list of MOB files for compaction if + * generational compaction is enabled. + */ + static ThreadLocal> compSelections = + new ThreadLocal>(); private final InternalScannerFactory scannerFactory = new InternalScannerFactory() { @Override public ScanType getScanType(CompactionRequestImpl request) { - // retain the delete markers until they are expired. - return ScanType.COMPACT_RETAIN_DELETES; + return request.isAllFiles() ? COMPACT_DROP_DELETES : COMPACT_RETAIN_DELETES; } @Override @@ -103,9 +151,34 @@ public DefaultMobStoreCompactor(Configuration conf, HStore store) { mobSizeThreshold = store.getColumnFamilyDescriptor().getMobThreshold(); } + @Override - public List compact(CompactionRequestImpl request, ThroughputController throughputController, - User user) throws IOException { + public List compact(CompactionRequestImpl request, + ThroughputController throughputController, User user) throws IOException { + LOG.info("Mob compaction: major=" + request.isMajor() + " isAll=" + request.isAllFiles() + + " priority=" + request.getPriority()); + if (request.getPriority() == HStore.PRIORITY_USER) { + userRequest.set(Boolean.TRUE); + } else { + userRequest.set(Boolean.FALSE); + } + LOG.info("Mob compaction files: " + request.getFiles()); + // Check if generational MOB compaction + compSelections.set(null); + if (conf.get(MobConstants.MOB_COMPACTION_TYPE_KEY, MobConstants.DEFAULT_MOB_COMPACTION_TYPE) + .equals(MobConstants.GENERATIONAL_MOB_COMPACTION_TYPE)) { + if (request.isMajor() && request.getPriority() == HStore.PRIORITY_USER) { + // Compact MOBs + List mobFiles = getReferencedMobFiles(request.getFiles()); + if (mobFiles.size() > 0) { + Generations gens = Generations.build(mobFiles, conf); + List list = gens.getCompactionSelections(); + if (list.size() > 0) { + compSelections.set(list); + } + } + } + } return compact(request, scannerFactory, writerFactory, throughputController, user); } @@ -135,23 +208,6 @@ public List compact(CompactionRequestImpl request, ThroughputController th * Otherwise, directly write this cell into the store file. * * - * 3. Decide how to write a Delete cell. - *
    - *
  1. - * If a Delete cell does not have a mob reference tag which means this delete marker have not - * been written to the mob del file, write this cell to the mob del file, and write this cell - * with a ref tag to a store file. - *
  2. - *
  3. - * Otherwise, directly write it to a store file. - *
  4. - *
- * After the major compaction on the normal hfiles, we have a guarantee that we have purged all - * deleted or old version mob refs, and the delete markers are written to a del file with the - * suffix _del. Because of this, it is safe to use the del file in the mob compaction. - * The mob compaction doesn't take place in the normal hfiles, it occurs directly in the - * mob files. When the small mob files are merged into bigger ones, the del file is added into - * the scanner to filter the deleted cells. * @param fd File details * @param scanner Where to read from. * @param writer Where to write to. @@ -169,6 +225,40 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel long bytesWrittenProgressForCloseCheck = 0; long bytesWrittenProgressForLog = 0; long bytesWrittenProgressForShippedCall = 0; + // Clear old mob references + mobRefSet.get().clear(); + boolean isUserRequest = userRequest.get(); + boolean compactMOBs = major && isUserRequest; + boolean generationalMob = conf.get(MobConstants.MOB_COMPACTION_TYPE_KEY, + MobConstants.DEFAULT_MOB_COMPACTION_TYPE) + .equals(MobConstants.GENERATIONAL_MOB_COMPACTION_TYPE); + if (generationalMob && compSelections.get() == null) { + LOG.warn("MOB compaction aborted, reason: generational compaction is enabled, "+ + "but compaction selection was empty."); + return true; + } + OutputMobWriters mobWriters = null; + + if (compactMOBs && generationalMob) { + List sel = compSelections.get(); + if (sel != null && sel.size() > 0) { + // Create output writers for compaction selections + mobWriters = new OutputMobWriters(sel); + int numWriters = mobWriters.getNumberOfWriters(); + List writers = new ArrayList(); + for (int i=0; i < numWriters; i++) { + StoreFileWriter sfw = mobStore.createWriterInTmp(new Date(fd.latestPutTs), fd.maxKeyCount, + compactionCompression, store.getRegionInfo().getStartKey(), true); + writers.add(sfw); + } + mobWriters.initOutputWriters(writers); + } + } + + boolean discardMobMiss = + conf.getBoolean(MobConstants.MOB_DISCARD_MISS_KEY, MobConstants.DEFAULT_MOB_DISCARD_MISS); + FileSystem fs = FileSystem.get(conf); + // Since scanner.next() can return 'false' but still be delivering data, // we have to use a do/while loop. List cells = new ArrayList<>(); @@ -183,36 +273,35 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel boolean hasMore; Path path = MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName()); byte[] fileName = null; - StoreFileWriter mobFileWriter = null, delFileWriter = null; - long mobCells = 0, deleteMarkersCount = 0; + StoreFileWriter mobFileWriter = null; + long mobCells = 0; long cellsCountCompactedToMob = 0, cellsCountCompactedFromMob = 0; long cellsSizeCompactedToMob = 0, cellsSizeCompactedFromMob = 0; boolean finished = false; + ScannerContext scannerContext = ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build(); throughputController.start(compactionName); - KeyValueScanner kvs = (scanner instanceof KeyValueScanner)? (KeyValueScanner)scanner : null; - long shippedCallSizeLimit = (long) numofFilesToCompact * this.store.getColumnFamilyDescriptor().getBlocksize(); + KeyValueScanner kvs = (scanner instanceof KeyValueScanner) ? (KeyValueScanner) scanner : null; + long shippedCallSizeLimit = + (long) numofFilesToCompact * this.store.getColumnFamilyDescriptor().getBlocksize(); + + Cell mobCell = null; try { try { - // If the mob file writer could not be created, directly write the cell to the store file. mobFileWriter = mobStore.createWriterInTmp(new Date(fd.latestPutTs), fd.maxKeyCount, compactionCompression, store.getRegionInfo().getStartKey(), true); fileName = Bytes.toBytes(mobFileWriter.getPath().getName()); } catch (IOException e) { - LOG.warn("Failed to create mob writer, " - + "we will continue the compaction by writing MOB cells directly in store files", e); + // Bailing out + LOG.error("Failed to create mob writer, ", e); + throw e; } - if (major) { - try { - delFileWriter = mobStore.createDelFileWriterInTmp(new Date(fd.latestPutTs), - fd.maxKeyCount, compactionCompression, store.getRegionInfo().getStartKey()); - } catch (IOException e) { - LOG.warn( - "Failed to create del writer, " - + "we will continue the compaction by writing delete markers directly in store files", - e); - } + if (compactMOBs) { + // Add the only reference we get for compact MOB case + // because new store file will have only one MOB reference + // in this case - of newly compacted MOB file + mobRefSet.get().add(mobFileWriter.getPath().getName()); } do { hasMore = scanner.next(cells, scannerContext); @@ -220,54 +309,111 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel now = EnvironmentEdgeManager.currentTime(); } for (Cell c : cells) { - if (major && CellUtil.isDelete(c)) { - if (MobUtils.isMobReferenceCell(c) || delFileWriter == null) { - // Directly write it to a store file - writer.append(c); + + if (compactMOBs) { + if (MobUtils.isMobReferenceCell(c)) { + String fName = MobUtils.getMobFileName(c); + Path pp = new Path(new Path(fs.getUri()), new Path(path, fName)); + + // Added to support migration + try { + mobCell = mobStore.resolve(c, true, false).getCell(); + } catch (FileNotFoundException fnfe) { + if (discardMobMiss) { + LOG.error("Missing MOB cell: file=" + pp + " not found"); + continue; + } else { + throw fnfe; + } + } + + if (discardMobMiss && mobCell.getValueLength() == 0) { + LOG.error("Missing MOB cell value: file=" + pp + " cell=" + mobCell); + continue; + } + + if (mobCell.getValueLength() > mobSizeThreshold) { + // put the mob data back to the store file + PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId()); + if (generationalMob) { + //TODO: verify fName + StoreFileWriter stw = mobWriters.getOutputWriterForInputFile(fName); + if (stw != null) { + stw.append(mobCell); + mobWriters.incrementMobCountForOutputWriter(stw, 1); + } // else leave mob cell in a MOB file which is not in compaction selections + } else { + mobFileWriter.append(mobCell); + mobCells++; + } + writer.append(MobUtils.createMobRefCell(mobCell, fileName, + this.mobStore.getRefCellTags())); + cellsCountCompactedFromMob++; + cellsSizeCompactedFromMob += mobCell.getValueLength(); + } else { + + // If MOB value is less than threshold, append it directly to a store file + PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId()); + writer.append(mobCell); + } + } else { - // Add a ref tag to this cell and write it to a store file. - writer.append(MobUtils.createMobRefDeleteMarker(c)); - // Write the cell to a del file - delFileWriter.append(c); - deleteMarkersCount++; + // Not a MOB reference cell + int size = c.getValueLength(); + if (size > mobSizeThreshold) { + // This MOB cell comes from a regular store file + // therefore we store it in original mob output + mobFileWriter.append(c); + writer + .append(MobUtils.createMobRefCell(c, fileName, this.mobStore.getRefCellTags())); + mobCells++; + } else { + writer.append(c); + } } - } else if (mobFileWriter == null || c.getTypeByte() != KeyValue.Type.Put.getCode()) { - // If the mob file writer is null or the kv type is not put, directly write the cell + } else if (c.getTypeByte() != KeyValue.Type.Put.getCode()) { + // Not a major compaction or major with MOB disabled + // If the kv type is not put, directly write the cell // to the store file. writer.append(c); } else if (MobUtils.isMobReferenceCell(c)) { + // Not a major MOB compaction, Put MOB reference if (MobUtils.hasValidMobRefCellValue(c)) { int size = MobUtils.getMobValueLength(c); if (size > mobSizeThreshold) { // If the value size is larger than the threshold, it's regarded as a mob. Since // its value is already in the mob file, directly write this cell to the store file writer.append(c); + // Add MOB reference to a set + mobRefSet.get().add(MobUtils.getMobFileName(c)); } else { // If the value is not larger than the threshold, it's not regarded a mob. Retrieve - // the mob cell from the mob file, and write it back to the store file. Must - // close the mob scanner once the life cycle finished. - try (MobCell mobCell = mobStore.resolve(c, false)) { - if (mobCell.getCell().getValueLength() != 0) { - // put the mob data back to the store file - PrivateCellUtil.setSequenceId(mobCell.getCell(), c.getSequenceId()); - writer.append(mobCell.getCell()); - cellsCountCompactedFromMob++; - cellsSizeCompactedFromMob += mobCell.getCell().getValueLength(); - } else { - // If the value of a file is empty, there might be issues when retrieving, - // directly write the cell to the store file, and leave it to be handled by the - // next compaction. - writer.append(c); - } + // the mob cell from the mob file, and write it back to the store file. + mobCell = mobStore.resolve(c, true, false).getCell(); + if (mobCell.getValueLength() != 0) { + // put the mob data back to the store file + PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId()); + writer.append(mobCell); + cellsCountCompactedFromMob++; + cellsSizeCompactedFromMob += mobCell.getValueLength(); + } else { + // If the value of a file is empty, there might be issues when retrieving, + // directly write the cell to the store file, and leave it to be handled by the + // next compaction. + LOG.error("Empty value for: " + c); + writer.append(c); + // Add MOB reference to a set + mobRefSet.get().add(MobUtils.getMobFileName(c)); } } } else { - LOG.warn("The value format of the KeyValue " + c - + " is wrong, its length is less than " + Bytes.SIZEOF_INT); + // TODO ???? + LOG.error("Corrupted MOB reference: " + c); writer.append(c); } } else if (c.getValueLength() <= mobSizeThreshold) { - //If value size of a cell is not larger than the threshold, directly write to store file + // If the value size of a cell is not larger than the threshold, directly write it to + // the store file. writer.append(c); } else { // If the value size of a cell is larger than the threshold, it's regarded as a mob, @@ -275,13 +421,15 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel mobCells++; // append the original keyValue in the mob file. mobFileWriter.append(c); - Cell reference = MobUtils.createMobRefCell(c, fileName, - this.mobStore.getRefCellTags()); + Cell reference = MobUtils.createMobRefCell(c, fileName, this.mobStore.getRefCellTags()); // write the cell whose value is the path of a mob file to the store file. writer.append(reference); cellsCountCompactedToMob++; cellsSizeCompactedToMob += c.getValueLength(); + // Add ref we get for compact MOB case + mobRefSet.get().add(mobFileWriter.getPath().getName()); } + int len = c.getSerializedSize(); ++progress.currentCompactedKVs; progress.totalCompactedSize += len; @@ -302,7 +450,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel } } if (kvs != null && bytesWrittenProgressForShippedCall > shippedCallSizeLimit) { - ((ShipperListener)writer).beforeShipped(); + ((ShipperListener) writer).beforeShipped(); kvs.shipped(); bytesWrittenProgressForShippedCall = 0; } @@ -326,6 +474,9 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel progress.cancel(); throw new InterruptedIOException( "Interrupted while control throughput of compacting " + compactionName); + } catch (IOException t) { + LOG.error("Mob compaction failed for region: " + store.getRegionInfo().getEncodedName()); + throw t; } finally { // Clone last cell in the final because writer will append last cell when committing. If // don't clone here and once the scanner get closed, then the memory of last cell will be @@ -333,24 +484,19 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel ((ShipperListener) writer).beforeShipped(); throughputController.finish(compactionName); if (!finished && mobFileWriter != null) { + // Remove all MOB references because compaction failed + mobRefSet.get().clear(); + // Abort writer abortWriter(mobFileWriter); - } - if (!finished && delFileWriter != null) { - abortWriter(delFileWriter); - } - } - if (delFileWriter != null) { - if (deleteMarkersCount > 0) { - // If the del file is not empty, commit it. - // If the commit fails, the compaction is re-performed again. - delFileWriter.appendMetadata(fd.maxSeqId, major, deleteMarkersCount); - delFileWriter.close(); - mobStore.commitFile(delFileWriter.getPath(), path); - } else { - // If the del file is empty, delete it instead of committing. - abortWriter(delFileWriter); + //Check if other writers exist + if (mobWriters != null) { + for(StoreFileWriter w: mobWriters.getOutputWriters()) { + abortWriter(w); + } + } } } + // Commit or abort major mob writer if (mobFileWriter != null) { if (mobCells > 0) { // If the mob file is not empty, commit it. @@ -362,6 +508,20 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel abortWriter(mobFileWriter); } } + // Commit or abort generational writers + if (mobWriters != null) { + for (StoreFileWriter w: mobWriters.getOutputWriters()) { + Long mobs = mobWriters.getMobCountForOutputWriter(w); + if (mobs != null && mobs > 0) { + mobRefSet.get().add(w.getPath().getName()); + w.appendMetadata(fd.maxSeqId, major, mobs); + w.close(); + mobStore.commitFile(w.getPath(), path); + } else { + abortWriter(w); + } + } + } mobStore.updateCellsCountCompactedFromMob(cellsCountCompactedFromMob); mobStore.updateCellsCountCompactedToMob(cellsCountCompactedToMob); mobStore.updateCellsSizeCompactedFromMob(cellsSizeCompactedFromMob); @@ -369,4 +529,356 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel progress.complete(); return true; } + + protected static String createKey(TableName tableName, String encodedName, + String columnFamilyName) { + return tableName.getNameAsString()+ "_" + encodedName + "_"+ columnFamilyName; + } + + @Override + protected List commitWriter(StoreFileWriter writer, FileDetails fd, + CompactionRequestImpl request) throws IOException { + List newFiles = Lists.newArrayList(writer.getPath()); + writer.appendMetadata(fd.maxSeqId, request.isAllFiles(), request.getFiles()); + // Append MOB references + Set refSet = mobRefSet.get(); + writer.appendMobMetadata(refSet); + writer.close(); + return newFiles; + } + + private List getReferencedMobFiles(Collection storeFiles) { + Path mobDir = MobUtils.getMobFamilyPath(conf, store.getTableName(), + store.getColumnFamilyName()); + Set mobSet = new HashSet(); + for (HStoreFile sf: storeFiles) { + byte[] value = sf.getMetadataValue(HStoreFile.MOB_FILE_REFS); + if (value != null) { + String s = new String(value); + String[] all = s.split(","); + Collections.addAll(mobSet, all); + } + } + List retList = new ArrayList(); + for(String name: mobSet) { + retList.add(new Path(mobDir, name)); + } + return retList; + } +} + +final class FileSelection implements Comparable { + + public final static String NULL_REGION = ""; + private Path path; + private long earliestTs; + private Configuration conf; + + public FileSelection(Path path, Configuration conf) throws IOException { + this.path = path; + this.conf = conf; + readEarliestTimestamp(); + } + + public String getEncodedRegionName() { + String fileName = path.getName(); + String[] parts = fileName.split("_"); + if (parts.length == 2) { + return parts[1]; + } else { + return NULL_REGION; + } + } + + public Path getPath() { + return path; + } + + public long getEarliestTimestamp() { + return earliestTs; + } + + private void readEarliestTimestamp() throws IOException { + FileSystem fs = path.getFileSystem(conf); + HStoreFile sf = new HStoreFile(fs, path, conf, CacheConfig.DISABLED, + BloomType.NONE, true); + sf.initReader(); + byte[] tsData = sf.getMetadataValue(HStoreFile.EARLIEST_PUT_TS); + if (tsData != null) { + this.earliestTs = Bytes.toLong(tsData); + } + sf.closeStoreFile(true); + } + + @Override + public int compareTo(FileSelection o) { + if (this.earliestTs > o.earliestTs) { + return +1; + } else if (this.earliestTs == o.earliestTs) { + return 0; + } else { + return -1; + } + } + +} + +final class Generations { + + private List generations; + private Configuration conf; + + private Generations(List gens, Configuration conf) { + this.generations = gens; + this.conf = conf; + } + + List getCompactionSelections() throws IOException { + int maxTotalFiles = this.conf.getInt(MobConstants.MOB_COMPACTION_MAX_TOTAL_FILES_KEY, + MobConstants.DEFAULT_MOB_COMPACTION_MAX_TOTAL_FILES); + int currentTotal = 0; + List list = new ArrayList(); + + for (Generation g: generations) { + List sel = g.getCompactionSelections(conf); + int size = getSize(sel); + if ((currentTotal + size > maxTotalFiles) && currentTotal > 0) { + break; + } else { + currentTotal += size; + list.addAll(sel); + } + } + return list; + } + + private int getSize(List sel) { + int size = 0; + for(CompactionSelection cs: sel) { + size += cs.size(); + } + return size; + } + + static Generations build(List files, Configuration conf) throws IOException { + Map > map = new HashMap>(); + for(Path p: files) { + String key = getRegionNameFromFileName(p.getName()); + ArrayList list = map.get(key); + if (list == null) { + list = new ArrayList(); + map.put(key, list); + } + list.add(new FileSelection(p, conf)); + } + + List gens = new ArrayList(); + for (Map.Entry> entry: map.entrySet()) { + String key = entry.getKey(); + Generation g = new Generation(key); + List selFiles = map.get(key); + for(FileSelection fs: selFiles) { + g.addFile(fs); + } + gens.add(g); + } + // Sort all generation files one-by-one + for(Generation gg: gens) { + gg.sortFiles(); + } + // Sort generations + Collections.sort(gens); + return new Generations(gens, conf); + } + + static String getRegionNameFromFileName(String name) { + int index = name.lastIndexOf("_"); + if (index < 0) { + return Generation.GEN0; + } + return name.substring(index+1); + } +} + +final class Generation implements Comparable { + + static final String GEN0 ="GEN0"; + private String regionName; + private long earliestTs = Long.MAX_VALUE; + private List files = new ArrayList<>(); + List compSelections; + + public Generation(String name) { + this.regionName = name; + } + + @SuppressWarnings("deprecation") + public List getCompactionSelections(Configuration conf) throws IOException { + + + int minFiles = conf.getInt(MobConstants.MOB_COMPACTION_MIN_FILES_KEY, + MobConstants.DEFAULT_MOB_COMPACTION_MIN_FILES); + int maxFiles = conf.getInt(MobConstants.MOB_COMPACTION_MAX_FILES_KEY, + MobConstants.DEFAULT_MOB_COMPACTION_MAX_FILES); + long maxSelectionSize = conf.getLong(MobConstants.MOB_COMPACTION_MAX_SELECTION_SIZE_KEY, + MobConstants.DEFAULT_MOB_COMPACTION_MAX_SELECTION_SIZE); + // Now it is ordered from oldest to newest ones + List rfiles = Lists.reverse(files); + List retList = new ArrayList(); + FileSystem fs = rfiles.get(0).getPath().getFileSystem(conf); + int off = 0; + while (off < rfiles.size()) { + if (fs.getLength(rfiles.get(off).getPath()) >= maxSelectionSize) { + off++; continue; + } + long selSize = 0; + int limit = Math.min(off + maxFiles, rfiles.size()); + int start = off; + List sel = new ArrayList(); + for (; off < limit; off++) { + Path p = rfiles.get(off).getPath(); + long fSize = fs.getLength(p); + if (selSize + fSize < maxSelectionSize) { + selSize+= fSize; + sel.add(new FileSelection(p, conf)); + } else { + if (sel.size() < minFiles) { + // discard + sel.clear(); + // advance by 1 + off = start +1; + } else { + // we have new selection + CompactionSelection cs = new CompactionSelection(sel); + retList.add(cs); + off++; + } + break; // continue outer loop + } + } + } + return retList; + } + + public boolean addFile(FileSelection f) { + if (f.getEncodedRegionName().equals(regionName)) { + files.add(f); + if (f.getEarliestTimestamp() < earliestTs) { + earliestTs = f.getEarliestTimestamp(); + } + return true; + } else { + return false; + } + } + + public void sortFiles() { + Collections.sort(files); + } + + public List getFiles() { + return files; + } + + public String getEncodedRegionName() { + return regionName; + } + + public long getEarliestTimestamp() { + return earliestTs; + } + + @Override + public int compareTo(Generation o) { + if (this.earliestTs > o.earliestTs) { + return +1; + } else if (this.earliestTs == o.earliestTs) { + return 0; + } else { + return -1; + } + } +} + +final class CompactionSelection { + private static AtomicLong idGen = new AtomicLong(); + private List files; + private long id; + + public CompactionSelection(List files) { + this.files = files; + this.id = idGen.getAndIncrement(); + } + + public List getFiles() { + return files; + } + + public long getId() { + return id; + } + + int size() { + return files.size(); + } +} + +final class OutputMobWriters { + + /* + * Input MOB file name -> output file writer + */ + private Map writerMap = new HashMap(); + /* + * Output file name -> MOB counter + */ + private Map mapMobCounts = new HashMap(); + /* + * List of compaction selections + */ + private List compSelections; + + public OutputMobWriters(List compSelections) { + this.compSelections = compSelections; + } + + int getNumberOfWriters() { + return compSelections.size(); + } + + StoreFileWriter getWriterForFile(String fileName) { + return writerMap.get(fileName); + } + + void initOutputWriters(List writers) { + for (int i = 0; i < writers.size(); i++) { + StoreFileWriter sw = writers.get(i); + mapMobCounts.put(sw.getPath().getName(), 0L); + CompactionSelection cs = compSelections.get(i); + for (FileSelection fs: cs.getFiles()) { + writerMap.put(fs.getPath().getName(), sw); + } + } + } + + Collection getOutputWriters() { + return writerMap.values(); + } + + StoreFileWriter getOutputWriterForInputFile(String name) { + return writerMap.get(name); + } + + long getMobCountForOutputWriter(StoreFileWriter writer) { + return mapMobCounts.get(writer.getPath().getName()); + } + + void incrementMobCountForOutputWriter(StoreFileWriter writer, int val) { + String key = writer.getPath().getName(); + Long count = mapMobCounts.get(key); + if (count == null) { + count = 0L; + } + mapMobCounts.put(key, count + val); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java index 3de7992cb121..c8a289db23a2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java @@ -22,7 +22,9 @@ import java.io.InterruptedIOException; import java.util.ArrayList; import java.util.Date; +import java.util.HashSet; import java.util.List; +import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -69,6 +71,13 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher { private long mobCellValueSizeThreshold = 0; private Path targetPath; private HMobStore mobStore; + // MOB file reference set + static ThreadLocal> mobRefSet = new ThreadLocal>() { + @Override + protected Set initialValue() { + return new HashSet(); + } + }; public DefaultMobStoreFlusher(Configuration conf, HStore store) throws IOException { super(conf, store); @@ -188,6 +197,8 @@ protected void performMobFlush(MemStoreSnapshot snapshot, long cacheFlushId, throughputController.start(flushName); } IOException ioe = null; + // Clear all past MOB references + mobRefSet.get().clear(); try { do { hasMore = scanner.next(cells, scannerContext); @@ -244,9 +255,12 @@ protected void performMobFlush(MemStoreSnapshot snapshot, long cacheFlushId, status.setStatus("Flushing mob file " + store + ": closing flushed file"); mobFileWriter.close(); mobStore.commitFile(mobFileWriter.getPath(), targetPath); + LOG.debug("Flush store file: {}", writer.getPath()); mobStore.updateMobFlushCount(); mobStore.updateMobFlushedCellsCount(mobCount); mobStore.updateMobFlushedCellsSize(mobSize); + // Add mob reference to store file metadata + mobRefSet.get().add(mobFileWriter.getPath().getName()); } else { try { status.setStatus("Flushing mob file " + store + ": no mob cells, closing flushed file"); @@ -258,4 +272,16 @@ protected void performMobFlush(MemStoreSnapshot snapshot, long cacheFlushId, } } } + + protected void finalizeWriter(StoreFileWriter writer, long cacheFlushSeqNum, + MonitoredTask status) throws IOException { + // Write out the log sequence number that corresponds to this output + // hfile. Also write current time in metadata as minFlushTime. + // The hfile is current up to and including cacheFlushSeqNum. + status.setStatus("Flushing " + store + ": appending metadata"); + writer.appendMetadata(cacheFlushSeqNum, false); + writer.appendMobMetadata(mobRefSet.get()); + status.setStatus("Flushing " + store + ": closing flushed file"); + writer.close(); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java index 42e78bbd007d..f82cf1a4cb14 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java @@ -57,7 +57,8 @@ public class ExpiredMobFileCleaner extends Configured implements Tool { * @param tableName The current table name. * @param family The current family. */ - public void cleanExpiredMobFiles(String tableName, ColumnFamilyDescriptor family) throws IOException { + public void cleanExpiredMobFiles(String tableName, ColumnFamilyDescriptor family) + throws IOException { Configuration conf = getConf(); TableName tn = TableName.valueOf(tableName); FileSystem fs = FileSystem.get(conf); @@ -67,7 +68,7 @@ public void cleanExpiredMobFiles(String tableName, ColumnFamilyDescriptor family copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f); CacheConfig cacheConfig = new CacheConfig(copyOfConf); MobUtils.cleanExpiredMobFiles(fs, conf, tn, family, cacheConfig, - EnvironmentEdgeManager.currentTime()); + EnvironmentEdgeManager.currentTime()); } public static void main(String[] args) throws Exception { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java index 4afd75b9c680..7e4daf3b9a0c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java @@ -43,7 +43,7 @@ public final class MobConstants { public static final String MOB_REGION_NAME = ".mob"; public static final byte[] MOB_REGION_NAME_BYTES = Bytes.toBytes(MOB_REGION_NAME); - public static final String MOB_CLEANER_PERIOD = "hbase.master.mob.ttl.cleaner.period"; + public static final String MOB_CLEANER_PERIOD = "hbase.master.mob.cleaner.period"; public static final int DEFAULT_MOB_CLEANER_PERIOD = 24 * 60 * 60; // one day public static final String MOB_CACHE_EVICT_PERIOD = "hbase.mob.cache.evict.period"; @@ -55,33 +55,23 @@ public final class MobConstants { public static final long DEFAULT_MOB_CACHE_EVICT_PERIOD = 3600L; public final static String TEMP_DIR_NAME = ".tmp"; - public final static String BULKLOAD_DIR_NAME = ".bulkload"; public final static byte[] MOB_TABLE_LOCK_SUFFIX = Bytes.toBytes(".mobLock"); - public final static String EMPTY_STRING = ""; - /** - * If the size of a mob file is less than this value, it's regarded as a small file and needs to - * be merged in mob compaction. The default value is 1280MB. - */ - public static final String MOB_COMPACTION_MERGEABLE_THRESHOLD = - "hbase.mob.compaction.mergeable.threshold"; - public static final long DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD = 10 * 128 * 1024 * 1024; + /** - * The max number of del files that is allowed in the mob file compaction. In the mob - * compaction, when the number of existing del files is larger than this value, they are merged - * until number of del files is not larger this value. The default value is 3. + * The max number of a MOB table regions that is allowed in a batch of the mob compaction. + * By setting this number to a custom value, users can control the overall effect + * of a major compaction of a large MOB-enabled table. */ - public static final String MOB_DELFILE_MAX_COUNT = "hbase.mob.delfile.max.count"; - public static final int DEFAULT_MOB_DELFILE_MAX_COUNT = 3; + + public static final String MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE = + "hbase.mob.compaction.batch.size"; + /** - * The max number of the mob files that is allowed in a batch of the mob compaction. - * The mob compaction merges the small mob files to bigger ones. If the number of the - * small files is very large, it could lead to a "too many opened file handlers" in the merge. - * And the merge has to be split into batches. This value limits the number of mob files - * that are selected in a batch of the mob compaction. The default value is 100. + * Default is 0 - means no limit - all regions of a MOB table will be compacted at once */ - public static final String MOB_COMPACTION_BATCH_SIZE = - "hbase.mob.compaction.batch.size"; - public static final int DEFAULT_MOB_COMPACTION_BATCH_SIZE = 100; + + public static final int DEFAULT_MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE = 0; + /** * The period that MobCompactionChore runs. The unit is second. * The default value is one week. @@ -91,12 +81,68 @@ public final class MobConstants { public static final int DEFAULT_MOB_COMPACTION_CHORE_PERIOD = 24 * 60 * 60 * 7; // a week public static final String MOB_COMPACTOR_CLASS_KEY = "hbase.mob.compactor.class"; + /** - * The max number of threads used in MobCompactor. + * Mob compaction type: "full", "generational" + * "full" - run full major compaction (during migration) + * "generational" - optimized version */ - public static final String MOB_COMPACTION_THREADS_MAX = - "hbase.mob.compaction.threads.max"; - public static final int DEFAULT_MOB_COMPACTION_THREADS_MAX = 1; + public final static String MOB_COMPACTION_TYPE_KEY = "hbase.mob.compaction.type"; + + public final static String DEFAULT_MOB_COMPACTION_TYPE = "full"; + + public final static String GENERATIONAL_MOB_COMPACTION_TYPE = "generational"; + + public final static String FULL_MOB_COMPACTION_TYPE = "full"; + + + /** + * Maximum size of a MOB compaction selection + */ + public static final String MOB_COMPACTION_MAX_SELECTION_SIZE_KEY = + "hbase.mob.compactions.max.selection.size"; + /** + * Default maximum selection size = 1GB + */ + public static final long DEFAULT_MOB_COMPACTION_MAX_SELECTION_SIZE = 1024 * 1024 * 1024; + + + /** + * Minimum number of MOB files eligible for compaction + */ + public static final String MOB_COMPACTION_MIN_FILES_KEY = "hbase.mob.compactions.min.files"; + + public static final int DEFAULT_MOB_COMPACTION_MIN_FILES = 3; + + /** + * Maximum number of MOB files (in one selection) eligible for compaction + */ + + public static final String MOB_COMPACTION_MAX_FILES_KEY = "hbase.mob.compactions.max.files"; + + public static final int DEFAULT_MOB_COMPACTION_MAX_FILES = 100; + + /** + * Maximum number of MOB files allowed in MOB compaction (per region) + */ + + public static final String MOB_COMPACTION_MAX_TOTAL_FILES_KEY = + "hbase.mob.compactions.max.total.files"; + + public static final int DEFAULT_MOB_COMPACTION_MAX_TOTAL_FILES = 1000; + + public static final String MOB_DISCARD_MISS_KEY = "hbase.mob.discard.miss"; + + public static final boolean DEFAULT_MOB_DISCARD_MISS = false; + + /** + * Minimum age required for MOB file to be archived + */ + public static final String MOB_MINIMUM_FILE_AGE_TO_ARCHIVE_KEY = + "mob.minimum.file.age.to.archive"; + + public static final long DEFAULT_MOB_MINIMUM_FILE_AGE_TO_ARCHIVE = 3600000; // 1 hour + private MobConstants() { } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileName.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileName.java index 3a29274285b6..64cfa4dfcafa 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileName.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileName.java @@ -23,8 +23,8 @@ /** * The mob file name. - * It consists of a md5 of a start key, a date and an uuid. - * It looks like md5(start) + date + uuid. + * It consists of a md5 of a start key, a date, uuid and encoded region name. + * It looks like md5(start) + date + uuid+ "_" + encoded region name. *
    *
  1. characters 0-31: md5 hex string of a start key. Since the length of the start key is not * fixed, have to use the md5 instead which has a fix length.
  2. @@ -45,10 +45,13 @@ public final class MobFileName { private final String startKey; private final String uuid; private final String fileName; + // Name of a region this MOB file belongs to + private final String regionName; private static final int STARTKEY_END_INDEX = 32; private static final int DATE_END_INDEX = 40; private static final int UUID_END_INDEX = 72; + public static final String REGION_SEP = "_"; /** * @param startKey @@ -57,12 +60,14 @@ public final class MobFileName { * The string of the latest timestamp of cells in this file, the format is yyyymmdd. * @param uuid * The uuid + * @param regionName name of a region, where this file was created during flush or compaction. */ - private MobFileName(byte[] startKey, String date, String uuid) { + private MobFileName(byte[] startKey, String date, String uuid, String regionName) { this.startKey = MD5Hash.getMD5AsHex(startKey, 0, startKey.length); this.uuid = uuid; this.date = date; - this.fileName = this.startKey + this.date + this.uuid; + this.regionName = regionName; + this.fileName = this.startKey + this.date + this.uuid + REGION_SEP + this.regionName; } /** @@ -72,12 +77,14 @@ private MobFileName(byte[] startKey, String date, String uuid) { * The string of the latest timestamp of cells in this file, the format is yyyymmdd. * @param uuid * The uuid + * @param regionName name of a region, where this file was created during flush or compaction. */ - private MobFileName(String startKey, String date, String uuid) { + private MobFileName(String startKey, String date, String uuid, String regionName) { this.startKey = startKey; this.uuid = uuid; this.date = date; - this.fileName = this.startKey + this.date + this.uuid; + this.regionName = regionName; + this.fileName = this.startKey + this.date + this.uuid + REGION_SEP + this.regionName; } /** @@ -88,10 +95,11 @@ private MobFileName(String startKey, String date, String uuid) { * @param date * The string of the latest timestamp of cells in this file, the format is yyyymmdd. * @param uuid The uuid. + * @param regionName name of a region, where this file was created during flush or compaction. * @return An instance of a MobFileName. */ - public static MobFileName create(byte[] startKey, String date, String uuid) { - return new MobFileName(startKey, date, uuid); + public static MobFileName create(byte[] startKey, String date, String uuid, String regionName) { + return new MobFileName(startKey, date, uuid, regionName); } /** @@ -102,10 +110,11 @@ public static MobFileName create(byte[] startKey, String date, String uuid) { * @param date * The string of the latest timestamp of cells in this file, the format is yyyymmdd. * @param uuid The uuid. + * @param regionName name of a region, where this file was created during flush or compaction. * @return An instance of a MobFileName. */ - public static MobFileName create(String startKey, String date, String uuid) { - return new MobFileName(startKey, date, uuid); + public static MobFileName create(String startKey, String date, String uuid, String regionName) { + return new MobFileName(startKey, date, uuid, regionName); } /** @@ -119,7 +128,8 @@ public static MobFileName create(String fileName) { String startKey = fileName.substring(0, STARTKEY_END_INDEX); String date = fileName.substring(STARTKEY_END_INDEX, DATE_END_INDEX); String uuid = fileName.substring(DATE_END_INDEX, UUID_END_INDEX); - return new MobFileName(startKey, date, uuid); + String regionName = fileName.substring(UUID_END_INDEX+1); + return new MobFileName(startKey, date, uuid, regionName); } /** @@ -148,6 +158,13 @@ public String getStartKey() { return startKey; } + /** + * Gets region name + * @return name of a region, where this file was created during flush or compaction. + */ + public String getRegionName() { + return regionName; + } /** * Gets the date string. Its format is yyyymmdd. * @return The date string. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java index ee1fe7db14fc..6adb4b58ded5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java @@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine; import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.yetus.audience.InterfaceAudience; /** @@ -29,7 +30,7 @@ */ @InterfaceAudience.Private public class MobStoreEngine extends DefaultStoreEngine { - + public final static String DEFAULT_MOB_COMPACTOR_CLASS_KEY = "hbase.mob.default.compactor"; @Override protected void createStoreFlusher(Configuration conf, HStore store) throws IOException { // When using MOB, we use DefaultMobStoreFlusher always @@ -43,6 +44,13 @@ protected void createStoreFlusher(Configuration conf, HStore store) throws IOExc */ @Override protected void createCompactor(Configuration conf, HStore store) throws IOException { - compactor = new DefaultMobStoreCompactor(conf, store); + String className = + conf.get(DEFAULT_MOB_COMPACTOR_CLASS_KEY, DefaultMobStoreCompactor.class.getName()); + try { + compactor = ReflectionUtils.instantiateWithCustomCtor(className, + new Class[] { Configuration.class, HStore.class }, new Object[] { conf, store }); + } catch (Exception e) { + throw new IOException("Unable to load configured compactor '" + className + "'", e); + } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java index 304a62e8901b..50a7f770500c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java @@ -29,12 +29,6 @@ import java.util.List; import java.util.Optional; import java.util.UUID; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.RejectedExecutionHandler; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -50,7 +44,6 @@ import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.backup.HFileArchiver; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.Scan; @@ -62,10 +55,6 @@ import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; -import org.apache.hadoop.hbase.master.locking.LockManager; -import org.apache.hadoop.hbase.mob.compactions.MobCompactor; -import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId; -import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactor; import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.HStoreFile; @@ -74,8 +63,6 @@ import org.apache.hadoop.hbase.util.ChecksumType; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.hbase.util.ReflectionUtils; -import org.apache.hadoop.hbase.util.Threads; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -87,23 +74,15 @@ public final class MobUtils { private static final Logger LOG = LoggerFactory.getLogger(MobUtils.class); - private final static long WEEKLY_THRESHOLD_MULTIPLIER = 7; - private final static long MONTHLY_THRESHOLD_MULTIPLIER = 4 * WEEKLY_THRESHOLD_MULTIPLIER; + public static final String SEP = "_"; private static final ThreadLocal LOCAL_FORMAT = new ThreadLocal() { - @Override - protected SimpleDateFormat initialValue() { - return new SimpleDateFormat("yyyyMMdd"); - } - }; - - private static final byte[] REF_DELETE_MARKER_TAG_BYTES; - static { - List tags = new ArrayList<>(); - tags.add(MobConstants.MOB_REF_TAG); - REF_DELETE_MARKER_TAG_BYTES = TagUtil.fromList(tags); - } + @Override + protected SimpleDateFormat initialValue() { + return new SimpleDateFormat("yyyyMMdd"); + } + }; /** * Private constructor to keep this class from being instantiated. @@ -130,45 +109,6 @@ public static Date parseDate(String dateString) throws ParseException { return LOCAL_FORMAT.get().parse(dateString); } - /** - * Get the first day of the input date's month - * @param calendar Calendar object - * @param date The date to find out its first day of that month - * @return The first day in the month - */ - public static Date getFirstDayOfMonth(final Calendar calendar, final Date date) { - - calendar.setTime(date); - calendar.set(Calendar.HOUR_OF_DAY, 0); - calendar.set(Calendar.MINUTE, 0); - calendar.set(Calendar.SECOND, 0); - calendar.set(Calendar.MILLISECOND, 0); - calendar.set(Calendar.DAY_OF_MONTH, 1); - - Date firstDayInMonth = calendar.getTime(); - return firstDayInMonth; - } - - /** - * Get the first day of the input date's week - * @param calendar Calendar object - * @param date The date to find out its first day of that week - * @return The first day in the week - */ - public static Date getFirstDayOfWeek(final Calendar calendar, final Date date) { - - calendar.setTime(date); - calendar.set(Calendar.HOUR_OF_DAY, 0); - calendar.set(Calendar.MINUTE, 0); - calendar.set(Calendar.SECOND, 0); - calendar.set(Calendar.MILLISECOND, 0); - calendar.setFirstDayOfWeek(Calendar.MONDAY); - calendar.set(Calendar.DAY_OF_WEEK, Calendar.MONDAY); - - Date firstDayInWeek = calendar.getTime(); - return firstDayInWeek; - } - /** * Whether the current cell is a mob reference cell. * @param cell The current cell. @@ -216,11 +156,10 @@ public static boolean hasMobReferenceTag(List tags) { } /** - * Indicates whether it's a raw scan. - * The information is set in the attribute "hbase.mob.scan.raw" of scan. - * For a mob cell, in a normal scan the scanners retrieves the mob cell from the mob file. - * In a raw scan, the scanner directly returns cell in HBase without retrieve the one in - * the mob file. + * Indicates whether it's a raw scan. The information is set in the attribute "hbase.mob.scan.raw" + * of scan. For a mob cell, in a normal scan the scanners retrieves the mob cell from the mob + * file. In a raw scan, the scanner directly returns cell in HBase without retrieve the one in the + * mob file. * @param scan The current scan. * @return True if it's a raw scan. */ @@ -234,9 +173,9 @@ public static boolean isRawMobScan(Scan scan) { } /** - * Indicates whether it's a reference only scan. - * The information is set in the attribute "hbase.mob.scan.ref.only" of scan. - * If it's a ref only scan, only the cells with ref tag are returned. + * Indicates whether it's a reference only scan. The information is set in the attribute + * "hbase.mob.scan.ref.only" of scan. If it's a ref only scan, only the cells with ref tag are + * returned. * @param scan The current scan. * @return True if it's a ref only scan. */ @@ -250,8 +189,8 @@ public static boolean isRefOnlyScan(Scan scan) { } /** - * Indicates whether the scan contains the information of caching blocks. - * The information is set in the attribute "hbase.mob.cache.blocks" of scan. + * Indicates whether the scan contains the information of caching blocks. The information is set + * in the attribute "hbase.mob.cache.blocks" of scan. * @param scan The current scan. * @return True when the Scan attribute specifies to cache the MOB blocks. */ @@ -266,22 +205,17 @@ public static boolean isCacheMobBlocks(Scan scan) { /** * Sets the attribute of caching blocks in the scan. - * - * @param scan - * The current scan. - * @param cacheBlocks - * True, set the attribute of caching blocks into the scan, the scanner with this scan - * caches blocks. - * False, the scanner doesn't cache blocks for this scan. + * @param scan The current scan. + * @param cacheBlocks True, set the attribute of caching blocks into the scan, the scanner with + * this scan caches blocks. False, the scanner doesn't cache blocks for this scan. */ public static void setCacheMobBlocks(Scan scan, boolean cacheBlocks) { scan.setAttribute(MobConstants.MOB_CACHE_BLOCKS, Bytes.toBytes(cacheBlocks)); } /** - * Cleans the expired mob files. - * Cleans the files whose creation date is older than (current - columnFamily.ttl), and - * the minVersions of that column family is 0. + * Cleans the expired mob files. Cleans the files whose creation date is older than (current - + * columnFamily.ttl), and the minVersions of that column family is 0. * @param fs The current file system. * @param conf The current configuration. * @param tableName The current table name. @@ -334,11 +268,11 @@ public static void cleanExpiredMobFiles(FileSystem fs, Configuration conf, Table Date fileDate = parseDate(MobFileName.getDateFromName(fileName)); if (LOG.isDebugEnabled()) { - LOG.debug("Checking file " + fileName); + LOG.debug("Checking file {}", fileName); } if (fileDate.getTime() < expireDate.getTime()) { if (LOG.isDebugEnabled()) { - LOG.debug(fileName + " is an expired file"); + LOG.debug("{} is an expired file", fileName); } filesToClean .add(new HStoreFile(fs, file.getPath(), conf, cacheConfig, BloomType.NONE, true)); @@ -349,19 +283,17 @@ public static void cleanExpiredMobFiles(FileSystem fs, Configuration conf, Table } if (!filesToClean.isEmpty()) { try { - removeMobFiles(conf, fs, tableName, mobTableDir, columnDescriptor.getName(), - filesToClean); + removeMobFiles(conf, fs, tableName, mobTableDir, columnDescriptor.getName(), filesToClean); deletedFileCount = filesToClean.size(); } catch (IOException e) { LOG.error("Failed to delete the mob files " + filesToClean, e); } } - LOG.info(deletedFileCount + " expired mob files are deleted"); + LOG.info("{} expired mob files are deleted", deletedFileCount); } /** - * Gets the root dir of the mob files. - * It's {HBASE_DIR}/mobdir. + * Gets the root dir of the mob files. It's {HBASE_DIR}/mobdir. * @param conf The current configuration. * @return the root dir of the mob file. */ @@ -371,8 +303,7 @@ public static Path getMobHome(Configuration conf) { } /** - * Gets the root dir of the mob files under the qualified HBase root dir. - * It's {rootDir}/mobdir. + * Gets the root dir of the mob files under the qualified HBase root dir. It's {rootDir}/mobdir. * @param rootDir The qualified path of HBase root directory. * @return The root dir of the mob file. */ @@ -394,8 +325,8 @@ public static Path getQualifiedMobRootDir(Configuration conf) throws IOException } /** - * Gets the table dir of the mob files under the qualified HBase root dir. - * It's {rootDir}/mobdir/data/${namespace}/${tableName} + * Gets the table dir of the mob files under the qualified HBase root dir. It's + * {rootDir}/mobdir/data/${namespace}/${tableName} * @param rootDir The qualified path of HBase root directory. * @param tableName The name of table. * @return The table dir of the mob file. @@ -405,8 +336,8 @@ public static Path getMobTableDir(Path rootDir, TableName tableName) { } /** - * Gets the region dir of the mob files. - * It's {HBASE_DIR}/mobdir/data/{namespace}/{tableName}/{regionEncodedName}. + * Gets the region dir of the mob files. It's + * {HBASE_DIR}/mobdir/data/{namespace}/{tableName}/{regionEncodedName}. * @param conf The current configuration. * @param tableName The current table name. * @return The region dir of the mob files. @@ -416,8 +347,8 @@ public static Path getMobRegionPath(Configuration conf, TableName tableName) { } /** - * Gets the region dir of the mob files under the specified root dir. - * It's {rootDir}/mobdir/data/{namespace}/{tableName}/{regionEncodedName}. + * Gets the region dir of the mob files under the specified root dir. It's + * {rootDir}/mobdir/data/{namespace}/{tableName}/{regionEncodedName}. * @param rootDir The qualified path of HBase root directory. * @param tableName The current table name. * @return The region dir of the mob files. @@ -429,8 +360,8 @@ public static Path getMobRegionPath(Path rootDir, TableName tableName) { } /** - * Gets the family dir of the mob files. - * It's {HBASE_DIR}/mobdir/{namespace}/{tableName}/{regionEncodedName}/{columnFamilyName}. + * Gets the family dir of the mob files. It's + * {HBASE_DIR}/mobdir/{namespace}/{tableName}/{regionEncodedName}/{columnFamilyName}. * @param conf The current configuration. * @param tableName The current table name. * @param familyName The current family name. @@ -441,8 +372,8 @@ public static Path getMobFamilyPath(Configuration conf, TableName tableName, Str } /** - * Gets the family dir of the mob files. - * It's {HBASE_DIR}/mobdir/{namespace}/{tableName}/{regionEncodedName}/{columnFamilyName}. + * Gets the family dir of the mob files. It's + * {HBASE_DIR}/mobdir/{namespace}/{tableName}/{regionEncodedName}/{columnFamilyName}. * @param regionPath The path of mob region which is a dummy one. * @param familyName The current family name. * @return The family dir of the mob files. @@ -452,19 +383,14 @@ public static Path getMobFamilyPath(Path regionPath, String familyName) { } /** - * Gets the RegionInfo of the mob files. - * This is a dummy region. The mob files are not saved in a region in HBase. - * This is only used in mob snapshot. It's internally used only. + * Gets the RegionInfo of the mob files. This is a dummy region. The mob files are not saved in a + * region in HBase. This is only used in mob snapshot. It's internally used only. * @param tableName * @return A dummy mob region info. */ public static RegionInfo getMobRegionInfo(TableName tableName) { - return RegionInfoBuilder.newBuilder(tableName) - .setStartKey(MobConstants.MOB_REGION_NAME_BYTES) - .setEndKey(HConstants.EMPTY_END_ROW) - .setSplit(false) - .setRegionId(0) - .build(); + return RegionInfoBuilder.newBuilder(tableName).setStartKey(MobConstants.MOB_REGION_NAME_BYTES) + .setEndKey(HConstants.EMPTY_END_ROW).setSplit(false).setRegionId(0).build(); } /** @@ -473,8 +399,9 @@ public static RegionInfo getMobRegionInfo(TableName tableName) { * @return If true, the current RegionInfo is a mob one. */ public static boolean isMobRegionInfo(RegionInfo regionInfo) { - return regionInfo == null ? false : getMobRegionInfo(regionInfo.getTable()).getEncodedName() - .equals(regionInfo.getEncodedName()); + return regionInfo == null ? false + : getMobRegionInfo(regionInfo.getTable()).getEncodedName() + .equals(regionInfo.getEncodedName()); } /** @@ -487,16 +414,6 @@ public static boolean isMobRegionName(TableName tableName, byte[] regionName) { return Bytes.equals(regionName, getMobRegionInfo(tableName).getRegionName()); } - /** - * Gets the working directory of the mob compaction. - * @param root The root directory of the mob compaction. - * @param jobName The current job name. - * @return The directory of the mob compaction for the current job. - */ - public static Path getCompactionWorkingPath(Path root, String jobName) { - return new Path(root, jobName); - } - /** * Archives the mob files. * @param conf The current configuration. @@ -510,16 +427,18 @@ public static Path getCompactionWorkingPath(Path root, String jobName) { public static void removeMobFiles(Configuration conf, FileSystem fs, TableName tableName, Path tableDir, byte[] family, Collection storeFiles) throws IOException { HFileArchiver.archiveStoreFiles(conf, fs, getMobRegionInfo(tableName), tableDir, family, - storeFiles); + storeFiles); } + + /** - * Creates a mob reference KeyValue. - * The value of the mob reference KeyValue is mobCellValueSize + mobFileName. + * Creates a mob reference KeyValue. The value of the mob reference KeyValue is mobCellValueSize + + * mobFileName. * @param cell The original Cell. * @param fileName The mob file name where the mob reference KeyValue is written. - * @param tableNameTag The tag of the current table name. It's very important in - * cloning the snapshot. + * @param tableNameTag The tag of the current table name. It's very important in cloning the + * snapshot. * @return The mob reference KeyValue. */ public static Cell createMobRefCell(Cell cell, byte[] fileName, Tag tableNameTag) { @@ -561,92 +480,14 @@ public static Cell createMobRefCell(Cell cell, byte[] fileName, byte[] refCellTa public static StoreFileWriter createWriter(Configuration conf, FileSystem fs, ColumnFamilyDescriptor family, String date, Path basePath, long maxKeyCount, Compression.Algorithm compression, String startKey, CacheConfig cacheConfig, - Encryption.Context cryptoContext, boolean isCompaction) - throws IOException { - MobFileName mobFileName = MobFileName.create(startKey, date, - UUID.randomUUID().toString().replaceAll("-", "")); - return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression, - cacheConfig, cryptoContext, isCompaction); - } - - /** - * Creates a writer for the ref file in temp directory. - * @param conf The current configuration. - * @param fs The current file system. - * @param family The descriptor of the current column family. - * @param basePath The basic path for a temp directory. - * @param maxKeyCount The key count. - * @param cacheConfig The current cache config. - * @param cryptoContext The encryption context. - * @param isCompaction If the writer is used in compaction. - * @return The writer for the mob file. - * @throws IOException - */ - public static StoreFileWriter createRefFileWriter(Configuration conf, FileSystem fs, - ColumnFamilyDescriptor family, Path basePath, long maxKeyCount, CacheConfig cacheConfig, - Encryption.Context cryptoContext, boolean isCompaction) - throws IOException { - return createWriter(conf, fs, family, - new Path(basePath, UUID.randomUUID().toString().replaceAll("-", "")), maxKeyCount, - family.getCompactionCompressionType(), cacheConfig, cryptoContext, - HStore.getChecksumType(conf), HStore.getBytesPerChecksum(conf), family.getBlocksize(), - family.getBloomFilterType(), isCompaction); - } - - /** - * Creates a writer for the mob file in temp directory. - * @param conf The current configuration. - * @param fs The current file system. - * @param family The descriptor of the current column family. - * @param date The date string, its format is yyyymmmdd. - * @param basePath The basic path for a temp directory. - * @param maxKeyCount The key count. - * @param compression The compression algorithm. - * @param startKey The start key. - * @param cacheConfig The current cache config. - * @param cryptoContext The encryption context. - * @param isCompaction If the writer is used in compaction. - * @return The writer for the mob file. - * @throws IOException - */ - public static StoreFileWriter createWriter(Configuration conf, FileSystem fs, - ColumnFamilyDescriptor family, String date, Path basePath, long maxKeyCount, - Compression.Algorithm compression, byte[] startKey, CacheConfig cacheConfig, - Encryption.Context cryptoContext, boolean isCompaction) + Encryption.Context cryptoContext, boolean isCompaction, String regionName) throws IOException { MobFileName mobFileName = MobFileName.create(startKey, date, - UUID.randomUUID().toString().replaceAll("-", "")); + UUID.randomUUID().toString().replaceAll("-", ""), regionName); return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression, cacheConfig, cryptoContext, isCompaction); } - /** - * Creates a writer for the del file in temp directory. - * @param conf The current configuration. - * @param fs The current file system. - * @param family The descriptor of the current column family. - * @param date The date string, its format is yyyymmmdd. - * @param basePath The basic path for a temp directory. - * @param maxKeyCount The key count. - * @param compression The compression algorithm. - * @param startKey The start key. - * @param cacheConfig The current cache config. - * @param cryptoContext The encryption context. - * @return The writer for the del file. - * @throws IOException - */ - public static StoreFileWriter createDelFileWriter(Configuration conf, FileSystem fs, - ColumnFamilyDescriptor family, String date, Path basePath, long maxKeyCount, - Compression.Algorithm compression, byte[] startKey, CacheConfig cacheConfig, - Encryption.Context cryptoContext) - throws IOException { - String suffix = UUID - .randomUUID().toString().replaceAll("-", "") + "_del"; - MobFileName mobFileName = MobFileName.create(startKey, date, suffix); - return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression, - cacheConfig, cryptoContext, true); - } - /** * Creates a writer for the mob file in temp directory. * @param conf The current configuration. @@ -663,14 +504,12 @@ public static StoreFileWriter createDelFileWriter(Configuration conf, FileSystem * @throws IOException */ public static StoreFileWriter createWriter(Configuration conf, FileSystem fs, - ColumnFamilyDescriptor family, MobFileName mobFileName, Path basePath, long maxKeyCount, + ColumnFamilyDescriptor family, MobFileName mobFileName, Path basePath, long maxKeyCount, Compression.Algorithm compression, CacheConfig cacheConfig, Encryption.Context cryptoContext, - boolean isCompaction) - throws IOException { - return createWriter(conf, fs, family, - new Path(basePath, mobFileName.getFileName()), maxKeyCount, compression, cacheConfig, - cryptoContext, HStore.getChecksumType(conf), HStore.getBytesPerChecksum(conf), - family.getBlocksize(), BloomType.NONE, isCompaction); + boolean isCompaction) throws IOException { + return createWriter(conf, fs, family, new Path(basePath, mobFileName.getFileName()), + maxKeyCount, compression, cacheConfig, cryptoContext, HStore.getChecksumType(conf), + HStore.getBytesPerChecksum(conf), family.getBlocksize(), BloomType.NONE, isCompaction); } /** @@ -692,10 +531,9 @@ public static StoreFileWriter createWriter(Configuration conf, FileSystem fs, * @throws IOException */ public static StoreFileWriter createWriter(Configuration conf, FileSystem fs, - ColumnFamilyDescriptor family, Path path, long maxKeyCount, - Compression.Algorithm compression, CacheConfig cacheConfig, Encryption.Context cryptoContext, - ChecksumType checksumType, int bytesPerChecksum, int blocksize, BloomType bloomType, - boolean isCompaction) + ColumnFamilyDescriptor family, Path path, long maxKeyCount, Compression.Algorithm compression, + CacheConfig cacheConfig, Encryption.Context cryptoContext, ChecksumType checksumType, + int bytesPerChecksum, int blocksize, BloomType bloomType, boolean isCompaction) throws IOException { if (compression == null) { compression = HFile.DEFAULT_COMPRESSION_ALGORITHM; @@ -708,17 +546,13 @@ public static StoreFileWriter createWriter(Configuration conf, FileSystem fs, writerCacheConf = cacheConfig; } HFileContext hFileContext = new HFileContextBuilder().withCompression(compression) - .withIncludesMvcc(true).withIncludesTags(true) - .withCompressTags(family.isCompressTags()) - .withChecksumType(checksumType) - .withBytesPerCheckSum(bytesPerChecksum) - .withBlockSize(blocksize) - .withHBaseCheckSum(true).withDataBlockEncoding(family.getDataBlockEncoding()) - .withEncryptionContext(cryptoContext) + .withIncludesMvcc(true).withIncludesTags(true).withCompressTags(family.isCompressTags()) + .withChecksumType(checksumType).withBytesPerCheckSum(bytesPerChecksum) + .withBlockSize(blocksize).withHBaseCheckSum(true) + .withDataBlockEncoding(family.getDataBlockEncoding()).withEncryptionContext(cryptoContext) .withCreateTime(EnvironmentEdgeManager.currentTime()).build(); - StoreFileWriter w = new StoreFileWriter.Builder(conf, writerCacheConf, fs) - .withFilePath(path) + StoreFileWriter w = new StoreFileWriter.Builder(conf, writerCacheConf, fs).withFilePath(path) .withComparator(CellComparator.getInstance()).withBloomType(bloomType) .withMaxKeyCount(maxKeyCount).withFileContext(hFileContext).build(); return w; @@ -777,11 +611,9 @@ private static void validateMobFile(Configuration conf, FileSystem fs, Path path } /** - * Indicates whether the current mob ref cell has a valid value. - * A mob ref cell has a mob reference tag. - * The value of a mob ref cell consists of two parts, real mob value length and mob file name. - * The real mob value length takes 4 bytes. - * The remaining part is the mob file name. + * Indicates whether the current mob ref cell has a valid value. A mob ref cell has a mob + * reference tag. The value of a mob ref cell consists of two parts, real mob value length and mob + * file name. The real mob value length takes 4 bytes. The remaining part is the mob file name. * @param cell The mob ref cell. * @return True if the cell has a valid value. */ @@ -790,11 +622,9 @@ public static boolean hasValidMobRefCellValue(Cell cell) { } /** - * Gets the mob value length from the mob ref cell. - * A mob ref cell has a mob reference tag. - * The value of a mob ref cell consists of two parts, real mob value length and mob file name. - * The real mob value length takes 4 bytes. - * The remaining part is the mob file name. + * Gets the mob value length from the mob ref cell. A mob ref cell has a mob reference tag. The + * value of a mob ref cell consists of two parts, real mob value length and mob file name. The + * real mob value length takes 4 bytes. The remaining part is the mob file name. * @param cell The mob ref cell. * @return The real mob value length. */ @@ -803,22 +633,20 @@ public static int getMobValueLength(Cell cell) { } /** - * Gets the mob file name from the mob ref cell. - * A mob ref cell has a mob reference tag. - * The value of a mob ref cell consists of two parts, real mob value length and mob file name. - * The real mob value length takes 4 bytes. - * The remaining part is the mob file name. + * Gets the mob file name from the mob ref cell. A mob ref cell has a mob reference tag. The value + * of a mob ref cell consists of two parts, real mob value length and mob file name. The real mob + * value length takes 4 bytes. The remaining part is the mob file name. * @param cell The mob ref cell. * @return The mob file name. */ public static String getMobFileName(Cell cell) { return Bytes.toString(cell.getValueArray(), cell.getValueOffset() + Bytes.SIZEOF_INT, - cell.getValueLength() - Bytes.SIZEOF_INT); + cell.getValueLength() - Bytes.SIZEOF_INT); } /** - * Gets the table name used in the table lock. - * The table lock name is a dummy one, it's not a table name. It's tableName + ".mobLock". + * Gets the table name used in the table lock. The table lock name is a dummy one, it's not a + * table name. It's tableName + ".mobLock". * @param tn The table name. * @return The table name used in table lock. */ @@ -827,71 +655,6 @@ public static TableName getTableLockName(TableName tn) { return TableName.valueOf(Bytes.add(tableName, MobConstants.MOB_TABLE_LOCK_SUFFIX)); } - /** - * Performs the mob compaction. - * @param conf the Configuration - * @param fs the file system - * @param tableName the table the compact - * @param hcd the column descriptor - * @param pool the thread pool - * @param allFiles Whether add all mob files into the compaction. - */ - public static void doMobCompaction(Configuration conf, FileSystem fs, TableName tableName, - ColumnFamilyDescriptor hcd, ExecutorService pool, boolean allFiles, LockManager.MasterLock lock) - throws IOException { - String className = conf.get(MobConstants.MOB_COMPACTOR_CLASS_KEY, - PartitionedMobCompactor.class.getName()); - // instantiate the mob compactor. - MobCompactor compactor = null; - try { - compactor = ReflectionUtils.instantiateWithCustomCtor(className, new Class[] { - Configuration.class, FileSystem.class, TableName.class, ColumnFamilyDescriptor.class, - ExecutorService.class }, new Object[] { conf, fs, tableName, hcd, pool }); - } catch (Exception e) { - throw new IOException("Unable to load configured mob file compactor '" + className + "'", e); - } - // compact only for mob-enabled column. - // obtain a write table lock before performing compaction to avoid race condition - // with major compaction in mob-enabled column. - try { - lock.acquire(); - compactor.compact(allFiles); - } catch (Exception e) { - LOG.error("Failed to compact the mob files for the column " + hcd.getNameAsString() - + " in the table " + tableName.getNameAsString(), e); - } finally { - lock.release(); - } - } - - /** - * Creates a thread pool. - * @param conf the Configuration - * @return A thread pool. - */ - public static ExecutorService createMobCompactorThreadPool(Configuration conf) { - int maxThreads = conf.getInt(MobConstants.MOB_COMPACTION_THREADS_MAX, - MobConstants.DEFAULT_MOB_COMPACTION_THREADS_MAX); - if (maxThreads == 0) { - maxThreads = 1; - } - final SynchronousQueue queue = new SynchronousQueue<>(); - ThreadPoolExecutor pool = new ThreadPoolExecutor(1, maxThreads, 60, TimeUnit.SECONDS, queue, - Threads.newDaemonThreadFactory("MobCompactor"), new RejectedExecutionHandler() { - @Override - public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) { - try { - // waiting for a thread to pick up instead of throwing exceptions. - queue.put(r); - } catch (InterruptedException e) { - throw new RejectedExecutionException(e); - } - } - }); - ((ThreadPoolExecutor) pool).allowCoreThreadTimeOut(true); - return pool; - } - /** * Checks whether this table has mob-enabled columns. * @param htd The current table descriptor. @@ -907,6 +670,37 @@ public static boolean hasMobColumns(TableDescriptor htd) { return false; } + /** + * Get list of Mob column families (if any exists) + * @param htd table descriptor + * @return list of Mob column families + */ + public static List getMobColumnFamilies(TableDescriptor htd) { + + List fams = new ArrayList(); + ColumnFamilyDescriptor[] hcds = htd.getColumnFamilies(); + for (ColumnFamilyDescriptor hcd : hcds) { + if (hcd.isMobEnabled()) { + fams.add(hcd); + } + } + return fams; + } + + + public static long getNumberOfMobFiles(Configuration conf, TableName tableName, String family) + throws IOException { + FileSystem fs = FileSystem.get(conf); + Path dir = getMobFamilyPath(conf, tableName, family); + FileStatus[] stat = fs.listStatus(dir); + for (FileStatus st : stat) { + LOG.info("MOB Directory content: {}", st.getPath()); + } + LOG.info("MOB Directory content total files: {}", stat.length); + + return stat.length; + } + /** * Indicates whether return null value when the mob file is missing or corrupt. * The information is set in the attribute "empty.value.on.mobcell.miss" of scan. @@ -915,7 +709,7 @@ public static boolean hasMobColumns(TableDescriptor htd) { */ public static boolean isReadEmptyValueOnMobCellMiss(Scan scan) { byte[] readEmptyValueOnMobCellMiss = - scan.getAttribute(MobConstants.EMPTY_VALUE_ON_MOBCELL_MISS); + scan.getAttribute(MobConstants.EMPTY_VALUE_ON_MOBCELL_MISS); try { return readEmptyValueOnMobCellMiss != null && Bytes.toBoolean(readEmptyValueOnMobCellMiss); } catch (IllegalArgumentException e) { @@ -923,15 +717,6 @@ public static boolean isReadEmptyValueOnMobCellMiss(Scan scan) { } } - /** - * Creates a mob ref delete marker. - * @param cell The current delete marker. - * @return A delete marker with the ref tag. - */ - public static Cell createMobRefDeleteMarker(Cell cell) { - return PrivateCellUtil.createCell(cell, TagUtil.concatTags(REF_DELETE_MARKER_TAG_BYTES, cell)); - } - /** * Checks if the mob file is expired. * @param column The descriptor of the current column family. @@ -939,7 +724,8 @@ public static Cell createMobRefDeleteMarker(Cell cell) { * @param fileDate The date string parsed from the mob file name. * @return True if the mob file is expired. */ - public static boolean isMobFileExpired(ColumnFamilyDescriptor column, long current, String fileDate) { + public static boolean isMobFileExpired(ColumnFamilyDescriptor column, long current, + String fileDate) { if (column.getMinVersions() > 0) { return false; } @@ -962,86 +748,4 @@ public static boolean isMobFileExpired(ColumnFamilyDescriptor column, long curre return false; } - /** - * fill out partition id based on compaction policy and date, threshold... - * @param id Partition id to be filled out - * @param firstDayOfCurrentMonth The first day in the current month - * @param firstDayOfCurrentWeek The first day in the current week - * @param dateStr Date string from the mob file - * @param policy Mob compaction policy - * @param calendar Calendar object - * @param threshold Mob compaciton threshold configured - * @return true if the file needs to be excluded from compaction - */ - public static boolean fillPartitionId(final CompactionPartitionId id, - final Date firstDayOfCurrentMonth, final Date firstDayOfCurrentWeek, final String dateStr, - final MobCompactPartitionPolicy policy, final Calendar calendar, final long threshold) { - - boolean skipCompcation = false; - id.setThreshold(threshold); - if (threshold <= 0) { - id.setDate(dateStr); - return skipCompcation; - } - - long finalThreshold; - Date date; - try { - date = MobUtils.parseDate(dateStr); - } catch (ParseException e) { - LOG.warn("Failed to parse date " + dateStr, e); - id.setDate(dateStr); - return true; - } - - /* The algorithm works as follows: - * For monthly policy: - * 1). If the file's date is in past months, apply 4 * 7 * threshold - * 2). If the file's date is in past weeks, apply 7 * threshold - * 3). If the file's date is in current week, exclude it from the compaction - * For weekly policy: - * 1). If the file's date is in past weeks, apply 7 * threshold - * 2). If the file's date in currently, apply threshold - * For daily policy: - * 1). apply threshold - */ - if (policy == MobCompactPartitionPolicy.MONTHLY) { - if (date.before(firstDayOfCurrentMonth)) { - // Check overflow - if (threshold < (Long.MAX_VALUE / MONTHLY_THRESHOLD_MULTIPLIER)) { - finalThreshold = MONTHLY_THRESHOLD_MULTIPLIER * threshold; - } else { - finalThreshold = Long.MAX_VALUE; - } - id.setThreshold(finalThreshold); - - // set to the date for the first day of that month - id.setDate(MobUtils.formatDate(MobUtils.getFirstDayOfMonth(calendar, date))); - return skipCompcation; - } - } - - if ((policy == MobCompactPartitionPolicy.MONTHLY) || - (policy == MobCompactPartitionPolicy.WEEKLY)) { - // Check if it needs to apply weekly multiplier - if (date.before(firstDayOfCurrentWeek)) { - // Check overflow - if (threshold < (Long.MAX_VALUE / WEEKLY_THRESHOLD_MULTIPLIER)) { - finalThreshold = WEEKLY_THRESHOLD_MULTIPLIER * threshold; - } else { - finalThreshold = Long.MAX_VALUE; - } - id.setThreshold(finalThreshold); - - id.setDate(MobUtils.formatDate(MobUtils.getFirstDayOfWeek(calendar, date))); - return skipCompcation; - } else if (policy == MobCompactPartitionPolicy.MONTHLY) { - skipCompcation = true; - } - } - - // Rest is daily - id.setDate(dateStr); - return skipCompcation; - } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactionRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactionRequest.java deleted file mode 100644 index 5025ed261229..000000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactionRequest.java +++ /dev/null @@ -1,64 +0,0 @@ -/** - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.mob.compactions; - -import org.apache.yetus.audience.InterfaceAudience; - -/** - * The compaction request for mob files. - */ -@InterfaceAudience.Private -public abstract class MobCompactionRequest { - - protected long selectionTime; - protected CompactionType type = CompactionType.PART_FILES; - - public void setCompactionType(CompactionType type) { - this.type = type; - } - - /** - * Gets the selection time. - * @return The selection time. - */ - public long getSelectionTime() { - return this.selectionTime; - } - - /** - * Gets the compaction type. - * @return The compaction type. - */ - public CompactionType getCompactionType() { - return type; - } - - protected enum CompactionType { - - /** - * Part of mob files are selected. - */ - PART_FILES, - - /** - * All of mob files are selected. - */ - ALL_FILES - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java deleted file mode 100644 index c5d93ea6e064..000000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java +++ /dev/null @@ -1,90 +0,0 @@ -/** - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.mob.compactions; - -import java.io.IOException; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.ExecutorService; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.TableName; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.mob.MobUtils; -import org.apache.hadoop.hbase.util.FSUtils; - -/** - * A mob compactor to directly compact the mob files. - */ -@InterfaceAudience.Private -public abstract class MobCompactor { - - protected FileSystem fs; - protected Configuration conf; - protected TableName tableName; - protected ColumnFamilyDescriptor column; - - protected Path mobTableDir; - protected Path mobFamilyDir; - protected ExecutorService pool; - - public MobCompactor(Configuration conf, FileSystem fs, TableName tableName, - ColumnFamilyDescriptor column, ExecutorService pool) { - this.conf = conf; - this.fs = fs; - this.tableName = tableName; - this.column = column; - this.pool = pool; - mobTableDir = FSUtils.getTableDir(MobUtils.getMobHome(conf), tableName); - mobFamilyDir = MobUtils.getMobFamilyPath(conf, tableName, column.getNameAsString()); - } - - /** - * Compacts the mob files for the current column family. - * @return The paths of new mob files generated in the compaction. - * @throws IOException - */ - public List compact() throws IOException { - return compact(false); - } - - /** - * Compacts the mob files by compaction type for the current column family. - * @param allFiles Whether add all mob files into the compaction. - * @return The paths of new mob files generated in the compaction. - * @throws IOException - */ - public List compact(boolean allFiles) throws IOException { - return compact(Arrays.asList(fs.listStatus(mobFamilyDir)), allFiles); - } - - /** - * Compacts the candidate mob files. - * @param files The candidate mob files. - * @param allFiles Whether add all mob files into the compaction. - * @return The paths of new mob files generated in the compaction. - * @throws IOException - */ - public abstract List compact(List files, boolean allFiles) - throws IOException; -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactionRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactionRequest.java deleted file mode 100644 index ab917a2d9228..000000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactionRequest.java +++ /dev/null @@ -1,333 +0,0 @@ -/** - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.mob.compactions; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; - -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.regionserver.HStoreFile; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.yetus.audience.InterfaceAudience; - -/** - * An implementation of {@link MobCompactionRequest} that is used in - * {@link PartitionedMobCompactor}. - * The mob files that have the same start key and date in their names belong to - * the same partition. - */ -@InterfaceAudience.Private -public class PartitionedMobCompactionRequest extends MobCompactionRequest { - - protected List delPartitions; - protected Collection compactionPartitions; - - public PartitionedMobCompactionRequest(Collection compactionPartitions, - List delPartitions) { - this.selectionTime = EnvironmentEdgeManager.currentTime(); - this.compactionPartitions = compactionPartitions; - this.delPartitions = delPartitions; - } - - /** - * Gets the compaction partitions. - * @return The compaction partitions. - */ - public Collection getCompactionPartitions() { - return this.compactionPartitions; - } - - /** - * Gets the del files. - * @return The del files. - */ - public List getDelPartitions() { - return this.delPartitions; - } - - /** - * The partition in the mob compaction. - * The mob files that have the same start key and date in their names belong to - * the same partition. - */ - protected static class CompactionPartition { - private List files = new ArrayList<>(); - private CompactionPartitionId partitionId; - - // The startKey and endKey of this partition, both are inclusive. - private byte[] startKey; - private byte[] endKey; - - public CompactionPartition(CompactionPartitionId partitionId) { - this.partitionId = partitionId; - } - - public CompactionPartitionId getPartitionId() { - return this.partitionId; - } - - public void addFile(FileStatus file) { - files.add(file); - } - - public List listFiles() { - return Collections.unmodifiableList(files); - } - - public int getFileCount () { - return files.size(); - } - - public byte[] getStartKey() { - return startKey; - } - - /** - * Set start key of this partition, only if the input startKey is less than - * the current start key. - */ - public void setStartKey(final byte[] startKey) { - if ((this.startKey == null) || (Bytes.compareTo(startKey, this.startKey) < 0)) { - this.startKey = startKey; - } - } - - public byte[] getEndKey() { - return endKey; - } - - /** - * Set end key of this partition, only if the input endKey is greater than - * the current end key. - */ - public void setEndKey(final byte[] endKey) { - if ((this.endKey == null) || (Bytes.compareTo(endKey, this.endKey) > 0)) { - this.endKey = endKey; - } - } - } - - /** - * The partition id that consists of start key and date of the mob file name. - */ - public static class CompactionPartitionId { - private String startKey; - private String date; - private String latestDate; - private long threshold; - - public CompactionPartitionId() { - // initialize these fields to empty string - this.startKey = ""; - this.date = ""; - this.latestDate = ""; - this.threshold = 0; - } - - public CompactionPartitionId(String startKey, String date) { - if (startKey == null || date == null) { - throw new IllegalArgumentException("Neither of start key and date could be null"); - } - this.startKey = startKey; - this.date = date; - this.latestDate = ""; - this.threshold = 0; - } - - public void setThreshold (final long threshold) { - this.threshold = threshold; - } - - public long getThreshold () { - return this.threshold; - } - - public String getStartKey() { - return this.startKey; - } - - public void setStartKey(final String startKey) { - this.startKey = startKey; - } - - public String getDate() { - return this.date; - } - - public void setDate(final String date) { - this.date = date; - } - - public String getLatestDate () { return this.latestDate; } - - public void updateLatestDate(final String latestDate) { - if (this.latestDate.compareTo(latestDate) < 0) { - this.latestDate = latestDate; - } - } - - @Override - public int hashCode() { - int result = 17; - result = 31 * result + startKey.hashCode(); - result = 31 * result + date.hashCode(); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (!(obj instanceof CompactionPartitionId)) { - return false; - } - CompactionPartitionId another = (CompactionPartitionId) obj; - if (!this.startKey.equals(another.startKey)) { - return false; - } - if (!this.date.equals(another.date)) { - return false; - } - return true; - } - - @Override - public String toString() { - return new StringBuilder(startKey).append(date).toString(); - } - } - - /** - * The delete file partition in the mob compaction. - * The delete partition is defined as [startKey, endKey] pair. - * The mob delete files that have the same start key and end key belong to - * the same partition. - */ - protected static class CompactionDelPartition { - private List delFiles = new ArrayList(); - private List storeFiles = new ArrayList<>(); - private CompactionDelPartitionId id; - - public CompactionDelPartition(CompactionDelPartitionId id) { - this.id = id; - } - - public CompactionDelPartitionId getId() { - return this.id; - } - - void addDelFile(FileStatus file) { - delFiles.add(file.getPath()); - } - public void addStoreFile(HStoreFile file) { - storeFiles.add(file); - } - - public List getStoreFiles() { - return storeFiles; - } - - List listDelFiles() { - return Collections.unmodifiableList(delFiles); - } - - void addDelFileList(final Collection list) { - delFiles.addAll(list); - } - - int getDelFileCount () { - return delFiles.size(); - } - - void cleanDelFiles() { - delFiles.clear(); - } - } - - /** - * The delete partition id that consists of start key and end key - */ - public static class CompactionDelPartitionId implements Comparable { - private byte[] startKey; - private byte[] endKey; - - public CompactionDelPartitionId() { - } - - public CompactionDelPartitionId(final byte[] startKey, final byte[] endKey) { - this.startKey = startKey; - this.endKey = endKey; - } - - public byte[] getStartKey() { - return this.startKey; - } - public void setStartKey(final byte[] startKey) { - this.startKey = startKey; - } - - public byte[] getEndKey() { - return this.endKey; - } - public void setEndKey(final byte[] endKey) { - this.endKey = endKey; - } - - @Override - public int compareTo(CompactionDelPartitionId o) { - /* - * 1). Compare the start key, if the k1 < k2, then k1 is less - * 2). If start Key is same, check endKey, k1 < k2, k1 is less - * If both are same, then they are equal. - */ - int result = Bytes.compareTo(this.startKey, o.getStartKey()); - if (result != 0) { - return result; - } - - return Bytes.compareTo(this.endKey, o.getEndKey()); - } - - @Override - public int hashCode() { - int result = 17; - result = 31 * result + java.util.Arrays.hashCode(startKey); - result = 31 * result + java.util.Arrays.hashCode(endKey); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (!(obj instanceof CompactionDelPartitionId)) { - return false; - } - CompactionDelPartitionId another = (CompactionDelPartitionId) obj; - - return (this.compareTo(another) == 0); - } - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java deleted file mode 100644 index a5823ec7d7bb..000000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java +++ /dev/null @@ -1,929 +0,0 @@ -/** - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.mob.compactions; - -import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TIME_KEY; -import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_CELLS_COUNT; -import static org.apache.hadoop.hbase.regionserver.HStoreFile.SKIP_RESET_SEQ_ID; - -import java.io.FileNotFoundException; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Calendar; -import java.util.Collection; -import java.util.Collections; -import java.util.Comparator; -import java.util.Date; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.NavigableMap; -import java.util.Objects; -import java.util.TreeMap; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.ArrayBackedTag; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellComparator; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.Tag; -import org.apache.hadoop.hbase.TagType; -import org.apache.hadoop.hbase.TagUtil; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.io.HFileLink; -import org.apache.hadoop.hbase.io.crypto.Encryption; -import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hadoop.hbase.io.hfile.HFile; -import org.apache.hadoop.hbase.io.hfile.HFile.Reader; -import org.apache.hadoop.hbase.mob.MobConstants; -import org.apache.hadoop.hbase.mob.MobFileName; -import org.apache.hadoop.hbase.mob.MobUtils; -import org.apache.hadoop.hbase.mob.compactions.MobCompactionRequest.CompactionType; -import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionDelPartition; -import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionDelPartitionId; -import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartition; -import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId; -import org.apache.hadoop.hbase.regionserver.BloomType; -import org.apache.hadoop.hbase.regionserver.HStore; -import org.apache.hadoop.hbase.regionserver.HStoreFile; -import org.apache.hadoop.hbase.regionserver.ScanInfo; -import org.apache.hadoop.hbase.regionserver.ScanType; -import org.apache.hadoop.hbase.regionserver.ScannerContext; -import org.apache.hadoop.hbase.regionserver.StoreFileInfo; -import org.apache.hadoop.hbase.regionserver.StoreFileScanner; -import org.apache.hadoop.hbase.regionserver.StoreFileWriter; -import org.apache.hadoop.hbase.regionserver.StoreScanner; -import org.apache.hadoop.hbase.security.EncryptionUtil; -import org.apache.hadoop.hbase.tool.BulkLoadHFiles; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - -/** - * An implementation of {@link MobCompactor} that compacts the mob files in partitions. - */ -@InterfaceAudience.Private -public class PartitionedMobCompactor extends MobCompactor { - - private static final Logger LOG = LoggerFactory.getLogger(PartitionedMobCompactor.class); - protected long mergeableSize; - protected int delFileMaxCount; - /** The number of files compacted in a batch */ - protected int compactionBatchSize; - protected int compactionKVMax; - - private final Path tempPath; - private final Path bulkloadPath; - private final CacheConfig compactionCacheConfig; - private final byte[] refCellTags; - private Encryption.Context cryptoContext = Encryption.Context.NONE; - - public PartitionedMobCompactor(Configuration conf, FileSystem fs, TableName tableName, - ColumnFamilyDescriptor column, ExecutorService pool) throws IOException { - super(conf, fs, tableName, column, pool); - mergeableSize = conf.getLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, - MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD); - delFileMaxCount = conf.getInt(MobConstants.MOB_DELFILE_MAX_COUNT, - MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT); - // default is 100 - compactionBatchSize = conf.getInt(MobConstants.MOB_COMPACTION_BATCH_SIZE, - MobConstants.DEFAULT_MOB_COMPACTION_BATCH_SIZE); - tempPath = new Path(MobUtils.getMobHome(conf), MobConstants.TEMP_DIR_NAME); - bulkloadPath = new Path(tempPath, new Path(MobConstants.BULKLOAD_DIR_NAME, new Path( - tableName.getNamespaceAsString(), tableName.getQualifierAsString()))); - compactionKVMax = this.conf.getInt(HConstants.COMPACTION_KV_MAX, - HConstants.COMPACTION_KV_MAX_DEFAULT); - Configuration copyOfConf = new Configuration(conf); - copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f); - compactionCacheConfig = new CacheConfig(copyOfConf); - List tags = new ArrayList<>(2); - tags.add(MobConstants.MOB_REF_TAG); - Tag tableNameTag = new ArrayBackedTag(TagType.MOB_TABLE_NAME_TAG_TYPE, tableName.getName()); - tags.add(tableNameTag); - this.refCellTags = TagUtil.fromList(tags); - cryptoContext = EncryptionUtil.createEncryptionContext(copyOfConf, column); - } - - @Override - public List compact(List files, boolean allFiles) throws IOException { - if (files == null || files.isEmpty()) { - LOG.info("No candidate mob files"); - return null; - } - LOG.info("is allFiles: " + allFiles); - - // find the files to compact. - PartitionedMobCompactionRequest request = select(files, allFiles); - // compact the files. - return performCompaction(request); - } - - /** - * Selects the compacted mob/del files. - * Iterates the candidates to find out all the del files and small mob files. - * @param candidates All the candidates. - * @param allFiles Whether add all mob files into the compaction. - * @return A compaction request. - * @throws IOException if IO failure is encountered - */ - protected PartitionedMobCompactionRequest select(List candidates, - boolean allFiles) throws IOException { - final Map filesToCompact = new HashMap<>(); - final CompactionPartitionId id = new CompactionPartitionId(); - final NavigableMap delFilesToCompact = new TreeMap<>(); - final CompactionDelPartitionId delId = new CompactionDelPartitionId(); - final ArrayList allDelPartitions = new ArrayList<>(); - int selectedFileCount = 0; - int irrelevantFileCount = 0; - int totalDelFiles = 0; - MobCompactPartitionPolicy policy = column.getMobCompactPartitionPolicy(); - - Calendar calendar = Calendar.getInstance(); - Date currentDate = new Date(); - Date firstDayOfCurrentMonth = null; - Date firstDayOfCurrentWeek = null; - - if (policy == MobCompactPartitionPolicy.MONTHLY) { - firstDayOfCurrentMonth = MobUtils.getFirstDayOfMonth(calendar, currentDate); - firstDayOfCurrentWeek = MobUtils.getFirstDayOfWeek(calendar, currentDate); - } else if (policy == MobCompactPartitionPolicy.WEEKLY) { - firstDayOfCurrentWeek = MobUtils.getFirstDayOfWeek(calendar, currentDate); - } - - // We check if there is any del files so the logic can be optimized for the following processing - // First step is to check if there is any delete files. If there is any delete files, - // For each Partition, it needs to read its startKey and endKey from files. - // If there is no delete file, there is no need to read startKey and endKey from files, this - // is an optimization. - boolean withDelFiles = false; - for (FileStatus file : candidates) { - if (!file.isFile()) { - continue; - } - // group the del files and small files. - FileStatus linkedFile = file; - if (HFileLink.isHFileLink(file.getPath())) { - HFileLink link = HFileLink.buildFromHFileLinkPattern(conf, file.getPath()); - linkedFile = getLinkedFileStatus(link); - if (linkedFile == null) { - continue; - } - } - if (StoreFileInfo.isDelFile(linkedFile.getPath())) { - withDelFiles = true; - break; - } - } - - for (FileStatus file : candidates) { - if (!file.isFile()) { - irrelevantFileCount++; - continue; - } - // group the del files and small files. - FileStatus linkedFile = file; - if (HFileLink.isHFileLink(file.getPath())) { - HFileLink link = HFileLink.buildFromHFileLinkPattern(conf, file.getPath()); - linkedFile = getLinkedFileStatus(link); - if (linkedFile == null) { - // If the linked file cannot be found, regard it as an irrelevantFileCount file - irrelevantFileCount++; - continue; - } - } - if (withDelFiles && StoreFileInfo.isDelFile(linkedFile.getPath())) { - // File in the Del Partition List - - // Get delId from the file - try (Reader reader = HFile.createReader(fs, linkedFile.getPath(), conf)) { - delId.setStartKey(reader.getFirstRowKey().get()); - delId.setEndKey(reader.getLastRowKey().get()); - } - CompactionDelPartition delPartition = delFilesToCompact.get(delId); - if (delPartition == null) { - CompactionDelPartitionId newDelId = - new CompactionDelPartitionId(delId.getStartKey(), delId.getEndKey()); - delPartition = new CompactionDelPartition(newDelId); - delFilesToCompact.put(newDelId, delPartition); - } - delPartition.addDelFile(file); - totalDelFiles ++; - } else { - String fileName = linkedFile.getPath().getName(); - String date = MobFileName.getDateFromName(fileName); - boolean skipCompaction = MobUtils - .fillPartitionId(id, firstDayOfCurrentMonth, firstDayOfCurrentWeek, date, policy, - calendar, mergeableSize); - if (allFiles || (!skipCompaction && (linkedFile.getLen() < id.getThreshold()))) { - // add all files if allFiles is true, - // otherwise add the small files to the merge pool - // filter out files which are not supposed to be compacted with the - // current policy - - id.setStartKey(MobFileName.getStartKeyFromName(fileName)); - CompactionPartition compactionPartition = filesToCompact.get(id); - if (compactionPartition == null) { - CompactionPartitionId newId = new CompactionPartitionId(id.getStartKey(), id.getDate()); - compactionPartition = new CompactionPartition(newId); - compactionPartition.addFile(file); - filesToCompact.put(newId, compactionPartition); - newId.updateLatestDate(date); - } else { - compactionPartition.addFile(file); - compactionPartition.getPartitionId().updateLatestDate(date); - } - - if (withDelFiles) { - // get startKey and endKey from the file and update partition - // TODO: is it possible to skip read of most hfiles? - try (Reader reader = HFile.createReader(fs, linkedFile.getPath(), conf)) { - compactionPartition.setStartKey(reader.getFirstRowKey().get()); - compactionPartition.setEndKey(reader.getLastRowKey().get()); - } - } - - selectedFileCount++; - } - } - } - - /* - * Merge del files so there are only non-overlapped del file lists - */ - for(Map.Entry entry : delFilesToCompact.entrySet()) { - if (allDelPartitions.size() > 0) { - // check if the current key range overlaps the previous one - CompactionDelPartition prev = allDelPartitions.get(allDelPartitions.size() - 1); - if (Bytes.compareTo(prev.getId().getEndKey(), entry.getKey().getStartKey()) >= 0) { - // merge them together - prev.getId().setEndKey(entry.getValue().getId().getEndKey()); - prev.addDelFileList(entry.getValue().listDelFiles()); - - } else { - allDelPartitions.add(entry.getValue()); - } - } else { - allDelPartitions.add(entry.getValue()); - } - } - - PartitionedMobCompactionRequest request = new PartitionedMobCompactionRequest( - filesToCompact.values(), allDelPartitions); - if (candidates.size() == (totalDelFiles + selectedFileCount + irrelevantFileCount)) { - // all the files are selected - request.setCompactionType(CompactionType.ALL_FILES); - } - LOG.info("The compaction type is " + request.getCompactionType() + ", the request has " - + totalDelFiles + " del files, " + selectedFileCount + " selected files, and " - + irrelevantFileCount + " irrelevant files"); - return request; - } - - /** - * Performs the compaction on the selected files. - *
      - *
    1. Compacts the del files.
    2. - *
    3. Compacts the selected small mob files and all the del files.
    4. - *
    5. If all the candidates are selected, delete the del files.
    6. - *
    - * @param request The compaction request. - * @return The paths of new mob files generated in the compaction. - * @throws IOException if IO failure is encountered - */ - protected List performCompaction(PartitionedMobCompactionRequest request) - throws IOException { - - // merge the del files, it is per del partition - for (CompactionDelPartition delPartition : request.getDelPartitions()) { - if (delPartition.getDelFileCount() <= 1) continue; - List newDelPaths = compactDelFiles(request, delPartition.listDelFiles()); - delPartition.cleanDelFiles(); - delPartition.addDelFileList(newDelPaths); - } - - List paths = null; - int totalDelFileCount = 0; - try { - for (CompactionDelPartition delPartition : request.getDelPartitions()) { - for (Path newDelPath : delPartition.listDelFiles()) { - HStoreFile sf = - new HStoreFile(fs, newDelPath, conf, compactionCacheConfig, BloomType.NONE, true); - // pre-create reader of a del file to avoid race condition when opening the reader in each - // partition. - sf.initReader(); - delPartition.addStoreFile(sf); - totalDelFileCount++; - } - } - LOG.info("After merging, there are " + totalDelFileCount + " del files"); - // compact the mob files by partitions. - paths = compactMobFiles(request); - LOG.info("After compaction, there are " + paths.size() + " mob files"); - } finally { - for (CompactionDelPartition delPartition : request.getDelPartitions()) { - closeStoreFileReaders(delPartition.getStoreFiles()); - } - } - - // archive the del files if all the mob files are selected. - if (request.type == CompactionType.ALL_FILES && !request.getDelPartitions().isEmpty()) { - LOG.info( - "After a mob compaction with all files selected, archiving the del files "); - for (CompactionDelPartition delPartition : request.getDelPartitions()) { - LOG.info(Objects.toString(delPartition.listDelFiles())); - try { - MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), - delPartition.getStoreFiles()); - } catch (IOException e) { - LOG.error("Failed to archive the del files " + delPartition.getStoreFiles(), e); - } - } - } - return paths; - } - - static class DelPartitionComparator implements Comparator { - private boolean compareStartKey; - - DelPartitionComparator(boolean compareStartKey) { - this.compareStartKey = compareStartKey; - } - - public boolean getCompareStartKey() { - return this.compareStartKey; - } - - public void setCompareStartKey(final boolean compareStartKey) { - this.compareStartKey = compareStartKey; - } - - @Override - public int compare(CompactionDelPartition o1, CompactionDelPartition o2) { - - if (compareStartKey) { - return Bytes.compareTo(o1.getId().getStartKey(), o2.getId().getStartKey()); - } else { - return Bytes.compareTo(o1.getId().getEndKey(), o2.getId().getEndKey()); - } - } - } - - @VisibleForTesting - List getListOfDelFilesForPartition(final CompactionPartition partition, - final List delPartitions) { - // Binary search for startKey and endKey - - List result = new ArrayList<>(); - - DelPartitionComparator comparator = new DelPartitionComparator(false); - CompactionDelPartitionId id = new CompactionDelPartitionId(null, partition.getStartKey()); - CompactionDelPartition target = new CompactionDelPartition(id); - int start = Collections.binarySearch(delPartitions, target, comparator); - - // Get the start index for partition - if (start < 0) { - // Calculate the insert point - start = (start + 1) * (-1); - if (start == delPartitions.size()) { - // no overlap - return result; - } else { - // Check another case which has no overlap - if (Bytes.compareTo(partition.getEndKey(), delPartitions.get(start).getId().getStartKey()) < 0) { - return result; - } - } - } - - // Search for end index for the partition - comparator.setCompareStartKey(true); - id.setStartKey(partition.getEndKey()); - int end = Collections.binarySearch(delPartitions, target, comparator); - - if (end < 0) { - end = (end + 1) * (-1); - if (end == 0) { - return result; - } else { - --end; - if (Bytes.compareTo(partition.getStartKey(), delPartitions.get(end).getId().getEndKey()) > 0) { - return result; - } - } - } - - for (int i = start; i <= end; ++i) { - result.addAll(delPartitions.get(i).getStoreFiles()); - } - - return result; - } - - /** - * Compacts the selected small mob files and all the del files. - * @param request The compaction request. - * @return The paths of new mob files after compactions. - * @throws IOException if IO failure is encountered - */ - protected List compactMobFiles(final PartitionedMobCompactionRequest request) - throws IOException { - Collection partitions = request.compactionPartitions; - if (partitions == null || partitions.isEmpty()) { - LOG.info("No partitions of mob files"); - return Collections.emptyList(); - } - List paths = new ArrayList<>(); - final Connection c = ConnectionFactory.createConnection(conf); - final Table table = c.getTable(tableName); - - try { - Map>> results = new HashMap<>(); - // compact the mob files by partitions in parallel. - for (final CompactionPartition partition : partitions) { - - // How to efficiently come up a list of delFiles for one partition? - // Search the delPartitions and collect all the delFiles for the partition - // One optimization can do is that if there is no del file, we do not need to - // come up with startKey/endKey. - List delFiles = getListOfDelFilesForPartition(partition, - request.getDelPartitions()); - - results.put(partition.getPartitionId(), pool.submit(new Callable>() { - @Override - public List call() throws Exception { - LOG.info("Compacting mob files for partition " + partition.getPartitionId()); - return compactMobFilePartition(request, partition, delFiles, c, table); - } - })); - } - // compact the partitions in parallel. - List failedPartitions = new ArrayList<>(); - for (Entry>> result : results.entrySet()) { - try { - paths.addAll(result.getValue().get()); - } catch (Exception e) { - // just log the error - LOG.error("Failed to compact the partition " + result.getKey(), e); - failedPartitions.add(result.getKey()); - } - } - if (!failedPartitions.isEmpty()) { - // if any partition fails in the compaction, directly throw an exception. - throw new IOException("Failed to compact the partitions " + failedPartitions); - } - } finally { - try { - table.close(); - } catch (IOException e) { - LOG.error("Failed to close the Table", e); - } - } - return paths; - } - - /** - * Compacts a partition of selected small mob files and all the del files. - * @param request The compaction request. - * @param partition A compaction partition. - * @param delFiles The del files. - * @param connection The connection to use. - * @param table The current table. - * @return The paths of new mob files after compactions. - * @throws IOException if IO failure is encountered - */ - private List compactMobFilePartition(PartitionedMobCompactionRequest request, - CompactionPartition partition, - List delFiles, - Connection connection, - Table table) throws IOException { - if (MobUtils.isMobFileExpired(column, EnvironmentEdgeManager.currentTime(), - partition.getPartitionId().getDate())) { - // If the files in the partition are expired, do not compact them and directly - // return an empty list. - return Collections.emptyList(); - } - List newFiles = new ArrayList<>(); - List files = partition.listFiles(); - int offset = 0; - Path bulkloadPathOfPartition = new Path(bulkloadPath, partition.getPartitionId().toString()); - Path bulkloadColumnPath = new Path(bulkloadPathOfPartition, column.getNameAsString()); - while (offset < files.size()) { - int batch = compactionBatchSize; - if (files.size() - offset < compactionBatchSize) { - batch = files.size() - offset; - } - if (batch == 1 && delFiles.isEmpty()) { - // only one file left and no del files, do not compact it, - // and directly add it to the new files. - newFiles.add(files.get(offset).getPath()); - offset++; - continue; - } - // clean the bulkload directory to avoid loading old files. - fs.delete(bulkloadPathOfPartition, true); - // add the selected mob files and del files into filesToCompact - List filesToCompact = new ArrayList<>(); - for (int i = offset; i < batch + offset; i++) { - HStoreFile sf = new HStoreFile(fs, files.get(i).getPath(), conf, compactionCacheConfig, - BloomType.NONE, true); - filesToCompact.add(sf); - } - filesToCompact.addAll(delFiles); - // compact the mob files in a batch. - compactMobFilesInBatch(request, partition, connection, table, filesToCompact, batch, - bulkloadPathOfPartition, bulkloadColumnPath, newFiles); - // move to the next batch. - offset += batch; - } - LOG.info("Compaction is finished. The number of mob files is changed from " + files.size() - + " to " + newFiles.size()); - return newFiles; - } - - /** - * Closes the readers of store files. - * @param storeFiles The store files to be closed. - */ - private void closeStoreFileReaders(List storeFiles) { - for (HStoreFile storeFile : storeFiles) { - try { - storeFile.closeStoreFile(true); - } catch (IOException e) { - LOG.warn("Failed to close the reader on store file " + storeFile.getPath(), e); - } - } - } - - /** - * Compacts a partition of selected small mob files and all the del files in a batch. - * @param request The compaction request. - * @param partition A compaction partition. - * @param connection To use for transport - * @param table The current table. - * @param filesToCompact The files to be compacted. - * @param batch The number of mob files to be compacted in a batch. - * @param bulkloadPathOfPartition The directory where the bulkload column of the current - * partition is saved. - * @param bulkloadColumnPath The directory where the bulkload files of current partition - * are saved. - * @param newFiles The paths of new mob files after compactions. - * @throws IOException if IO failure is encountered - */ - private void compactMobFilesInBatch(PartitionedMobCompactionRequest request, - CompactionPartition partition, - Connection connection, Table table, - List filesToCompact, int batch, - Path bulkloadPathOfPartition, Path bulkloadColumnPath, - List newFiles) - throws IOException { - // open scanner to the selected mob files and del files. - StoreScanner scanner = createScanner(filesToCompact, ScanType.COMPACT_DROP_DELETES); - // the mob files to be compacted, not include the del files. - List mobFilesToCompact = filesToCompact.subList(0, batch); - // Pair(maxSeqId, cellsCount) - Pair fileInfo = getFileInfo(mobFilesToCompact); - // open writers for the mob files and new ref store files. - StoreFileWriter writer = null; - StoreFileWriter refFileWriter = null; - Path filePath = null; - long mobCells = 0; - boolean cleanupTmpMobFile = false; - boolean cleanupBulkloadDirOfPartition = false; - boolean cleanupCommittedMobFile = false; - boolean closeReaders= true; - - try { - try { - writer = MobUtils - .createWriter(conf, fs, column, partition.getPartitionId().getLatestDate(), tempPath, - Long.MAX_VALUE, column.getCompactionCompressionType(), - partition.getPartitionId().getStartKey(), compactionCacheConfig, cryptoContext, - true); - cleanupTmpMobFile = true; - filePath = writer.getPath(); - byte[] fileName = Bytes.toBytes(filePath.getName()); - // create a temp file and open a writer for it in the bulkloadPath - refFileWriter = MobUtils.createRefFileWriter(conf, fs, column, bulkloadColumnPath, - fileInfo.getSecond().longValue(), compactionCacheConfig, cryptoContext, true); - cleanupBulkloadDirOfPartition = true; - List cells = new ArrayList<>(); - boolean hasMore; - ScannerContext scannerContext = - ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build(); - do { - hasMore = scanner.next(cells, scannerContext); - for (Cell cell : cells) { - // write the mob cell to the mob file. - writer.append(cell); - // write the new reference cell to the store file. - Cell reference = MobUtils.createMobRefCell(cell, fileName, this.refCellTags); - refFileWriter.append(reference); - mobCells++; - } - cells.clear(); - } while (hasMore); - } finally { - // close the scanner. - scanner.close(); - - if (cleanupTmpMobFile) { - // append metadata to the mob file, and close the mob file writer. - closeMobFileWriter(writer, fileInfo.getFirst(), mobCells); - } - - if (cleanupBulkloadDirOfPartition) { - // append metadata and bulkload info to the ref mob file, and close the writer. - closeRefFileWriter(refFileWriter, fileInfo.getFirst(), request.selectionTime); - } - } - - if (mobCells > 0) { - // commit mob file - MobUtils.commitFile(conf, fs, filePath, mobFamilyDir, compactionCacheConfig); - cleanupTmpMobFile = false; - cleanupCommittedMobFile = true; - // bulkload the ref file - bulkloadRefFile(table.getName(), bulkloadPathOfPartition, filePath.getName()); - cleanupCommittedMobFile = false; - newFiles.add(new Path(mobFamilyDir, filePath.getName())); - } - - // archive the old mob files, do not archive the del files. - try { - closeStoreFileReaders(mobFilesToCompact); - closeReaders = false; - MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), mobFilesToCompact); - } catch (IOException e) { - LOG.error("Failed to archive the files " + mobFilesToCompact, e); - } - } finally { - if (closeReaders) { - closeStoreFileReaders(mobFilesToCompact); - } - - if (cleanupTmpMobFile) { - deletePath(filePath); - } - - if (cleanupBulkloadDirOfPartition) { - // delete the bulkload files in bulkloadPath - deletePath(bulkloadPathOfPartition); - } - - if (cleanupCommittedMobFile) { - deletePath(new Path(mobFamilyDir, filePath.getName())); - } - } - } - - /** - * Compacts the del files in batches which avoids opening too many files. - * @param request The compaction request. - * @param delFilePaths Del file paths to compact - * @return The paths of new del files after merging or the original files if no merging - * is necessary. - * @throws IOException if IO failure is encountered - */ - protected List compactDelFiles(PartitionedMobCompactionRequest request, - List delFilePaths) throws IOException { - if (delFilePaths.size() <= delFileMaxCount) { - return delFilePaths; - } - // when there are more del files than the number that is allowed, merge it firstly. - int offset = 0; - List paths = new ArrayList<>(); - while (offset < delFilePaths.size()) { - // get the batch - int batch = compactionBatchSize; - if (delFilePaths.size() - offset < compactionBatchSize) { - batch = delFilePaths.size() - offset; - } - List batchedDelFiles = new ArrayList<>(); - if (batch == 1) { - // only one file left, do not compact it, directly add it to the new files. - paths.add(delFilePaths.get(offset)); - offset++; - continue; - } - for (int i = offset; i < batch + offset; i++) { - batchedDelFiles.add(new HStoreFile(fs, delFilePaths.get(i), conf, compactionCacheConfig, - BloomType.NONE, true)); - } - // compact the del files in a batch. - paths.add(compactDelFilesInBatch(request, batchedDelFiles)); - // move to the next batch. - offset += batch; - } - return compactDelFiles(request, paths); - } - - /** - * Compacts the del file in a batch. - * @param request The compaction request. - * @param delFiles The del files. - * @return The path of new del file after merging. - * @throws IOException if IO failure is encountered - */ - private Path compactDelFilesInBatch(PartitionedMobCompactionRequest request, - List delFiles) throws IOException { - // create a scanner for the del files. - StoreScanner scanner = createScanner(delFiles, ScanType.COMPACT_RETAIN_DELETES); - StoreFileWriter writer = null; - Path filePath = null; - try { - writer = MobUtils.createDelFileWriter(conf, fs, column, - MobUtils.formatDate(new Date(request.selectionTime)), tempPath, Long.MAX_VALUE, - column.getCompactionCompressionType(), HConstants.EMPTY_START_ROW, compactionCacheConfig, - cryptoContext); - filePath = writer.getPath(); - List cells = new ArrayList<>(); - boolean hasMore; - ScannerContext scannerContext = - ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build(); - do { - hasMore = scanner.next(cells, scannerContext); - for (Cell cell : cells) { - writer.append(cell); - } - cells.clear(); - } while (hasMore); - } finally { - scanner.close(); - if (writer != null) { - try { - writer.close(); - } catch (IOException e) { - LOG.error("Failed to close the writer of the file " + filePath, e); - } - } - } - // commit the new del file - Path path = MobUtils.commitFile(conf, fs, filePath, mobFamilyDir, compactionCacheConfig); - // archive the old del files - try { - MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), delFiles); - } catch (IOException e) { - LOG.error("Failed to archive the old del files " + delFiles, e); - } - return path; - } - - /** - * Creates a store scanner. - * @param filesToCompact The files to be compacted. - * @param scanType The scan type. - * @return The store scanner. - * @throws IOException if IO failure is encountered - */ - private StoreScanner createScanner(List filesToCompact, ScanType scanType) - throws IOException { - List scanners = StoreFileScanner.getScannersForStoreFiles(filesToCompact, - false, true, false, false, HConstants.LATEST_TIMESTAMP); - long ttl = HStore.determineTTLFromFamily(column); - ScanInfo scanInfo = new ScanInfo(conf, column, ttl, 0, CellComparator.getInstance()); - return new StoreScanner(scanInfo, scanType, scanners); - } - - /** - * Bulkloads the current file. - * @param tableName The table to load into. - * @param bulkloadDirectory The path of bulkload directory. - * @param fileName The current file name. - * @throws IOException if IO failure is encountered - */ - private void bulkloadRefFile(TableName tableName, Path bulkloadDirectory, String fileName) - throws IOException { - // bulkload the ref file - try { - BulkLoadHFiles.create(conf).bulkLoad(tableName, bulkloadDirectory); - } catch (Exception e) { - throw new IOException(e); - } - } - - /** - * Closes the mob file writer. - * @param writer The mob file writer. - * @param maxSeqId Maximum sequence id. - * @param mobCellsCount The number of mob cells. - * @throws IOException if IO failure is encountered - */ - private void closeMobFileWriter(StoreFileWriter writer, long maxSeqId, long mobCellsCount) - throws IOException { - if (writer != null) { - writer.appendMetadata(maxSeqId, false, mobCellsCount); - try { - writer.close(); - } catch (IOException e) { - LOG.error("Failed to close the writer of the file " + writer.getPath(), e); - } - } - } - - /** - * Closes the ref file writer. - * @param writer The ref file writer. - * @param maxSeqId Maximum sequence id. - * @param bulkloadTime The timestamp at which the bulk load file is created. - * @throws IOException if IO failure is encountered - */ - private void closeRefFileWriter(StoreFileWriter writer, long maxSeqId, long bulkloadTime) - throws IOException { - if (writer != null) { - writer.appendMetadata(maxSeqId, false); - writer.appendFileInfo(BULKLOAD_TIME_KEY, Bytes.toBytes(bulkloadTime)); - writer.appendFileInfo(SKIP_RESET_SEQ_ID, Bytes.toBytes(true)); - try { - writer.close(); - } catch (IOException e) { - LOG.error("Failed to close the writer of the ref file " + writer.getPath(), e); - } - } - } - - /** - * Gets the max seqId and number of cells of the store files. - * @param storeFiles The store files. - * @return The pair of the max seqId and number of cells of the store files. - * @throws IOException if IO failure is encountered - */ - private Pair getFileInfo(List storeFiles) throws IOException { - long maxSeqId = 0; - long maxKeyCount = 0; - for (HStoreFile sf : storeFiles) { - // the readers will be closed later after the merge. - maxSeqId = Math.max(maxSeqId, sf.getMaxSequenceId()); - sf.initReader(); - byte[] count = sf.getReader().loadFileInfo().get(MOB_CELLS_COUNT); - if (count != null) { - maxKeyCount += Bytes.toLong(count); - } - } - return new Pair<>(maxSeqId, maxKeyCount); - } - - /** - * Deletes a file. - * @param path The path of the file to be deleted. - */ - private void deletePath(Path path) { - try { - if (path != null) { - fs.delete(path, true); - } - } catch (IOException e) { - LOG.error("Failed to delete the file " + path, e); - } - } - - private FileStatus getLinkedFileStatus(HFileLink link) throws IOException { - Path[] locations = link.getLocations(); - FileStatus file; - for (Path location : locations) { - - if (location != null) { - try { - file = fs.getFileStatus(location); - if (file != null) { - return file; - } - } catch (FileNotFoundException e) { - } - } - } - LOG.warn("The file " + link + " links to can not be found"); - return null; - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java index b8ea9609e2bc..8f7b69068aa3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java @@ -193,28 +193,6 @@ public StoreFileWriter createWriterInTmp(Date date, long maxKeyCount, isCompaction); } - /** - * Creates the writer for the del file in temp directory. - * The del file keeps tracking the delete markers. Its name has a suffix _del, - * the format is [0-9a-f]+(_del)?. - * @param date The latest date of written cells. - * @param maxKeyCount The key count. - * @param compression The compression algorithm. - * @param startKey The start key. - * @return The writer for the del file. - * @throws IOException - */ - public StoreFileWriter createDelFileWriterInTmp(Date date, long maxKeyCount, - Compression.Algorithm compression, byte[] startKey) throws IOException { - if (startKey == null) { - startKey = HConstants.EMPTY_START_ROW; - } - Path path = getTempDir(); - String suffix = UUID - .randomUUID().toString().replaceAll("-", "") + "_del"; - MobFileName mobFileName = MobFileName.create(startKey, MobUtils.formatDate(date), suffix); - return createWriterInTmp(mobFileName, path, maxKeyCount, compression, true); - } /** * Creates the writer for the mob file in temp directory. @@ -231,7 +209,7 @@ public StoreFileWriter createWriterInTmp(String date, Path basePath, long maxKey Compression.Algorithm compression, byte[] startKey, boolean isCompaction) throws IOException { MobFileName mobFileName = MobFileName.create(startKey, date, UUID.randomUUID() - .toString().replaceAll("-", "")); + .toString().replaceAll("-", ""), region.getRegionInfo().getEncodedName()); return createWriterInTmp(mobFileName, basePath, maxKeyCount, compression, isCompaction); } @@ -265,7 +243,7 @@ public void commitFile(final Path sourceFile, Path targetPath) throws IOExceptio } Path dstPath = new Path(targetPath, sourceFile.getName()); validateMobFile(sourceFile); - String msg = "Renaming flushed file from " + sourceFile + " to " + dstPath; + String msg = " FLUSH Renaming flushed file from " + sourceFile + " to " + dstPath; LOG.info(msg); Path parent = dstPath.getParent(); if (!region.getFilesystem().exists(parent)) { @@ -309,6 +287,20 @@ public MobCell resolve(Cell reference, boolean cacheBlocks) throws IOException { return resolve(reference, cacheBlocks, -1, true); } + /** + * Reads the cell from the mob file with readEmptyValueOnMobCellMiss + * @param reference The cell found in the HBase, its value is a path to a mob file. + * @param cacheBlocks Whether the scanner should cache blocks. + * @param readEmptyValueOnMobCellMiss should return empty mob cell if reference + * can not be resolved. + * @return The cell found in the mob file. + * @throws IOException + */ + public MobCell resolve(Cell reference, boolean cacheBlocks, boolean readEmptyValueOnMobCellMiss) + throws IOException { + return resolve(reference, cacheBlocks, -1, readEmptyValueOnMobCellMiss); + } + /** * Reads the cell from the mob file. * @param reference The cell found in the HBase, its value is a path to a mob file. @@ -520,4 +512,6 @@ public long getMobScanCellsSize() { public byte[] getRefCellTags() { return this.refCellTags; } + + } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index b5e64673146f..5138e05ea1a1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -1905,7 +1905,7 @@ public void cancelRequestedCompaction(CompactionContext compaction) { finishCompactionRequest(compaction.getRequest()); } - private void finishCompactionRequest(CompactionRequestImpl cr) { + protected void finishCompactionRequest(CompactionRequestImpl cr) { this.region.reportCompactionRequestEnd(cr.isMajor(), cr.getFiles().size(), cr.getSize()); if (cr.isOffPeak()) { offPeakCompactionTracker.set(false); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java index c14e3a8a3b7e..2346fd886fee 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java @@ -111,6 +111,9 @@ public class HStoreFile implements StoreFile { /** Key for the number of mob cells in metadata */ public static final byte[] MOB_CELLS_COUNT = Bytes.toBytes("MOB_CELLS_COUNT"); + /** Key for the list of MOB file references */ + public static final byte[] MOB_FILE_REFS = Bytes.toBytes("MOB_FILE_REFS"); + /** Meta key set when store file is a result of a bulk load */ public static final byte[] BULKLOAD_TASK_KEY = Bytes.toBytes("BULKLOAD_SOURCE_TASK"); public static final byte[] BULKLOAD_TIME_KEY = Bytes.toBytes("BULKLOAD_TIMESTAMP"); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java index 76144f08d5dc..61cd34c6d235 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java @@ -114,4 +114,5 @@ public void close() { LOG.warn("Failed to free referenced mob cells: ", e); } } + } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java index 578f80688560..7af748ce38fe 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java @@ -30,15 +30,16 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HDFSBlocksDistribution; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.io.HalfStoreFileReader; import org.apache.hadoop.hbase.io.Reference; import org.apache.hadoop.hbase.io.hfile.CacheConfig; +import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Describe a StoreFile (hfile, reference, link) @@ -141,7 +142,7 @@ private StoreFileInfo(final Configuration conf, final FileSystem fs, } if (LOG.isTraceEnabled()) LOG.trace(p + " is a " + reference.getFileRegion() + " reference to " + referencePath); - } else if (isHFile(p)) { + } else if (isHFile(p) || isMobFile(p) || isMobRefFile(p)) { // HFile if (fileStatus != null) { this.createdTimestamp = fileStatus.getModificationTime(); @@ -436,6 +437,30 @@ public static boolean isHFile(final String fileName) { return m.matches() && m.groupCount() > 0; } + public static boolean isMobFile(final Path path) { + String fileName = path.getName(); + String[] parts = fileName.split(MobUtils.SEP); + if (parts.length != 2) { + return false; + } + Matcher m = HFILE_NAME_PATTERN.matcher(parts[0]); + Matcher mm = HFILE_NAME_PATTERN.matcher(parts[1]); + return m.matches() && mm.matches(); + } + + public static boolean isMobRefFile(final Path path) { + String fileName = path.getName(); + int lastIndex = fileName.lastIndexOf(MobUtils.SEP); + if (lastIndex < 0) { + return false; + } + String[] parts = new String[2]; + parts[0] = fileName.substring(0, lastIndex); + parts[1] = fileName.substring(lastIndex + 1); + String name = parts[0] + "." + parts[1]; + Matcher m = REF_NAME_PATTERN.matcher(name); + return m.matches() && m.groupCount() > 1; + } /** * @param path Path to check. * @return True if the path has format of a del file. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java index 7b4fb4a0c06f..522efb5ea18d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java @@ -25,6 +25,7 @@ import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY; import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAX_SEQ_ID_KEY; import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_CELLS_COUNT; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_FILE_REFS; import static org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY; import java.io.IOException; @@ -242,6 +243,28 @@ public void appendMetadata(final long maxSequenceId, final boolean majorCompacti appendTrackedTimestampsToMetadata(); } + /** + * Appends MOB - specific metadata + * @param mobRefSet - set of MOB file names + * @throws IOException problem writing to FS + */ + public void appendMobMetadata(Set mobRefSet) throws IOException { + if (mobRefSet.isEmpty()) { + return; + } + StringBuilder sb = new StringBuilder(2 * mobRefSet.size() - 1); + String[] arr = new String[mobRefSet.size()]; + arr = mobRefSet.toArray(arr); + for (int i = 0; i < arr.length; i++) { + sb.append(arr[i]); + if (i < arr.length - 1) { + sb.append(","); + } + } + byte[] bytes = sb.toString().getBytes(); + writer.appendFileInfo(MOB_FILE_REFS, bytes); + } + /** * Add TimestampRange and earliest put timestamp to Metadata */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java index 503325a1a374..3eab002dba5a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java @@ -115,7 +115,7 @@ public CompactionProgress getProgress() { } /** The sole reason this class exists is that java has no ref/out/pointer parameters. */ - protected static class FileDetails { + public static class FileDetails { /** Maximum key count after compaction (for blooms) */ public long maxKeyCount = 0; /** Earliest put timestamp if major compaction */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java index b30d3499f212..a9be1bae730d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java @@ -1126,7 +1126,7 @@ public HFileFilter(FileSystem fs) { @Override protected boolean accept(Path p, @CheckForNull Boolean isDir) { - if (!StoreFileInfo.isHFile(p)) { + if (!StoreFileInfo.isHFile(p) && !StoreFileInfo.isMobFile(p)) { return false; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java new file mode 100644 index 000000000000..eb0a097fee6f --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java @@ -0,0 +1,356 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.mob; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InterruptedIOException; +import java.util.ArrayList; +import java.util.Date; +import java.util.List; +import java.util.Random; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.PrivateCellUtil; +import org.apache.hadoop.hbase.io.hfile.CorruptHFileException; +import org.apache.hadoop.hbase.regionserver.CellSink; +import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.regionserver.KeyValueScanner; +import org.apache.hadoop.hbase.regionserver.ScannerContext; +import org.apache.hadoop.hbase.regionserver.ShipperListener; +import org.apache.hadoop.hbase.regionserver.StoreFileWriter; +import org.apache.hadoop.hbase.regionserver.throttle.ThroughputControlUtil; +import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@InterfaceAudience.Private +public class FaultyMobStoreCompactor extends DefaultMobStoreCompactor { + + private static final Logger LOG = LoggerFactory.getLogger(FaultyMobStoreCompactor.class); + + public static AtomicLong mobCounter = new AtomicLong(); + public static AtomicLong totalFailures = new AtomicLong(); + public static AtomicLong totalCompactions = new AtomicLong(); + public static AtomicLong totalMajorCompactions = new AtomicLong(); + + static double failureProb = 0.1d; + static Random rnd = new Random(); + + public FaultyMobStoreCompactor(Configuration conf, HStore store) { + super(conf, store); + failureProb = conf.getDouble("injected.fault.probability", 0.1); + } + + @Override + protected boolean performCompaction(FileDetails fd, InternalScanner scanner, CellSink writer, + long smallestReadPoint, boolean cleanSeqId, ThroughputController throughputController, + boolean major, int numofFilesToCompact) throws IOException { + + totalCompactions.incrementAndGet(); + if (major) { + totalMajorCompactions.incrementAndGet(); + } + long bytesWrittenProgressForCloseCheck = 0; + long bytesWrittenProgressForLog = 0; + long bytesWrittenProgressForShippedCall = 0; + // Clear old mob references + mobRefSet.get().clear(); + boolean isUserRequest = userRequest.get(); + boolean compactMOBs = major && isUserRequest; + boolean discardMobMiss = + conf.getBoolean(MobConstants.MOB_DISCARD_MISS_KEY, MobConstants.DEFAULT_MOB_DISCARD_MISS); + + boolean mustFail = false; + if (compactMOBs) { + mobCounter.incrementAndGet(); + double dv = rnd.nextDouble(); + if (dv < failureProb) { + mustFail = true; + totalFailures.incrementAndGet(); + } + } + + FileSystem fs = FileSystem.get(conf); + + // Since scanner.next() can return 'false' but still be delivering data, + // we have to use a do/while loop. + List cells = new ArrayList<>(); + // Limit to "hbase.hstore.compaction.kv.max" (default 10) to avoid OOME + int closeCheckSizeLimit = HStore.getCloseCheckInterval(); + long lastMillis = 0; + if (LOG.isDebugEnabled()) { + lastMillis = EnvironmentEdgeManager.currentTime(); + } + String compactionName = ThroughputControlUtil.getNameForThrottling(store, "compaction"); + long now = 0; + boolean hasMore; + Path path = MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName()); + byte[] fileName = null; + StoreFileWriter mobFileWriter = null; + long mobCells = 0; + long cellsCountCompactedToMob = 0, cellsCountCompactedFromMob = 0; + long cellsSizeCompactedToMob = 0, cellsSizeCompactedFromMob = 0; + boolean finished = false; + + ScannerContext scannerContext = + ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build(); + throughputController.start(compactionName); + KeyValueScanner kvs = (scanner instanceof KeyValueScanner) ? (KeyValueScanner) scanner : null; + long shippedCallSizeLimit = + (long) numofFilesToCompact * this.store.getColumnFamilyDescriptor().getBlocksize(); + + Cell mobCell = null; + + long counter = 0; + long countFailAt = -1; + if (mustFail) { + countFailAt = rnd.nextInt(100); // randomly fail fast + } + + try { + try { + // If the mob file writer could not be created, directly write the cell to the store file. + mobFileWriter = mobStore.createWriterInTmp(new Date(fd.latestPutTs), fd.maxKeyCount, + compactionCompression, store.getRegionInfo().getStartKey(), true); + fileName = Bytes.toBytes(mobFileWriter.getPath().getName()); + } catch (IOException e) { + // Bailing out + LOG.error("Failed to create mob writer, ", e); + throw e; + } + if (compactMOBs) { + // Add the only reference we get for compact MOB case + // because new store file will have only one MOB reference + // in this case - of newly compacted MOB file + mobRefSet.get().add(mobFileWriter.getPath().getName()); + } + do { + hasMore = scanner.next(cells, scannerContext); + if (LOG.isDebugEnabled()) { + now = EnvironmentEdgeManager.currentTime(); + } + for (Cell c : cells) { + counter++; + if (compactMOBs) { + if (MobUtils.isMobReferenceCell(c)) { + if (counter == countFailAt) { + LOG.warn("\n\n INJECTED FAULT mobCounter=" + mobCounter.get() + "\n\n"); + throw new CorruptHFileException("injected fault"); + } + String fName = MobUtils.getMobFileName(c); + Path pp = new Path(new Path(fs.getUri()), new Path(path, fName)); + + // Added to support migration + try { + mobCell = mobStore.resolve(c, true, false).getCell(); + } catch (FileNotFoundException fnfe) { + if (discardMobMiss) { + LOG.error("Missing MOB cell: file=" + pp + " not found"); + continue; + } else { + throw fnfe; + } + } + + if (discardMobMiss && mobCell.getValueLength() == 0) { + LOG.error("Missing MOB cell value: file=" + pp + " cell=" + mobCell); + continue; + } + + if (mobCell.getValueLength() > mobSizeThreshold) { + // put the mob data back to the store file + PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId()); + mobFileWriter.append(mobCell); + writer.append( + MobUtils.createMobRefCell(mobCell, fileName, this.mobStore.getRefCellTags())); + cellsCountCompactedFromMob++; + cellsSizeCompactedFromMob += mobCell.getValueLength(); + mobCells++; + } else { + // If MOB value is less than threshold, append it directly to a store file + PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId()); + writer.append(mobCell); + } + + } else { + // Not a MOB reference cell + int size = c.getValueLength(); + if (size > mobSizeThreshold) { + mobFileWriter.append(c); + writer + .append(MobUtils.createMobRefCell(c, fileName, this.mobStore.getRefCellTags())); + mobCells++; + } else { + writer.append(c); + } + } + } else if (c.getTypeByte() != KeyValue.Type.Put.getCode()) { + // Not a major compaction or major with MOB disabled + // If the kv type is not put, directly write the cell + // to the store file. + writer.append(c); + } else if (MobUtils.isMobReferenceCell(c)) { + // Not a major MOB compaction, Put MOB reference + if (MobUtils.hasValidMobRefCellValue(c)) { + int size = MobUtils.getMobValueLength(c); + if (size > mobSizeThreshold) { + // If the value size is larger than the threshold, it's regarded as a mob. Since + // its value is already in the mob file, directly write this cell to the store file + writer.append(c); + // Add MOB reference to a set + mobRefSet.get().add(MobUtils.getMobFileName(c)); + } else { + // If the value is not larger than the threshold, it's not regarded a mob. Retrieve + // the mob cell from the mob file, and write it back to the store file. + mobCell = mobStore.resolve(c, true, false).getCell(); + if (mobCell.getValueLength() != 0) { + // put the mob data back to the store file + PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId()); + writer.append(mobCell); + cellsCountCompactedFromMob++; + cellsSizeCompactedFromMob += mobCell.getValueLength(); + } else { + // If the value of a file is empty, there might be issues when retrieving, + // directly write the cell to the store file, and leave it to be handled by the + // next compaction. + LOG.error("Empty value for: " + c); + writer.append(c); + // Add MOB reference to a set + mobRefSet.get().add(MobUtils.getMobFileName(c)); + } + } + } else { + // TODO ???? + LOG.error("Corrupted MOB reference: " + c); + writer.append(c); + } + } else if (c.getValueLength() <= mobSizeThreshold) { + // If the value size of a cell is not larger than the threshold, directly write it to + // the store file. + writer.append(c); + } else { + // If the value size of a cell is larger than the threshold, it's regarded as a mob, + // write this cell to a mob file, and write the path to the store file. + mobCells++; + // append the original keyValue in the mob file. + mobFileWriter.append(c); + Cell reference = MobUtils.createMobRefCell(c, fileName, this.mobStore.getRefCellTags()); + // write the cell whose value is the path of a mob file to the store file. + writer.append(reference); + cellsCountCompactedToMob++; + cellsSizeCompactedToMob += c.getValueLength(); + // Add ref we get for compact MOB case + mobRefSet.get().add(mobFileWriter.getPath().getName()); + } + + int len = c.getSerializedSize(); + ++progress.currentCompactedKVs; + progress.totalCompactedSize += len; + bytesWrittenProgressForShippedCall += len; + if (LOG.isDebugEnabled()) { + bytesWrittenProgressForLog += len; + } + throughputController.control(compactionName, len); + // check periodically to see if a system stop is requested + if (closeCheckSizeLimit > 0) { + bytesWrittenProgressForCloseCheck += len; + if (bytesWrittenProgressForCloseCheck > closeCheckSizeLimit) { + bytesWrittenProgressForCloseCheck = 0; + if (!store.areWritesEnabled()) { + progress.cancel(); + return false; + } + } + } + if (kvs != null && bytesWrittenProgressForShippedCall > shippedCallSizeLimit) { + ((ShipperListener) writer).beforeShipped(); + kvs.shipped(); + bytesWrittenProgressForShippedCall = 0; + } + } + // Log the progress of long running compactions every minute if + // logging at DEBUG level + if (LOG.isDebugEnabled()) { + if ((now - lastMillis) >= COMPACTION_PROGRESS_LOG_INTERVAL) { + String rate = String.format("%.2f", + (bytesWrittenProgressForLog / 1024.0) / ((now - lastMillis) / 1000.0)); + LOG.debug("Compaction progress: {} {}, rate={} KB/sec, throughputController is {}", + compactionName, progress, rate, throughputController); + lastMillis = now; + bytesWrittenProgressForLog = 0; + } + } + cells.clear(); + } while (hasMore); + finished = true; + } catch (InterruptedException e) { + progress.cancel(); + throw new InterruptedIOException( + "Interrupted while control throughput of compacting " + compactionName); + } catch (FileNotFoundException e) { + LOG.error("MOB Stress Test FAILED, region: " + store.getRegionInfo().getEncodedName(), e); + System.exit(-1); + } catch (IOException t) { + LOG.error("Mob compaction failed for region: " + store.getRegionInfo().getEncodedName()); + throw t; + } finally { + // Clone last cell in the final because writer will append last cell when committing. If + // don't clone here and once the scanner get closed, then the memory of last cell will be + // released. (HBASE-22582) + ((ShipperListener) writer).beforeShipped(); + throughputController.finish(compactionName); + if (!finished && mobFileWriter != null) { + // Remove all MOB references because compaction failed + mobRefSet.get().clear(); + // Abort writer + abortWriter(mobFileWriter); + } + } + + if (mobFileWriter != null) { + if (mobCells > 0) { + // If the mob file is not empty, commit it. + mobFileWriter.appendMetadata(fd.maxSeqId, major, mobCells); + mobFileWriter.close(); + mobStore.commitFile(mobFileWriter.getPath(), path); + } else { + // If the mob file is empty, delete it instead of committing. + abortWriter(mobFileWriter); + } + } + mobStore.updateCellsCountCompactedFromMob(cellsCountCompactedFromMob); + mobStore.updateCellsCountCompactedToMob(cellsCountCompactedToMob); + mobStore.updateCellsSizeCompactedFromMob(cellsSizeCompactedFromMob); + mobStore.updateCellsSizeCompactedToMob(cellsSizeCompactedToMob); + progress.complete(); + return true; + + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java new file mode 100644 index 000000000000..5357de0780d9 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java @@ -0,0 +1,79 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.mob; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.util.AbstractHBaseTool; +import org.apache.hadoop.util.ToolRunner; +import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.yetus.audience.InterfaceStability; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class MobStressTool extends AbstractHBaseTool { + private static final Logger LOG = LoggerFactory.getLogger(MobStressTool.class); + private CommandLine cmd; + + public MobStressTool() throws IOException { + init(); + } + + protected void init() throws IOException { + // define supported options + addOptWithArg("n", "Number of MOB key-values to insert, default - 10000000"); + } + + @Override + protected void addOptions() { + } + + @Override + protected void processOptions(CommandLine cmd) { + this.cmd = cmd; + } + + @Override + protected int doWork() throws Exception { + long numRowsToInsert = 10000000; + if (cmd.hasOption("n")) { + numRowsToInsert = Long.parseLong(cmd.getOptionValue("n")); + if (numRowsToInsert < 0) { + LOG.warn("Ignore wrong option '-n'"); + numRowsToInsert = 10000000; + } + } + + TestMobCompaction test = new TestMobCompaction(); + test.init(getConf(), numRowsToInsert); + test.testMobCompaction(); + return 0; + } + + public static void main(String[] args) throws Exception { + Configuration conf = HBaseConfiguration.create(); + int ret = ToolRunner.run(conf, new MobStressTool(), args); + System.exit(ret); + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java new file mode 100644 index 000000000000..9266f874d95a --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java @@ -0,0 +1,339 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.mob; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.Arrays; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.KeepDeletedCells; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.master.MobFileCleanerChore; +import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +/** + Reproduction for MOB data loss + + 1. Settings: Region Size 200 MB, Flush threshold 800 KB. + 2. Insert 10 Million records + 3. MOB Compaction and Archiver + a) Trigger MOB Compaction (every 2 minutes) + b) Trigger major compaction (every 2 minutes) + c) Trigger archive cleaner (every 3 minutes) + 4. Validate MOB data after complete data load. + + */ +@SuppressWarnings("deprecation") +@Category(LargeTests.class) +public class TestMobCompaction { + private static final Logger LOG = LoggerFactory.getLogger(TestMobCompaction.class); + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobCompaction.class); + @Rule + public TestName testName = new TestName(); + + private HBaseTestingUtility HTU; + + private final static String famStr = "f1"; + private final static byte[] fam = Bytes.toBytes(famStr); + private final static byte[] qualifier = Bytes.toBytes("q1"); + private final static long mobLen = 10; + private final static byte[] mobVal = Bytes + .toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789"); + + private Configuration conf; + private HTableDescriptor hdt; + private HColumnDescriptor hcd; + private Admin admin; + private long count = 500000; + private double failureProb = 0.1; + private Table table = null; + private MobFileCleanerChore chore = new MobFileCleanerChore(); + + private static volatile boolean run = true; + + public TestMobCompaction() { + + } + + public void init(Configuration conf, long numRows) throws IOException { + this.conf = conf; + this.count = numRows; + printConf(); + hdt = createTableDescriptor("testMobCompactTable"); + Connection conn = ConnectionFactory.createConnection(this.conf); + this.admin = conn.getAdmin(); + this.hcd = new HColumnDescriptor(fam); + this.hcd.setMobEnabled(true); + this.hcd.setMobThreshold(mobLen); + this.hcd.setMaxVersions(1); + this.hdt.addFamily(hcd); + if (admin.tableExists(hdt.getTableName())) { + admin.disableTable(hdt.getTableName()); + admin.deleteTable(hdt.getTableName()); + } + admin.createTable(hdt); + table = conn.getTable(hdt.getTableName()); + } + + private void printConf() { + LOG.info("To run stress test, please change HBase configuration as following:"); + LOG.info("hfile.format.version=3"); + LOG.info("hbase.master.hfilecleaner.ttl=0"); + LOG.info("hbase.hregion.max.filesize=200000000"); + LOG.info("hbase.client.retries.number=100"); + LOG.info("hbase.hregion.memstore.flush.size=800000"); + LOG.info("hbase.hstore.blockingStoreFiles=150"); + LOG.info("hbase.hstore.compaction.throughput.lower.bound=50000000"); + LOG.info("hbase.hstore.compaction.throughput.higher.bound=100000000"); + LOG.info("hbase.master.mob.cleaner.period=0"); + LOG.info("hbase.mob.default.compactor=org.apache.hadoop.hbase.mob.FaultyMobStoreCompactor"); + LOG.warn("injected.fault.probability=x, where x is between 0. and 1."); + + } + + private HTableDescriptor createTableDescriptor(final String name, final int minVersions, + final int versions, final int ttl, KeepDeletedCells keepDeleted) { + HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name)); + return htd; + } + + private HTableDescriptor createTableDescriptor(final String name) { + return createTableDescriptor(name, HColumnDescriptor.DEFAULT_MIN_VERSIONS, 1, + HConstants.FOREVER, HColumnDescriptor.DEFAULT_KEEP_DELETED); + } + + @Before + public void setUp() throws Exception { + HTU = new HBaseTestingUtility(); + hdt = HTU.createTableDescriptor("testMobCompactTable"); + conf = HTU.getConfiguration(); + + initConf(); + + // HTU.getConfiguration().setInt("hbase.mob.compaction.chore.period", 0); + HTU.startMiniCluster(); + admin = HTU.getAdmin(); + + hcd = new HColumnDescriptor(fam); + hcd.setMobEnabled(true); + hcd.setMobThreshold(mobLen); + hcd.setMaxVersions(1); + hdt.addFamily(hcd); + table = HTU.createTable(hdt, null); + } + + private void initConf() { + + conf.setInt("hfile.format.version", 3); + conf.setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0); + conf.setInt("hbase.client.retries.number", 100); + conf.setInt("hbase.hregion.max.filesize", 200000000); + conf.setInt("hbase.hregion.memstore.flush.size", 800000); + conf.setInt("hbase.hstore.blockingStoreFiles", 150); + conf.setInt("hbase.hstore.compaction.throughput.lower.bound", 52428800); + conf.setInt("hbase.hstore.compaction.throughput.higher.bound", 2 * 52428800); + conf.setDouble("injected.fault.probability", failureProb); + conf.set(MobStoreEngine.DEFAULT_MOB_COMPACTOR_CLASS_KEY, + FaultyMobStoreCompactor.class.getName()); + + } + + @After + public void tearDown() throws Exception { + HTU.shutdownMiniCluster(); + } + + class MajorCompaction implements Runnable { + + @Override + public void run() { + while (run) { + try { + admin.majorCompact(hdt.getTableName(), fam); + Thread.sleep(120000); + } catch (Exception e) { + LOG.error("MOB Stress Test FAILED", e); + System.exit(-1); + } + } + } + } + + class CleanMobAndArchive implements Runnable { + + @Override + public void run() { + while (run) { + try { + LOG.info("MOB cleanup chore started ..."); + chore.cleanupObsoleteMobFiles(conf, table.getName()); + LOG.info("MOB cleanup chore finished"); + + Thread.sleep(130000); + } catch (Exception e) { + e.printStackTrace(); + } + } + } + } + + class WriteData implements Runnable { + + private long rows = -1; + + public WriteData(long rows) { + this.rows = rows; + } + + @Override + public void run() { + try { + + // BufferedMutator bm = admin.getConnection().getBufferedMutator(table.getName()); + // Put Operation + for (int i = 0; i < rows; i++) { + Put p = new Put(Bytes.toBytes(i)); + p.addColumn(fam, qualifier, mobVal); + table.put(p); + + // bm.mutate(p); + if (i % 10000 == 0) { + LOG.info("LOADED=" + i); + try { + Thread.sleep(500); + } catch (InterruptedException ee) { + + } + } + if (i % 100000 == 0) { + printStats(i); + } + } + // bm.flush(); + admin.flush(table.getName()); + run = false; + } catch (Exception e) { + LOG.error("MOB Stress Test FAILED", e); + System.exit(-1); + } + } + } + + @Test + public void testMobCompaction() throws InterruptedException, IOException { + + try { + + Thread writeData = new Thread(new WriteData(count)); + writeData.start(); + + Thread majorcompact = new Thread(new MajorCompaction()); + majorcompact.start(); + + Thread cleaner = new Thread(new CleanMobAndArchive()); + cleaner.start(); + + while (run) { + Thread.sleep(1000); + } + + MobUtils.getNumberOfMobFiles(conf, table.getName(), new String(fam)); + LOG.info("Waiting for write thread to finish ..."); + writeData.join(); + // Cleanup again + chore.cleanupObsoleteMobFiles(conf, table.getName()); + + if (HTU != null) { + LOG.info("Archive cleaner started ..."); + // Call archive cleaner again + HTU.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting(); + LOG.info("Archive cleaner finished"); + } + + scanTable(); + + } finally { + + admin.disableTable(hdt.getTableName()); + admin.deleteTable(hdt.getTableName()); + } + LOG.info("MOB Stress Test finished OK"); + printStats(count); + + } + + public void printStats(long loaded) { + LOG.info("MOB Stress Test: loaded=" + loaded + " compactions=" + + FaultyMobStoreCompactor.totalCompactions.get() + " major=" + + FaultyMobStoreCompactor.totalMajorCompactions.get() + " mob=" + + FaultyMobStoreCompactor.mobCounter.get() + " injected failures=" + + FaultyMobStoreCompactor.totalFailures.get()); + } + + private void scanTable() { + try { + + Result result; + ResultScanner scanner = table.getScanner(fam); + int counter = 0; + while ((result = scanner.next()) != null) { + assertTrue(Arrays.equals(result.getValue(fam, qualifier), mobVal)); + if (counter % 10000 == 0) { + LOG.info("GET=" + counter); + } + counter++; + } + assertEquals(count, counter); + } catch (Exception e) { + e.printStackTrace(); + LOG.error("MOB Stress Test FAILED"); + if (HTU != null) { + assertTrue(false); + } else { + System.exit(-1); + } + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java index ae53ff21feba..a6a2ee55009c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java @@ -47,6 +47,7 @@ public class TestMobFileName { private Date date; private String dateStr; private byte[] startKey; + private String regionName = "region"; @Before public void setUp() { @@ -59,32 +60,32 @@ public void setUp() { @Test public void testHashCode() { - assertEquals(MobFileName.create(startKey, dateStr, uuid).hashCode(), - MobFileName.create(startKey, dateStr, uuid).hashCode()); - assertNotSame(MobFileName.create(startKey, dateStr, uuid), - MobFileName.create(startKey, dateStr, uuid)); + assertEquals(MobFileName.create(startKey, dateStr, uuid, regionName).hashCode(), + MobFileName.create(startKey, dateStr, uuid, regionName).hashCode()); + assertNotSame(MobFileName.create(startKey, dateStr, uuid, regionName), + MobFileName.create(startKey, dateStr, uuid, regionName)); } @Test public void testCreate() { - MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid); + MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid, regionName); assertEquals(mobFileName, MobFileName.create(mobFileName.getFileName())); } @Test public void testGet() { - MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid); + MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid, regionName); assertEquals(MD5Hash.getMD5AsHex(startKey, 0, startKey.length), mobFileName.getStartKey()); assertEquals(dateStr, mobFileName.getDate()); assertEquals(mobFileName.getFileName(), MD5Hash.getMD5AsHex(startKey, 0, startKey.length) - + dateStr + uuid); + + dateStr + uuid+"_"+regionName); } @Test public void testEquals() { - MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid); + MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid, regionName); assertTrue(mobFileName.equals(mobFileName)); assertFalse(mobFileName.equals(this)); - assertTrue(mobFileName.equals(MobFileName.create(startKey, dateStr, uuid))); + assertTrue(mobFileName.equals(MobFileName.create(startKey, dateStr, uuid, regionName))); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java deleted file mode 100644 index 130a6d31cba6..000000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java +++ /dev/null @@ -1,1202 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.mob.compactions; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; -import java.security.Key; -import java.security.SecureRandom; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Calendar; -import java.util.Collections; -import java.util.List; -import java.util.Objects; -import java.util.Optional; -import java.util.Random; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.RejectedExecutionHandler; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import javax.crypto.spec.SecretKeySpec; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.NamespaceDescriptor; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.BufferedMutator; -import org.apache.hadoop.hbase.client.CompactType; -import org.apache.hadoop.hbase.client.CompactionState; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.Delete; -import org.apache.hadoop.hbase.client.Durability; -import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.ResultScanner; -import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.coprocessor.ObserverContext; -import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; -import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; -import org.apache.hadoop.hbase.coprocessor.RegionObserver; -import org.apache.hadoop.hbase.io.HFileLink; -import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; -import org.apache.hadoop.hbase.io.crypto.aes.AES; -import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hadoop.hbase.io.hfile.HFile; -import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner; -import org.apache.hadoop.hbase.mob.MobConstants; -import org.apache.hadoop.hbase.mob.MobFileName; -import org.apache.hadoop.hbase.mob.MobTestUtil; -import org.apache.hadoop.hbase.mob.MobUtils; -import org.apache.hadoop.hbase.regionserver.BloomType; -import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.HStoreFile; -import org.apache.hadoop.hbase.regionserver.Store; -import org.apache.hadoop.hbase.regionserver.StoreFile; -import org.apache.hadoop.hbase.regionserver.StoreFileInfo; -import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker; -import org.apache.hadoop.hbase.security.EncryptionUtil; -import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.Threads; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Category(LargeTests.class) -public class TestMobCompactor { - - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestMobCompactor.class); - - private static final Logger LOG = LoggerFactory.getLogger(TestMobCompactor.class); - private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private static Configuration conf = null; - private TableName tableName; - private static Connection conn; - private BufferedMutator bufMut; - private Table table; - private static Admin admin; - private HTableDescriptor desc; - private HColumnDescriptor hcd1; - private HColumnDescriptor hcd2; - private static FileSystem fs; - private static final String family1 = "family1"; - private static final String family2 = "family2"; - private static final String qf1 = "qualifier1"; - private static final String qf2 = "qualifier2"; - - private static long tsFor20150907Monday; - private static long tsFor20151120Sunday; - private static long tsFor20151128Saturday; - private static long tsFor20151130Monday; - private static long tsFor20151201Tuesday; - private static long tsFor20151205Saturday; - private static long tsFor20151228Monday; - private static long tsFor20151231Thursday; - private static long tsFor20160101Friday; - private static long tsFor20160103Sunday; - - private static final byte[] mobKey01 = Bytes.toBytes("r01"); - private static final byte[] mobKey02 = Bytes.toBytes("r02"); - private static final byte[] mobKey03 = Bytes.toBytes("r03"); - private static final byte[] mobKey04 = Bytes.toBytes("r04"); - private static final byte[] mobKey05 = Bytes.toBytes("r05"); - private static final byte[] mobKey06 = Bytes.toBytes("r05"); - private static final byte[] mobKey1 = Bytes.toBytes("r1"); - private static final byte[] mobKey2 = Bytes.toBytes("r2"); - private static final byte[] mobKey3 = Bytes.toBytes("r3"); - private static final byte[] mobKey4 = Bytes.toBytes("r4"); - private static final byte[] mobKey5 = Bytes.toBytes("r5"); - private static final byte[] mobKey6 = Bytes.toBytes("r6"); - private static final byte[] mobKey7 = Bytes.toBytes("r7"); - private static final byte[] mobKey8 = Bytes.toBytes("r8"); - private static final String mobValue0 = "mobValue00000000000000000000000000"; - private static final String mobValue1 = "mobValue00000111111111111111111111"; - private static final String mobValue2 = "mobValue00000222222222222222222222"; - private static final String mobValue3 = "mobValue00000333333333333333333333"; - private static final String mobValue4 = "mobValue00000444444444444444444444"; - private static final String mobValue5 = "mobValue00000666666666666666666666"; - private static final String mobValue6 = "mobValue00000777777777777777777777"; - private static final String mobValue7 = "mobValue00000888888888888888888888"; - private static final String mobValue8 = "mobValue00000888888888888888888899"; - - private static byte[] KEYS = Bytes.toBytes("012"); - private static int regionNum = KEYS.length; - private static int delRowNum = 1; - private static int delCellNum = 6; - private static int cellNumPerRow = 3; - private static int rowNumPerFile = 2; - private static ExecutorService pool; - - @Rule - public TestName name = new TestName(); - - @BeforeClass - public static void setUpBeforeClass() throws Exception { - TEST_UTIL.getConfiguration().setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, 5000); - TEST_UTIL.getConfiguration() - .set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, KeyProviderForTesting.class.getName()); - TEST_UTIL.getConfiguration().set(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, "hbase"); - TEST_UTIL.getConfiguration().setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0); - TEST_UTIL.getConfiguration().setInt("hbase.client.retries.number", 1); - TEST_UTIL.getConfiguration().setInt("hbase.hfile.compaction.discharger.interval", 100); - TEST_UTIL.startMiniCluster(1); - pool = createThreadPool(TEST_UTIL.getConfiguration()); - conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration(), pool); - fs = TEST_UTIL.getTestFileSystem(); - conf = TEST_UTIL.getConfiguration(); - admin = TEST_UTIL.getAdmin(); - - // Initialize timestamps for these days - Calendar calendar = Calendar.getInstance(); - calendar.set(2015, 8, 7, 10, 20); - tsFor20150907Monday = calendar.getTimeInMillis(); - - calendar.set(2015, 10, 20, 10, 20); - tsFor20151120Sunday = calendar.getTimeInMillis(); - - calendar.set(2015, 10, 28, 10, 20); - tsFor20151128Saturday = calendar.getTimeInMillis(); - - calendar.set(2015, 10, 30, 10, 20); - tsFor20151130Monday = calendar.getTimeInMillis(); - - calendar.set(2015, 11, 1, 10, 20); - tsFor20151201Tuesday = calendar.getTimeInMillis(); - - calendar.set(2015, 11, 5, 10, 20); - tsFor20151205Saturday = calendar.getTimeInMillis(); - - calendar.set(2015, 11, 28, 10, 20); - tsFor20151228Monday = calendar.getTimeInMillis(); - - calendar.set(2015, 11, 31, 10, 20); - tsFor20151231Thursday = calendar.getTimeInMillis(); - - calendar.set(2016, 0, 1, 10, 20); - tsFor20160101Friday = calendar.getTimeInMillis(); - - calendar.set(2016, 0, 3, 10, 20); - tsFor20160103Sunday = calendar.getTimeInMillis(); - } - - @AfterClass - public static void tearDownAfterClass() throws Exception { - pool.shutdown(); - conn.close(); - TEST_UTIL.shutdownMiniCluster(); - } - - public void setUp(String tableNameAsString) throws IOException { - tableName = TableName.valueOf(tableNameAsString); - hcd1 = new HColumnDescriptor(family1); - hcd1.setMobEnabled(true); - hcd1.setMobThreshold(5); - hcd2 = new HColumnDescriptor(family2); - hcd2.setMobEnabled(true); - hcd2.setMobThreshold(5); - desc = new HTableDescriptor(tableName); - desc.addFamily(hcd1); - desc.addFamily(hcd2); - admin.createTable(desc, getSplitKeys()); - table = conn.getTable(tableName); - bufMut = conn.getBufferedMutator(tableName); - } - - // Set up for mob compaction policy testing - private void setUpForPolicyTest(String tableNameAsString, MobCompactPartitionPolicy type) - throws IOException { - tableName = TableName.valueOf(tableNameAsString); - hcd1 = new HColumnDescriptor(family1); - hcd1.setMobEnabled(true); - hcd1.setMobThreshold(10); - hcd1.setMobCompactPartitionPolicy(type); - desc = new HTableDescriptor(tableName); - desc.addFamily(hcd1); - admin.createTable(desc); - table = conn.getTable(tableName); - bufMut = conn.getBufferedMutator(tableName); - } - - // alter mob compaction policy - private void alterForPolicyTest(final MobCompactPartitionPolicy type) - throws Exception { - - hcd1.setMobCompactPartitionPolicy(type); - desc.modifyFamily(hcd1); - admin.modifyTable(desc); - LOG.info("alter status finished"); - } - - @Test - public void testMinorCompaction() throws Exception { - resetConf(); - int mergeSize = 5000; - // change the mob compaction merge size - conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize); - - // create a table with namespace - NamespaceDescriptor namespaceDescriptor = NamespaceDescriptor.create("ns").build(); - String tableNameAsString = "ns:testMinorCompaction"; - admin.createNamespace(namespaceDescriptor); - setUp(tableNameAsString); - int count = 4; - // generate mob files - loadData(admin, bufMut, tableName, count, rowNumPerFile); - int rowNumPerRegion = count * rowNumPerFile; - - assertEquals("Before deleting: mob rows count", regionNum * rowNumPerRegion, - MobTestUtil.countMobRows(table)); - assertEquals("Before deleting: mob cells count", regionNum * cellNumPerRow * rowNumPerRegion, - countMobCells(table)); - assertEquals("Before deleting: mob file count", regionNum * count, - countFiles(tableName, true, family1)); - - int largeFilesCount = countLargeFiles(mergeSize, tableName, family1); - createDelFile(table, tableName, Bytes.toBytes(family1), Bytes.toBytes(qf1)); - - assertEquals("Before compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum), - MobTestUtil.countMobRows(table)); - assertEquals("Before compaction: mob cells count", regionNum - * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table)); - assertEquals("Before compaction: family1 mob file count", regionNum * count, - countFiles(tableName, true, family1)); - assertEquals("Before compaction: family2 mob file count", regionNum * count, - countFiles(tableName, true, family2)); - assertEquals("Before compaction: family1 del file count", regionNum, - countFiles(tableName, false, family1)); - assertEquals("Before compaction: family2 del file count", regionNum, - countFiles(tableName, false, family2)); - - // do the mob file compaction - MobCompactor compactor = new PartitionedMobCompactor(conf, fs, tableName, hcd1, pool); - compactor.compact(); - - assertEquals("After compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum), - MobTestUtil.countMobRows(table)); - assertEquals("After compaction: mob cells count", regionNum - * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table)); - // After the compaction, the files smaller than the mob compaction merge size - // is merge to one file - assertEquals("After compaction: family1 mob file count", largeFilesCount + regionNum, - countFiles(tableName, true, family1)); - assertEquals("After compaction: family2 mob file count", regionNum * count, - countFiles(tableName, true, family2)); - assertEquals("After compaction: family1 del file count", regionNum, - countFiles(tableName, false, family1)); - assertEquals("After compaction: family2 del file count", regionNum, - countFiles(tableName, false, family2)); - } - - @Test - public void testMinorCompactionWithWeeklyPolicy() throws Exception { - resetConf(); - int mergeSize = 5000; - // change the mob compaction merge size - conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize); - - commonPolicyTestLogic("testMinorCompactionWithWeeklyPolicy", - MobCompactPartitionPolicy.WEEKLY, false, 6, - new String[] { "20150907", "20151120", "20151128", "20151130", "20151205", "20160103" }, - true); - } - - @Test - public void testMajorCompactionWithWeeklyPolicy() throws Exception { - resetConf(); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyPolicy", - MobCompactPartitionPolicy.WEEKLY, true, 5, - new String[] { "20150907", "20151120", "20151128", "20151205", "20160103" }, true); - } - - @Test - public void testMinorCompactionWithMonthlyPolicy() throws Exception { - resetConf(); - int mergeSize = 5000; - // change the mob compaction merge size - conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize); - - commonPolicyTestLogic("testMinorCompactionWithMonthlyPolicy", - MobCompactPartitionPolicy.MONTHLY, false, 4, - new String[] { "20150907", "20151130", "20151231", "20160103" }, true); - } - - @Test - public void testMajorCompactionWithMonthlyPolicy() throws Exception { - resetConf(); - - commonPolicyTestLogic("testMajorCompactionWithMonthlyPolicy", - MobCompactPartitionPolicy.MONTHLY, true, 4, - new String[] {"20150907", "20151130", "20151231", "20160103"}, true); - } - - @Test - public void testMajorCompactionWithWeeklyFollowedByMonthly() throws Exception { - resetConf(); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthly", - MobCompactPartitionPolicy.WEEKLY, true, 5, - new String[] { "20150907", "20151120", "20151128", "20151205", "20160103" }, true); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthly", - MobCompactPartitionPolicy.MONTHLY, true, 4, - new String[] {"20150907", "20151128", "20151205", "20160103" }, false); - } - - @Test - public void testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByWeekly() throws Exception { - resetConf(); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByWeekly", - MobCompactPartitionPolicy.WEEKLY, true, 5, - new String[] { "20150907", "20151120", "20151128", "20151205", "20160103" }, true); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByWeekly", - MobCompactPartitionPolicy.MONTHLY, true, 4, - new String[] { "20150907", "20151128", "20151205", "20160103" }, false); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByWeekly", - MobCompactPartitionPolicy.WEEKLY, true, 4, - new String[] { "20150907", "20151128", "20151205", "20160103" }, false); - } - - @Test - public void testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByDaily() throws Exception { - resetConf(); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByDaily", - MobCompactPartitionPolicy.WEEKLY, true, 5, - new String[] { "20150907", "20151120", "20151128", "20151205", "20160103" }, true); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByDaily", - MobCompactPartitionPolicy.MONTHLY, true, 4, - new String[] { "20150907", "20151128", "20151205", "20160103" }, false); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByDaily", - MobCompactPartitionPolicy.DAILY, true, 4, - new String[] { "20150907", "20151128", "20151205", "20160103" }, false); - } - - @Test - public void testCompactionWithHFileLink() throws IOException, InterruptedException { - resetConf(); - String tableNameAsString = "testCompactionWithHFileLink"; - setUp(tableNameAsString); - int count = 4; - // generate mob files - loadData(admin, bufMut, tableName, count, rowNumPerFile); - int rowNumPerRegion = count * rowNumPerFile; - - long tid = System.currentTimeMillis(); - String snapshotName1 = "snaptb-" + tid; - // take a snapshot - admin.snapshot(snapshotName1, tableName); - - createDelFile(table, tableName, Bytes.toBytes(family1), Bytes.toBytes(qf1)); - - assertEquals("Before compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum), - MobTestUtil.countMobRows(table)); - assertEquals("Before compaction: mob cells count", regionNum - * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table)); - assertEquals("Before compaction: family1 mob file count", regionNum * count, - countFiles(tableName, true, family1)); - assertEquals("Before compaction: family2 mob file count", regionNum * count, - countFiles(tableName, true, family2)); - assertEquals("Before compaction: family1 del file count", regionNum, - countFiles(tableName, false, family1)); - assertEquals("Before compaction: family2 del file count", regionNum, - countFiles(tableName, false, family2)); - - // do the mob compaction - MobCompactor compactor = new PartitionedMobCompactor(conf, fs, tableName, hcd1, pool); - compactor.compact(); - - assertEquals("After first compaction: mob rows count", regionNum - * (rowNumPerRegion - delRowNum), MobTestUtil.countMobRows(table)); - assertEquals("After first compaction: mob cells count", regionNum - * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table)); - assertEquals("After first compaction: family1 mob file count", regionNum, - countFiles(tableName, true, family1)); - assertEquals("After first compaction: family2 mob file count", regionNum * count, - countFiles(tableName, true, family2)); - assertEquals("After first compaction: family1 del file count", 0, - countFiles(tableName, false, family1)); - assertEquals("After first compaction: family2 del file count", regionNum, - countFiles(tableName, false, family2)); - assertEquals("After first compaction: family1 hfilelink count", 0, countHFileLinks(family1)); - assertEquals("After first compaction: family2 hfilelink count", 0, countHFileLinks(family2)); - - admin.disableTable(tableName); - // Restore from snapshot, the hfilelink will exist in mob dir - admin.restoreSnapshot(snapshotName1); - admin.enableTable(tableName); - - assertEquals("After restoring snapshot: mob rows count", regionNum * rowNumPerRegion, - MobTestUtil.countMobRows(table)); - assertEquals("After restoring snapshot: mob cells count", regionNum * cellNumPerRow - * rowNumPerRegion, countMobCells(table)); - assertEquals("After restoring snapshot: family1 mob file count", regionNum * count, - countFiles(tableName, true, family1)); - assertEquals("After restoring snapshot: family2 mob file count", regionNum * count, - countFiles(tableName, true, family2)); - assertEquals("After restoring snapshot: family1 del file count", 0, - countFiles(tableName, false, family1)); - assertEquals("After restoring snapshot: family2 del file count", 0, - countFiles(tableName, false, family2)); - assertEquals("After restoring snapshot: family1 hfilelink count", regionNum * count, - countHFileLinks(family1)); - assertEquals("After restoring snapshot: family2 hfilelink count", 0, countHFileLinks(family2)); - - compactor.compact(); - - assertEquals("After second compaction: mob rows count", regionNum * rowNumPerRegion, - MobTestUtil.countMobRows(table)); - assertEquals("After second compaction: mob cells count", regionNum * cellNumPerRow - * rowNumPerRegion, countMobCells(table)); - assertEquals("After second compaction: family1 mob file count", regionNum, - countFiles(tableName, true, family1)); - assertEquals("After second compaction: family2 mob file count", regionNum * count, - countFiles(tableName, true, family2)); - assertEquals("After second compaction: family1 del file count", 0, - countFiles(tableName, false, family1)); - assertEquals("After second compaction: family2 del file count", 0, - countFiles(tableName, false, family2)); - assertEquals("After second compaction: family1 hfilelink count", 0, countHFileLinks(family1)); - assertEquals("After second compaction: family2 hfilelink count", 0, countHFileLinks(family2)); - assertRefFileNameEqual(family1); - } - - @Test - public void testMajorCompactionFromAdmin() throws Exception { - resetConf(); - int mergeSize = 5000; - // change the mob compaction merge size - conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize); - SecureRandom rng = new SecureRandom(); - byte[] keyBytes = new byte[AES.KEY_LENGTH]; - rng.nextBytes(keyBytes); - String algorithm = conf.get(HConstants.CRYPTO_KEY_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES); - Key cfKey = new SecretKeySpec(keyBytes, algorithm); - byte[] encryptionKey = EncryptionUtil.wrapKey(conf, - conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, User.getCurrent().getShortName()), cfKey); - final TableName tableName = TableName.valueOf(name.getMethodName()); - HTableDescriptor desc = new HTableDescriptor(tableName); - HColumnDescriptor hcd1 = new HColumnDescriptor(family1); - hcd1.setMobEnabled(true); - hcd1.setMobThreshold(0); - hcd1.setEncryptionType(algorithm); - hcd1.setEncryptionKey(encryptionKey); - HColumnDescriptor hcd2 = new HColumnDescriptor(family2); - hcd2.setMobEnabled(true); - hcd2.setMobThreshold(0); - desc.addFamily(hcd1); - desc.addFamily(hcd2); - admin.createTable(desc, getSplitKeys()); - Table table = conn.getTable(tableName); - BufferedMutator bufMut = conn.getBufferedMutator(tableName); - int count = 4; - // generate mob files - loadData(admin, bufMut, tableName, count, rowNumPerFile); - int rowNumPerRegion = count * rowNumPerFile; - - assertEquals("Before deleting: mob rows count", regionNum * rowNumPerRegion, - MobTestUtil.countMobRows(table)); - assertEquals("Before deleting: mob cells count", regionNum * cellNumPerRow * rowNumPerRegion, - countMobCells(table)); - assertEquals("Before deleting: mob file count", regionNum * count, - countFiles(tableName, true, family1)); - - createDelFile(table, tableName, Bytes.toBytes(family1), Bytes.toBytes(qf1)); - - assertEquals("Before compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum), - MobTestUtil.countMobRows(table)); - assertEquals("Before compaction: mob cells count", regionNum - * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table)); - assertEquals("Before compaction: family1 mob file count", regionNum * count, - countFiles(tableName, true, family1)); - assertEquals("Before compaction: family2 mob file count", regionNum * count, - countFiles(tableName, true, family2)); - assertEquals("Before compaction: family1 del file count", regionNum, - countFiles(tableName, false, family1)); - assertEquals("Before compaction: family2 del file count", regionNum, - countFiles(tableName, false, family2)); - - // do the major mob compaction, it will force all files to compaction - admin.majorCompact(tableName, hcd1.getName(), CompactType.MOB); - - waitUntilMobCompactionFinished(tableName); - assertEquals("After compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum), - MobTestUtil.countMobRows(table)); - assertEquals("After compaction: mob cells count", regionNum - * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table)); - assertEquals("After compaction: family1 mob file count", regionNum, - countFiles(tableName, true, family1)); - assertEquals("After compaction: family2 mob file count", regionNum * count, - countFiles(tableName, true, family2)); - assertEquals("After compaction: family1 del file count", 0, - countFiles(tableName, false, family1)); - assertEquals("After compaction: family2 del file count", regionNum, - countFiles(tableName, false, family2)); - Assert.assertTrue(verifyEncryption(tableName, family1)); - table.close(); - } - - @Test - public void testScannerOnBulkLoadRefHFiles() throws Exception { - resetConf(); - setUp("testScannerOnBulkLoadRefHFiles"); - long ts = EnvironmentEdgeManager.currentTime(); - byte[] key0 = Bytes.toBytes("k0"); - byte[] key1 = Bytes.toBytes("k1"); - String value0 = "mobValue0"; - String value1 = "mobValue1"; - String newValue0 = "new"; - Put put0 = new Put(key0); - put0.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), ts, Bytes.toBytes(value0)); - loadData(admin, bufMut, tableName, new Put[] { put0 }); - put0 = new Put(key0); - put0.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), ts, Bytes.toBytes(newValue0)); - Put put1 = new Put(key1); - put1.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), ts, Bytes.toBytes(value1)); - loadData(admin, bufMut, tableName, new Put[] { put0, put1 }); - // read the latest cell of key0. - Get get = new Get(key0); - Result result = table.get(get); - Cell cell = result.getColumnLatestCell(hcd1.getName(), Bytes.toBytes(qf1)); - assertEquals("Before compaction: mob value of k0", newValue0, - Bytes.toString(CellUtil.cloneValue(cell))); - admin.majorCompact(tableName, hcd1.getName(), CompactType.MOB); - waitUntilMobCompactionFinished(tableName); - // read the latest cell of key0, the cell seqId in bulk loaded file is not reset in the - // scanner. The cell that has "new" value is still visible. - result = table.get(get); - cell = result.getColumnLatestCell(hcd1.getName(), Bytes.toBytes(qf1)); - assertEquals("After compaction: mob value of k0", newValue0, - Bytes.toString(CellUtil.cloneValue(cell))); - // read the ref cell, not read further to the mob cell. - get = new Get(key1); - get.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(true)); - result = table.get(get); - cell = result.getColumnLatestCell(hcd1.getName(), Bytes.toBytes(qf1)); - // the ref name is the new file - Path mobFamilyPath = - MobUtils.getMobFamilyPath(TEST_UTIL.getConfiguration(), tableName, hcd1.getNameAsString()); - List paths = new ArrayList<>(); - if (fs.exists(mobFamilyPath)) { - FileStatus[] files = fs.listStatus(mobFamilyPath); - for (FileStatus file : files) { - if (!StoreFileInfo.isDelFile(file.getPath())) { - paths.add(file.getPath()); - } - } - } - assertEquals("After compaction: number of mob files:", 1, paths.size()); - assertEquals("After compaction: mob file name:", MobUtils.getMobFileName(cell), paths.get(0) - .getName()); - } - - /** - * This case tests the following mob compaction and normal compaction scenario, - * after mob compaction, the mob reference in new bulkloaded hfile will win even after it - * is compacted with some other normal hfiles. This is to make sure the mvcc is included - * after compaction for mob enabled store files. - */ - @Test - public void testGetAfterCompaction() throws Exception { - resetConf(); - conf.setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0); - String famStr = "f1"; - byte[] fam = Bytes.toBytes(famStr); - byte[] qualifier = Bytes.toBytes("q1"); - byte[] mobVal = Bytes.toBytes("01234567890"); - HTableDescriptor hdt = new HTableDescriptor(TableName.valueOf(name.getMethodName())); - hdt.addCoprocessor(CompactTwoLatestHfilesCopro.class.getName()); - HColumnDescriptor hcd = new HColumnDescriptor(fam); - hcd.setMobEnabled(true); - hcd.setMobThreshold(10); - hcd.setMaxVersions(1); - hdt.addFamily(hcd); - try { - Table table = TEST_UTIL.createTable(hdt, null); - HRegion r = TEST_UTIL.getMiniHBaseCluster().getRegions(hdt.getTableName()).get(0); - Put p = new Put(Bytes.toBytes("r1")); - p.addColumn(fam, qualifier, mobVal); - table.put(p); - // Create mob file mob1 and reference file ref1 - TEST_UTIL.flush(table.getName()); - // Make sure that it is flushed. - FileSystem fs = r.getRegionFileSystem().getFileSystem(); - Path path = r.getRegionFileSystem().getStoreDir(famStr); - waitUntilFilesShowup(fs, path, 1); - - p = new Put(Bytes.toBytes("r2")); - p.addColumn(fam, qualifier, mobVal); - table.put(p); - // Create mob file mob2 and reference file ref2 - TEST_UTIL.flush(table.getName()); - waitUntilFilesShowup(fs, path, 2); - // Do mob compaction to create mob3 and ref3 - TEST_UTIL.getAdmin().compact(hdt.getTableName(), fam, CompactType.MOB); - waitUntilFilesShowup(fs, path, 3); - - // Compact ref3 and ref2 into ref4 - TEST_UTIL.getAdmin().compact(hdt.getTableName(), fam); - waitUntilFilesShowup(fs, path, 2); - - // Sleep for some time, since TimeToLiveHFileCleaner is 0, the next run of - // clean chore is guaranteed to clean up files in archive - Thread.sleep(100); - // Run cleaner to make sure that files in archive directory are cleaned up - TEST_UTIL.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting(); - - // Get "r2" - Get get = new Get(Bytes.toBytes("r2")); - try { - Result result = table.get(get); - assertTrue(Arrays.equals(result.getValue(fam, qualifier), mobVal)); - } catch (IOException e) { - assertTrue("The MOB file doesn't exist", false); - } - } finally { - TEST_UTIL.deleteTable(hdt.getTableName()); - } - } - - private void waitUntilFilesShowup(final FileSystem fs, final Path path, final int num) - throws InterruptedException, IOException { - FileStatus[] fileList = fs.listStatus(path); - while (fileList.length != num) { - Thread.sleep(50); - fileList = fs.listStatus(path); - for (FileStatus fileStatus: fileList) { - LOG.info(Objects.toString(fileStatus)); - } - } - } - - /** - * This copro overwrites the default compaction policy. It always chooses two latest hfiles and - * compacts them into a new one. - */ - public static class CompactTwoLatestHfilesCopro implements RegionCoprocessor, RegionObserver { - - @Override - public Optional getRegionObserver() { - return Optional.of(this); - } - - @Override - public void preCompactSelection(ObserverContext c, Store store, - List candidates, CompactionLifeCycleTracker tracker) - throws IOException { - int count = candidates.size(); - if (count >= 2) { - for (int i = 0; i < count - 2; i++) { - candidates.remove(0); - } - c.bypass(); - } - } - } - - private void waitUntilMobCompactionFinished(TableName tableName) throws IOException, - InterruptedException { - long finished = EnvironmentEdgeManager.currentTime() + 60000; - CompactionState state = admin.getCompactionState(tableName, CompactType.MOB); - while (EnvironmentEdgeManager.currentTime() < finished) { - if (state == CompactionState.NONE) { - break; - } - state = admin.getCompactionState(tableName, CompactType.MOB); - Thread.sleep(10); - } - assertEquals(CompactionState.NONE, state); - } - - /** - * Gets the number of cells in the given table. - * @param table to get the scanner - * @return the number of cells - */ - private int countMobCells(final Table table) throws IOException { - Scan scan = new Scan(); - // Do not retrieve the mob data when scanning - scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE)); - ResultScanner results = table.getScanner(scan); - int count = 0; - for (Result res : results) { - count += res.size(); - } - results.close(); - return count; - } - - /** - * Gets the number of files in the mob path. - * @param isMobFile gets number of the mob files or del files - * @param familyName the family name - * @return the number of the files - */ - private int countFiles(TableName tableName, boolean isMobFile, String familyName) - throws IOException { - Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, familyName); - int count = 0; - if (fs.exists(mobDirPath)) { - FileStatus[] files = fs.listStatus(mobDirPath); - for (FileStatus file : files) { - if (isMobFile == true) { - if (!StoreFileInfo.isDelFile(file.getPath())) { - count++; - } - } else { - if (StoreFileInfo.isDelFile(file.getPath())) { - count++; - } - } - } - } - return count; - } - - private boolean verifyEncryption(TableName tableName, String familyName) throws IOException { - Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, familyName); - boolean hasFiles = false; - if (fs.exists(mobDirPath)) { - FileStatus[] files = fs.listStatus(mobDirPath); - hasFiles = files != null && files.length > 0; - Assert.assertTrue(hasFiles); - Path path = files[0].getPath(); - CacheConfig cacheConf = new CacheConfig(conf); - HStoreFile sf = new HStoreFile(TEST_UTIL.getTestFileSystem(), path, conf, cacheConf, - BloomType.NONE, true); - sf.initReader(); - HFile.Reader reader = sf.getReader().getHFileReader(); - byte[] encryptionKey = reader.getTrailer().getEncryptionKey(); - Assert.assertTrue(null != encryptionKey); - Assert.assertTrue(reader.getFileContext().getEncryptionContext().getCipher().getName() - .equals(HConstants.CIPHER_AES)); - } - return hasFiles; - } - - /** - * Gets the number of HFileLink in the mob path. - * @param familyName the family name - * @return the number of the HFileLink - */ - private int countHFileLinks(String familyName) throws IOException { - Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, familyName); - int count = 0; - if (fs.exists(mobDirPath)) { - FileStatus[] files = fs.listStatus(mobDirPath); - for (FileStatus file : files) { - if (HFileLink.isHFileLink(file.getPath())) { - count++; - } - } - } - return count; - } - - /** - * Gets the number of files. - * @param size the size of the file - * @param tableName the current table name - * @param familyName the family name - * @return the number of files large than the size - */ - private int countLargeFiles(int size, TableName tableName, String familyName) throws IOException { - Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, familyName); - int count = 0; - if (fs.exists(mobDirPath)) { - FileStatus[] files = fs.listStatus(mobDirPath); - for (FileStatus file : files) { - // ignore the del files in the mob path - if ((!StoreFileInfo.isDelFile(file.getPath())) && (file.getLen() > size)) { - count++; - } - } - } - return count; - } - - /** - * loads some data to the table. - */ - private void loadData(Admin admin, BufferedMutator table, TableName tableName, int fileNum, - int rowNumPerFile) throws IOException, InterruptedException { - if (fileNum <= 0) { - throw new IllegalArgumentException(); - } - for (int i = 0; i < fileNum * rowNumPerFile; i++) { - for (byte k0 : KEYS) { - byte[] k = new byte[] { k0 }; - byte[] key = Bytes.add(k, Bytes.toBytes(i)); - byte[] mobVal = makeDummyData(10 * (i + 1)); - Put put = new Put(key); - put.setDurability(Durability.SKIP_WAL); - put.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), mobVal); - put.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf2), mobVal); - put.addColumn(Bytes.toBytes(family2), Bytes.toBytes(qf1), mobVal); - table.mutate(put); - } - if ((i + 1) % rowNumPerFile == 0) { - table.flush(); - admin.flush(tableName); - } - } - } - - private void loadData(Admin admin, BufferedMutator table, TableName tableName, Put[] puts) - throws IOException { - table.mutate(Arrays.asList(puts)); - table.flush(); - admin.flush(tableName); - } - - private void loadDataForPartitionPolicy(Admin admin, BufferedMutator table, TableName tableName) - throws IOException { - - Put[] pArray = new Put[1000]; - - for (int i = 0; i < 1000; i ++) { - Put put0 = new Put(Bytes.toBytes("r0" + i)); - put0.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), - tsFor20151130Monday, Bytes.toBytes(mobValue0)); - pArray[i] = put0; - } - loadData(admin, bufMut, tableName, pArray); - - Put put06 = new Put(mobKey06); - put06.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), - tsFor20151128Saturday, Bytes.toBytes(mobValue0)); - - loadData(admin, bufMut, tableName, new Put[] { put06 }); - - Put put1 = new Put(mobKey1); - put1.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151201Tuesday, - Bytes.toBytes(mobValue1)); - loadData(admin, bufMut, tableName, new Put[] { put1 }); - - Put put2 = new Put(mobKey2); - put2.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151205Saturday, - Bytes.toBytes(mobValue2)); - loadData(admin, bufMut, tableName, new Put[] { put2 }); - - Put put3 = new Put(mobKey3); - put3.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151228Monday, - Bytes.toBytes(mobValue3)); - loadData(admin, bufMut, tableName, new Put[] { put3 }); - - Put put4 = new Put(mobKey4); - put4.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151231Thursday, - Bytes.toBytes(mobValue4)); - loadData(admin, bufMut, tableName, new Put[] { put4 }); - - Put put5 = new Put(mobKey5); - put5.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20160101Friday, - Bytes.toBytes(mobValue5)); - loadData(admin, bufMut, tableName, new Put[] { put5 }); - - Put put6 = new Put(mobKey6); - put6.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20160103Sunday, - Bytes.toBytes(mobValue6)); - loadData(admin, bufMut, tableName, new Put[] { put6 }); - - Put put7 = new Put(mobKey7); - put7.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20150907Monday, - Bytes.toBytes(mobValue7)); - loadData(admin, bufMut, tableName, new Put[] { put7 }); - - Put put8 = new Put(mobKey8); - put8.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151120Sunday, - Bytes.toBytes(mobValue8)); - loadData(admin, bufMut, tableName, new Put[] { put8 }); - } - - - /** - * delete the row, family and cell to create the del file - */ - private void createDelFile(Table table, TableName tableName, byte[] family, byte[] qf) - throws IOException, InterruptedException { - for (byte k0 : KEYS) { - byte[] k = new byte[] { k0 }; - // delete a family - byte[] key1 = Bytes.add(k, Bytes.toBytes(0)); - Delete delete1 = new Delete(key1); - delete1.addFamily(family); - table.delete(delete1); - // delete one row - byte[] key2 = Bytes.add(k, Bytes.toBytes(2)); - Delete delete2 = new Delete(key2); - table.delete(delete2); - // delete one cell - byte[] key3 = Bytes.add(k, Bytes.toBytes(4)); - Delete delete3 = new Delete(key3); - delete3.addColumn(family, qf); - table.delete(delete3); - } - admin.flush(tableName); - List regions = TEST_UTIL.getHBaseCluster().getRegions(tableName); - for (HRegion region : regions) { - region.waitForFlushesAndCompactions(); - region.compact(true); - } - } - /** - * Creates the dummy data with a specific size. - * @param size the size of value - * @return the dummy data - */ - private byte[] makeDummyData(int size) { - byte[] dummyData = new byte[size]; - new Random().nextBytes(dummyData); - return dummyData; - } - - /** - * Gets the split keys - */ - private byte[][] getSplitKeys() { - byte[][] splitKeys = new byte[KEYS.length - 1][]; - for (int i = 0; i < splitKeys.length; ++i) { - splitKeys[i] = new byte[] { KEYS[i + 1] }; - } - return splitKeys; - } - - private static ExecutorService createThreadPool(Configuration conf) { - int maxThreads = 10; - long keepAliveTime = 60; - final SynchronousQueue queue = new SynchronousQueue<>(); - ThreadPoolExecutor pool = new ThreadPoolExecutor(1, maxThreads, - keepAliveTime, TimeUnit.SECONDS, queue, - Threads.newDaemonThreadFactory("MobFileCompactionChore"), - new RejectedExecutionHandler() { - @Override - public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) { - try { - // waiting for a thread to pick up instead of throwing exceptions. - queue.put(r); - } catch (InterruptedException e) { - throw new RejectedExecutionException(e); - } - } - }); - pool.allowCoreThreadTimeOut(true); - return pool; - } - - private void assertRefFileNameEqual(String familyName) throws IOException { - Scan scan = new Scan(); - scan.addFamily(Bytes.toBytes(familyName)); - // Do not retrieve the mob data when scanning - scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE)); - ResultScanner results = table.getScanner(scan); - Path mobFamilyPath = MobUtils.getMobFamilyPath(TEST_UTIL.getConfiguration(), - tableName, familyName); - List actualFilePaths = new ArrayList<>(); - List expectFilePaths = new ArrayList<>(); - for (Result res : results) { - for (Cell cell : res.listCells()) { - byte[] referenceValue = CellUtil.cloneValue(cell); - String fileName = Bytes.toString(referenceValue, Bytes.SIZEOF_INT, - referenceValue.length - Bytes.SIZEOF_INT); - Path targetPath = new Path(mobFamilyPath, fileName); - if(!actualFilePaths.contains(targetPath)) { - actualFilePaths.add(targetPath); - } - } - } - results.close(); - if (fs.exists(mobFamilyPath)) { - FileStatus[] files = fs.listStatus(mobFamilyPath); - for (FileStatus file : files) { - if (!StoreFileInfo.isDelFile(file.getPath())) { - expectFilePaths.add(file.getPath()); - } - } - } - Collections.sort(actualFilePaths); - Collections.sort(expectFilePaths); - assertEquals(expectFilePaths, actualFilePaths); - } - - /** - * Resets the configuration. - */ - private void resetConf() { - conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, - MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD); - conf.setInt(MobConstants.MOB_COMPACTION_BATCH_SIZE, - MobConstants.DEFAULT_MOB_COMPACTION_BATCH_SIZE); - } - - /** - * Verify mob partition policy compaction values. - */ - private void verifyPolicyValues() throws Exception { - Get get = new Get(mobKey01); - Result result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue0))); - - get = new Get(mobKey02); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue0))); - - get = new Get(mobKey03); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue0))); - - get = new Get(mobKey04); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue0))); - - get = new Get(mobKey05); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue0))); - - get = new Get(mobKey06); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue0))); - - get = new Get(mobKey1); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue1))); - - get = new Get(mobKey2); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue2))); - - get = new Get(mobKey3); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue3))); - - get = new Get(mobKey4); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue4))); - - get = new Get(mobKey5); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue5))); - - get = new Get(mobKey6); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue6))); - - get = new Get(mobKey7); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue7))); - - get = new Get(mobKey8); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue8))); - } - - private void commonPolicyTestLogic (final String tableNameAsString, - final MobCompactPartitionPolicy pType, final boolean majorCompact, - final int expectedFileNumbers, final String[] expectedFileNames, - final boolean setupAndLoadData - ) throws Exception { - if (setupAndLoadData) { - setUpForPolicyTest(tableNameAsString, pType); - - loadDataForPartitionPolicy(admin, bufMut, tableName); - } else { - alterForPolicyTest(pType); - } - - if (majorCompact) { - admin.majorCompact(tableName, hcd1.getName(), CompactType.MOB); - } else { - admin.compact(tableName, hcd1.getName(), CompactType.MOB); - } - - waitUntilMobCompactionFinished(tableName); - - // Run cleaner to make sure that files in archive directory are cleaned up - TEST_UTIL.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting(); - - //check the number of files - Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, family1); - FileStatus[] fileList = fs.listStatus(mobDirPath); - - assertTrue(fileList.length == expectedFileNumbers); - - // the file names are expected - ArrayList fileNames = new ArrayList<>(expectedFileNumbers); - for (FileStatus file : fileList) { - fileNames.add(MobFileName.getDateFromName(file.getPath().getName())); - } - int index = 0; - for (String fileName : expectedFileNames) { - index = fileNames.indexOf(fileName); - assertTrue(index >= 0); - fileNames.remove(index); - } - - // Check daily mob files are removed from the mobdir, and only weekly mob files are there. - // Also check that there is no data loss. - - verifyPolicyValues(); - } - } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactionRequest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactionRequest.java deleted file mode 100644 index 27801533dc5a..000000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactionRequest.java +++ /dev/null @@ -1,65 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.mob.compactions; - -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartition; -import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -@Category(SmallTests.class) -public class TestPartitionedMobCompactionRequest { - - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestPartitionedMobCompactionRequest.class); - - @Test - public void testCompactedPartitionId() { - String startKey1 = "startKey1"; - String startKey2 = "startKey2"; - String date1 = "date1"; - String date2 = "date2"; - CompactionPartitionId partitionId1 = new CompactionPartitionId(startKey1, date1); - CompactionPartitionId partitionId2 = new CompactionPartitionId(startKey2, date2); - CompactionPartitionId partitionId3 = new CompactionPartitionId(startKey1, date2); - - Assert.assertTrue(partitionId1.equals(partitionId1)); - Assert.assertFalse(partitionId1.equals(partitionId2)); - Assert.assertFalse(partitionId1.equals(partitionId3)); - Assert.assertFalse(partitionId2.equals(partitionId3)); - - Assert.assertEquals(startKey1, partitionId1.getStartKey()); - Assert.assertEquals(date1, partitionId1.getDate()); - } - - @Test - public void testCompactedPartition() { - CompactionPartitionId partitionId = new CompactionPartitionId("startKey1", "date1"); - CompactionPartition partition = new CompactionPartition(partitionId); - FileStatus file = new FileStatus(1, false, 1, 1024, 1, new Path("/test")); - partition.addFile(file); - Assert.assertEquals(file, partition.listFiles().get(0)); - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java deleted file mode 100644 index 4e6461aaf800..000000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java +++ /dev/null @@ -1,958 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.mob.compactions; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; -import java.text.ParseException; -import java.util.ArrayList; -import java.util.Calendar; -import java.util.Collection; -import java.util.Collections; -import java.util.Date; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.RejectedExecutionHandler; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellComparatorImpl; -import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValue.Type; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy; -import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hadoop.hbase.io.hfile.HFileContext; -import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; -import org.apache.hadoop.hbase.mob.MobConstants; -import org.apache.hadoop.hbase.mob.MobFileName; -import org.apache.hadoop.hbase.mob.MobUtils; -import org.apache.hadoop.hbase.mob.compactions.MobCompactionRequest.CompactionType; -import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionDelPartition; -import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartition; -import org.apache.hadoop.hbase.regionserver.BloomType; -import org.apache.hadoop.hbase.regionserver.HStore; -import org.apache.hadoop.hbase.regionserver.HStoreFile; -import org.apache.hadoop.hbase.regionserver.KeyValueScanner; -import org.apache.hadoop.hbase.regionserver.ScanInfo; -import org.apache.hadoop.hbase.regionserver.ScanType; -import org.apache.hadoop.hbase.regionserver.StoreFileScanner; -import org.apache.hadoop.hbase.regionserver.StoreFileWriter; -import org.apache.hadoop.hbase.regionserver.StoreScanner; -import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.hbase.util.Threads; -import org.apache.hadoop.hdfs.DistributedFileSystem; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Category(LargeTests.class) -public class TestPartitionedMobCompactor { - - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestPartitionedMobCompactor.class); - - private static final Logger LOG = LoggerFactory.getLogger(TestPartitionedMobCompactor.class); - private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private final static String family = "family"; - private final static String qf = "qf"; - private final long DAY_IN_MS = 1000 * 60 * 60 * 24; - private static byte[] KEYS = Bytes.toBytes("012"); - private HColumnDescriptor hcd = new HColumnDescriptor(family); - private Configuration conf = TEST_UTIL.getConfiguration(); - private CacheConfig cacheConf = new CacheConfig(conf); - private FileSystem fs; - private List mobFiles = new ArrayList<>(); - private List delFiles = new ArrayList<>(); - private List allFiles = new ArrayList<>(); - private Path basePath; - private String mobSuffix; - private String delSuffix; - private static ExecutorService pool; - - @Rule - public TestName name = new TestName(); - - @BeforeClass - public static void setUpBeforeClass() throws Exception { - TEST_UTIL.getConfiguration().setInt("hfile.format.version", 3); - // Inject our customized DistributedFileSystem - TEST_UTIL.getConfiguration().setClass("fs.hdfs.impl", FaultyDistributedFileSystem.class, - DistributedFileSystem.class); - TEST_UTIL.startMiniCluster(1); - pool = createThreadPool(); - } - - @AfterClass - public static void tearDownAfterClass() throws Exception { - pool.shutdown(); - TEST_UTIL.shutdownMiniCluster(); - } - - private void init(String tableName) throws Exception { - fs = FileSystem.get(conf); - Path testDir = FSUtils.getRootDir(conf); - Path mobTestDir = new Path(testDir, MobConstants.MOB_DIR_NAME); - basePath = new Path(new Path(mobTestDir, tableName), family); - mobSuffix = TEST_UTIL.getRandomUUID().toString().replaceAll("-", ""); - delSuffix = TEST_UTIL.getRandomUUID().toString().replaceAll("-", "") + "_del"; - allFiles.clear(); - mobFiles.clear(); - delFiles.clear(); - } - - @Test - public void testCompactionSelectAllFilesWeeklyPolicy() throws Exception { - String tableName = "testCompactionSelectAllFilesWeeklyPolicy"; - testCompactionAtMergeSize(tableName, MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD, - CompactionType.ALL_FILES, false, false, new Date(), MobCompactPartitionPolicy.WEEKLY, 1); - } - - @Test - public void testCompactionSelectPartFilesWeeklyPolicy() throws Exception { - String tableName = "testCompactionSelectPartFilesWeeklyPolicy"; - testCompactionAtMergeSize(tableName, 4000, CompactionType.PART_FILES, false, false, - new Date(), MobCompactPartitionPolicy.WEEKLY, 1); - } - - @Test - public void testCompactionSelectPartFilesWeeklyPolicyWithPastWeek() throws Exception { - String tableName = "testCompactionSelectPartFilesWeeklyPolicyWithPastWeek"; - Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS)); - testCompactionAtMergeSize(tableName, 700, CompactionType.PART_FILES, false, false, dateLastWeek, - MobCompactPartitionPolicy.WEEKLY, 7); - } - - @Test - public void testCompactionSelectAllFilesWeeklyPolicyWithPastWeek() throws Exception { - String tableName = "testCompactionSelectAllFilesWeeklyPolicyWithPastWeek"; - Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS)); - testCompactionAtMergeSize(tableName, 3000, CompactionType.ALL_FILES, - false, false, dateLastWeek, MobCompactPartitionPolicy.WEEKLY, 7); - } - - @Test - public void testCompactionSelectAllFilesMonthlyPolicy() throws Exception { - String tableName = "testCompactionSelectAllFilesMonthlyPolicy"; - Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS)); - testCompactionAtMergeSize(tableName, MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD, - CompactionType.ALL_FILES, false, false, dateLastWeek, - MobCompactPartitionPolicy.MONTHLY, 7); - } - - @Test - public void testCompactionSelectNoFilesWithinCurrentWeekMonthlyPolicy() throws Exception { - String tableName = "testCompactionSelectNoFilesWithinCurrentWeekMonthlyPolicy"; - testCompactionAtMergeSize(tableName, MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD, - CompactionType.PART_FILES, false, false, new Date(), MobCompactPartitionPolicy.MONTHLY, 1); - } - - @Test - public void testCompactionSelectPartFilesMonthlyPolicy() throws Exception { - String tableName = "testCompactionSelectPartFilesMonthlyPolicy"; - testCompactionAtMergeSize(tableName, 4000, CompactionType.PART_FILES, false, false, - new Date(), MobCompactPartitionPolicy.MONTHLY, 1); - } - - @Test - public void testCompactionSelectPartFilesMonthlyPolicyWithPastWeek() throws Exception { - String tableName = "testCompactionSelectPartFilesMonthlyPolicyWithPastWeek"; - Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS)); - Calendar calendar = Calendar.getInstance(); - Date firstDayOfCurrentMonth = MobUtils.getFirstDayOfMonth(calendar, new Date()); - CompactionType type = CompactionType.PART_FILES; - long mergeSizeMultiFactor = 7; - - - // The dateLastWeek may not really be last week, suppose that it runs at 2/1/2017, it is going - // to be last month and the monthly policy is going to be applied here. - if (dateLastWeek.before(firstDayOfCurrentMonth)) { - type = CompactionType.ALL_FILES; - mergeSizeMultiFactor *= 4; - } - - testCompactionAtMergeSize(tableName, 700, type, false, false, dateLastWeek, - MobCompactPartitionPolicy.MONTHLY, mergeSizeMultiFactor); - } - - @Test - public void testCompactionSelectAllFilesMonthlyPolicyWithPastWeek() throws Exception { - String tableName = "testCompactionSelectAllFilesMonthlyPolicyWithPastWeek"; - Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS)); - - testCompactionAtMergeSize(tableName, 3000, CompactionType.ALL_FILES, - false, false, dateLastWeek, MobCompactPartitionPolicy.MONTHLY, 7); - } - - @Test - public void testCompactionSelectPartFilesMonthlyPolicyWithPastMonth() throws Exception { - String tableName = "testCompactionSelectPartFilesMonthlyPolicyWithPastMonth"; - - // back 5 weeks, it is going to be a past month - Date dateLastMonth = new Date(System.currentTimeMillis() - (7 * 5 * DAY_IN_MS)); - testCompactionAtMergeSize(tableName, 200, CompactionType.PART_FILES, false, false, dateLastMonth, - MobCompactPartitionPolicy.MONTHLY, 28); - } - - @Test - public void testCompactionSelectAllFilesMonthlyPolicyWithPastMonth() throws Exception { - String tableName = "testCompactionSelectAllFilesMonthlyPolicyWithPastMonth"; - - // back 5 weeks, it is going to be a past month - Date dateLastMonth = new Date(System.currentTimeMillis() - (7 * 5 * DAY_IN_MS)); - testCompactionAtMergeSize(tableName, 750, CompactionType.ALL_FILES, - false, false, dateLastMonth, MobCompactPartitionPolicy.MONTHLY, 28); - } - - @Test - public void testCompactionSelectWithAllFiles() throws Exception { - String tableName = "testCompactionSelectWithAllFiles"; - // If there is only 1 file, it will not be compacted with _del files, so - // It wont be CompactionType.ALL_FILES in this case, do not create with _del files. - testCompactionAtMergeSize(tableName, MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD, - CompactionType.ALL_FILES, false, false); - } - - @Test - public void testCompactionSelectWithPartFiles() throws Exception { - String tableName = "testCompactionSelectWithPartFiles"; - testCompactionAtMergeSize(tableName, 4000, CompactionType.PART_FILES, false); - } - - @Test - public void testCompactionSelectWithForceAllFiles() throws Exception { - String tableName = "testCompactionSelectWithForceAllFiles"; - testCompactionAtMergeSize(tableName, Long.MAX_VALUE, CompactionType.ALL_FILES, true); - } - - private void testCompactionAtMergeSize(final String tableName, - final long mergeSize, final CompactionType type, final boolean isForceAllFiles) - throws Exception { - testCompactionAtMergeSize(tableName, mergeSize, type, isForceAllFiles, true); - } - - private void testCompactionAtMergeSize(final String tableName, - final long mergeSize, final CompactionType type, final boolean isForceAllFiles, - final boolean createDelFiles) - throws Exception { - Date date = new Date(); - testCompactionAtMergeSize(tableName, mergeSize, type, isForceAllFiles, createDelFiles, date); - } - - private void testCompactionAtMergeSize(final String tableName, - final long mergeSize, final CompactionType type, final boolean isForceAllFiles, - final boolean createDelFiles, final Date date) - throws Exception { - testCompactionAtMergeSize(tableName, mergeSize, type, isForceAllFiles, createDelFiles, date, - MobCompactPartitionPolicy.DAILY, 1); - } - - private void testCompactionAtMergeSize(final String tableName, - final long mergeSize, final CompactionType type, final boolean isForceAllFiles, - final boolean createDelFiles, final Date date, final MobCompactPartitionPolicy policy, - final long mergeSizeMultiFactor) - throws Exception { - resetConf(); - init(tableName); - int count = 10; - // create 10 mob files. - createStoreFiles(basePath, family, qf, count, Type.Put, date); - - if (createDelFiles) { - // create 10 del files - createStoreFiles(basePath, family, qf, count, Type.Delete, date); - } - - Calendar calendar = Calendar.getInstance(); - Date firstDayOfCurrentWeek = MobUtils.getFirstDayOfWeek(calendar, new Date()); - - listFiles(); - List expectedStartKeys = new ArrayList<>(); - for(FileStatus file : mobFiles) { - if(file.getLen() < mergeSize * mergeSizeMultiFactor) { - String fileName = file.getPath().getName(); - String startKey = fileName.substring(0, 32); - - // If the policy is monthly and files are in current week, they will be skipped - // in minor compcation. - boolean skipCompaction = false; - if (policy == MobCompactPartitionPolicy.MONTHLY) { - String fileDateStr = MobFileName.getDateFromName(fileName); - Date fileDate; - try { - fileDate = MobUtils.parseDate(fileDateStr); - } catch (ParseException e) { - LOG.warn("Failed to parse date " + fileDateStr, e); - fileDate = new Date(); - } - if (!fileDate.before(firstDayOfCurrentWeek)) { - skipCompaction = true; - } - } - - // If it is not an major mob compaction and del files are there, - // these mob files wont be compacted. - if (isForceAllFiles || (!createDelFiles && !skipCompaction)) { - expectedStartKeys.add(startKey); - } - } - } - - // Set the policy - this.hcd.setMobCompactPartitionPolicy(policy); - // set the mob compaction mergeable threshold - conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize); - testSelectFiles(tableName, type, isForceAllFiles, expectedStartKeys); - // go back to the default daily policy - this.hcd.setMobCompactPartitionPolicy(MobCompactPartitionPolicy.DAILY); - } - - @Test - public void testCompactDelFilesWithDefaultBatchSize() throws Exception { - testCompactDelFilesAtBatchSize(name.getMethodName(), MobConstants.DEFAULT_MOB_COMPACTION_BATCH_SIZE, - MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT); - } - - @Test - public void testCompactDelFilesWithSmallBatchSize() throws Exception { - testCompactDelFilesAtBatchSize(name.getMethodName(), 4, MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT); - } - - @Test - public void testCompactDelFilesChangeMaxDelFileCount() throws Exception { - testCompactDelFilesAtBatchSize(name.getMethodName(), 4, 2); - } - - @Test - public void testCompactFilesWithDstDirFull() throws Exception { - String tableName = name.getMethodName(); - fs = FileSystem.get(conf); - FaultyDistributedFileSystem faultyFs = (FaultyDistributedFileSystem)fs; - Path testDir = FSUtils.getRootDir(conf); - Path mobTestDir = new Path(testDir, MobConstants.MOB_DIR_NAME); - basePath = new Path(new Path(mobTestDir, tableName), family); - - try { - int count = 2; - // create 2 mob files. - createStoreFiles(basePath, family, qf, count, Type.Put, true, new Date()); - listFiles(); - - TableName tName = TableName.valueOf(tableName); - MobCompactor compactor = new PartitionedMobCompactor(conf, faultyFs, tName, hcd, pool); - faultyFs.setThrowException(true); - try { - compactor.compact(allFiles, true); - } catch (IOException e) { - System.out.println("Expected exception, ignore"); - } - - // Verify that all the files in tmp directory are cleaned up - Path tempPath = new Path(MobUtils.getMobHome(conf), MobConstants.TEMP_DIR_NAME); - FileStatus[] ls = faultyFs.listStatus(tempPath); - - // Only .bulkload under this directory - assertTrue(ls.length == 1); - assertTrue(MobConstants.BULKLOAD_DIR_NAME.equalsIgnoreCase(ls[0].getPath().getName())); - - Path bulkloadPath = new Path(tempPath, new Path(MobConstants.BULKLOAD_DIR_NAME, new Path( - tName.getNamespaceAsString(), tName.getQualifierAsString()))); - - // Nothing in bulkLoad directory - FileStatus[] lsBulkload = faultyFs.listStatus(bulkloadPath); - assertTrue(lsBulkload.length == 0); - - } finally { - faultyFs.setThrowException(false); - } - } - - /** - * Create mulitple partition files - */ - private void createMobFile(Path basePath) throws IOException { - HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build(); - MobFileName mobFileName = null; - int ii = 0; - Date today = new Date(); - for (byte k0 : KEYS) { - byte[] startRow = Bytes.toBytes(ii++); - - mobFileName = MobFileName.create(startRow, MobUtils.formatDate(today), mobSuffix); - - StoreFileWriter mobFileWriter = - new StoreFileWriter.Builder(conf, cacheConf, fs).withFileContext(meta) - .withFilePath(new Path(basePath, mobFileName.getFileName())).build(); - - long now = System.currentTimeMillis(); - try { - for (int i = 0; i < 10; i++) { - byte[] key = Bytes.add(Bytes.toBytes(k0), Bytes.toBytes(i)); - byte[] dummyData = new byte[5000]; - new Random().nextBytes(dummyData); - mobFileWriter.append( - new KeyValue(key, Bytes.toBytes(family), Bytes.toBytes(qf), now, Type.Put, dummyData)); - } - } finally { - mobFileWriter.close(); - } - } - } - - /** - * Create mulitple partition delete files - */ - private void createMobDelFile(Path basePath, int startKey) throws IOException { - HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build(); - MobFileName mobFileName = null; - Date today = new Date(); - - byte[] startRow = Bytes.toBytes(startKey); - - mobFileName = MobFileName.create(startRow, MobUtils.formatDate(today), delSuffix); - - StoreFileWriter mobFileWriter = - new StoreFileWriter.Builder(conf, cacheConf, fs).withFileContext(meta) - .withFilePath(new Path(basePath, mobFileName.getFileName())).build(); - - long now = System.currentTimeMillis(); - try { - byte[] key = Bytes.add(Bytes.toBytes(KEYS[startKey]), Bytes.toBytes(0)); - byte[] dummyData = new byte[5000]; - new Random().nextBytes(dummyData); - mobFileWriter.append( - new KeyValue(key, Bytes.toBytes(family), Bytes.toBytes(qf), now, Type.Delete, dummyData)); - key = Bytes.add(Bytes.toBytes(KEYS[startKey]), Bytes.toBytes(2)); - mobFileWriter.append( - new KeyValue(key, Bytes.toBytes(family), Bytes.toBytes(qf), now, Type.Delete, dummyData)); - key = Bytes.add(Bytes.toBytes(KEYS[startKey]), Bytes.toBytes(4)); - mobFileWriter.append( - new KeyValue(key, Bytes.toBytes(family), Bytes.toBytes(qf), now, Type.Delete, dummyData)); - - } finally { - mobFileWriter.close(); - } - } - - @Test - public void testCompactFilesWithoutDelFile() throws Exception { - String tableName = "testCompactFilesWithoutDelFile"; - resetConf(); - init(tableName); - - createMobFile(basePath); - - listFiles(); - - PartitionedMobCompactor compactor = new PartitionedMobCompactor(conf, fs, - TableName.valueOf(tableName), hcd, pool) { - @Override - public List compact(List files, boolean isForceAllFiles) - throws IOException { - if (files == null || files.isEmpty()) { - return null; - } - - PartitionedMobCompactionRequest request = select(files, isForceAllFiles); - - // Make sure that there is no del Partitions - assertTrue(request.getDelPartitions().size() == 0); - - // Make sure that when there is no startKey/endKey for partition. - for (CompactionPartition p : request.getCompactionPartitions()) { - assertTrue(p.getStartKey() == null); - assertTrue(p.getEndKey() == null); - } - return null; - } - }; - - compactor.compact(allFiles, true); - } - - static class MyPartitionedMobCompactor extends PartitionedMobCompactor { - int delPartitionSize = 0; - int PartitionsIncludeDelFiles = 0; - CacheConfig cacheConfig = null; - - MyPartitionedMobCompactor(Configuration conf, FileSystem fs, TableName tableName, - ColumnFamilyDescriptor column, ExecutorService pool, final int delPartitionSize, - final CacheConfig cacheConf, final int PartitionsIncludeDelFiles) - throws IOException { - super(conf, fs, tableName, column, pool); - this.delPartitionSize = delPartitionSize; - this.cacheConfig = cacheConf; - this.PartitionsIncludeDelFiles = PartitionsIncludeDelFiles; - } - - @Override public List compact(List files, boolean isForceAllFiles) - throws IOException { - if (files == null || files.isEmpty()) { - return null; - } - PartitionedMobCompactionRequest request = select(files, isForceAllFiles); - - assertTrue(request.getDelPartitions().size() == delPartitionSize); - if (request.getDelPartitions().size() > 0) { - for (CompactionPartition p : request.getCompactionPartitions()) { - assertTrue(p.getStartKey() != null); - assertTrue(p.getEndKey() != null); - } - } - - try { - for (CompactionDelPartition delPartition : request.getDelPartitions()) { - for (Path newDelPath : delPartition.listDelFiles()) { - HStoreFile sf = - new HStoreFile(fs, newDelPath, conf, this.cacheConfig, BloomType.NONE, true); - // pre-create reader of a del file to avoid race condition when opening the reader in - // each partition. - sf.initReader(); - delPartition.addStoreFile(sf); - } - } - - // Make sure that CompactionDelPartitions does not overlap - CompactionDelPartition prevDelP = null; - for (CompactionDelPartition delP : request.getDelPartitions()) { - assertTrue( - Bytes.compareTo(delP.getId().getStartKey(), delP.getId().getEndKey()) <= 0); - - if (prevDelP != null) { - assertTrue( - Bytes.compareTo(prevDelP.getId().getEndKey(), delP.getId().getStartKey()) < 0); - } - } - - int affectedPartitions = 0; - - // Make sure that only del files within key range for a partition is included in compaction. - // compact the mob files by partitions in parallel. - for (CompactionPartition partition : request.getCompactionPartitions()) { - List delFiles = getListOfDelFilesForPartition(partition, request.getDelPartitions()); - if (!request.getDelPartitions().isEmpty()) { - if (!((Bytes.compareTo(request.getDelPartitions().get(0).getId().getStartKey(), - partition.getEndKey()) > 0) || (Bytes.compareTo( - request.getDelPartitions().get(request.getDelPartitions().size() - 1).getId() - .getEndKey(), partition.getStartKey()) < 0))) { - - if (delFiles.size() > 0) { - assertTrue(delFiles.size() == 1); - affectedPartitions += delFiles.size(); - assertTrue(Bytes.compareTo(partition.getStartKey(), - CellUtil.cloneRow(delFiles.get(0).getLastKey().get())) <= 0); - assertTrue(Bytes.compareTo(partition.getEndKey(), - CellUtil.cloneRow(delFiles.get(delFiles.size() - 1).getFirstKey().get())) >= 0); - } - } - } - } - // The del file is only included in one partition - assertTrue(affectedPartitions == PartitionsIncludeDelFiles); - } finally { - for (CompactionDelPartition delPartition : request.getDelPartitions()) { - for (HStoreFile storeFile : delPartition.getStoreFiles()) { - try { - storeFile.closeStoreFile(true); - } catch (IOException e) { - LOG.warn("Failed to close the reader on store file " + storeFile.getPath(), e); - } - } - } - } - - return null; - } - } - - @Test - public void testCompactFilesWithOneDelFile() throws Exception { - String tableName = "testCompactFilesWithOneDelFile"; - resetConf(); - init(tableName); - - // Create only del file. - createMobFile(basePath); - createMobDelFile(basePath, 2); - - listFiles(); - - MyPartitionedMobCompactor compactor = new MyPartitionedMobCompactor(conf, fs, - TableName.valueOf(tableName), hcd, pool, 1, cacheConf, 1); - - compactor.compact(allFiles, true); - } - - @Test - public void testCompactFilesWithMultiDelFiles() throws Exception { - String tableName = "testCompactFilesWithMultiDelFiles"; - resetConf(); - init(tableName); - - // Create only del file. - createMobFile(basePath); - createMobDelFile(basePath, 0); - createMobDelFile(basePath, 1); - createMobDelFile(basePath, 2); - - listFiles(); - - MyPartitionedMobCompactor compactor = new MyPartitionedMobCompactor(conf, fs, - TableName.valueOf(tableName), hcd, pool, 3, cacheConf, 3); - compactor.compact(allFiles, true); - } - - private void testCompactDelFilesAtBatchSize(String tableName, int batchSize, - int delfileMaxCount) throws Exception { - resetConf(); - init(tableName); - // create 20 mob files. - createStoreFiles(basePath, family, qf, 20, Type.Put, new Date()); - // create 13 del files - createStoreFiles(basePath, family, qf, 13, Type.Delete, new Date()); - listFiles(); - - // set the max del file count - conf.setInt(MobConstants.MOB_DELFILE_MAX_COUNT, delfileMaxCount); - // set the mob compaction batch size - conf.setInt(MobConstants.MOB_COMPACTION_BATCH_SIZE, batchSize); - testCompactDelFiles(tableName, 1, 13, false); - } - - /** - * Tests the selectFiles - * @param tableName the table name - * @param type the expected compaction type - * @param isForceAllFiles whether all the mob files are selected - * @param expected the expected start keys - */ - private void testSelectFiles(String tableName, final CompactionType type, - final boolean isForceAllFiles, final List expected) throws IOException { - PartitionedMobCompactor compactor = new PartitionedMobCompactor(conf, fs, - TableName.valueOf(tableName), hcd, pool) { - @Override - public List compact(List files, boolean isForceAllFiles) - throws IOException { - if (files == null || files.isEmpty()) { - return null; - } - PartitionedMobCompactionRequest request = select(files, isForceAllFiles); - - // Make sure that when there is no del files, there will be no startKey/endKey for partition. - if (request.getDelPartitions().size() == 0) { - for (CompactionPartition p : request.getCompactionPartitions()) { - assertTrue(p.getStartKey() == null); - assertTrue(p.getEndKey() == null); - } - } - - // Make sure that CompactionDelPartitions does not overlap - CompactionDelPartition prevDelP = null; - for (CompactionDelPartition delP : request.getDelPartitions()) { - assertTrue(Bytes.compareTo(delP.getId().getStartKey(), - delP.getId().getEndKey()) <= 0); - - if (prevDelP != null) { - assertTrue(Bytes.compareTo(prevDelP.getId().getEndKey(), - delP.getId().getStartKey()) < 0); - } - } - - // Make sure that only del files within key range for a partition is included in compaction. - // compact the mob files by partitions in parallel. - for (CompactionPartition partition : request.getCompactionPartitions()) { - List delFiles = getListOfDelFilesForPartition(partition, request.getDelPartitions()); - if (!request.getDelPartitions().isEmpty()) { - if (!((Bytes.compareTo(request.getDelPartitions().get(0).getId().getStartKey(), - partition.getEndKey()) > 0) || (Bytes.compareTo( - request.getDelPartitions().get(request.getDelPartitions().size() - 1).getId() - .getEndKey(), partition.getStartKey()) < 0))) { - if (delFiles.size() > 0) { - assertTrue(Bytes.compareTo(partition.getStartKey(), - delFiles.get(0).getFirstKey().get().getRowArray()) >= 0); - assertTrue(Bytes.compareTo(partition.getEndKey(), - delFiles.get(delFiles.size() - 1).getLastKey().get().getRowArray()) <= 0); - } - } - } - } - - // assert the compaction type - assertEquals(type, request.type); - // assert get the right partitions - compareCompactedPartitions(expected, request.compactionPartitions); - // assert get the right del files - compareDelFiles(request.getDelPartitions()); - return null; - } - }; - compactor.compact(allFiles, isForceAllFiles); - } - - /** - * Tests the compacteDelFile - * @param tableName the table name - * @param expectedFileCount the expected file count - * @param expectedCellCount the expected cell count - * @param isForceAllFiles whether all the mob files are selected - */ - private void testCompactDelFiles(String tableName, final int expectedFileCount, - final int expectedCellCount, boolean isForceAllFiles) throws IOException { - PartitionedMobCompactor compactor = new PartitionedMobCompactor(conf, fs, - TableName.valueOf(tableName), hcd, pool) { - @Override - protected List performCompaction(PartitionedMobCompactionRequest request) - throws IOException { - List delFilePaths = new ArrayList<>(); - for (CompactionDelPartition delPartition: request.getDelPartitions()) { - for (Path p : delPartition.listDelFiles()) { - delFilePaths.add(p); - } - } - List newDelPaths = compactDelFiles(request, delFilePaths); - // assert the del files are merged. - assertEquals(expectedFileCount, newDelPaths.size()); - assertEquals(expectedCellCount, countDelCellsInDelFiles(newDelPaths)); - return null; - } - }; - compactor.compact(allFiles, isForceAllFiles); - } - - /** - * Lists the files in the path - */ - private void listFiles() throws IOException { - for (FileStatus file : fs.listStatus(basePath)) { - allFiles.add(file); - if (file.getPath().getName().endsWith("_del")) { - delFiles.add(file.getPath()); - } else { - mobFiles.add(file); - } - } - } - - /** - * Compares the compacted partitions. - * @param partitions the collection of CompactedPartitions - */ - private void compareCompactedPartitions(List expected, - Collection partitions) { - List actualKeys = new ArrayList<>(); - for (CompactionPartition partition : partitions) { - actualKeys.add(partition.getPartitionId().getStartKey()); - } - Collections.sort(expected); - Collections.sort(actualKeys); - assertEquals(expected.size(), actualKeys.size()); - for (int i = 0; i < expected.size(); i++) { - assertEquals(expected.get(i), actualKeys.get(i)); - } - } - - /** - * Compares the del files. - * @param delPartitions all del partitions - */ - private void compareDelFiles(List delPartitions) { - Map delMap = new HashMap<>(); - for (CompactionDelPartition delPartition : delPartitions) { - for (Path f : delPartition.listDelFiles()) { - delMap.put(f, f); - } - } - for (Path f : delFiles) { - assertTrue(delMap.containsKey(f)); - } - } - - /** - * Creates store files. - * @param basePath the path to create file - * @param family the column family name - * @param qualifier the column qualifier assigned to data values - * @param count the store file number - * @param type the row key type - * @param date the latest timestamp when an instance of MobFileName is created - */ - private void createStoreFiles(Path basePath, String family, String qualifier, int count, - Type type, final Date date) throws IOException { - createStoreFiles(basePath, family, qualifier, count, type, false, date); - } - - private void createStoreFiles(Path basePath, String family, String qualifier, int count, - Type type, boolean sameStartKey, final Date date) throws IOException { - HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build(); - String startKey = "row_"; - MobFileName mobFileName = null; - for (int i = 0; i < count; i++) { - byte[] startRow; - if (sameStartKey) { - // When creating multiple files under one partition, suffix needs to be different. - startRow = Bytes.toBytes(startKey); - mobSuffix = TEST_UTIL.getRandomUUID().toString().replaceAll("-", ""); - delSuffix = TEST_UTIL.getRandomUUID().toString().replaceAll("-", "") + "_del"; - } else { - startRow = Bytes.toBytes(startKey + i); - } - if(type.equals(Type.Delete)) { - mobFileName = MobFileName.create(startRow, MobUtils.formatDate(date), delSuffix); - } - if(type.equals(Type.Put)){ - mobFileName = MobFileName.create(startRow, MobUtils.formatDate(date), mobSuffix); - } - StoreFileWriter mobFileWriter = new StoreFileWriter.Builder(conf, cacheConf, fs) - .withFileContext(meta).withFilePath(new Path(basePath, mobFileName.getFileName())).build(); - writeStoreFile(mobFileWriter, startRow, Bytes.toBytes(family), Bytes.toBytes(qualifier), - type, (i+1)*1000); - } - } - - /** - * Writes data to store file. - * @param writer the store file writer - * @param row the row key - * @param family the family name - * @param qualifier the column qualifier - * @param type the key type - * @param size the size of value - */ - private static void writeStoreFile(final StoreFileWriter writer, byte[]row, byte[] family, - byte[] qualifier, Type type, int size) throws IOException { - long now = System.currentTimeMillis(); - try { - byte[] dummyData = new byte[size]; - new Random().nextBytes(dummyData); - writer.append(new KeyValue(row, family, qualifier, now, type, dummyData)); - } finally { - writer.close(); - } - } - - /** - * Gets the number of del cell in the del files - * @param paths the del file paths - * @return the cell size - */ - private int countDelCellsInDelFiles(List paths) throws IOException { - List sfs = new ArrayList<>(); - int size = 0; - for (Path path : paths) { - HStoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true); - sfs.add(sf); - } - List scanners = new ArrayList<>(StoreFileScanner.getScannersForStoreFiles(sfs, - false, true, false, false, HConstants.LATEST_TIMESTAMP)); - long timeToPurgeDeletes = Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0); - long ttl = HStore.determineTTLFromFamily(hcd); - ScanInfo scanInfo = new ScanInfo(conf, hcd, ttl, timeToPurgeDeletes, CellComparatorImpl.COMPARATOR); - StoreScanner scanner = new StoreScanner(scanInfo, ScanType.COMPACT_RETAIN_DELETES, scanners); - List results = new ArrayList<>(); - boolean hasMore = true; - - while (hasMore) { - hasMore = scanner.next(results); - size += results.size(); - results.clear(); - } - scanner.close(); - return size; - } - - private static ExecutorService createThreadPool() { - int maxThreads = 10; - long keepAliveTime = 60; - final SynchronousQueue queue = new SynchronousQueue<>(); - ThreadPoolExecutor pool = new ThreadPoolExecutor(1, maxThreads, keepAliveTime, - TimeUnit.SECONDS, queue, Threads.newDaemonThreadFactory("MobFileCompactionChore"), - new RejectedExecutionHandler() { - @Override - public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) { - try { - // waiting for a thread to pick up instead of throwing exceptions. - queue.put(r); - } catch (InterruptedException e) { - throw new RejectedExecutionException(e); - } - } - }); - ((ThreadPoolExecutor) pool).allowCoreThreadTimeOut(true); - return pool; - } - - /** - * Resets the configuration. - */ - private void resetConf() { - conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, - MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD); - conf.setInt(MobConstants.MOB_DELFILE_MAX_COUNT, MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT); - conf.setInt(MobConstants.MOB_COMPACTION_BATCH_SIZE, - MobConstants.DEFAULT_MOB_COMPACTION_BATCH_SIZE); - } - - /** - * The customized Distributed File System Implementation - */ - static class FaultyDistributedFileSystem extends DistributedFileSystem { - private volatile boolean throwException = false; - - public FaultyDistributedFileSystem() { - super(); - } - - public void setThrowException(boolean throwException) { - this.throwException = throwException; - } - - @Override - public boolean rename(Path src, Path dst) throws IOException { - if (throwException) { - throw new IOException("No more files allowed"); - } - return super.rename(src, dst); - } - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java index cccc97005398..822a85651350 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java @@ -270,30 +270,6 @@ public void testMajorCompactionAfterDelete() throws Exception { // region.compactStores(); region.compact(true); assertEquals("After compaction: store files", 1, countStoreFiles()); - // still have original mob hfiles and now added a mob del file - assertEquals("After compaction: mob files", numHfiles + 1, countMobFiles()); - - Scan scan = new Scan(); - scan.setRaw(true); - InternalScanner scanner = region.getScanner(scan); - List results = new ArrayList<>(); - scanner.next(results); - int deleteCount = 0; - while (!results.isEmpty()) { - for (Cell c : results) { - if (c.getTypeByte() == KeyValue.Type.DeleteFamily.getCode()) { - deleteCount++; - assertTrue(Bytes.equals(CellUtil.cloneRow(c), deleteRow)); - } - } - results.clear(); - scanner.next(results); - } - // assert the delete mark is retained after the major compaction - assertEquals(1, deleteCount); - scanner.close(); - // assert the deleted cell is not counted - assertEquals("The cells in mob files", numHfiles - 1, countMobCellsInMobFiles(1)); } private int countStoreFiles() throws IOException { @@ -424,38 +400,4 @@ private int countReferencedMobFiles() throws IOException { return files.size(); } - private int countMobCellsInMobFiles(int expectedNumDelfiles) throws IOException { - Configuration copyOfConf = new Configuration(conf); - copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f); - CacheConfig cacheConfig = new CacheConfig(copyOfConf); - Path mobDirPath = MobUtils.getMobFamilyPath(conf, htd.getTableName(), hcd.getNameAsString()); - List sfs = new ArrayList<>(); - int numDelfiles = 0; - int size = 0; - if (fs.exists(mobDirPath)) { - for (FileStatus f : fs.listStatus(mobDirPath)) { - HStoreFile sf = new HStoreFile(fs, f.getPath(), conf, cacheConfig, BloomType.NONE, true); - sfs.add(sf); - if (StoreFileInfo.isDelFile(sf.getPath())) { - numDelfiles++; - } - } - - List scanners = StoreFileScanner.getScannersForStoreFiles(sfs, false, true, - false, false, HConstants.LATEST_TIMESTAMP); - long timeToPurgeDeletes = Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0); - long ttl = HStore.determineTTLFromFamily(hcd); - ScanInfo scanInfo = new ScanInfo(copyOfConf, hcd, ttl, timeToPurgeDeletes, - CellComparatorImpl.COMPARATOR); - StoreScanner scanner = new StoreScanner(scanInfo, ScanType.COMPACT_DROP_DELETES, scanners); - try { - size += UTIL.countRows(scanner); - } finally { - scanner.close(); - } - } - // assert the number of the existing del files - assertEquals(expectedNumDelfiles, numDelfiles); - return size; - } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java index e8096fb95106..5bbca546389d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java @@ -402,7 +402,7 @@ String createMobFileName(String oldFileName) { String startKey = mobFileName.getStartKey(); String date = mobFileName.getDate(); return MobFileName.create(startKey, date, - TEST_UTIL.getRandomUUID().toString().replaceAll("-", "")) + TEST_UTIL.getRandomUUID().toString().replaceAll("-", ""), "abcdef") .getFileName(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java index 09ae96fc0a2b..4412dc85c79f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java @@ -98,6 +98,7 @@ public void setUp() { /** * This creates a table and then corrupts a mob file. Hbck should quarantine the file. */ + @SuppressWarnings("deprecation") @Test public void testQuarantineCorruptMobFile() throws Exception { TableName table = TableName.valueOf(name.getMethodName()); From 34962d992d7d9625a877359222fb1c53b5531775 Mon Sep 17 00:00:00 2001 From: Vladimir Rodionov Date: Thu, 17 Oct 2019 17:42:17 -0700 Subject: [PATCH 04/15] HBASE-22749: add-on #1 --- .../hbase/master/MasterRpcServices.java | 2 +- .../hbase/master/MobFileCleanerChore.java | 16 ++++++----- .../hbase/mob/DefaultMobStoreCompactor.java | 4 +-- .../org/apache/hadoop/hbase/mob/MobUtils.java | 14 ---------- .../hadoop/hbase/regionserver/HStoreFile.java | 3 ++ .../hbase/regionserver/MobStoreScanner.java | 1 - .../hbase/regionserver/StoreFileWriter.java | 28 ++++++++++--------- .../hadoop/hbase/mob/MobStressTool.java | 2 +- .../hadoop/hbase/mob/TestMobCompaction.java | 18 +++++++++++- 9 files changed, 48 insertions(+), 40 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java index a411e85c47e7..e53afa1aea24 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java @@ -1746,8 +1746,8 @@ public CompactRegionResponse compactRegion(final RpcController controller, // if the region is a mob region, do the mob file compaction. if (MobUtils.isMobRegionName(tableName, regionName)) { checkHFileFormatVersionForMob(); - //return compactMob(request, tableName); //TODO: support CompactType.MOB + LOG.warn("CompactType.MOB is not supported yet, will run regular compaction."); return super.compactRegion(controller, request); } else { return super.compactRegion(controller, request); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java index 046c09c546bf..5e0739c1cb45 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java @@ -164,19 +164,21 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) new HStoreFile(fs, pp, conf, CacheConfig.DISABLED, BloomType.NONE, true); sf.initReader(); byte[] mobRefData = sf.getMetadataValue(HStoreFile.MOB_FILE_REFS); - byte[] mobCellCountData = sf.getMetadataValue(HStoreFile.MOB_CELLS_COUNT); byte[] bulkloadMarkerData = sf.getMetadataValue(HStoreFile.BULKLOAD_TASK_KEY); - if (mobRefData == null - && (mobCellCountData != null || bulkloadMarkerData == null)) { + if (mobRefData == null && bulkloadMarkerData == null) { LOG.info("Found old store file with no MOB_FILE_REFS: {} - " + "can not proceed until all old files will be MOB-compacted.", pp); return; - } else if (mobRefData == null) { - LOG.info("Skipping file without MOB references (can be bulkloaded file):{}", pp); + } else if (mobRefData == null && bulkloadMarkerData != null) { + LOG.info("Skipping file without MOB references (bulkloaded file):{}", pp); continue; } - String[] mobs = new String(mobRefData).split(","); - regionMobs.addAll(Arrays.asList(mobs)); + if (mobRefData.length > 1) { + // if length = 1 means NULL, that there are no MOB references + // in this store file, but the file was created by new MOB code + String[] mobs = new String(mobRefData).split(","); + regionMobs.addAll(Arrays.asList(mobs)); + } } } catch (FileNotFoundException e) { // TODO diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java index b4468836ce89..9ce068807332 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java @@ -553,8 +553,8 @@ private List getReferencedMobFiles(Collection storeFiles) { Set mobSet = new HashSet(); for (HStoreFile sf: storeFiles) { byte[] value = sf.getMetadataValue(HStoreFile.MOB_FILE_REFS); - if (value != null) { - String s = new String(value); + if (value != null && value.length > 1) { + String s = Bytes.toString(value); String[] all = s.split(","); Collections.addAll(mobSet, all); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java index 50a7f770500c..38ec2ff8b46f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java @@ -687,20 +687,6 @@ public static List getMobColumnFamilies(TableDescriptor return fams; } - - public static long getNumberOfMobFiles(Configuration conf, TableName tableName, String family) - throws IOException { - FileSystem fs = FileSystem.get(conf); - Path dir = getMobFamilyPath(conf, tableName, family); - FileStatus[] stat = fs.listStatus(dir); - for (FileStatus st : stat) { - LOG.info("MOB Directory content: {}", st.getPath()); - } - LOG.info("MOB Directory content total files: {}", stat.length); - - return stat.length; - } - /** * Indicates whether return null value when the mob file is missing or corrupt. * The information is set in the attribute "empty.value.on.mobcell.miss" of scan. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java index 2346fd886fee..413c43a1e0fb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java @@ -110,6 +110,9 @@ public class HStoreFile implements StoreFile { /** Key for the number of mob cells in metadata */ public static final byte[] MOB_CELLS_COUNT = Bytes.toBytes("MOB_CELLS_COUNT"); + + /** Null data */ + public static final byte[] NULL_VALUE = new byte[] {0}; /** Key for the list of MOB file references */ public static final byte[] MOB_FILE_REFS = Bytes.toBytes("MOB_FILE_REFS"); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java index 61cd34c6d235..76144f08d5dc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java @@ -114,5 +114,4 @@ public void close() { LOG.warn("Failed to free referenced mob cells: ", e); } } - } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java index 522efb5ea18d..4381a701081f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java @@ -26,6 +26,7 @@ import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAX_SEQ_ID_KEY; import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_CELLS_COUNT; import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_FILE_REFS; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.NULL_VALUE; import static org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY; import java.io.IOException; @@ -244,25 +245,26 @@ public void appendMetadata(final long maxSequenceId, final boolean majorCompacti } /** - * Appends MOB - specific metadata + * Appends MOB - specific metadata (even if it is empty) * @param mobRefSet - set of MOB file names * @throws IOException problem writing to FS */ public void appendMobMetadata(Set mobRefSet) throws IOException { - if (mobRefSet.isEmpty()) { - return; - } - StringBuilder sb = new StringBuilder(2 * mobRefSet.size() - 1); - String[] arr = new String[mobRefSet.size()]; - arr = mobRefSet.toArray(arr); - for (int i = 0; i < arr.length; i++) { - sb.append(arr[i]); - if (i < arr.length - 1) { - sb.append(","); + if (mobRefSet != null && mobRefSet.size() > 0) { + StringBuilder sb = new StringBuilder(2 * mobRefSet.size() - 1); + String[] arr = new String[mobRefSet.size()]; + arr = mobRefSet.toArray(arr); + for (int i = 0; i < arr.length; i++) { + sb.append(arr[i]); + if (i < arr.length - 1) { + sb.append(","); + } } + byte[] bytes = Bytes.toBytes(sb.toString()); + writer.appendFileInfo(MOB_FILE_REFS, bytes); + } else { + writer.appendFileInfo(MOB_FILE_REFS, NULL_VALUE); } - byte[] bytes = sb.toString().getBytes(); - writer.appendFileInfo(MOB_FILE_REFS, bytes); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java index 5357de0780d9..146fb216a08f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java @@ -29,7 +29,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@InterfaceAudience.Public +@InterfaceAudience.Private @InterfaceStability.Evolving public class MobStressTool extends AbstractHBaseTool { private static final Logger LOG = LoggerFactory.getLogger(MobStressTool.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java index 9266f874d95a..435042c70010 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java @@ -24,6 +24,9 @@ import java.util.Arrays; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -279,7 +282,7 @@ public void testMobCompaction() throws InterruptedException, IOException { Thread.sleep(1000); } - MobUtils.getNumberOfMobFiles(conf, table.getName(), new String(fam)); + getNumberOfMobFiles(conf, table.getName(), new String(fam)); LOG.info("Waiting for write thread to finish ..."); writeData.join(); // Cleanup again @@ -303,6 +306,19 @@ public void testMobCompaction() throws InterruptedException, IOException { printStats(count); } + + private long getNumberOfMobFiles(Configuration conf, TableName tableName, String family) + throws IOException { + FileSystem fs = FileSystem.get(conf); + Path dir = MobUtils.getMobFamilyPath(conf, tableName, family); + FileStatus[] stat = fs.listStatus(dir); + for (FileStatus st : stat) { + LOG.debug("MOB Directory content: {}", st.getPath()); + } + LOG.debug("MOB Directory content total files: {}", stat.length); + + return stat.length; + } public void printStats(long loaded) { LOG.info("MOB Stress Test: loaded=" + loaded + " compactions=" From 89b5684d259a9839a7fea87e0d03a09dc52a807a Mon Sep 17 00:00:00 2001 From: Vladimir Rodionov Date: Fri, 18 Oct 2019 15:52:59 -0700 Subject: [PATCH 05/15] HBASE-22749: add-on #2 --- .../hbase/master/MobFileCleanerChore.java | 2 +- .../hbase/mob/DefaultMobStoreCompactor.java | 8 ++------ .../hbase/mob/FaultyMobStoreCompactor.java | 19 ++++++++++++++++++- 3 files changed, 21 insertions(+), 8 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java index 5e0739c1cb45..83f9903e4f68 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java @@ -166,7 +166,7 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) byte[] mobRefData = sf.getMetadataValue(HStoreFile.MOB_FILE_REFS); byte[] bulkloadMarkerData = sf.getMetadataValue(HStoreFile.BULKLOAD_TASK_KEY); if (mobRefData == null && bulkloadMarkerData == null) { - LOG.info("Found old store file with no MOB_FILE_REFS: {} - " + LOG.warn("Found old store file with no MOB_FILE_REFS: {} - " + "can not proceed until all old files will be MOB-compacted.", pp); return; } else if (mobRefData == null && bulkloadMarkerData != null) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java index 9ce068807332..bb610cdb8175 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java @@ -80,7 +80,7 @@ public class DefaultMobStoreCompactor extends DefaultCompactor { /* * MOB file reference set thread local variable. It contains set of - * to a MOB file names, which newly compacted store file has references to. + * a MOB file names, which newly compacted store file has references to. * This variable is populated during compaction and the content of it is * written into meta section of a newly created store file at the final step * of compaction process. @@ -875,10 +875,6 @@ long getMobCountForOutputWriter(StoreFileWriter writer) { void incrementMobCountForOutputWriter(StoreFileWriter writer, int val) { String key = writer.getPath().getName(); - Long count = mapMobCounts.get(key); - if (count == null) { - count = 0L; - } - mapMobCounts.put(key, count + val); + mapMobCounts.compute(key, (k,v) -> v == null? val: v + val); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java index eb0a097fee6f..f8d22c320a03 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java @@ -48,7 +48,24 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - +/** + * This class is used for testing only. The main purpose is to emulate + * random failures during MOB compaction process. + * Example of usage: + *
    {@code 
    + * public class SomeTest {
    + * 
    + *   public void initConfiguration(Configuration conf){
    + *     conf.set(MobStoreEngine.DEFAULT_MOB_COMPACTOR_CLASS_KEY,
    +         FaultyMobStoreCompactor.class.getName());
    +       conf.setDouble("injected.fault.probability", 0.1);  
    + *   }
    + * } 
    + * }
    + * @see org.apache.hadoop.hbase.mob.TestMobCompaction on how to use and configure + * this class. + * + */ @InterfaceAudience.Private public class FaultyMobStoreCompactor extends DefaultMobStoreCompactor { From 705d780bb1658a69c4b552dff5a299b44e2ef180 Mon Sep 17 00:00:00 2001 From: Vladimir Rodionov Date: Tue, 22 Oct 2019 16:56:57 -0700 Subject: [PATCH 06/15] HBASE-22749: add-on #3 --- .../hbase/master/MobFileCleanerChore.java | 41 +++++++++---------- .../hbase/mob/DefaultMobStoreCompactor.java | 8 ++-- .../apache/hadoop/hbase/mob/MobConstants.java | 15 +++++-- .../hbase/mob/FaultyMobStoreCompactor.java | 2 +- 4 files changed, 36 insertions(+), 30 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java index 83f9903e4f68..1347a207bd1d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java @@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.procedure2.LockType; import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.HStoreFile; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.HFileArchiveUtil; import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; @@ -95,17 +96,7 @@ protected void chore() { for (TableDescriptor htd : map.values()) { for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) { if (hcd.isMobEnabled() && hcd.getMinVersions() == 0) { - // clean only for mob-enabled column. - // obtain a read table lock before cleaning, synchronize with MobFileCompactionChore. - final LockManager.MasterLock lock = master.getLockManager().createMasterLock( - MobUtils.getTableLockName(htd.getTableName()), LockType.SHARED, - this.getClass().getSimpleName() + ": Cleaning expired mob files"); - try { - lock.acquire(); - cleaner.cleanExpiredMobFiles(htd.getTableName().getNameAsString(), hcd); - } finally { - lock.release(); - } + cleaner.cleanExpiredMobFiles(htd.getTableName().getNameAsString(), hcd); } } // Now clean obsolete files for a table @@ -134,6 +125,15 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) LOG.info("Skipping non-MOB table [{}]", table); return; } + // We check only those MOB files, which creation time is less + // than maxTimeToArchive. This is a current time - 1h. 1 hour gap + // gives us full confidence that all corresponding store files will + // exist at the time cleaning procedure begins and will be examined. + // So, if MOB file creation time is greater than this maxTimeToArchive, + // this will be skipped and won't be archived. + long maxCreationTimeToArchive = EnvironmentEdgeManager.currentTime() - 3600000; + LOG.info("Only MOB files whose creation time less than {} will be archived", + maxCreationTimeToArchive); Path rootDir = FSUtils.getRootDir(conf); Path tableDir = FSUtils.getTableDir(rootDir, table); // How safe is this call? @@ -148,7 +148,7 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) boolean succeed = false; Set regionMobs = new HashSet(); while (!succeed) { - // TODO handle FNFE + RemoteIterator rit = fs.listLocatedStatus(storePath); List storeFiles = new ArrayList(); // Load list of store files first @@ -165,6 +165,8 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) sf.initReader(); byte[] mobRefData = sf.getMetadataValue(HStoreFile.MOB_FILE_REFS); byte[] bulkloadMarkerData = sf.getMetadataValue(HStoreFile.BULKLOAD_TASK_KEY); + // close store file to avoid memory leaks + sf.closeStoreFile(true); if (mobRefData == null && bulkloadMarkerData == null) { LOG.warn("Found old store file with no MOB_FILE_REFS: {} - " + "can not proceed until all old files will be MOB-compacted.", pp); @@ -181,8 +183,7 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) } } } catch (FileNotFoundException e) { - // TODO - LOG.warn(e.getMessage()); + LOG.warn("Starting MOB cleaning cycle from the beginning due to error:",e); continue; } succeed = true; @@ -197,9 +198,6 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) allActiveMobFileName.size()); } // Now scan MOB directories and find MOB files with no references to them - long now = System.currentTimeMillis(); - long minAgeToArchive = conf.getLong(MobConstants.MOB_MINIMUM_FILE_AGE_TO_ARCHIVE_KEY, - MobConstants.DEFAULT_MOB_MINIMUM_FILE_AGE_TO_ARCHIVE); for (ColumnFamilyDescriptor hcd : list) { List toArchive = new ArrayList(); String family = hcd.getNameAsString(); @@ -211,12 +209,13 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) if (!allActiveMobFileName.contains(p.getName())) { // MOB is not in a list of active references, but it can be too // fresh, skip it in this case - /* DEBUG */ LOG.debug( - " Age=" + (now - fs.getFileStatus(p).getModificationTime()) + " MOB file=" + p); - if (now - fs.getFileStatus(p).getModificationTime() > minAgeToArchive) { + long creationTime = fs.getFileStatus(p).getModificationTime(); + if ( creationTime < maxCreationTimeToArchive) { + /* DEBUG */ LOG.info( + " Archiving MOB file{} creation time=" + (fs.getFileStatus(p).getModificationTime()), p); toArchive.add(p); } else { - LOG.debug("Skipping fresh file: {}", p); + LOG.info("Skipping fresh file: {}", p); } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java index bb610cdb8175..c0e4f66e3ca9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java @@ -256,7 +256,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel } boolean discardMobMiss = - conf.getBoolean(MobConstants.MOB_DISCARD_MISS_KEY, MobConstants.DEFAULT_MOB_DISCARD_MISS); + conf.getBoolean(MobConstants.MOB_UNSAFE_DISCARD_MISS_KEY, MobConstants.DEFAULT_MOB_DISCARD_MISS); FileSystem fs = FileSystem.get(conf); // Since scanner.next() can return 'false' but still be delivering data, @@ -340,7 +340,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel StoreFileWriter stw = mobWriters.getOutputWriterForInputFile(fName); if (stw != null) { stw.append(mobCell); - mobWriters.incrementMobCountForOutputWriter(stw, 1); + mobWriters.incrementMobCountForOutputWriter(stw); } // else leave mob cell in a MOB file which is not in compaction selections } else { mobFileWriter.append(mobCell); @@ -873,8 +873,8 @@ long getMobCountForOutputWriter(StoreFileWriter writer) { return mapMobCounts.get(writer.getPath().getName()); } - void incrementMobCountForOutputWriter(StoreFileWriter writer, int val) { + void incrementMobCountForOutputWriter(StoreFileWriter writer) { String key = writer.getPath().getName(); - mapMobCounts.compute(key, (k,v) -> v == null? val: v + val); + mapMobCounts.compute(key, (k,v) -> v == null? 1: v + 1); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java index 7e4daf3b9a0c..a96bb4cfb74b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java @@ -64,7 +64,7 @@ public final class MobConstants { */ public static final String MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE = - "hbase.mob.compaction.batch.size"; + "hbase.mob.major.compaction.region.batch.size"; /** * Default is 0 - means no limit - all regions of a MOB table will be compacted at once @@ -115,7 +115,10 @@ public final class MobConstants { public static final int DEFAULT_MOB_COMPACTION_MIN_FILES = 3; /** - * Maximum number of MOB files (in one selection) eligible for compaction + * Maximum number of MOB files (in one compaction selection per region) eligible for compaction. + * If the number of a files in compaction selection is very large, it could lead + * to a "too many opened file handlers" error in a file system or can degrade overall I/O + * performance. */ public static final String MOB_COMPACTION_MAX_FILES_KEY = "hbase.mob.compactions.max.files"; @@ -131,7 +134,11 @@ public final class MobConstants { public static final int DEFAULT_MOB_COMPACTION_MAX_TOTAL_FILES = 1000; - public static final String MOB_DISCARD_MISS_KEY = "hbase.mob.discard.miss"; + /** + * Use this configuration option with caution, only during upgrade procedure + * to handle missing MOB cells during compaction. + */ + public static final String MOB_UNSAFE_DISCARD_MISS_KEY = "hbase.unsafe.mob.discard.miss"; public static final boolean DEFAULT_MOB_DISCARD_MISS = false; @@ -141,7 +148,7 @@ public final class MobConstants { public static final String MOB_MINIMUM_FILE_AGE_TO_ARCHIVE_KEY = "mob.minimum.file.age.to.archive"; - public static final long DEFAULT_MOB_MINIMUM_FILE_AGE_TO_ARCHIVE = 3600000; // 1 hour + public static final long DEFAULT_MOB_MINIMUM_FILE_AGE_TO_ARCHIVE = 24*3600000; // 1 day private MobConstants() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java index f8d22c320a03..4c754e4d57e8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java @@ -101,7 +101,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel boolean isUserRequest = userRequest.get(); boolean compactMOBs = major && isUserRequest; boolean discardMobMiss = - conf.getBoolean(MobConstants.MOB_DISCARD_MISS_KEY, MobConstants.DEFAULT_MOB_DISCARD_MISS); + conf.getBoolean(MobConstants.MOB_UNSAFE_DISCARD_MISS_KEY, MobConstants.DEFAULT_MOB_DISCARD_MISS); boolean mustFail = false; if (compactMOBs) { From 0868ff96a591a360fde6fb0c6aa10b033c0a5569 Mon Sep 17 00:00:00 2001 From: Vladimir Rodionov Date: Thu, 24 Oct 2019 18:04:04 -0700 Subject: [PATCH 07/15] HBASE-22749: add-on #4 --- .../hbase/mob/DefaultMobStoreCompactor.java | 114 ++++++++++-------- .../org/apache/hadoop/hbase/mob/MobUtils.java | 43 ++++++- 2 files changed, 107 insertions(+), 50 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java index c0e4f66e3ca9..263b0ffd3969 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java @@ -34,6 +34,7 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicLong; +import org.apache.commons.lang3.mutable.MutableLong; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -169,7 +170,9 @@ public List compact(CompactionRequestImpl request, .equals(MobConstants.GENERATIONAL_MOB_COMPACTION_TYPE)) { if (request.isMajor() && request.getPriority() == HStore.PRIORITY_USER) { // Compact MOBs - List mobFiles = getReferencedMobFiles(request.getFiles()); + Path mobDir = MobUtils.getMobFamilyPath(conf, store.getTableName(), + store.getColumnFamilyName()); + List mobFiles = MobUtils.getReferencedMobFiles(request.getFiles(), mobDir); if (mobFiles.size() > 0) { Generations gens = Generations.build(mobFiles, conf); List list = gens.getCompactionSelections(); @@ -274,6 +277,12 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel Path path = MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName()); byte[] fileName = null; StoreFileWriter mobFileWriter = null; + /* + * mobCells are used only to decide if we need to commit or abort major MOB output file. + * This file is present in both: regular and generational compaction. In regular compaction - + * it is the only MOB file output, in generational compaction it is the output for new MOB cells, + * which come from store files - not from MOB files. + */ long mobCells = 0; long cellsCountCompactedToMob = 0, cellsCountCompactedFromMob = 0; long cellsSizeCompactedToMob = 0, cellsSizeCompactedFromMob = 0; @@ -333,7 +342,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel } if (mobCell.getValueLength() > mobSizeThreshold) { - // put the mob data back to the store file + // put the mob data back to the MOB store file PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId()); if (generationalMob) { //TODO: verify fName @@ -341,33 +350,44 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel if (stw != null) { stw.append(mobCell); mobWriters.incrementMobCountForOutputWriter(stw); - } // else leave mob cell in a MOB file which is not in compaction selections + byte[] fname = Bytes.toBytes(stw.getPath().getName()); + // Update MOB reference with new MOB file + writer.append(MobUtils.createMobRefCell(mobCell, fname, + this.mobStore.getRefCellTags())); + } else { + // else leave mob cell in a MOB file which is not in compaction selections + // write MOB cell reference to output store + // add MOB file reference to mobRefSet + writer.append(mobCell); + mobRefSet.get().add(fName); + } } else { mobFileWriter.append(mobCell); mobCells++; - } - writer.append(MobUtils.createMobRefCell(mobCell, fileName, - this.mobStore.getRefCellTags())); - cellsCountCompactedFromMob++; - cellsSizeCompactedFromMob += mobCell.getValueLength(); + writer.append(MobUtils.createMobRefCell(mobCell, fileName, + this.mobStore.getRefCellTags())); + } } else { - // If MOB value is less than threshold, append it directly to a store file PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId()); writer.append(mobCell); + cellsCountCompactedFromMob++; + cellsSizeCompactedFromMob += mobCell.getValueLength(); } - } else { // Not a MOB reference cell int size = c.getValueLength(); if (size > mobSizeThreshold) { // This MOB cell comes from a regular store file - // therefore we store it in original mob output + // therefore we store it into original mob output mobFileWriter.append(c); writer .append(MobUtils.createMobRefCell(c, fileName, this.mobStore.getRefCellTags())); mobCells++; + cellsCountCompactedToMob++; + cellsSizeCompactedToMob += c.getValueLength(); } else { + // Not a MOB cell, write it directly to a store file writer.append(c); } } @@ -384,7 +404,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel // If the value size is larger than the threshold, it's regarded as a mob. Since // its value is already in the mob file, directly write this cell to the store file writer.append(c); - // Add MOB reference to a set + // Add MOB reference to a MOB reference set mobRefSet.get().add(MobUtils.getMobFileName(c)); } else { // If the value is not larger than the threshold, it's not regarded a mob. Retrieve @@ -475,7 +495,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel throw new InterruptedIOException( "Interrupted while control throughput of compacting " + compactionName); } catch (IOException t) { - LOG.error("Mob compaction failed for region: " + store.getRegionInfo().getEncodedName()); + LOG.error("Mob compaction failed for region:{} ", store.getRegionInfo().getEncodedName()); throw t; } finally { // Clone last cell in the final because writer will append last cell when committing. If @@ -487,16 +507,24 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel // Remove all MOB references because compaction failed mobRefSet.get().clear(); // Abort writer + LOG.debug("Aborting writer for {} because of compaction failure", + mobFileWriter.getPath()); abortWriter(mobFileWriter); //Check if other writers exist if (mobWriters != null) { for(StoreFileWriter w: mobWriters.getOutputWriters()) { + LOG.debug("Aborting writer for {} because of compaction failure", + w.getPath()); abortWriter(w); } } } } // Commit or abort major mob writer + // If IOException happens during below operation, some + // MOB files can be committed partially, but corresponding + // store file won't be committed, therefore these MOB files + // become orphans and will be deleted during next MOB cleaning chore cycle if (mobFileWriter != null) { if (mobCells > 0) { // If the mob file is not empty, commit it. @@ -505,6 +533,10 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel mobStore.commitFile(mobFileWriter.getPath(), path); } else { // If the mob file is empty, delete it instead of committing. + LOG.debug("Aborting writer for {} because there are no MOB cells", + mobFileWriter.getPath()); + // Remove MOB file from reference set + mobRefSet.get().remove(mobFileWriter.getPath().getName()); abortWriter(mobFileWriter); } } @@ -518,6 +550,9 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel w.close(); mobStore.commitFile(w.getPath(), path); } else { + LOG.debug("Aborting writer for {} because there are no MOB cells", w.getPath()); + // Remove MOB file from reference set + mobRefSet.get().remove(w.getPath().getName()); abortWriter(w); } } @@ -547,24 +582,6 @@ protected List commitWriter(StoreFileWriter writer, FileDetails fd, return newFiles; } - private List getReferencedMobFiles(Collection storeFiles) { - Path mobDir = MobUtils.getMobFamilyPath(conf, store.getTableName(), - store.getColumnFamilyName()); - Set mobSet = new HashSet(); - for (HStoreFile sf: storeFiles) { - byte[] value = sf.getMetadataValue(HStoreFile.MOB_FILE_REFS); - if (value != null && value.length > 1) { - String s = Bytes.toString(value); - String[] all = s.split(","); - Collections.addAll(mobSet, all); - } - } - List retList = new ArrayList(); - for(String name: mobSet) { - retList.add(new Path(mobDir, name)); - } - return retList; - } } final class FileSelection implements Comparable { @@ -581,10 +598,9 @@ public FileSelection(Path path, Configuration conf) throws IOException { } public String getEncodedRegionName() { - String fileName = path.getName(); - String[] parts = fileName.split("_"); - if (parts.length == 2) { - return parts[1]; + String name = MobUtils.getEncodedRegionName(path.getName()); + if (name != null) { + return name; } else { return NULL_REGION; } @@ -664,12 +680,7 @@ static Generations build(List files, Configuration conf) throws IOExceptio Map > map = new HashMap>(); for(Path p: files) { String key = getRegionNameFromFileName(p.getName()); - ArrayList list = map.get(key); - if (list == null) { - list = new ArrayList(); - map.put(key, list); - } - list.add(new FileSelection(p, conf)); + map.computeIfAbsent(key, k -> new ArrayList()).add(new FileSelection(p, conf)); } List gens = new ArrayList(); @@ -691,12 +702,12 @@ static Generations build(List files, Configuration conf) throws IOExceptio return new Generations(gens, conf); } - static String getRegionNameFromFileName(String name) { - int index = name.lastIndexOf("_"); - if (index < 0) { + static String getRegionNameFromFileName(String mobFileName) { + String name = MobUtils.getEncodedRegionName(mobFileName); + if (name == null) { return Generation.GEN0; } - return name.substring(index+1); + return name; } } @@ -832,7 +843,7 @@ final class OutputMobWriters { /* * Output file name -> MOB counter */ - private Map mapMobCounts = new HashMap(); + private Map mapMobCounts = new HashMap(); /* * List of compaction selections */ @@ -853,7 +864,7 @@ StoreFileWriter getWriterForFile(String fileName) { void initOutputWriters(List writers) { for (int i = 0; i < writers.size(); i++) { StoreFileWriter sw = writers.get(i); - mapMobCounts.put(sw.getPath().getName(), 0L); + mapMobCounts.put(sw.getPath().getName(), new MutableLong(0L)); CompactionSelection cs = compSelections.get(i); for (FileSelection fs: cs.getFiles()) { writerMap.put(fs.getPath().getName(), sw); @@ -870,11 +881,16 @@ StoreFileWriter getOutputWriterForInputFile(String name) { } long getMobCountForOutputWriter(StoreFileWriter writer) { - return mapMobCounts.get(writer.getPath().getName()); + return mapMobCounts.get(writer.getPath().getName()).longValue(); } void incrementMobCountForOutputWriter(StoreFileWriter writer) { String key = writer.getPath().getName(); - mapMobCounts.compute(key, (k,v) -> v == null? 1: v + 1); + MutableLong v = mapMobCounts.get(key); + if (v == null) { + mapMobCounts.put(key, new MutableLong(1L)); + } else { + v.add(1); + } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java index 38ec2ff8b46f..ae11a6ca97ca 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java @@ -25,9 +25,12 @@ import java.util.ArrayList; import java.util.Calendar; import java.util.Collection; +import java.util.Collections; import java.util.Date; +import java.util.HashSet; import java.util.List; import java.util.Optional; +import java.util.Set; import java.util.UUID; import org.apache.hadoop.conf.Configuration; @@ -733,5 +736,43 @@ public static boolean isMobFileExpired(ColumnFamilyDescriptor column, long curre } return false; } + + /** + * Gets encoded region name from a MOB file name + * @param mobFileName MOB file name + * @return encoded region name or null + */ + public static String getEncodedRegionName(String mobFileName) { + int index = mobFileName.lastIndexOf(MobFileName.REGION_SEP); + if (index < 0) { + return null; + } + return mobFileName.substring(index+1); + } + + /** + * Get list of referenced MOB files from a given collection + * of store files + * @param storeFiles store files + * @param mobDir MOB file directory + * @return list of MOB file paths + */ + + public static List getReferencedMobFiles(Collection storeFiles, Path mobDir) { -} + Set mobSet = new HashSet(); + for (HStoreFile sf : storeFiles) { + byte[] value = sf.getMetadataValue(HStoreFile.MOB_FILE_REFS); + if (value != null && value.length > 1) { + String s = Bytes.toString(value); + String[] all = s.split(","); + Collections.addAll(mobSet, all); + } + } + List retList = new ArrayList(); + for (String name : mobSet) { + retList.add(new Path(mobDir, name)); + } + return retList; + } +} \ No newline at end of file From 2bd707c1c469bfc2d2d7a21ccdbf886354e26980 Mon Sep 17 00:00:00 2001 From: Vladimir Rodionov Date: Mon, 28 Oct 2019 18:13:12 -0700 Subject: [PATCH 08/15] HBASE-22749: add-on #5 --- .../hbase/master/MobFileCleanerChore.java | 26 ++++++++++++++++--- .../hbase/master/MobFileCompactionChore.java | 9 +++++++ .../hbase/mob/DefaultMobStoreCompactor.java | 24 ++++++++++++----- .../apache/hadoop/hbase/mob/MobConstants.java | 8 ------ 4 files changed, 49 insertions(+), 18 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java index 1347a207bd1d..5d3bda6c57b5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java @@ -43,11 +43,9 @@ import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hadoop.hbase.master.locking.LockManager; -import org.apache.hadoop.hbase.mob.MobConstants; import org.apache.hadoop.hbase.mob.ExpiredMobFileCleaner; +import org.apache.hadoop.hbase.mob.MobConstants; import org.apache.hadoop.hbase.mob.MobUtils; -import org.apache.hadoop.hbase.procedure2.LockType; import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -60,7 +58,7 @@ /** * The Class ExpiredMobFileCleanerChore for running cleaner regularly to remove the expired - * mob files. + * and obsolete (files which have no active references to) mob files. */ @InterfaceAudience.Private public class MobFileCleanerChore extends ScheduledChore { @@ -77,6 +75,26 @@ public MobFileCleanerChore(HMaster master) { this.master = master; cleaner = new ExpiredMobFileCleaner(); cleaner.setConf(master.getConfiguration()); + checkObsoleteConfigurations(); + } + + private void checkObsoleteConfigurations() { + Configuration conf = master.getConfiguration(); + if (conf.get("hbase.master.mob.ttl.cleaner.period") != null) { + LOG.warn("'hbase.master.mob.ttl.cleaner.period' is obsolete and not used anymore."); + } + if (conf.get("hbase.mob.compaction.mergeable.threshold") != null) { + LOG.warn("'hbase.mob.compaction.mergeable.threshold' is obsolete and not used anymore."); + } + if (conf.get("hbase.mob.delfile.max.count") != null) { + LOG.warn("'hbase.mob.delfile.max.count' is obsolete and not used anymore."); + } + if (conf.get("hbase.mob.compaction.threads.max") != null) { + LOG.warn("'hbase.mob.compaction.threads.max' is obsolete and not used anymore."); + } + if (conf.get("hbase.mob.compaction.batch.size") != null) { + LOG.warn("'hbase.mob.compaction.batch.size' is obsolete and not used anymore."); + } } @VisibleForTesting diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java index 91b9e8db9281..03357a2cb784 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java @@ -42,6 +42,15 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * Periodic MOB compaction chore. + * It runs MOB compaction on region servers in parallel, thus + * utilizing distributed cluster resources. To avoid possible major + * compaction storms, one can specify maximum number regions to be compacted + * in parallel by setting configuration parameter:
    + * 'hbase.mob.major.compaction.region.batch.size', which by default is 0 (unlimited). + * + */ @InterfaceAudience.Private public class MobFileCompactionChore extends ScheduledChore { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java index 263b0ffd3969..7b39c427a8f4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java @@ -187,9 +187,18 @@ public List compact(CompactionRequestImpl request, /** * Performs compaction on a column family with the mob flag enabled. - * This is for when the mob threshold size has changed or if the mob - * column family mode has been toggled via an alter table statement. - * Compacts the files by the following rules. + * This is for when MOB compaction is explicitly requested (by User). + * There are two modes of a MOB compaction:
    + *

    + *

      + *
    • 1. General mode - when all data is compacted into a single MOB file. + *
    • 2. Generational mode - when MOB files are collected (and compacted) by generations,
      + * which are defined by history of a region splits. The oldest generations are compacted first.
      + * In this mode the output of a compaction process will contain multiple MOB files. The main
      + * idea behind generational compaction is to limit maximum size of a MOB file and to limit I/O
      + * write/read amplification during MOB compaction. + *
    + * The basic algorithm of compaction is the following:
    * 1. If the Put cell has a mob reference tag, the cell's value is the path of the mob file. *
      *
    1. @@ -307,9 +316,10 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel throw e; } if (compactMOBs) { - // Add the only reference we get for compact MOB case + // Add the only reference we get for compact MOB general (not generational) case // because new store file will have only one MOB reference - // in this case - of newly compacted MOB file + // in this case - of a newly compacted MOB file. In generational compaction mode, + // this reference is present as well along with (potentially) many others. mobRefSet.get().add(mobFileWriter.getPath().getName()); } do { @@ -329,7 +339,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel mobCell = mobStore.resolve(c, true, false).getCell(); } catch (FileNotFoundException fnfe) { if (discardMobMiss) { - LOG.error("Missing MOB cell: file=" + pp + " not found"); + LOG.debug("Missing MOB cell: file={} not found cell={}", pp, c); continue; } else { throw fnfe; @@ -339,6 +349,8 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel if (discardMobMiss && mobCell.getValueLength() == 0) { LOG.error("Missing MOB cell value: file=" + pp + " cell=" + mobCell); continue; + } else if (mobCell.getValueLength() == 0) { + LOG.warn("Found 0 length MOB cell in a file={} cell={}", pp, mobCell); } if (mobCell.getValueLength() > mobSizeThreshold) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java index a96bb4cfb74b..e2fa9f81c480 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java @@ -142,14 +142,6 @@ public final class MobConstants { public static final boolean DEFAULT_MOB_DISCARD_MISS = false; - /** - * Minimum age required for MOB file to be archived - */ - public static final String MOB_MINIMUM_FILE_AGE_TO_ARCHIVE_KEY = - "mob.minimum.file.age.to.archive"; - - public static final long DEFAULT_MOB_MINIMUM_FILE_AGE_TO_ARCHIVE = 24*3600000; // 1 day - private MobConstants() { } From 231ca5f7a5030c608e9f4c7e697e5186623387b1 Mon Sep 17 00:00:00 2001 From: Vladimir Rodionov Date: Thu, 31 Oct 2019 11:57:40 -0700 Subject: [PATCH 09/15] HBASE-23209: Simplify logic in DefaultMobStoreCompactor --- .../hbase/mob/DefaultMobStoreCompactor.java | 32 +++---------------- 1 file changed, 5 insertions(+), 27 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java index 7b39c427a8f4..623d655157ae 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java @@ -411,33 +411,11 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel } else if (MobUtils.isMobReferenceCell(c)) { // Not a major MOB compaction, Put MOB reference if (MobUtils.hasValidMobRefCellValue(c)) { - int size = MobUtils.getMobValueLength(c); - if (size > mobSizeThreshold) { - // If the value size is larger than the threshold, it's regarded as a mob. Since - // its value is already in the mob file, directly write this cell to the store file - writer.append(c); - // Add MOB reference to a MOB reference set - mobRefSet.get().add(MobUtils.getMobFileName(c)); - } else { - // If the value is not larger than the threshold, it's not regarded a mob. Retrieve - // the mob cell from the mob file, and write it back to the store file. - mobCell = mobStore.resolve(c, true, false).getCell(); - if (mobCell.getValueLength() != 0) { - // put the mob data back to the store file - PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId()); - writer.append(mobCell); - cellsCountCompactedFromMob++; - cellsSizeCompactedFromMob += mobCell.getValueLength(); - } else { - // If the value of a file is empty, there might be issues when retrieving, - // directly write the cell to the store file, and leave it to be handled by the - // next compaction. - LOG.error("Empty value for: " + c); - writer.append(c); - // Add MOB reference to a set - mobRefSet.get().add(MobUtils.getMobFileName(c)); - } - } + // We do not check mobSizeThreshold during normal compaction, + // leaving it to a MOB compaction run + writer.append(c); + // Add MOB reference to a MOB reference set + mobRefSet.get().add(MobUtils.getMobFileName(c)); } else { // TODO ???? LOG.error("Corrupted MOB reference: " + c); From 44a3e8c744a226c2a4f1f6d11f0c8aeb3af4e0b7 Mon Sep 17 00:00:00 2001 From: Vladimir Rodionov Date: Tue, 5 Nov 2019 12:37:34 -0800 Subject: [PATCH 10/15] HBASE-23190: Convert MOB compaction test to integration test --- .../hbase/IntegrationTestMobCompaction.java | 413 ++++++++++++++++++ .../hbase/master/MobFileCleanerChore.java | 5 +- .../apache/hadoop/hbase/mob/MobConstants.java | 7 + .../hadoop/hbase/mob/TestMobCompaction.java | 2 +- 4 files changed, 425 insertions(+), 2 deletions(-) create mode 100644 hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java new file mode 100644 index 000000000000..b4812b997b11 --- /dev/null +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java @@ -0,0 +1,413 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Set; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.master.MobFileCleanerChore; +import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner; +import org.apache.hadoop.hbase.mob.FaultyMobStoreCompactor; +import org.apache.hadoop.hbase.mob.MobConstants; +import org.apache.hadoop.hbase.mob.MobStoreEngine; +import org.apache.hadoop.hbase.mob.MobUtils; + +import org.apache.hadoop.hbase.testclassification.IntegrationTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.util.ToolRunner; +import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects; +import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * An integration test to detect regressions in HBASE-22749. Test creates + * MOB-enabled table, and runs in parallel, the following tasks: loads data, + * runs MOB compactions, runs MOB cleaning chore. The failure injections into MOB + * compaction cycle is implemented via specific sub-class of DefaultMobStoreCompactor - + * FaultyMobStoreCompactor. The probability of failure is controlled by command-line + * argument 'failprob'. + * @see HBASE-22749 + */ +@SuppressWarnings("deprecation") + +@Category(IntegrationTests.class) +public class IntegrationTestMobCompaction extends IntegrationTestBase { + protected static final Logger LOG = LoggerFactory.getLogger(IntegrationTestMobCompaction.class); + + protected static final String REGIONSERVER_COUNT_KEY = "servers"; + protected static final String ROWS_COUNT_KEY = "rows"; + protected static final String FAILURE_PROB_KEY = "failprob"; + + protected static final int DEFAULT_REGIONSERVER_COUNT = 3; + protected static final int DEFAULT_ROWS_COUNT = 5000000; + protected static final double DEFAULT_FAILURE_PROB = 0.1; + + protected static int regionServerCount = DEFAULT_REGIONSERVER_COUNT; + protected static long rowsToLoad = DEFAULT_ROWS_COUNT; + protected static double failureProb = DEFAULT_FAILURE_PROB; + + protected static String famStr = "f1"; + protected static byte[] fam = Bytes.toBytes(famStr); + protected static byte[] qualifier = Bytes.toBytes("q1"); + protected static long mobLen = 10; + protected static byte[] mobVal = Bytes + .toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789"); + + private static Configuration conf; + private static HTableDescriptor hdt; + private static HColumnDescriptor hcd; + private static Admin admin; + private static Table table = null; + private static MobFileCleanerChore chore; + + private static volatile boolean run = true; + + @Override + @Before + public void setUp() throws Exception { + util = getTestingUtil(getConf()); + conf = util.getConfiguration(); + // Initialize with test-specific configuration values + initConf(conf); + regionServerCount = + conf.getInt(REGIONSERVER_COUNT_KEY, DEFAULT_REGIONSERVER_COUNT); + LOG.info("Initializing cluster with {} region servers.", regionServerCount); + util.initializeCluster(regionServerCount); + admin = util.getAdmin(); + + createTestTable(); + + LOG.info("Cluster initialized and ready"); + } + + private void createTestTable() throws IOException { + // Create test table + hdt = util.createTableDescriptor("testMobCompactTable"); + hcd = new HColumnDescriptor(fam); + hcd.setMobEnabled(true); + hcd.setMobThreshold(mobLen); + hcd.setMaxVersions(1); + hdt.addFamily(hcd); + table = util.createTable(hdt, null); + } + + @After + public void tearDown() throws IOException { + LOG.info("Cleaning up after test."); + if(util.isDistributedCluster()) { + deleteTablesIfAny(); + // TODO + } + LOG.info("Restoring cluster."); + util.restoreCluster(); + LOG.info("Cluster restored."); + } + + @Override + public void setUpMonkey() throws Exception { + // Sorry, no Monkey + } + + private void deleteTablesIfAny() throws IOException { + if (table != null) { + util.deleteTableIfAny(table.getName()); + } + } + + + + @Override + public void setUpCluster() throws Exception { + util = getTestingUtil(getConf()); + LOG.debug("Initializing/checking cluster has {} servers",regionServerCount); + util.initializeCluster(regionServerCount); + LOG.debug("Done initializing/checking cluster"); + } + + /** + * + * @return status of CLI execution + */ + @Override + public int runTestFromCommandLine() throws Exception { + testMobCompaction(); + return 0; + } + + @Override + public TableName getTablename() { + // That is only valid when Monkey is CALM (no monkey) + return null; + } + + @Override + protected Set getColumnFamilies() { + // That is only valid when Monkey is CALM (no monkey) + return null; + } + + @Override + protected void addOptions() { + addOptWithArg(REGIONSERVER_COUNT_KEY, + "Total number of region servers. Default: '" + DEFAULT_REGIONSERVER_COUNT + "'"); + addOptWithArg(ROWS_COUNT_KEY, + "Total number of data rows to load. Default: '" + DEFAULT_ROWS_COUNT + "'"); + addOptWithArg(FAILURE_PROB_KEY, + "Probability of a failure of a region MOB compaction request. Default: '" + + DEFAULT_FAILURE_PROB + "'"); + } + + @Override + protected void processOptions(CommandLine cmd) { + super.processOptions(cmd); + + regionServerCount = + Integer.parseInt(cmd.getOptionValue(REGIONSERVER_COUNT_KEY, + Integer.toString(DEFAULT_REGIONSERVER_COUNT))); + rowsToLoad = + Long.parseLong(cmd.getOptionValue(ROWS_COUNT_KEY, + Long.toString(DEFAULT_ROWS_COUNT))); + failureProb = Double.parseDouble(cmd.getOptionValue(FAILURE_PROB_KEY, + Double.toString(DEFAULT_FAILURE_PROB))); + + LOG.info(MoreObjects.toStringHelper("Parsed Options") + .add(REGIONSERVER_COUNT_KEY, regionServerCount) + .add(ROWS_COUNT_KEY, rowsToLoad) + .add(FAILURE_PROB_KEY, failureProb) + .toString()); + } + + private static void initConf(Configuration conf) { + + conf.setInt("hfile.format.version", 3); + conf.setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0); + conf.setInt("hbase.client.retries.number", 100); + conf.setInt("hbase.hregion.max.filesize", 200000000); + conf.setInt("hbase.hregion.memstore.flush.size", 800000); + conf.setInt("hbase.hstore.blockingStoreFiles", 150); + conf.setInt("hbase.hstore.compaction.throughput.lower.bound", 52428800); + conf.setInt("hbase.hstore.compaction.throughput.higher.bound", 2 * 52428800); + conf.setDouble("injected.fault.probability", failureProb); + conf.set(MobStoreEngine.DEFAULT_MOB_COMPACTOR_CLASS_KEY, + FaultyMobStoreCompactor.class.getName()); + conf.setBoolean("hbase.table.sanity.checks", false); + conf.setLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, 20000); + + } + + + class MajorCompaction implements Runnable { + + @Override + public void run() { + while (run) { + try { + admin.majorCompact(hdt.getTableName(), fam); + Thread.sleep(120000); + } catch (Exception e) { + LOG.error("MOB Stress Test FAILED", e); + System.exit(-1); + } + } + } + } + + class CleanMobAndArchive implements Runnable { + + @Override + public void run() { + while (run) { + try { + LOG.info("MOB cleanup chore started ..."); + if (chore == null) { + chore = new MobFileCleanerChore(); + } + chore.cleanupObsoleteMobFiles(conf, table.getName()); + LOG.info("MOB cleanup chore finished"); + + Thread.sleep(130000); + } catch (Exception e) { + e.printStackTrace(); + } + } + } + } + + class WriteData implements Runnable { + + private long rows = -1; + + public WriteData(long rows) { + this.rows = rows; + } + + @Override + public void run() { + try { + + // BufferedMutator bm = admin.getConnection().getBufferedMutator(table.getName()); + // Put Operation + for (int i = 0; i < rows; i++) { + Put p = new Put(Bytes.toBytes(i)); + p.addColumn(fam, qualifier, mobVal); + table.put(p); + + // bm.mutate(p); + if (i % 10000 == 0) { + LOG.info("LOADED=" + i); + try { + Thread.sleep(500); + } catch (InterruptedException ee) { + + } + } + if (i % 100000 == 0) { + printStats(i); + } + } + // bm.flush(); + admin.flush(table.getName()); + run = false; + } catch (Exception e) { + LOG.error("MOB Stress Test FAILED", e); + System.exit(-1); + } + } + } + + @Test + public void testMobCompaction() throws InterruptedException, IOException { + + try { + + Thread writeData = new Thread(new WriteData(rowsToLoad)); + writeData.start(); + + Thread majorcompact = new Thread(new MajorCompaction()); + majorcompact.start(); + + Thread cleaner = new Thread(new CleanMobAndArchive()); + cleaner.start(); + + while (run) { + Thread.sleep(1000); + } + + getNumberOfMobFiles(conf, table.getName(), new String(fam)); + LOG.info("Waiting for write thread to finish ..."); + writeData.join(); + // Cleanup again + chore.cleanupObsoleteMobFiles(conf, table.getName()); + + if (util != null) { + LOG.info("Archive cleaner started ..."); + // Call archive cleaner again + util.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting(); + LOG.info("Archive cleaner finished"); + } + + scanTable(); + + } finally { + + admin.disableTable(hdt.getTableName()); + admin.deleteTable(hdt.getTableName()); + } + LOG.info("MOB Stress Test finished OK"); + printStats(rowsToLoad); + + } + + private long getNumberOfMobFiles(Configuration conf, TableName tableName, String family) + throws IOException { + FileSystem fs = FileSystem.get(conf); + Path dir = MobUtils.getMobFamilyPath(conf, tableName, family); + FileStatus[] stat = fs.listStatus(dir); + for (FileStatus st : stat) { + LOG.debug("MOB Directory content: {}", st.getPath()); + } + LOG.debug("MOB Directory content total files: {}", stat.length); + + return stat.length; + } + + public void printStats(long loaded) { + LOG.info("MOB Stress Test: loaded=" + loaded + " compactions=" + + FaultyMobStoreCompactor.totalCompactions.get() + " major=" + + FaultyMobStoreCompactor.totalMajorCompactions.get() + " mob=" + + FaultyMobStoreCompactor.mobCounter.get() + " injected failures=" + + FaultyMobStoreCompactor.totalFailures.get()); + } + + private void scanTable() { + try { + + Result result; + ResultScanner scanner = table.getScanner(fam); + int counter = 0; + while ((result = scanner.next()) != null) { + assertTrue(Arrays.equals(result.getValue(fam, qualifier), mobVal)); + if (counter % 10000 == 0) { + LOG.info("GET=" + counter); + } + counter++; + } + assertEquals(rowsToLoad, counter); + } catch (Exception e) { + e.printStackTrace(); + LOG.error("MOB Stress Test FAILED"); + if (util != null) { + assertTrue(false); + } else { + System.exit(-1); + } + } + } + + /** + * + * @param args argument list + */ + public static void main(String[] args) throws Exception { + Configuration conf = HBaseConfiguration.create(); + initConf(conf); + IntegrationTestingUtility.setUseDistributedCluster(conf); + int status = ToolRunner.run(conf, new IntegrationTestMobCompaction(), args); + System.exit(status); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java index 5d3bda6c57b5..2996f8d1640d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java @@ -66,6 +66,7 @@ public class MobFileCleanerChore extends ScheduledChore { private static final Logger LOG = LoggerFactory.getLogger(MobFileCleanerChore.class); private final HMaster master; private ExpiredMobFileCleaner cleaner; + private long minAgeToArchive; public MobFileCleanerChore(HMaster master) { super(master.getServerName() + "-ExpiredMobFileCleanerChore", master, master.getConfiguration() @@ -75,6 +76,8 @@ public MobFileCleanerChore(HMaster master) { this.master = master; cleaner = new ExpiredMobFileCleaner(); cleaner.setConf(master.getConfiguration()); + this.minAgeToArchive = master.getConfiguration().getLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, + MobConstants.DEFAULT_MIN_AGE_TO_ARCHIVE); checkObsoleteConfigurations(); } @@ -149,7 +152,7 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) // exist at the time cleaning procedure begins and will be examined. // So, if MOB file creation time is greater than this maxTimeToArchive, // this will be skipped and won't be archived. - long maxCreationTimeToArchive = EnvironmentEdgeManager.currentTime() - 3600000; + long maxCreationTimeToArchive = EnvironmentEdgeManager.currentTime() - minAgeToArchive; LOG.info("Only MOB files whose creation time less than {} will be archived", maxCreationTimeToArchive); Path rootDir = FSUtils.getRootDir(conf); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java index e2fa9f81c480..b72774aa7711 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java @@ -141,6 +141,13 @@ public final class MobConstants { public static final String MOB_UNSAFE_DISCARD_MISS_KEY = "hbase.unsafe.mob.discard.miss"; public static final boolean DEFAULT_MOB_DISCARD_MISS = false; + + /** + * Minimum MOB file age to archive, default (3600000 - 1h) + */ + public static final String MIN_AGE_TO_ARCHIVE_KEY = "hbase.mob.min.age.archive"; + + public static final long DEFAULT_MIN_AGE_TO_ARCHIVE = 3600000; private MobConstants() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java index 435042c70010..ab989f7e0448 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java @@ -89,7 +89,7 @@ public class TestMobCompaction { private HTableDescriptor hdt; private HColumnDescriptor hcd; private Admin admin; - private long count = 500000; + private long count = 5000000; private double failureProb = 0.1; private Table table = null; private MobFileCleanerChore chore = new MobFileCleanerChore(); From d0d70ff560566e04c1d39eeb5e0a878f784c2531 Mon Sep 17 00:00:00 2001 From: Vladimir Rodionov Date: Tue, 5 Nov 2019 18:40:15 -0800 Subject: [PATCH 11/15] HBASE-23188: MobFileCleanerChore test case --- .../hbase/master/MobFileCleanerChore.java | 4 +- .../hbase/mob/TesMobFileCleanerChore.java | 236 ++++++++++++++++++ .../hadoop/hbase/mob/TestMobCompaction.java | 2 +- 3 files changed, 239 insertions(+), 3 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TesMobFileCleanerChore.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java index 2996f8d1640d..cc1e9b7884e8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java @@ -76,8 +76,6 @@ public MobFileCleanerChore(HMaster master) { this.master = master; cleaner = new ExpiredMobFileCleaner(); cleaner.setConf(master.getConfiguration()); - this.minAgeToArchive = master.getConfiguration().getLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, - MobConstants.DEFAULT_MIN_AGE_TO_ARCHIVE); checkObsoleteConfigurations(); } @@ -138,6 +136,8 @@ protected void chore() { public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws IOException { + this.minAgeToArchive = conf.getLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, + MobConstants.DEFAULT_MIN_AGE_TO_ARCHIVE); try (final Connection conn = ConnectionFactory.createConnection(conf); final Admin admin = conn.getAdmin();) { TableDescriptor htd = admin.getDescriptor(table); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TesMobFileCleanerChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TesMobFileCleanerChore.java new file mode 100644 index 000000000000..2b83310011a5 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TesMobFileCleanerChore.java @@ -0,0 +1,236 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.mob; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.Arrays; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.CompactionState; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.master.MobFileCleanerChore; +import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Mob file cleaner chore test. + * 1. Creates MOB table + * 2. Load MOB data and flushes it N times + * 3. Runs major MOB compaction (N MOB files go to archive) + * 4. Verifies that number of MOB files in a mob directory is N+1 + * 5. Waits for a period of time larger than minimum age to archive + * 6. Runs Mob cleaner chore + * 7 Verifies that number of MOB files in a mob directory is 1. + */ +@SuppressWarnings("deprecation") +@Category(MediumTests.class) +public class TesMobFileCleanerChore { + private static final Logger LOG = LoggerFactory.getLogger(TesMobFileCleanerChore.class); + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TesMobFileCleanerChore.class); + @Rule + public TestName testName = new TestName(); + + private HBaseTestingUtility HTU; + + private final static String famStr = "f1"; + private final static byte[] fam = Bytes.toBytes(famStr); + private final static byte[] qualifier = Bytes.toBytes("q1"); + private final static long mobLen = 10; + private final static byte[] mobVal = Bytes + .toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789"); + + private Configuration conf; + private HTableDescriptor hdt; + private HColumnDescriptor hcd; + private Admin admin; + private Table table = null; + private MobFileCleanerChore chore; + private long minAgeToArchive = 10000; + + public TesMobFileCleanerChore() { + } + + + @Before + public void setUp() throws Exception { + HTU = new HBaseTestingUtility(); + hdt = HTU.createTableDescriptor("testMobCompactTable"); + conf = HTU.getConfiguration(); + + initConf(); + + HTU.startMiniCluster(); + admin = HTU.getAdmin(); + chore = new MobFileCleanerChore(); + hcd = new HColumnDescriptor(fam); + hcd.setMobEnabled(true); + hcd.setMobThreshold(mobLen); + hcd.setMaxVersions(1); + hdt.addFamily(hcd); + table = HTU.createTable(hdt, null); + } + + private void initConf() { + + conf.setInt("hfile.format.version", 3); + conf.setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0); + conf.setInt("hbase.client.retries.number", 100); + conf.setInt("hbase.hregion.max.filesize", 200000000); + conf.setInt("hbase.hregion.memstore.flush.size", 800000); + conf.setInt("hbase.hstore.blockingStoreFiles", 150); + conf.setInt("hbase.hstore.compaction.throughput.lower.bound", 52428800); + conf.setInt("hbase.hstore.compaction.throughput.higher.bound", 2 * 52428800); + //conf.set(MobStoreEngine.DEFAULT_MOB_COMPACTOR_CLASS_KEY, + // FaultyMobStoreCompactor.class.getName()); + // Disable automatic MOB compaction + conf.setLong(MobConstants.MOB_COMPACTION_CHORE_PERIOD, 0); + // Disable automatic MOB file cleaner chore + conf.setLong(MobConstants.MOB_CLEANER_PERIOD, 0); + // Set minimum age to archive to 10 sec + conf.setLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, minAgeToArchive); + // Set compacted file discharger interval to a half minAgeToArchive + conf.setLong("hbase.hfile.compaction.discharger.interval", minAgeToArchive/2); + } + + private void loadData(int start, int num) { + try { + + for (int i = 0; i < num; i++) { + Put p = new Put(Bytes.toBytes(start + i)); + p.addColumn(fam, qualifier, mobVal); + table.put(p); + } + admin.flush(table.getName()); + } catch (Exception e) { + LOG.error("MOB file cleaner chore test FAILED", e); + assertTrue(false); + } + } + + @After + public void tearDown() throws Exception { + HTU.shutdownMiniCluster(); + } + + @Test + public void testMobFileCleanerChore() throws InterruptedException, IOException { + + try { + + loadData(0, 10); + loadData(10, 10); + loadData(20, 10); + long num = getNumberOfMobFiles(conf, table.getName(), new String(fam)); + assertEquals(3, num); + // Major compact + admin.majorCompact(hdt.getTableName(), fam); + // wait until compaction is complete + while (admin.getCompactionState(hdt.getTableName()) != CompactionState.NONE) { + Thread.sleep(100); + } + + num = getNumberOfMobFiles(conf, table.getName(), new String(fam)); + assertEquals(4, num); + // We have guarantee, that compcated file discharger will run during this pause + // because it has interval less than this wait time + LOG.info("Waiting for {}ms", minAgeToArchive + 1000); + + Thread.sleep(minAgeToArchive + 1000); + LOG.info("Cleaning up MOB files"); + // Cleanup again + chore.cleanupObsoleteMobFiles(conf, table.getName()); + + num = getNumberOfMobFiles(conf, table.getName(), new String(fam)); + assertEquals(1, num); + + long scanned = scanTable(); + assertEquals(30, scanned); + + } finally { + + admin.disableTable(hdt.getTableName()); + admin.deleteTable(hdt.getTableName()); + } + LOG.info("MOB Stress Test finished OK"); + + } + + private long getNumberOfMobFiles(Configuration conf, TableName tableName, String family) + throws IOException { + FileSystem fs = FileSystem.get(conf); + Path dir = MobUtils.getMobFamilyPath(conf, tableName, family); + FileStatus[] stat = fs.listStatus(dir); + for (FileStatus st : stat) { + LOG.debug("MOB Directory content: {}", st.getPath()); + } + LOG.debug("MOB Directory content total files: {}", stat.length); + + return stat.length; + } + + + private long scanTable() { + try { + + Result result; + ResultScanner scanner = table.getScanner(fam); + long counter = 0; + while ((result = scanner.next()) != null) { + assertTrue(Arrays.equals(result.getValue(fam, qualifier), mobVal)); + counter++; + } + return counter; + } catch (Exception e) { + e.printStackTrace(); + LOG.error("MOB file cleaner chore test FAILED"); + if (HTU != null) { + assertTrue(false); + } else { + System.exit(-1); + } + } + return 0; + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java index ab989f7e0448..2531c67b8e93 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java @@ -89,7 +89,7 @@ public class TestMobCompaction { private HTableDescriptor hdt; private HColumnDescriptor hcd; private Admin admin; - private long count = 5000000; + private long count = 100000; private double failureProb = 0.1; private Table table = null; private MobFileCleanerChore chore = new MobFileCleanerChore(); From 0f0707a42b491847fbc8987ea5dac4877e9004e0 Mon Sep 17 00:00:00 2001 From: Vladimir Rodionov Date: Wed, 6 Nov 2019 12:27:13 -0800 Subject: [PATCH 12/15] HBASE-23266: Test case for MOB compaction in a region's batch mode --- .../hbase/master/MobFileCompactionChore.java | 28 +- .../mob/TestMobCompactionRegionBatchMode.java | 256 ++++++++++++++++++ 2 files changed, 276 insertions(+), 8 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegionBatchMode.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java index 03357a2cb784..02559da75fa0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.mob.MobConstants; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,8 +56,8 @@ public class MobFileCompactionChore extends ScheduledChore { private static final Logger LOG = LoggerFactory.getLogger(MobFileCompactionChore.class); - private final Configuration conf; - private final HMaster master; + private Configuration conf; + private HMaster master; private int regionBatchSize = 0;// not set - compact all public MobFileCompactionChore(HMaster master) { @@ -73,6 +74,12 @@ public MobFileCompactionChore(HMaster master) { } + @VisibleForTesting + public MobFileCompactionChore(Configuration conf, int batchSize) { + this.conf = conf; + this.regionBatchSize = batchSize; + } + @Override protected void chore() { @@ -134,7 +141,8 @@ protected void chore() { } } - private void performMajorCompactionInBatches(Admin admin, TableDescriptor htd, + @VisibleForTesting + public void performMajorCompactionInBatches(Admin admin, TableDescriptor htd, ColumnFamilyDescriptor hcd) throws IOException, InterruptedException { List regions = admin.getRegions(htd.getTableName()); @@ -158,13 +166,15 @@ private void performMajorCompactionInBatches(Admin admin, TableDescriptor htd, } List compacted = new ArrayList(); + int totalCompacted = 0; while(!toCompact.isEmpty()) { // Check status of active compactions for (RegionInfo ri: toCompact) { try { if (admin.getCompactionStateForRegion(ri.getRegionName()) == CompactionState.NONE) { - LOG.info("Finished major compaction: table={} region={}", htd.getTableName(), - ri.getRegionNameAsString()); + totalCompacted++; + LOG.info("Finished major compaction: table={} region={}, compacted regions={}", + htd.getTableName(),ri.getRegionNameAsString(), totalCompacted); compacted.add(ri); } } catch (IOException e) { @@ -181,7 +191,9 @@ private void performMajorCompactionInBatches(Admin admin, TableDescriptor htd, } } compacted.clear(); - Thread.sleep(60000); + LOG.debug("Wait for 10 sec, toCompact size={} regions left={} compacted so far={}", + toCompact.size(), regions.size(), totalCompacted); + Thread.sleep(10000); } LOG.info("Finished major compacting {}. cf={}", htd.getTableName(), hcd.getNameAsString()); @@ -201,8 +213,8 @@ private void startCompaction(Admin admin, TableName table, RegionInfo region, b // Is 1 second too aggressive? Thread.sleep(1000); if (EnvironmentEdgeManager.currentTime() - startTime > waitTime) { - LOG.warn("Waited for {} ms to start major compaction on table: {} region: {}", waitTime, - table.getNameAsString(), region.getRegionNameAsString()); + LOG.warn("Waited for {} ms to start major compaction on table: {} region: {}. Aborted.", + waitTime, table.getNameAsString(), region.getRegionNameAsString()); break; } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegionBatchMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegionBatchMode.java new file mode 100644 index 000000000000..42b41db9c9cb --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegionBatchMode.java @@ -0,0 +1,256 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.mob; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Random; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.CompactionState; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.master.MobFileCleanerChore; +import org.apache.hadoop.hbase.master.MobFileCompactionChore; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.RegionSplitter; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Mob file compaction chore in batch mode test. + * 1. Enables batch mode for regular MOB compaction, + * Sets batch size to 7 regions. + * 2. Disables periodic MOB compactions, sets minimum age to archive to 10 sec + * 3. Creates MOB table with 20 regions + * 4. Loads MOB data (randomized keys, 1000 rows), flushes data. + * 5. Repeats 4. two more times + * 6. Verifies that we have 20 *3 = 60 mob files (equals to number of regions x 3) + * 7. Runs major MOB compaction. + * 8. Verifies that number of MOB files in a mob directory is 20 x4 = 80 + * 9. Waits for a period of time larger than minimum age to archive + * 10. Runs Mob cleaner chore + * 11 Verifies that number of MOB files in a mob directory is 20. + * 12 Runs scanner and checks all 10000 rows. + */ +@SuppressWarnings("deprecation") +@Category(LargeTests.class) +public class TestMobCompactionRegionBatchMode { + private static final Logger LOG = + LoggerFactory.getLogger(TestMobCompactionRegionBatchMode.class); + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobCompactionRegionBatchMode.class); + @Rule + public TestName testName = new TestName(); + + private HBaseTestingUtility HTU; + + private final static String famStr = "f1"; + private final static byte[] fam = Bytes.toBytes(famStr); + private final static byte[] qualifier = Bytes.toBytes("q1"); + private final static long mobLen = 10; + private final static byte[] mobVal = Bytes + .toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789"); + + private Configuration conf; + private HTableDescriptor hdt; + private HColumnDescriptor hcd; + private Admin admin; + private Table table = null; + private long minAgeToArchive = 10000; + private int numRegions = 20; + private int rows = 1000; + private int batchSize = 7; + private MobFileCompactionChore compactionChore; + private MobFileCleanerChore cleanerChore; + + public TestMobCompactionRegionBatchMode() { + } + + + @Before + public void setUp() throws Exception { + HTU = new HBaseTestingUtility(); + hdt = HTU.createTableDescriptor("testMobCompactTable"); + conf = HTU.getConfiguration(); + + initConf(); + + HTU.startMiniCluster(); + admin = HTU.getAdmin(); + compactionChore = new MobFileCompactionChore(conf, batchSize); + cleanerChore = new MobFileCleanerChore(); + hcd = new HColumnDescriptor(fam); + hcd.setMobEnabled(true); + hcd.setMobThreshold(mobLen); + hcd.setMaxVersions(1); + hdt.addFamily(hcd); + byte[][] splitKeys = generateSplitKeys(); + table = HTU.createTable(hdt, splitKeys); + + } + + private byte[][] generateSplitKeys() { + RegionSplitter.UniformSplit splitAlgo = new RegionSplitter.UniformSplit(); + return splitAlgo.split(numRegions); + } + + + private void initConf() { + + conf.setInt("hfile.format.version", 3); + // Disable automatic MOB compaction + conf.setLong(MobConstants.MOB_COMPACTION_CHORE_PERIOD, 0); + // Disable automatic MOB file cleaner chore + conf.setLong(MobConstants.MOB_CLEANER_PERIOD, 0); + // Set minimum age to archive to 10 sec + conf.setLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, minAgeToArchive); + // Set compacted file discharger interval to a half minAgeToArchive + conf.setLong("hbase.hfile.compaction.discharger.interval", minAgeToArchive/2); + conf.setInt(MobConstants.MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE, 7); + } + + private void loadData(int num) { + + Random r = new Random(); + try { + LOG.info("Started loading {} rows", num); + for (int i = 0; i < num; i++) { + byte[] key = new byte[32]; + r.nextBytes(key); + Put p = new Put(key); + p.addColumn(fam, qualifier, mobVal); + table.put(p); + } + admin.flush(table.getName()); + LOG.info("Finished loading {} rows", num); + } catch (Exception e) { + LOG.error("MOB file compaction chore test FAILED", e); + assertTrue(false); + } + } + + @After + public void tearDown() throws Exception { + HTU.shutdownMiniCluster(); + } + + @Test + public void testMobFileCompactionBatchMode() throws InterruptedException, IOException { + + try { + + // Load and flush data 3 times + loadData(rows); + loadData(rows); + loadData(rows); + long num = getNumberOfMobFiles(conf, table.getName(), new String(fam)); + assertEquals(numRegions * 3, num); + // Major compact with batch mode enabled + compactionChore.performMajorCompactionInBatches(admin, hdt, hcd); + // wait until compaction is complete + while (admin.getCompactionState(hdt.getTableName()) != CompactionState.NONE) { + Thread.sleep(100); + } + + num = getNumberOfMobFiles(conf, table.getName(), new String(fam)); + assertEquals(numRegions * 4, num); + // We have guarantee, that compcated file discharger will run during this pause + // because it has interval less than this wait time + LOG.info("Waiting for {}ms", minAgeToArchive + 1000); + + Thread.sleep(minAgeToArchive + 1000); + LOG.info("Cleaning up MOB files"); + // Cleanup again + cleanerChore.cleanupObsoleteMobFiles(conf, table.getName()); + + num = getNumberOfMobFiles(conf, table.getName(), new String(fam)); + assertEquals(numRegions, num); + + long scanned = scanTable(); + assertEquals(3 * rows, scanned); + + } finally { + + admin.disableTable(hdt.getTableName()); + admin.deleteTable(hdt.getTableName()); + } + LOG.info("MOB compaction chore finished OK"); + + } + + private long getNumberOfMobFiles(Configuration conf, TableName tableName, String family) + throws IOException { + FileSystem fs = FileSystem.get(conf); + Path dir = MobUtils.getMobFamilyPath(conf, tableName, family); + FileStatus[] stat = fs.listStatus(dir); + for (FileStatus st : stat) { + LOG.debug("MOB Directory content: {}", st.getPath()); + } + LOG.debug("MOB Directory content total files: {}", stat.length); + + return stat.length; + } + + + private long scanTable() { + try { + + Result result; + ResultScanner scanner = table.getScanner(fam); + long counter = 0; + while ((result = scanner.next()) != null) { + assertTrue(Arrays.equals(result.getValue(fam, qualifier), mobVal)); + counter++; + } + return counter; + } catch (Exception e) { + e.printStackTrace(); + LOG.error("MOB file compaction chore test FAILED"); + if (HTU != null) { + assertTrue(false); + } else { + System.exit(-1); + } + } + return 0; + } +} From dcb54c978da1ac8a0f7003955fede2274d163313 Mon Sep 17 00:00:00 2001 From: Vladimir Rodionov Date: Wed, 6 Nov 2019 13:36:26 -0800 Subject: [PATCH 13/15] HBASE-23267: Test case for MOB compaction in a regular mode --- .../hbase/mob/TestMobCompactionBase.java | 243 ++++++++++++++++++ .../mob/TestMobCompactionRegionBatchMode.java | 190 ++------------ .../mob/TestMobCompactionRegularMode.java | 80 ++++++ 3 files changed, 338 insertions(+), 175 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java new file mode 100644 index 000000000000..4f751e897bde --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java @@ -0,0 +1,243 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.mob; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Random; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.CompactionState; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.master.MobFileCleanerChore; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.RegionSplitter; +import org.junit.After; +import org.junit.Before; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Mob file compaction base test. + * 1. Enables batch mode for regular MOB compaction, + * Sets batch size to 7 regions. (Optional) + * 2. Disables periodic MOB compactions, sets minimum age to archive to 10 sec + * 3. Creates MOB table with 20 regions + * 4. Loads MOB data (randomized keys, 1000 rows), flushes data. + * 5. Repeats 4. two more times + * 6. Verifies that we have 20 *3 = 60 mob files (equals to number of regions x 3) + * 7. Runs major MOB compaction. + * 8. Verifies that number of MOB files in a mob directory is 20 x4 = 80 + * 9. Waits for a period of time larger than minimum age to archive + * 10. Runs Mob cleaner chore + * 11 Verifies that number of MOB files in a mob directory is 20. + * 12 Runs scanner and checks all 3 * 1000 rows. + */ +@SuppressWarnings("deprecation") +public abstract class TestMobCompactionBase { + private static final Logger LOG = + LoggerFactory.getLogger(TestMobCompactionBase.class); + + protected HBaseTestingUtility HTU; + + protected final static String famStr = "f1"; + protected final static byte[] fam = Bytes.toBytes(famStr); + protected final static byte[] qualifier = Bytes.toBytes("q1"); + protected final static long mobLen = 10; + protected final static byte[] mobVal = Bytes + .toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789"); + + protected Configuration conf; + protected HTableDescriptor hdt; + private HColumnDescriptor hcd; + protected Admin admin; + protected Table table = null; + protected long minAgeToArchive = 10000; + protected int numRegions = 20; + protected int rows = 1000; + + protected MobFileCleanerChore cleanerChore; + + public TestMobCompactionBase() { + } + + + @Before + public void setUp() throws Exception { + HTU = new HBaseTestingUtility(); + hdt = HTU.createTableDescriptor(getClass().getName()); + conf = HTU.getConfiguration(); + + initConf(); + + HTU.startMiniCluster(); + admin = HTU.getAdmin(); + cleanerChore = new MobFileCleanerChore(); + hcd = new HColumnDescriptor(fam); + hcd.setMobEnabled(true); + hcd.setMobThreshold(mobLen); + hcd.setMaxVersions(1); + hdt.addFamily(hcd); + byte[][] splitKeys = generateSplitKeys(); + table = HTU.createTable(hdt, splitKeys); + + } + + private byte[][] generateSplitKeys() { + RegionSplitter.UniformSplit splitAlgo = new RegionSplitter.UniformSplit(); + return splitAlgo.split(numRegions); + } + + + protected void initConf() { + + conf.setInt("hfile.format.version", 3); + // Disable automatic MOB compaction + conf.setLong(MobConstants.MOB_COMPACTION_CHORE_PERIOD, 0); + // Disable automatic MOB file cleaner chore + conf.setLong(MobConstants.MOB_CLEANER_PERIOD, 0); + // Set minimum age to archive to 10 sec + conf.setLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, minAgeToArchive); + // Set compacted file discharger interval to a half minAgeToArchive + conf.setLong("hbase.hfile.compaction.discharger.interval", minAgeToArchive/2); + //conf.setInt(MobConstants.MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE, 7); + } + + private void loadData(int num) { + + Random r = new Random(); + try { + LOG.info("Started loading {} rows", num); + for (int i = 0; i < num; i++) { + byte[] key = new byte[32]; + r.nextBytes(key); + Put p = new Put(key); + p.addColumn(fam, qualifier, mobVal); + table.put(p); + } + admin.flush(table.getName()); + LOG.info("Finished loading {} rows", num); + } catch (Exception e) { + LOG.error("MOB file compaction chore test FAILED", e); + assertTrue(false); + } + } + + @After + public void tearDown() throws Exception { + HTU.shutdownMiniCluster(); + } + + + public void baseTestMobFileCompaction() throws InterruptedException, IOException { + + try { + + // Load and flush data 3 times + loadData(rows); + loadData(rows); + loadData(rows); + long num = getNumberOfMobFiles(conf, table.getName(), new String(fam)); + assertEquals(numRegions * 3, num); + // Major MOB compact + mobCompact(admin, hdt, hcd); + // wait until compaction is complete + while (admin.getCompactionState(hdt.getTableName()) != CompactionState.NONE) { + Thread.sleep(100); + } + + num = getNumberOfMobFiles(conf, table.getName(), new String(fam)); + assertEquals(numRegions * 4, num); + // We have guarantee, that compcated file discharger will run during this pause + // because it has interval less than this wait time + LOG.info("Waiting for {}ms", minAgeToArchive + 1000); + + Thread.sleep(minAgeToArchive + 1000); + LOG.info("Cleaning up MOB files"); + // Cleanup again + cleanerChore.cleanupObsoleteMobFiles(conf, table.getName()); + + num = getNumberOfMobFiles(conf, table.getName(), new String(fam)); + assertEquals(numRegions, num); + + long scanned = scanTable(); + assertEquals(3 * rows, scanned); + + } finally { + + admin.disableTable(hdt.getTableName()); + admin.deleteTable(hdt.getTableName()); + } + + } + + protected abstract void mobCompact(Admin admin2, HTableDescriptor hdt2, HColumnDescriptor hcd2) + throws IOException, InterruptedException; + + + protected long getNumberOfMobFiles(Configuration conf, TableName tableName, String family) + throws IOException { + FileSystem fs = FileSystem.get(conf); + Path dir = MobUtils.getMobFamilyPath(conf, tableName, family); + FileStatus[] stat = fs.listStatus(dir); + for (FileStatus st : stat) { + LOG.debug("MOB Directory content: {}", st.getPath()); + } + LOG.debug("MOB Directory content total files: {}", stat.length); + + return stat.length; + } + + + protected long scanTable() { + try { + + Result result; + ResultScanner scanner = table.getScanner(fam); + long counter = 0; + while ((result = scanner.next()) != null) { + assertTrue(Arrays.equals(result.getValue(fam, qualifier), mobVal)); + counter++; + } + return counter; + } catch (Exception e) { + e.printStackTrace(); + LOG.error("MOB file compaction test FAILED"); + if (HTU != null) { + assertTrue(false); + } else { + System.exit(-1); + } + } + return 0; + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegionBatchMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegionBatchMode.java index 42b41db9c9cb..be2fdb5ff1c2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegionBatchMode.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegionBatchMode.java @@ -17,34 +17,14 @@ * limitations under the License. */ package org.apache.hadoop.hbase.mob; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - import java.io.IOException; -import java.util.Arrays; -import java.util.Random; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.CompactionState; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.ResultScanner; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.master.MobFileCleanerChore; import org.apache.hadoop.hbase.master.MobFileCompactionChore; import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.RegionSplitter; -import org.junit.After; import org.junit.Before; import org.junit.ClassRule; import org.junit.Rule; @@ -68,11 +48,11 @@ * 9. Waits for a period of time larger than minimum age to archive * 10. Runs Mob cleaner chore * 11 Verifies that number of MOB files in a mob directory is 20. - * 12 Runs scanner and checks all 10000 rows. + * 12 Runs scanner and checks all 3 * 1000 rows. */ @SuppressWarnings("deprecation") @Category(LargeTests.class) -public class TestMobCompactionRegionBatchMode { +public class TestMobCompactionRegionBatchMode extends TestMobCompactionBase{ private static final Logger LOG = LoggerFactory.getLogger(TestMobCompactionRegionBatchMode.class); @ClassRule @@ -81,176 +61,36 @@ public class TestMobCompactionRegionBatchMode { @Rule public TestName testName = new TestName(); - private HBaseTestingUtility HTU; - - private final static String famStr = "f1"; - private final static byte[] fam = Bytes.toBytes(famStr); - private final static byte[] qualifier = Bytes.toBytes("q1"); - private final static long mobLen = 10; - private final static byte[] mobVal = Bytes - .toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789"); - - private Configuration conf; - private HTableDescriptor hdt; - private HColumnDescriptor hcd; - private Admin admin; - private Table table = null; - private long minAgeToArchive = 10000; - private int numRegions = 20; - private int rows = 1000; private int batchSize = 7; private MobFileCompactionChore compactionChore; - private MobFileCleanerChore cleanerChore; public TestMobCompactionRegionBatchMode() { } - @Before public void setUp() throws Exception { - HTU = new HBaseTestingUtility(); - hdt = HTU.createTableDescriptor("testMobCompactTable"); - conf = HTU.getConfiguration(); - - initConf(); - - HTU.startMiniCluster(); - admin = HTU.getAdmin(); + super.setUp(); compactionChore = new MobFileCompactionChore(conf, batchSize); - cleanerChore = new MobFileCleanerChore(); - hcd = new HColumnDescriptor(fam); - hcd.setMobEnabled(true); - hcd.setMobThreshold(mobLen); - hcd.setMaxVersions(1); - hdt.addFamily(hcd); - byte[][] splitKeys = generateSplitKeys(); - table = HTU.createTable(hdt, splitKeys); - - } - - private byte[][] generateSplitKeys() { - RegionSplitter.UniformSplit splitAlgo = new RegionSplitter.UniformSplit(); - return splitAlgo.split(numRegions); - } - - - private void initConf() { - - conf.setInt("hfile.format.version", 3); - // Disable automatic MOB compaction - conf.setLong(MobConstants.MOB_COMPACTION_CHORE_PERIOD, 0); - // Disable automatic MOB file cleaner chore - conf.setLong(MobConstants.MOB_CLEANER_PERIOD, 0); - // Set minimum age to archive to 10 sec - conf.setLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, minAgeToArchive); - // Set compacted file discharger interval to a half minAgeToArchive - conf.setLong("hbase.hfile.compaction.discharger.interval", minAgeToArchive/2); - conf.setInt(MobConstants.MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE, 7); } - private void loadData(int num) { - - Random r = new Random(); - try { - LOG.info("Started loading {} rows", num); - for (int i = 0; i < num; i++) { - byte[] key = new byte[32]; - r.nextBytes(key); - Put p = new Put(key); - p.addColumn(fam, qualifier, mobVal); - table.put(p); - } - admin.flush(table.getName()); - LOG.info("Finished loading {} rows", num); - } catch (Exception e) { - LOG.error("MOB file compaction chore test FAILED", e); - assertTrue(false); - } - } - - @After - public void tearDown() throws Exception { - HTU.shutdownMiniCluster(); + protected void initConf() { + super.initConf(); + conf.setInt(MobConstants.MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE, batchSize); } @Test public void testMobFileCompactionBatchMode() throws InterruptedException, IOException { - - try { - - // Load and flush data 3 times - loadData(rows); - loadData(rows); - loadData(rows); - long num = getNumberOfMobFiles(conf, table.getName(), new String(fam)); - assertEquals(numRegions * 3, num); - // Major compact with batch mode enabled - compactionChore.performMajorCompactionInBatches(admin, hdt, hcd); - // wait until compaction is complete - while (admin.getCompactionState(hdt.getTableName()) != CompactionState.NONE) { - Thread.sleep(100); - } - - num = getNumberOfMobFiles(conf, table.getName(), new String(fam)); - assertEquals(numRegions * 4, num); - // We have guarantee, that compcated file discharger will run during this pause - // because it has interval less than this wait time - LOG.info("Waiting for {}ms", minAgeToArchive + 1000); - - Thread.sleep(minAgeToArchive + 1000); - LOG.info("Cleaning up MOB files"); - // Cleanup again - cleanerChore.cleanupObsoleteMobFiles(conf, table.getName()); - - num = getNumberOfMobFiles(conf, table.getName(), new String(fam)); - assertEquals(numRegions, num); - - long scanned = scanTable(); - assertEquals(3 * rows, scanned); - - } finally { - - admin.disableTable(hdt.getTableName()); - admin.deleteTable(hdt.getTableName()); - } - LOG.info("MOB compaction chore finished OK"); + LOG.info("MOB compaction chore batch mode started"); + baseTestMobFileCompaction(); + LOG.info("MOB compaction chore batch mode finished OK"); } - - private long getNumberOfMobFiles(Configuration conf, TableName tableName, String family) - throws IOException { - FileSystem fs = FileSystem.get(conf); - Path dir = MobUtils.getMobFamilyPath(conf, tableName, family); - FileStatus[] stat = fs.listStatus(dir); - for (FileStatus st : stat) { - LOG.debug("MOB Directory content: {}", st.getPath()); - } - LOG.debug("MOB Directory content total files: {}", stat.length); - - return stat.length; - } - - private long scanTable() { - try { - - Result result; - ResultScanner scanner = table.getScanner(fam); - long counter = 0; - while ((result = scanner.next()) != null) { - assertTrue(Arrays.equals(result.getValue(fam, qualifier), mobVal)); - counter++; - } - return counter; - } catch (Exception e) { - e.printStackTrace(); - LOG.error("MOB file compaction chore test FAILED"); - if (HTU != null) { - assertTrue(false); - } else { - System.exit(-1); - } - } - return 0; + @Override + protected void mobCompact(Admin admin, HTableDescriptor hdt, HColumnDescriptor hcd) + throws IOException, InterruptedException { + // Major compact with batch mode enabled + compactionChore.performMajorCompactionInBatches(admin, hdt, hcd); } + } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java new file mode 100644 index 000000000000..053c1f051e2c --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java @@ -0,0 +1,80 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.mob; +import java.io.IOException; + +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Mob file compaction chore in batch mode test. + * 1. Use default (non-batch) mode for regular MOB compaction, + * 2. Disables periodic MOB compactions, sets minimum age to archive to 10 sec + * 3. Creates MOB table with 20 regions + * 4. Loads MOB data (randomized keys, 1000 rows), flushes data. + * 5. Repeats 4. two more times + * 6. Verifies that we have 20 *3 = 60 mob files (equals to number of regions x 3) + * 7. Runs major MOB compaction. + * 8. Verifies that number of MOB files in a mob directory is 20 x4 = 80 + * 9. Waits for a period of time larger than minimum age to archive + * 10. Runs Mob cleaner chore + * 11 Verifies that number of MOB files in a mob directory is 20. + * 12 Runs scanner and checks all 3 * 1000 rows. + */ +@SuppressWarnings("deprecation") +@Category(LargeTests.class) +public class TestMobCompactionRegularMode extends TestMobCompactionBase{ + private static final Logger LOG = + LoggerFactory.getLogger(TestMobCompactionRegularMode.class); + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobCompactionRegularMode.class); + @Rule + public TestName testName = new TestName(); + + + public TestMobCompactionRegularMode() { + } + + @Test + public void testMobFileCompactionBatchMode() throws InterruptedException, IOException { + LOG.info("MOB compaction regular mode started"); + baseTestMobFileCompaction(); + LOG.info("MOB compaction regular mode finished OK"); + + } + + @Override + protected void mobCompact(Admin admin, HTableDescriptor hdt, HColumnDescriptor hcd) + throws IOException, InterruptedException { + // Major compact MOB table + admin.majorCompact(hdt.getTableName(), hcd.getName()); + } + +} From 212cc513840d5ad9035e61515eab0f8ae4c2f282 Mon Sep 17 00:00:00 2001 From: Vladimir Rodionov Date: Wed, 20 Nov 2019 16:59:25 -0800 Subject: [PATCH 14/15] HBASE-23189: Finilize I/O optimized MOB compaction --- .../hbase/io/hfile/HFileWriterImpl.java | 4 + .../hbase/master/MobFileCleanerChore.java | 7 + .../hbase/mob/DefaultMobStoreCompactor.java | 593 +++++------------- .../apache/hadoop/hbase/mob/MobConstants.java | 43 +- .../org/apache/hadoop/hbase/mob/MobUtils.java | 62 -- .../hbase/regionserver/StoreFileWriter.java | 4 + .../hadoop/hbase/mob/TestMobCompaction.java | 30 +- .../hbase/mob/TestMobCompactionBase.java | 1 - .../hbase/mob/TestMobCompactionOptMode.java | 88 +++ .../TestMobCompactionOptRegionBatchMode.java | 99 +++ .../mob/TestMobCompactionRegularMode.java | 6 +- ...tMobCompactionRegularRegionBatchMode.java} | 14 +- 12 files changed, 397 insertions(+), 554 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java rename hbase-server/src/test/java/org/apache/hadoop/hbase/mob/{TestMobCompactionRegionBatchMode.java => TestMobCompactionRegularRegionBatchMode.java} (86%) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java index 93cca8bd3623..5e48205c173e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java @@ -227,6 +227,10 @@ protected final void writeFileInfo(FixedFileTrailer trailer, DataOutputStream ou HFile.updateWriteLatency(System.currentTimeMillis() - startTime); } + public long getPos() throws IOException { + return outputStream.getPos(); + + } /** * Checks that the given Cell's key does not violate the key order. * diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java index cc1e9b7884e8..fbe82f551390 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java @@ -181,6 +181,7 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) } try { for (Path pp : storeFiles) { + /*DEBUG*/LOG.debug("DDDD Store file: {}", pp); HStoreFile sf = new HStoreFile(fs, pp, conf, CacheConfig.DISABLED, BloomType.NONE, true); sf.initReader(); @@ -200,6 +201,7 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) // if length = 1 means NULL, that there are no MOB references // in this store file, but the file was created by new MOB code String[] mobs = new String(mobRefData).split(","); + /*DEBUG*/LOG.debug("DDDD found: {} mob refs: ", mobs.length, Arrays.toString(mobs)); regionMobs.addAll(Arrays.asList(mobs)); } } @@ -218,6 +220,9 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) LOG.warn("Found too many active MOB files: {}, this may result in high memory pressure.", allActiveMobFileName.size()); } + /*DEBUG*/LOG.debug("DDDD Found: {} active mob refs", allActiveMobFileName.size()); + allActiveMobFileName.stream().forEach(LOG::debug); + // Now scan MOB directories and find MOB files with no references to them for (ColumnFamilyDescriptor hcd : list) { List toArchive = new ArrayList(); @@ -238,6 +243,8 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) } else { LOG.info("Skipping fresh file: {}", p); } + } else { + LOG.info("Keepeing active MOB file: {}", p); } } LOG.info(" MOB Cleaner found {} files for family={}", toArchive.size() , family); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java index 623d655157ae..74d7e812fa15 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java @@ -24,30 +24,23 @@ import java.io.IOException; import java.io.InterruptedIOException; import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; import java.util.Date; import java.util.HashMap; import java.util.HashSet; import java.util.List; -import java.util.Map; import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.commons.lang3.mutable.MutableLong; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.CellSink; import org.apache.hadoop.hbase.regionserver.HMobStore; import org.apache.hadoop.hbase.regionserver.HStore; -import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.KeyValueScanner; import org.apache.hadoop.hbase.regionserver.ScanInfo; @@ -106,12 +99,17 @@ protected Boolean initialValue() { }; /* - * Contains list of MOB files for compaction if - * generational compaction is enabled. + * Map : MOB file name - file length + * Can be expensive for large amount of MOB files? */ - static ThreadLocal> compSelections = - new ThreadLocal>(); - + static ThreadLocal> mobLengthMap = + new ThreadLocal>() { + @Override + protected HashMap initialValue() { + return new HashMap(); + } + }; + private final InternalScannerFactory scannerFactory = new InternalScannerFactory() { @Override @@ -164,39 +162,51 @@ public List compact(CompactionRequestImpl request, userRequest.set(Boolean.FALSE); } LOG.info("Mob compaction files: " + request.getFiles()); - // Check if generational MOB compaction - compSelections.set(null); + // Check if I/O optimized MOB compaction if (conf.get(MobConstants.MOB_COMPACTION_TYPE_KEY, MobConstants.DEFAULT_MOB_COMPACTION_TYPE) - .equals(MobConstants.GENERATIONAL_MOB_COMPACTION_TYPE)) { + .equals(MobConstants.IO_OPTIMIZED_MOB_COMPACTION_TYPE)) { if (request.isMajor() && request.getPriority() == HStore.PRIORITY_USER) { - // Compact MOBs Path mobDir = MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName()); List mobFiles = MobUtils.getReferencedMobFiles(request.getFiles(), mobDir); if (mobFiles.size() > 0) { - Generations gens = Generations.build(mobFiles, conf); - List list = gens.getCompactionSelections(); - if (list.size() > 0) { - compSelections.set(list); - } + calculateMobLengthMap(mobFiles); } + LOG.info("I/O optimized MOB compaction. Total referenced MOB files: {}", mobFiles.size()); } } return compact(request, scannerFactory, writerFactory, throughputController, user); } + private void calculateMobLengthMap(List mobFiles) throws IOException { + FileSystem fs = mobFiles.get(0).getFileSystem(this.conf); + HashMap map = mobLengthMap.get(); + map.clear(); + long maxMobFileSize = conf.getLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, + MobConstants.DEFAULT_MOB_COMPACTION_MAX_FILE_SIZE); + for (Path p: mobFiles) { + FileStatus st = fs.getFileStatus(p); + long size = st.getLen(); + /*DEBUG*/ + if (size > 2 * maxMobFileSize) { + LOG.debug("DDDD FOUND BIG ref MOB size={} file={}", size, p.getName()); + } + + map.put(p.getName(), fs.getFileStatus(p).getLen()); + } + } + + /** * Performs compaction on a column family with the mob flag enabled. - * This is for when MOB compaction is explicitly requested (by User). + * This works only when MOB compaction is explicitly requested (by User), or by Master * There are two modes of a MOB compaction:
      *

      *

        - *
      • 1. General mode - when all data is compacted into a single MOB file. - *
      • 2. Generational mode - when MOB files are collected (and compacted) by generations,
        - * which are defined by history of a region splits. The oldest generations are compacted first.
        - * In this mode the output of a compaction process will contain multiple MOB files. The main
        - * idea behind generational compaction is to limit maximum size of a MOB file and to limit I/O
        - * write/read amplification during MOB compaction. + *
      • 1. Full mode - when all MOB data for a region is compacted into a single MOB file. + *
      • 2. I/O optimized mode - for use cases with no or infrequent updates/deletes of a
        + * MOB data. The main idea behind i/o optimized compaction is to limit maximum size of a MOB + * file produced during compaction and to limit I/O write/read amplification. *
      * The basic algorithm of compaction is the following:
      * 1. If the Put cell has a mob reference tag, the cell's value is the path of the mob file. @@ -241,34 +251,19 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel mobRefSet.get().clear(); boolean isUserRequest = userRequest.get(); boolean compactMOBs = major && isUserRequest; - boolean generationalMob = conf.get(MobConstants.MOB_COMPACTION_TYPE_KEY, + boolean ioOptimizedMob = conf.get(MobConstants.MOB_COMPACTION_TYPE_KEY, MobConstants.DEFAULT_MOB_COMPACTION_TYPE) - .equals(MobConstants.GENERATIONAL_MOB_COMPACTION_TYPE); - if (generationalMob && compSelections.get() == null) { - LOG.warn("MOB compaction aborted, reason: generational compaction is enabled, "+ - "but compaction selection was empty."); - return true; - } - OutputMobWriters mobWriters = null; - - if (compactMOBs && generationalMob) { - List sel = compSelections.get(); - if (sel != null && sel.size() > 0) { - // Create output writers for compaction selections - mobWriters = new OutputMobWriters(sel); - int numWriters = mobWriters.getNumberOfWriters(); - List writers = new ArrayList(); - for (int i=0; i < numWriters; i++) { - StoreFileWriter sfw = mobStore.createWriterInTmp(new Date(fd.latestPutTs), fd.maxKeyCount, - compactionCompression, store.getRegionInfo().getStartKey(), true); - writers.add(sfw); - } - mobWriters.initOutputWriters(writers); - } - } + .equals(MobConstants.IO_OPTIMIZED_MOB_COMPACTION_TYPE); boolean discardMobMiss = - conf.getBoolean(MobConstants.MOB_UNSAFE_DISCARD_MISS_KEY, MobConstants.DEFAULT_MOB_DISCARD_MISS); + conf.getBoolean(MobConstants.MOB_UNSAFE_DISCARD_MISS_KEY, + MobConstants.DEFAULT_MOB_DISCARD_MISS); + + long maxMobFileSize = conf.getLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, + MobConstants.DEFAULT_MOB_COMPACTION_MAX_FILE_SIZE); + LOG.info("Compact MOB={} optimized={} maximum MOB file size={}", compactMOBs, + ioOptimizedMob, maxMobFileSize); + FileSystem fs = FileSystem.get(conf); // Since scanner.next() can return 'false' but still be delivering data, @@ -287,10 +282,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel byte[] fileName = null; StoreFileWriter mobFileWriter = null; /* - * mobCells are used only to decide if we need to commit or abort major MOB output file. - * This file is present in both: regular and generational compaction. In regular compaction - - * it is the only MOB file output, in generational compaction it is the output for new MOB cells, - * which come from store files - not from MOB files. + * mobCells are used only to decide if we need to commit or abort current MOB output file. */ long mobCells = 0; long cellsCountCompactedToMob = 0, cellsCountCompactedFromMob = 0; @@ -306,22 +298,10 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel Cell mobCell = null; try { - try { - mobFileWriter = mobStore.createWriterInTmp(new Date(fd.latestPutTs), fd.maxKeyCount, - compactionCompression, store.getRegionInfo().getStartKey(), true); - fileName = Bytes.toBytes(mobFileWriter.getPath().getName()); - } catch (IOException e) { - // Bailing out - LOG.error("Failed to create mob writer, ", e); - throw e; - } - if (compactMOBs) { - // Add the only reference we get for compact MOB general (not generational) case - // because new store file will have only one MOB reference - // in this case - of a newly compacted MOB file. In generational compaction mode, - // this reference is present as well along with (potentially) many others. - mobRefSet.get().add(mobFileWriter.getPath().getName()); - } + + mobFileWriter = newMobWriter(fd, compactMOBs); + fileName = Bytes.toBytes(mobFileWriter.getPath().getName()); + do { hasMore = scanner.next(cells, scannerContext); if (LOG.isDebugEnabled()) { @@ -350,35 +330,59 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel LOG.error("Missing MOB cell value: file=" + pp + " cell=" + mobCell); continue; } else if (mobCell.getValueLength() == 0) { + //TODO: what to do here? This is data corruption? LOG.warn("Found 0 length MOB cell in a file={} cell={}", pp, mobCell); } if (mobCell.getValueLength() > mobSizeThreshold) { // put the mob data back to the MOB store file PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId()); - if (generationalMob) { - //TODO: verify fName - StoreFileWriter stw = mobWriters.getOutputWriterForInputFile(fName); - if (stw != null) { - stw.append(mobCell); - mobWriters.incrementMobCountForOutputWriter(stw); - byte[] fname = Bytes.toBytes(stw.getPath().getName()); - // Update MOB reference with new MOB file - writer.append(MobUtils.createMobRefCell(mobCell, fname, - this.mobStore.getRefCellTags())); - } else { - // else leave mob cell in a MOB file which is not in compaction selections - // write MOB cell reference to output store - // add MOB file reference to mobRefSet - writer.append(mobCell); - mobRefSet.get().add(fName); - } - } else { + if (!ioOptimizedMob) { mobFileWriter.append(mobCell); mobCells++; writer.append(MobUtils.createMobRefCell(mobCell, fileName, this.mobStore.getRefCellTags())); - } + } else { + // I/O optimized mode + // Check if MOB cell origin file size is + // greater than threshold + Long size = mobLengthMap.get().get(fName); + if (size == null) { + // FATAL error, abort compaction + String msg = + String.format("Found unreferenced MOB file during compaction %s, aborting.", + fName); + LOG.error(msg); + throw new IOException(msg); + } + // Can not be null + if (size < maxMobFileSize) { + // If MOB cell origin file is below threshold + // it is get compacted + mobFileWriter.append(mobCell); + // Update number of mobCells in a current mob writer + mobCells++; + writer.append(MobUtils.createMobRefCell(mobCell, fileName, + this.mobStore.getRefCellTags())); + // Update total size of the output (we do not take into account + // file compression yet) + long len = getLength(mobFileWriter); + + if (len > maxMobFileSize) { + LOG.debug("DDDD Output File Length={} file=", len, new String(fileName)); + commitOrAbortMobWriter(mobFileWriter, fd.maxSeqId, mobCells, major); + mobFileWriter = newMobWriter(fd, compactMOBs); + fileName = Bytes.toBytes(mobFileWriter.getPath().getName()); + mobCells = 0; + } + } else { + // We leave large MOB file as is (is not compacted), + // then we update set of MOB file references + // and append mob cell directly to the store's writer + mobRefSet.get().add(fName); + writer.append(mobCell); + } + } } else { // If MOB value is less than threshold, append it directly to a store file PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId()); @@ -398,6 +402,20 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel mobCells++; cellsCountCompactedToMob++; cellsSizeCompactedToMob += c.getValueLength(); + if (ioOptimizedMob) { + // Update total size of the output (we do not take into account + // file compression yet) + long len = getLength(mobFileWriter); + if (len > 2 * maxMobFileSize) { + LOG.debug("DDDD Output MOB size={} file={}", len, fileName); + } + if (len > maxMobFileSize) { + commitOrAbortMobWriter(mobFileWriter, fd.maxSeqId, mobCells, major); + mobFileWriter = newMobWriter(fd, compactMOBs); + fileName = Bytes.toBytes(mobFileWriter.getPath().getName()); + mobCells = 0; + } + } } else { // Not a MOB cell, write it directly to a store file writer.append(c); @@ -497,28 +515,62 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel // Remove all MOB references because compaction failed mobRefSet.get().clear(); // Abort writer - LOG.debug("Aborting writer for {} because of compaction failure", + LOG.debug("Aborting writer for {} because of a compaction failure", mobFileWriter.getPath()); abortWriter(mobFileWriter); - //Check if other writers exist - if (mobWriters != null) { - for(StoreFileWriter w: mobWriters.getOutputWriters()) { - LOG.debug("Aborting writer for {} because of compaction failure", - w.getPath()); - abortWriter(w); - } - } } } + + // Commit last MOB writer + commitOrAbortMobWriter(mobFileWriter, fd.maxSeqId, mobCells, major); + + mobStore.updateCellsCountCompactedFromMob(cellsCountCompactedFromMob); + mobStore.updateCellsCountCompactedToMob(cellsCountCompactedToMob); + mobStore.updateCellsSizeCompactedFromMob(cellsSizeCompactedFromMob); + mobStore.updateCellsSizeCompactedToMob(cellsSizeCompactedToMob); + progress.complete(); + return true; + } + + private long getLength(StoreFileWriter mobFileWriter) throws IOException { + return mobFileWriter.getPos(); + } + + + private StoreFileWriter newMobWriter(FileDetails fd, boolean compactMOBs) + throws IOException { + try { + StoreFileWriter mobFileWriter = mobStore.createWriterInTmp(new Date(fd.latestPutTs), + fd.maxKeyCount, compactionCompression, store.getRegionInfo().getStartKey(), true); + LOG.debug("DDDD New MOB created={}", mobFileWriter.getPath().getName()); + + if (compactMOBs) { + // Add reference we get for compact MOB + mobRefSet.get().add(mobFileWriter.getPath().getName()); + } + return mobFileWriter; + } catch (IOException e) { + // Bailing out + LOG.error("Failed to create mob writer, ", e); + throw e; + } + } + + private void commitOrAbortMobWriter(StoreFileWriter mobFileWriter, long maxSeqId, + long mobCells, boolean major) throws IOException + { // Commit or abort major mob writer // If IOException happens during below operation, some // MOB files can be committed partially, but corresponding // store file won't be committed, therefore these MOB files // become orphans and will be deleted during next MOB cleaning chore cycle + LOG.debug("DDDD Commit or Abort size={} mobCells={} major={} file={}", + mobFileWriter.getPos(), mobCells, major, mobFileWriter.getPath().getName()); + Path path = MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName()); if (mobFileWriter != null) { if (mobCells > 0) { // If the mob file is not empty, commit it. - mobFileWriter.appendMetadata(fd.maxSeqId, major, mobCells); + mobFileWriter.appendMetadata(maxSeqId, major, mobCells); mobFileWriter.close(); mobStore.commitFile(mobFileWriter.getPath(), path); } else { @@ -530,31 +582,8 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel abortWriter(mobFileWriter); } } - // Commit or abort generational writers - if (mobWriters != null) { - for (StoreFileWriter w: mobWriters.getOutputWriters()) { - Long mobs = mobWriters.getMobCountForOutputWriter(w); - if (mobs != null && mobs > 0) { - mobRefSet.get().add(w.getPath().getName()); - w.appendMetadata(fd.maxSeqId, major, mobs); - w.close(); - mobStore.commitFile(w.getPath(), path); - } else { - LOG.debug("Aborting writer for {} because there are no MOB cells", w.getPath()); - // Remove MOB file from reference set - mobRefSet.get().remove(w.getPath().getName()); - abortWriter(w); - } - } - } - mobStore.updateCellsCountCompactedFromMob(cellsCountCompactedFromMob); - mobStore.updateCellsCountCompactedToMob(cellsCountCompactedToMob); - mobStore.updateCellsSizeCompactedFromMob(cellsSizeCompactedFromMob); - mobStore.updateCellsSizeCompactedToMob(cellsSizeCompactedToMob); - progress.complete(); - return true; } - + protected static String createKey(TableName tableName, String encodedName, String columnFamilyName) { return tableName.getNameAsString()+ "_" + encodedName + "_"+ columnFamilyName; @@ -574,313 +603,3 @@ protected List commitWriter(StoreFileWriter writer, FileDetails fd, } -final class FileSelection implements Comparable { - - public final static String NULL_REGION = ""; - private Path path; - private long earliestTs; - private Configuration conf; - - public FileSelection(Path path, Configuration conf) throws IOException { - this.path = path; - this.conf = conf; - readEarliestTimestamp(); - } - - public String getEncodedRegionName() { - String name = MobUtils.getEncodedRegionName(path.getName()); - if (name != null) { - return name; - } else { - return NULL_REGION; - } - } - - public Path getPath() { - return path; - } - - public long getEarliestTimestamp() { - return earliestTs; - } - - private void readEarliestTimestamp() throws IOException { - FileSystem fs = path.getFileSystem(conf); - HStoreFile sf = new HStoreFile(fs, path, conf, CacheConfig.DISABLED, - BloomType.NONE, true); - sf.initReader(); - byte[] tsData = sf.getMetadataValue(HStoreFile.EARLIEST_PUT_TS); - if (tsData != null) { - this.earliestTs = Bytes.toLong(tsData); - } - sf.closeStoreFile(true); - } - - @Override - public int compareTo(FileSelection o) { - if (this.earliestTs > o.earliestTs) { - return +1; - } else if (this.earliestTs == o.earliestTs) { - return 0; - } else { - return -1; - } - } - -} - -final class Generations { - - private List generations; - private Configuration conf; - - private Generations(List gens, Configuration conf) { - this.generations = gens; - this.conf = conf; - } - - List getCompactionSelections() throws IOException { - int maxTotalFiles = this.conf.getInt(MobConstants.MOB_COMPACTION_MAX_TOTAL_FILES_KEY, - MobConstants.DEFAULT_MOB_COMPACTION_MAX_TOTAL_FILES); - int currentTotal = 0; - List list = new ArrayList(); - - for (Generation g: generations) { - List sel = g.getCompactionSelections(conf); - int size = getSize(sel); - if ((currentTotal + size > maxTotalFiles) && currentTotal > 0) { - break; - } else { - currentTotal += size; - list.addAll(sel); - } - } - return list; - } - - private int getSize(List sel) { - int size = 0; - for(CompactionSelection cs: sel) { - size += cs.size(); - } - return size; - } - - static Generations build(List files, Configuration conf) throws IOException { - Map > map = new HashMap>(); - for(Path p: files) { - String key = getRegionNameFromFileName(p.getName()); - map.computeIfAbsent(key, k -> new ArrayList()).add(new FileSelection(p, conf)); - } - - List gens = new ArrayList(); - for (Map.Entry> entry: map.entrySet()) { - String key = entry.getKey(); - Generation g = new Generation(key); - List selFiles = map.get(key); - for(FileSelection fs: selFiles) { - g.addFile(fs); - } - gens.add(g); - } - // Sort all generation files one-by-one - for(Generation gg: gens) { - gg.sortFiles(); - } - // Sort generations - Collections.sort(gens); - return new Generations(gens, conf); - } - - static String getRegionNameFromFileName(String mobFileName) { - String name = MobUtils.getEncodedRegionName(mobFileName); - if (name == null) { - return Generation.GEN0; - } - return name; - } -} - -final class Generation implements Comparable { - - static final String GEN0 ="GEN0"; - private String regionName; - private long earliestTs = Long.MAX_VALUE; - private List files = new ArrayList<>(); - List compSelections; - - public Generation(String name) { - this.regionName = name; - } - - @SuppressWarnings("deprecation") - public List getCompactionSelections(Configuration conf) throws IOException { - - - int minFiles = conf.getInt(MobConstants.MOB_COMPACTION_MIN_FILES_KEY, - MobConstants.DEFAULT_MOB_COMPACTION_MIN_FILES); - int maxFiles = conf.getInt(MobConstants.MOB_COMPACTION_MAX_FILES_KEY, - MobConstants.DEFAULT_MOB_COMPACTION_MAX_FILES); - long maxSelectionSize = conf.getLong(MobConstants.MOB_COMPACTION_MAX_SELECTION_SIZE_KEY, - MobConstants.DEFAULT_MOB_COMPACTION_MAX_SELECTION_SIZE); - // Now it is ordered from oldest to newest ones - List rfiles = Lists.reverse(files); - List retList = new ArrayList(); - FileSystem fs = rfiles.get(0).getPath().getFileSystem(conf); - int off = 0; - while (off < rfiles.size()) { - if (fs.getLength(rfiles.get(off).getPath()) >= maxSelectionSize) { - off++; continue; - } - long selSize = 0; - int limit = Math.min(off + maxFiles, rfiles.size()); - int start = off; - List sel = new ArrayList(); - for (; off < limit; off++) { - Path p = rfiles.get(off).getPath(); - long fSize = fs.getLength(p); - if (selSize + fSize < maxSelectionSize) { - selSize+= fSize; - sel.add(new FileSelection(p, conf)); - } else { - if (sel.size() < minFiles) { - // discard - sel.clear(); - // advance by 1 - off = start +1; - } else { - // we have new selection - CompactionSelection cs = new CompactionSelection(sel); - retList.add(cs); - off++; - } - break; // continue outer loop - } - } - } - return retList; - } - - public boolean addFile(FileSelection f) { - if (f.getEncodedRegionName().equals(regionName)) { - files.add(f); - if (f.getEarliestTimestamp() < earliestTs) { - earliestTs = f.getEarliestTimestamp(); - } - return true; - } else { - return false; - } - } - - public void sortFiles() { - Collections.sort(files); - } - - public List getFiles() { - return files; - } - - public String getEncodedRegionName() { - return regionName; - } - - public long getEarliestTimestamp() { - return earliestTs; - } - - @Override - public int compareTo(Generation o) { - if (this.earliestTs > o.earliestTs) { - return +1; - } else if (this.earliestTs == o.earliestTs) { - return 0; - } else { - return -1; - } - } -} - -final class CompactionSelection { - private static AtomicLong idGen = new AtomicLong(); - private List files; - private long id; - - public CompactionSelection(List files) { - this.files = files; - this.id = idGen.getAndIncrement(); - } - - public List getFiles() { - return files; - } - - public long getId() { - return id; - } - - int size() { - return files.size(); - } -} - -final class OutputMobWriters { - - /* - * Input MOB file name -> output file writer - */ - private Map writerMap = new HashMap(); - /* - * Output file name -> MOB counter - */ - private Map mapMobCounts = new HashMap(); - /* - * List of compaction selections - */ - private List compSelections; - - public OutputMobWriters(List compSelections) { - this.compSelections = compSelections; - } - - int getNumberOfWriters() { - return compSelections.size(); - } - - StoreFileWriter getWriterForFile(String fileName) { - return writerMap.get(fileName); - } - - void initOutputWriters(List writers) { - for (int i = 0; i < writers.size(); i++) { - StoreFileWriter sw = writers.get(i); - mapMobCounts.put(sw.getPath().getName(), new MutableLong(0L)); - CompactionSelection cs = compSelections.get(i); - for (FileSelection fs: cs.getFiles()) { - writerMap.put(fs.getPath().getName(), sw); - } - } - } - - Collection getOutputWriters() { - return writerMap.values(); - } - - StoreFileWriter getOutputWriterForInputFile(String name) { - return writerMap.get(name); - } - - long getMobCountForOutputWriter(StoreFileWriter writer) { - return mapMobCounts.get(writer.getPath().getName()).longValue(); - } - - void incrementMobCountForOutputWriter(StoreFileWriter writer) { - String key = writer.getPath().getName(); - MutableLong v = mapMobCounts.get(key); - if (v == null) { - mapMobCounts.put(key, new MutableLong(1L)); - } else { - v.add(1); - } - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java index b72774aa7711..dc0c0808b2c1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java @@ -55,7 +55,6 @@ public final class MobConstants { public static final long DEFAULT_MOB_CACHE_EVICT_PERIOD = 3600L; public final static String TEMP_DIR_NAME = ".tmp"; - public final static byte[] MOB_TABLE_LOCK_SUFFIX = Bytes.toBytes(".mobLock"); /** * The max number of a MOB table regions that is allowed in a batch of the mob compaction. @@ -83,15 +82,15 @@ public final class MobConstants { public static final String MOB_COMPACTOR_CLASS_KEY = "hbase.mob.compactor.class"; /** - * Mob compaction type: "full", "generational" + * Mob compaction type: "full", "io_optimized" * "full" - run full major compaction (during migration) - * "generational" - optimized version + * "io_optimized" - optimized version for use case with infrequent updates/deletes */ public final static String MOB_COMPACTION_TYPE_KEY = "hbase.mob.compaction.type"; public final static String DEFAULT_MOB_COMPACTION_TYPE = "full"; - public final static String GENERATIONAL_MOB_COMPACTION_TYPE = "generational"; + public final static String IO_OPTIMIZED_MOB_COMPACTION_TYPE = "optimized"; public final static String FULL_MOB_COMPACTION_TYPE = "full"; @@ -99,40 +98,12 @@ public final class MobConstants { /** * Maximum size of a MOB compaction selection */ - public static final String MOB_COMPACTION_MAX_SELECTION_SIZE_KEY = - "hbase.mob.compactions.max.selection.size"; + public static final String MOB_COMPACTION_MAX_FILE_SIZE_KEY = + "hbase.mob.compactions.max.file.size"; /** * Default maximum selection size = 1GB */ - public static final long DEFAULT_MOB_COMPACTION_MAX_SELECTION_SIZE = 1024 * 1024 * 1024; - - - /** - * Minimum number of MOB files eligible for compaction - */ - public static final String MOB_COMPACTION_MIN_FILES_KEY = "hbase.mob.compactions.min.files"; - - public static final int DEFAULT_MOB_COMPACTION_MIN_FILES = 3; - - /** - * Maximum number of MOB files (in one compaction selection per region) eligible for compaction. - * If the number of a files in compaction selection is very large, it could lead - * to a "too many opened file handlers" error in a file system or can degrade overall I/O - * performance. - */ - - public static final String MOB_COMPACTION_MAX_FILES_KEY = "hbase.mob.compactions.max.files"; - - public static final int DEFAULT_MOB_COMPACTION_MAX_FILES = 100; - - /** - * Maximum number of MOB files allowed in MOB compaction (per region) - */ - - public static final String MOB_COMPACTION_MAX_TOTAL_FILES_KEY = - "hbase.mob.compactions.max.total.files"; - - public static final int DEFAULT_MOB_COMPACTION_MAX_TOTAL_FILES = 1000; + public static final long DEFAULT_MOB_COMPACTION_MAX_FILE_SIZE = 1024 * 1024 * 1024; /** * Use this configuration option with caution, only during upgrade procedure @@ -147,7 +118,7 @@ public final class MobConstants { */ public static final String MIN_AGE_TO_ARCHIVE_KEY = "hbase.mob.min.age.archive"; - public static final long DEFAULT_MIN_AGE_TO_ARCHIVE = 3600000; + public static final long DEFAULT_MIN_AGE_TO_ARCHIVE = 3600000; // 1h private MobConstants() { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java index ae11a6ca97ca..2356b927fa12 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java @@ -561,57 +561,6 @@ public static StoreFileWriter createWriter(Configuration conf, FileSystem fs, return w; } - /** - * Commits the mob file. - * @param conf The current configuration. - * @param fs The current file system. - * @param sourceFile The path where the mob file is saved. - * @param targetPath The directory path where the source file is renamed to. - * @param cacheConfig The current cache config. - * @return The target file path the source file is renamed to. - * @throws IOException - */ - public static Path commitFile(Configuration conf, FileSystem fs, final Path sourceFile, - Path targetPath, CacheConfig cacheConfig) throws IOException { - if (sourceFile == null) { - return null; - } - Path dstPath = new Path(targetPath, sourceFile.getName()); - validateMobFile(conf, fs, sourceFile, cacheConfig, true); - String msg = "Renaming flushed file from " + sourceFile + " to " + dstPath; - LOG.info(msg); - Path parent = dstPath.getParent(); - if (!fs.exists(parent)) { - fs.mkdirs(parent); - } - if (!fs.rename(sourceFile, dstPath)) { - throw new IOException("Failed rename of " + sourceFile + " to " + dstPath); - } - return dstPath; - } - - /** - * Validates a mob file by opening and closing it. - * @param conf The current configuration. - * @param fs The current file system. - * @param path The path where the mob file is saved. - * @param cacheConfig The current cache config. - */ - private static void validateMobFile(Configuration conf, FileSystem fs, Path path, - CacheConfig cacheConfig, boolean primaryReplica) throws IOException { - HStoreFile storeFile = null; - try { - storeFile = new HStoreFile(fs, path, conf, cacheConfig, BloomType.NONE, primaryReplica); - storeFile.initReader(); - } catch (IOException e) { - LOG.error("Failed to open mob file[" + path + "], keep it in temp directory.", e); - throw e; - } finally { - if (storeFile != null) { - storeFile.closeStoreFile(false); - } - } - } /** * Indicates whether the current mob ref cell has a valid value. A mob ref cell has a mob @@ -647,17 +596,6 @@ public static String getMobFileName(Cell cell) { cell.getValueLength() - Bytes.SIZEOF_INT); } - /** - * Gets the table name used in the table lock. The table lock name is a dummy one, it's not a - * table name. It's tableName + ".mobLock". - * @param tn The table name. - * @return The table name used in table lock. - */ - public static TableName getTableLockName(TableName tn) { - byte[] tableName = tn.getName(); - return TableName.valueOf(Bytes.add(tableName, MobConstants.MOB_TABLE_LOCK_SUFFIX)); - } - /** * Checks whether this table has mob-enabled columns. * @param htd The current table descriptor. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java index 4381a701081f..cc010ebcbb74 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java @@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileContext; +import org.apache.hadoop.hbase.io.hfile.HFileWriterImpl; import org.apache.hadoop.hbase.util.BloomContext; import org.apache.hadoop.hbase.util.BloomFilterFactory; import org.apache.hadoop.hbase.util.BloomFilterUtil; @@ -171,6 +172,9 @@ private StoreFileWriter(FileSystem fs, Path path, final Configuration conf, Cach } } + public long getPos() throws IOException { + return ((HFileWriterImpl) writer).getPos(); + } /** * Writes meta data. * Call before {@link #close()} since its written as meta data to this file. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java index 2531c67b8e93..fce6be341421 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.util.Arrays; +import java.util.Random; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -43,7 +44,7 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.master.MobFileCleanerChore; import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner; -import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.IntegrationTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; @@ -67,7 +68,7 @@ */ @SuppressWarnings("deprecation") -@Category(LargeTests.class) +@Category(IntegrationTests.class) public class TestMobCompaction { private static final Logger LOG = LoggerFactory.getLogger(TestMobCompaction.class); @ClassRule @@ -89,7 +90,7 @@ public class TestMobCompaction { private HTableDescriptor hdt; private HColumnDescriptor hcd; private Admin admin; - private long count = 100000; + private long count = 1000000; private double failureProb = 0.1; private Table table = null; private MobFileCleanerChore chore = new MobFileCleanerChore(); @@ -178,8 +179,14 @@ private void initConf() { conf.setInt("hbase.hstore.compaction.throughput.lower.bound", 52428800); conf.setInt("hbase.hstore.compaction.throughput.higher.bound", 2 * 52428800); conf.setDouble("injected.fault.probability", failureProb); - conf.set(MobStoreEngine.DEFAULT_MOB_COMPACTOR_CLASS_KEY, - FaultyMobStoreCompactor.class.getName()); + //conf.set(MobStoreEngine.DEFAULT_MOB_COMPACTOR_CLASS_KEY, + // FaultyMobStoreCompactor.class.getName()); + conf.setLong(MobConstants.MOB_COMPACTION_CHORE_PERIOD, 0); + conf.setLong(MobConstants.MOB_CLEANER_PERIOD, 0); + conf.setLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, 20000); + conf.set(MobConstants.MOB_COMPACTION_TYPE_KEY, + MobConstants.IO_OPTIMIZED_MOB_COMPACTION_TYPE); + conf.setLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, 1000000); } @@ -236,8 +243,12 @@ public void run() { // BufferedMutator bm = admin.getConnection().getBufferedMutator(table.getName()); // Put Operation + Random r = new Random(); for (int i = 0; i < rows; i++) { - Put p = new Put(Bytes.toBytes(i)); + //Put p = new Put(Bytes.toBytes(i)); + byte[] key = new byte[16]; + r.nextBytes(key); + Put p = new Put(key); p.addColumn(fam, qualifier, mobVal); table.put(p); @@ -287,6 +298,7 @@ public void testMobCompaction() throws InterruptedException, IOException { writeData.join(); // Cleanup again chore.cleanupObsoleteMobFiles(conf, table.getName()); + getNumberOfMobFiles(conf, table.getName(), new String(fam)); if (HTU != null) { LOG.info("Archive cleaner started ..."); @@ -312,10 +324,12 @@ private long getNumberOfMobFiles(Configuration conf, TableName tableName, Strin FileSystem fs = FileSystem.get(conf); Path dir = MobUtils.getMobFamilyPath(conf, tableName, family); FileStatus[] stat = fs.listStatus(dir); + long size = 0; for (FileStatus st : stat) { - LOG.debug("MOB Directory content: {}", st.getPath()); + LOG.debug("MOB Directory content: {} len={}", st.getPath(), st.getLen()); + size+= st.getLen(); } - LOG.debug("MOB Directory content total files: {}", stat.length); + LOG.debug("MOB Directory content total files: {}, total size={}", stat.length, size); return stat.length; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java index 4f751e897bde..773a5e182b83 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java @@ -129,7 +129,6 @@ protected void initConf() { conf.setLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, minAgeToArchive); // Set compacted file discharger interval to a half minAgeToArchive conf.setLong("hbase.hfile.compaction.discharger.interval", minAgeToArchive/2); - //conf.setInt(MobConstants.MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE, 7); } private void loadData(int num) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java new file mode 100644 index 000000000000..fdb7ed9f6cc3 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java @@ -0,0 +1,88 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.mob; +import java.io.IOException; + +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Mob file compaction chore in a generational non-batch mode test. + * 1. Uses default (non-batch) mode for regular MOB compaction, sets generational mode ON + * 2. Disables periodic MOB compactions, sets minimum age to archive to 10 sec + * 3. Creates MOB table with 20 regions + * 4. Loads MOB data (randomized keys, 1000 rows), flushes data. + * 5. Repeats 4. two more times + * 6. Verifies that we have 20 *3 = 60 mob files (equals to number of regions x 3) + * 7. Runs major MOB compaction. + * 8. Verifies that number of MOB files in a mob directory is 20 x4 = 80 + * 9. Waits for a period of time larger than minimum age to archive + * 10. Runs Mob cleaner chore + * 11 Verifies that number of MOB files in a mob directory is 20. + * 12 Runs scanner and checks all 3 * 1000 rows. + */ +@SuppressWarnings("deprecation") +@Category(LargeTests.class) +public class TestMobCompactionOptMode extends TestMobCompactionBase{ + private static final Logger LOG = + LoggerFactory.getLogger(TestMobCompactionOptMode.class); + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobCompactionOptMode.class); + @Rule + public TestName testName = new TestName(); + + + public TestMobCompactionOptMode() { + } + + @Override + protected void initConf() { + super.initConf(); + conf.set(MobConstants.MOB_COMPACTION_TYPE_KEY, + MobConstants.IO_OPTIMIZED_MOB_COMPACTION_TYPE); + conf.setLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, 1000000); + } + + @Test + public void testMobFileCompactionBatchMode() throws InterruptedException, IOException { + LOG.info("MOB compaction generational (non-batch) mode started"); + baseTestMobFileCompaction(); + LOG.info("MOB compaction generational (non-batch) mode finished OK"); + + } + + @Override + protected void mobCompact(Admin admin, HTableDescriptor hdt, HColumnDescriptor hcd) + throws IOException, InterruptedException { + // Major compact MOB table + admin.majorCompact(hdt.getTableName(), hcd.getName()); + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java new file mode 100644 index 000000000000..fd451daee933 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java @@ -0,0 +1,99 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.mob; +import java.io.IOException; + +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.master.MobFileCompactionChore; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Mob file compaction chore in a generational batch mode test. + * 1. Enables batch mode for regular MOB compaction, + * Sets batch size to 7 regions. Enables generational mode. + * 2. Disables periodic MOB compactions, sets minimum age to archive to 10 sec + * 3. Creates MOB table with 20 regions + * 4. Loads MOB data (randomized keys, 1000 rows), flushes data. + * 5. Repeats 4. two more times + * 6. Verifies that we have 20 *3 = 60 mob files (equals to number of regions x 3) + * 7. Runs major MOB compaction. + * 8. Verifies that number of MOB files in a mob directory is 20 x4 = 80 + * 9. Waits for a period of time larger than minimum age to archive + * 10. Runs Mob cleaner chore + * 11 Verifies that number of MOB files in a mob directory is 20. + * 12 Runs scanner and checks all 3 * 1000 rows. + */ +@SuppressWarnings("deprecation") +@Category(LargeTests.class) +public class TestMobCompactionOptRegionBatchMode extends TestMobCompactionBase{ + private static final Logger LOG = + LoggerFactory.getLogger(TestMobCompactionOptRegionBatchMode.class); + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobCompactionOptRegionBatchMode.class); + @Rule + public TestName testName = new TestName(); + + private int batchSize = 7; + private MobFileCompactionChore compactionChore; + + public TestMobCompactionOptRegionBatchMode() { + } + + @Before + public void setUp() throws Exception { + super.setUp(); + compactionChore = new MobFileCompactionChore(conf, batchSize); + } + + protected void initConf() { + super.initConf(); + conf.setInt(MobConstants.MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE, batchSize); + conf.set(MobConstants.MOB_COMPACTION_TYPE_KEY, + MobConstants.IO_OPTIMIZED_MOB_COMPACTION_TYPE); + conf.setLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, 1000000); + } + + @Test + public void testMobFileCompactionBatchMode() throws InterruptedException, IOException { + LOG.info("MOB compaction chore generational batch mode started"); + baseTestMobFileCompaction(); + LOG.info("MOB compaction chore generational batch mode finished OK"); + + } + + @Override + protected void mobCompact(Admin admin, HTableDescriptor hdt, HColumnDescriptor hcd) + throws IOException, InterruptedException { + // Major compact with batch mode enabled + compactionChore.performMajorCompactionInBatches(admin, hdt, hcd); + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java index 053c1f051e2c..017eb545cc41 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java @@ -33,8 +33,8 @@ import org.slf4j.LoggerFactory; /** - * Mob file compaction chore in batch mode test. - * 1. Use default (non-batch) mode for regular MOB compaction, + * Mob file compaction chore in a regular non-batch mode test. + * 1. Uses default (non-batch) mode for regular MOB compaction, * 2. Disables periodic MOB compactions, sets minimum age to archive to 10 sec * 3. Creates MOB table with 20 regions * 4. Loads MOB data (randomized keys, 1000 rows), flushes data. @@ -61,7 +61,7 @@ public class TestMobCompactionRegularMode extends TestMobCompactionBase{ public TestMobCompactionRegularMode() { } - + @Test public void testMobFileCompactionBatchMode() throws InterruptedException, IOException { LOG.info("MOB compaction regular mode started"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegionBatchMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java similarity index 86% rename from hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegionBatchMode.java rename to hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java index be2fdb5ff1c2..0903a635d998 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegionBatchMode.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java @@ -35,7 +35,7 @@ import org.slf4j.LoggerFactory; /** - * Mob file compaction chore in batch mode test. + * Mob file compaction chore in a regular batch mode test. * 1. Enables batch mode for regular MOB compaction, * Sets batch size to 7 regions. * 2. Disables periodic MOB compactions, sets minimum age to archive to 10 sec @@ -52,19 +52,19 @@ */ @SuppressWarnings("deprecation") @Category(LargeTests.class) -public class TestMobCompactionRegionBatchMode extends TestMobCompactionBase{ +public class TestMobCompactionRegularRegionBatchMode extends TestMobCompactionBase{ private static final Logger LOG = - LoggerFactory.getLogger(TestMobCompactionRegionBatchMode.class); + LoggerFactory.getLogger(TestMobCompactionRegularRegionBatchMode.class); @ClassRule public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestMobCompactionRegionBatchMode.class); + HBaseClassTestRule.forClass(TestMobCompactionRegularRegionBatchMode.class); @Rule public TestName testName = new TestName(); private int batchSize = 7; private MobFileCompactionChore compactionChore; - public TestMobCompactionRegionBatchMode() { + public TestMobCompactionRegularRegionBatchMode() { } @Before @@ -80,9 +80,9 @@ protected void initConf() { @Test public void testMobFileCompactionBatchMode() throws InterruptedException, IOException { - LOG.info("MOB compaction chore batch mode started"); + LOG.info("MOB compaction chore regular batch mode started"); baseTestMobFileCompaction(); - LOG.info("MOB compaction chore batch mode finished OK"); + LOG.info("MOB compaction chore regular batch mode finished OK"); } From 73cc635833daf68f6fc4ee91ac335ad2e7002eab Mon Sep 17 00:00:00 2001 From: Vladimir Rodionov Date: Fri, 22 Nov 2019 11:23:14 -0800 Subject: [PATCH 15/15] HBASE-23189: Finilize I/O optimized MOB compaction (clean up and minor improvs) --- .../hbase/master/MobFileCleanerChore.java | 6 +- .../hbase/mob/DefaultMobStoreCompactor.java | 59 ++++++++----------- .../hbase/mob/DefaultMobStoreFlusher.java | 3 +- .../hadoop/hbase/mob/TestMobCompaction.java | 44 ++++++++------ 4 files changed, 55 insertions(+), 57 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java index fbe82f551390..718bc0f21b03 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java @@ -181,7 +181,7 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) } try { for (Path pp : storeFiles) { - /*DEBUG*/LOG.debug("DDDD Store file: {}", pp); + LOG.debug("Store file: {}", pp); HStoreFile sf = new HStoreFile(fs, pp, conf, CacheConfig.DISABLED, BloomType.NONE, true); sf.initReader(); @@ -201,7 +201,7 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) // if length = 1 means NULL, that there are no MOB references // in this store file, but the file was created by new MOB code String[] mobs = new String(mobRefData).split(","); - /*DEBUG*/LOG.debug("DDDD found: {} mob refs: ", mobs.length, Arrays.toString(mobs)); + LOG.debug("Found: {} mob refs: ", mobs.length, Arrays.toString(mobs)); regionMobs.addAll(Arrays.asList(mobs)); } } @@ -220,7 +220,7 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) LOG.warn("Found too many active MOB files: {}, this may result in high memory pressure.", allActiveMobFileName.size()); } - /*DEBUG*/LOG.debug("DDDD Found: {} active mob refs", allActiveMobFileName.size()); + LOG.debug("Found: {} active mob refs", allActiveMobFileName.size()); allActiveMobFileName.stream().forEach(LOG::debug); // Now scan MOB directories and find MOB files with no references to them diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java index 74d7e812fa15..cc28a8cd9124 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java @@ -182,16 +182,10 @@ private void calculateMobLengthMap(List mobFiles) throws IOException { FileSystem fs = mobFiles.get(0).getFileSystem(this.conf); HashMap map = mobLengthMap.get(); map.clear(); - long maxMobFileSize = conf.getLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, - MobConstants.DEFAULT_MOB_COMPACTION_MAX_FILE_SIZE); for (Path p: mobFiles) { FileStatus st = fs.getFileStatus(p); long size = st.getLen(); - /*DEBUG*/ - if (size > 2 * maxMobFileSize) { - LOG.debug("DDDD FOUND BIG ref MOB size={} file={}", size, p.getName()); - } - + LOG.info("Ref MOB file={} size={}", p, size); map.put(p.getName(), fs.getFileStatus(p).getLen()); } } @@ -251,7 +245,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel mobRefSet.get().clear(); boolean isUserRequest = userRequest.get(); boolean compactMOBs = major && isUserRequest; - boolean ioOptimizedMob = conf.get(MobConstants.MOB_COMPACTION_TYPE_KEY, + boolean ioOptimizedMode = conf.get(MobConstants.MOB_COMPACTION_TYPE_KEY, MobConstants.DEFAULT_MOB_COMPACTION_TYPE) .equals(MobConstants.IO_OPTIMIZED_MOB_COMPACTION_TYPE); @@ -261,8 +255,8 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel long maxMobFileSize = conf.getLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, MobConstants.DEFAULT_MOB_COMPACTION_MAX_FILE_SIZE); - LOG.info("Compact MOB={} optimized={} maximum MOB file size={}", compactMOBs, - ioOptimizedMob, maxMobFileSize); + LOG.info("Compact MOB={} optimized={} maximum MOB file size={} major={}", compactMOBs, + ioOptimizedMode, maxMobFileSize, major); FileSystem fs = FileSystem.get(conf); @@ -299,16 +293,13 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel Cell mobCell = null; try { - mobFileWriter = newMobWriter(fd, compactMOBs); + mobFileWriter = newMobWriter(fd); fileName = Bytes.toBytes(mobFileWriter.getPath().getName()); do { hasMore = scanner.next(cells, scannerContext); - if (LOG.isDebugEnabled()) { - now = EnvironmentEdgeManager.currentTime(); - } + now = EnvironmentEdgeManager.currentTime(); for (Cell c : cells) { - if (compactMOBs) { if (MobUtils.isMobReferenceCell(c)) { String fName = MobUtils.getMobFileName(c); @@ -337,7 +328,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel if (mobCell.getValueLength() > mobSizeThreshold) { // put the mob data back to the MOB store file PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId()); - if (!ioOptimizedMob) { + if (!ioOptimizedMode) { mobFileWriter.append(mobCell); mobCells++; writer.append(MobUtils.createMobRefCell(mobCell, fileName, @@ -369,9 +360,10 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel long len = getLength(mobFileWriter); if (len > maxMobFileSize) { - LOG.debug("DDDD Output File Length={} file=", len, new String(fileName)); + LOG.debug("Closing output MOB File, length={} file={}", + len, Bytes.toString(fileName)); commitOrAbortMobWriter(mobFileWriter, fd.maxSeqId, mobCells, major); - mobFileWriter = newMobWriter(fd, compactMOBs); + mobFileWriter = newMobWriter(fd); fileName = Bytes.toBytes(mobFileWriter.getPath().getName()); mobCells = 0; } @@ -402,16 +394,13 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel mobCells++; cellsCountCompactedToMob++; cellsSizeCompactedToMob += c.getValueLength(); - if (ioOptimizedMob) { + if (ioOptimizedMode) { // Update total size of the output (we do not take into account // file compression yet) long len = getLength(mobFileWriter); - if (len > 2 * maxMobFileSize) { - LOG.debug("DDDD Output MOB size={} file={}", len, fileName); - } if (len > maxMobFileSize) { commitOrAbortMobWriter(mobFileWriter, fd.maxSeqId, mobCells, major); - mobFileWriter = newMobWriter(fd, compactMOBs); + mobFileWriter = newMobWriter(fd); fileName = Bytes.toBytes(mobFileWriter.getPath().getName()); mobCells = 0; } @@ -454,8 +443,15 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel writer.append(reference); cellsCountCompactedToMob++; cellsSizeCompactedToMob += c.getValueLength(); - // Add ref we get for compact MOB case - mobRefSet.get().add(mobFileWriter.getPath().getName()); + if (ioOptimizedMode) { + long len = getLength(mobFileWriter); + if (len > maxMobFileSize) { + commitOrAbortMobWriter(mobFileWriter, fd.maxSeqId, mobCells, major); + mobFileWriter = newMobWriter(fd); + fileName = Bytes.toBytes(mobFileWriter.getPath().getName()); + mobCells = 0; + } + } } int len = c.getSerializedSize(); @@ -537,17 +533,14 @@ private long getLength(StoreFileWriter mobFileWriter) throws IOException { } - private StoreFileWriter newMobWriter(FileDetails fd, boolean compactMOBs) + private StoreFileWriter newMobWriter(FileDetails fd/*, boolean compactMOBs*/) throws IOException { try { StoreFileWriter mobFileWriter = mobStore.createWriterInTmp(new Date(fd.latestPutTs), fd.maxKeyCount, compactionCompression, store.getRegionInfo().getStartKey(), true); - LOG.debug("DDDD New MOB created={}", mobFileWriter.getPath().getName()); - - if (compactMOBs) { - // Add reference we get for compact MOB - mobRefSet.get().add(mobFileWriter.getPath().getName()); - } + LOG.debug("New MOB writer created={}", mobFileWriter.getPath().getName()); + // Add reference we get for compact MOB + mobRefSet.get().add(mobFileWriter.getPath().getName()); return mobFileWriter; } catch (IOException e) { // Bailing out @@ -564,7 +557,7 @@ private void commitOrAbortMobWriter(StoreFileWriter mobFileWriter, long maxSeqId // MOB files can be committed partially, but corresponding // store file won't be committed, therefore these MOB files // become orphans and will be deleted during next MOB cleaning chore cycle - LOG.debug("DDDD Commit or Abort size={} mobCells={} major={} file={}", + LOG.debug("Commit or abort size={} mobCells={} major={} file={}", mobFileWriter.getPos(), mobCells, major, mobFileWriter.getPath().getName()); Path path = MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName()); if (mobFileWriter != null) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java index c8a289db23a2..08fb2c69a6a8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java @@ -214,8 +214,7 @@ protected void performMobFlush(MemStoreSnapshot snapshot, long cacheFlushId, // append the original keyValue in the mob file. mobFileWriter.append(c); mobSize += c.getValueLength(); - mobCount++; - + mobCount++; // append the tags to the KeyValue. // The key is same, the value is the filename of the mob file Cell reference = MobUtils.createMobRefCell(c, fileName, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java index fce6be341421..6b7302819e22 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.util.Arrays; -import java.util.Random; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -44,11 +43,12 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.master.MobFileCleanerChore; import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner; -import org.apache.hadoop.hbase.testclassification.IntegrationTests; +import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; import org.junit.ClassRule; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -68,7 +68,7 @@ */ @SuppressWarnings("deprecation") -@Category(IntegrationTests.class) +@Category(LargeTests.class) public class TestMobCompaction { private static final Logger LOG = LoggerFactory.getLogger(TestMobCompaction.class); @ClassRule @@ -90,7 +90,7 @@ public class TestMobCompaction { private HTableDescriptor hdt; private HColumnDescriptor hcd; private Admin admin; - private long count = 1000000; + private long count = 500000; private double failureProb = 0.1; private Table table = null; private MobFileCleanerChore chore = new MobFileCleanerChore(); @@ -179,11 +179,11 @@ private void initConf() { conf.setInt("hbase.hstore.compaction.throughput.lower.bound", 52428800); conf.setInt("hbase.hstore.compaction.throughput.higher.bound", 2 * 52428800); conf.setDouble("injected.fault.probability", failureProb); - //conf.set(MobStoreEngine.DEFAULT_MOB_COMPACTOR_CLASS_KEY, - // FaultyMobStoreCompactor.class.getName()); +// conf.set(MobStoreEngine.DEFAULT_MOB_COMPACTOR_CLASS_KEY, +// FaultyMobStoreCompactor.class.getName()); conf.setLong(MobConstants.MOB_COMPACTION_CHORE_PERIOD, 0); conf.setLong(MobConstants.MOB_CLEANER_PERIOD, 0); - conf.setLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, 20000); + conf.setLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, 120000); conf.set(MobConstants.MOB_COMPACTION_TYPE_KEY, MobConstants.IO_OPTIMIZED_MOB_COMPACTION_TYPE); conf.setLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, 1000000); @@ -241,31 +241,23 @@ public WriteData(long rows) { public void run() { try { - // BufferedMutator bm = admin.getConnection().getBufferedMutator(table.getName()); // Put Operation - Random r = new Random(); for (int i = 0; i < rows; i++) { - //Put p = new Put(Bytes.toBytes(i)); - byte[] key = new byte[16]; - r.nextBytes(key); + byte[] key = Bytes.toBytes(i); Put p = new Put(key); - p.addColumn(fam, qualifier, mobVal); + p.addColumn(fam, qualifier, Bytes.add(key,mobVal)); table.put(p); - - // bm.mutate(p); if (i % 10000 == 0) { LOG.info("LOADED=" + i); try { Thread.sleep(500); } catch (InterruptedException ee) { - } } if (i % 100000 == 0) { printStats(i); } } - // bm.flush(); admin.flush(table.getName()); run = false; } catch (Exception e) { @@ -275,6 +267,7 @@ public void run() { } } + @Ignore @Test public void testMobCompaction() throws InterruptedException, IOException { @@ -349,12 +342,25 @@ private void scanTable() { ResultScanner scanner = table.getScanner(fam); int counter = 0; while ((result = scanner.next()) != null) { - assertTrue(Arrays.equals(result.getValue(fam, qualifier), mobVal)); + byte[] key = result.getRow(); + assertTrue(Arrays.equals(result.getValue(fam, qualifier), + Bytes.add(key,mobVal))); if (counter % 10000 == 0) { - LOG.info("GET=" + counter); + LOG.info("GET=" + counter+" key=" + Bytes.toInt(key)); } counter++; } + +// for (int i=0; i < count; i++) { +// byte[] key = Bytes.toBytes(i); +// Get get = new Get(key); +// Result res = table.get(get); +// assertTrue(Arrays.equals(res.getValue(fam, qualifier), +// Bytes.add(key,mobVal))); +// if (i % 1000 == 0) { +// LOG.info("GET=" + i); +// } +// } assertEquals(count, counter); } catch (Exception e) { e.printStackTrace();