HBASE-12975 Supportable SplitTransaction and RegionMergeTransaction interfaces
This commit is contained in:
parent
452ce33ef5
commit
e156ed619c
|
@ -6239,7 +6239,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
b.getRegionFileSystem().logFileSystemState(LOG);
|
b.getRegionFileSystem().logFileSystemState(LOG);
|
||||||
}
|
}
|
||||||
|
|
||||||
RegionMergeTransaction rmt = new RegionMergeTransaction(a, b, true);
|
RegionMergeTransactionImpl rmt = new RegionMergeTransactionImpl(a, b, true);
|
||||||
if (!rmt.prepare(null)) {
|
if (!rmt.prepare(null)) {
|
||||||
throw new IOException("Unable to merge regions " + a + " and " + b);
|
throw new IOException("Unable to merge regions " + a + " and " + b);
|
||||||
}
|
}
|
||||||
|
@ -6252,7 +6252,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
+ Bytes.toStringBinary(mergedRegionInfo.getEndKey()) + ">");
|
+ Bytes.toStringBinary(mergedRegionInfo.getEndKey()) + ">");
|
||||||
HRegion dstRegion;
|
HRegion dstRegion;
|
||||||
try {
|
try {
|
||||||
dstRegion = rmt.execute(null, null);
|
dstRegion = (HRegion)rmt.execute(null, null);
|
||||||
} catch (IOException ioe) {
|
} catch (IOException ioe) {
|
||||||
rmt.rollback(null, null);
|
rmt.rollback(null, null);
|
||||||
throw new IOException("Failed merging region " + a + " and " + b
|
throw new IOException("Failed merging region " + a + " and " + b
|
||||||
|
|
|
@ -23,9 +23,7 @@ import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.Cell;
|
import org.apache.hadoop.hbase.Cell;
|
||||||
import org.apache.hadoop.hbase.DroppedSnapshotException;
|
|
||||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
|
||||||
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
||||||
import org.apache.hadoop.hbase.HRegionInfo;
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
|
|
|
@ -65,7 +65,7 @@ class RegionMergeRequest implements Runnable {
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
final long startTime = EnvironmentEdgeManager.currentTime();
|
final long startTime = EnvironmentEdgeManager.currentTime();
|
||||||
RegionMergeTransaction mt = new RegionMergeTransaction(region_a,
|
RegionMergeTransactionImpl mt = new RegionMergeTransactionImpl(region_a,
|
||||||
region_b, forcible);
|
region_b, forcible);
|
||||||
|
|
||||||
//acquire a shared read lock on the table, so that table schema modifications
|
//acquire a shared read lock on the table, so that table schema modifications
|
||||||
|
|
|
@ -13,35 +13,20 @@
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||||
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||||
* License for the specific language governing permissions and limitationsME
|
* License for the specific language governing permissions and limitations
|
||||||
* under the License.
|
* under the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.regionserver;
|
package org.apache.hadoop.hbase.regionserver;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.ListIterator;
|
|
||||||
import java.util.Map;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.regionserver.SplitTransaction.TransactionListener;
|
||||||
|
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.HRegionInfo;
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
import org.apache.hadoop.hbase.MetaMutationAnnotation;
|
|
||||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
|
||||||
import org.apache.hadoop.hbase.Server;
|
import org.apache.hadoop.hbase.Server;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
|
||||||
import org.apache.hadoop.hbase.client.Delete;
|
|
||||||
import org.apache.hadoop.hbase.client.Mutation;
|
|
||||||
import org.apache.hadoop.hbase.client.Put;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.SplitTransaction.LoggingProgressable;
|
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Executes region merge as a "transaction". It is similar with
|
* Executes region merge as a "transaction". It is similar with
|
||||||
|
@ -50,12 +35,21 @@ import org.apache.hadoop.hbase.util.Pair;
|
||||||
* transaction and {@link #rollback(Server, RegionServerServices)} to cleanup if
|
* transaction and {@link #rollback(Server, RegionServerServices)} to cleanup if
|
||||||
* execute fails.
|
* execute fails.
|
||||||
*
|
*
|
||||||
* <p>
|
* <p>Here is an example of how you would use this interface:
|
||||||
* Here is an example of how you would use this class:
|
|
||||||
*
|
|
||||||
* <pre>
|
* <pre>
|
||||||
* RegionMergeTransaction mt = new RegionMergeTransaction(this.conf, parent, midKey)
|
* RegionMergeTransactionFactory factory = new RegionMergeTransactionFactory(conf);
|
||||||
* if (!mt.prepare(services)) return;
|
* RegionMergeTransaction mt = factory.create(parent, midKey)
|
||||||
|
* .registerTransactionListener(new TransactionListener() {
|
||||||
|
* public void transition(RegionMergeTransaction transaction,
|
||||||
|
* RegionMergeTransactionPhase from, RegionMergeTransactionPhase to) throws IOException {
|
||||||
|
* // ...
|
||||||
|
* }
|
||||||
|
* public void rollback(RegionMergeTransaction transaction,
|
||||||
|
* RegionMergeTransactionPhase from, RegionMergeTransactionPhase to) {
|
||||||
|
* // ...
|
||||||
|
* }
|
||||||
|
* });
|
||||||
|
* if (!mt.prepare()) return;
|
||||||
* try {
|
* try {
|
||||||
* mt.execute(server, services);
|
* mt.execute(server, services);
|
||||||
* } catch (IOException ioe) {
|
* } catch (IOException ioe) {
|
||||||
|
@ -63,33 +57,25 @@ import org.apache.hadoop.hbase.util.Pair;
|
||||||
* mt.rollback(server, services);
|
* mt.rollback(server, services);
|
||||||
* return;
|
* return;
|
||||||
* } catch (RuntimeException e) {
|
* } catch (RuntimeException e) {
|
||||||
* myAbortable.abort("Failed merge, abort");
|
* // abort the server
|
||||||
* }
|
* }
|
||||||
* }
|
* }
|
||||||
* </Pre>
|
* </Pre>
|
||||||
* <p>
|
* <p>A merge transaction is not thread safe. Callers must ensure a split is run by
|
||||||
* This class is not thread safe. Caller needs ensure merge is run by one thread
|
* one thread only.
|
||||||
* only.
|
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||||
public class RegionMergeTransaction {
|
@InterfaceStability.Evolving
|
||||||
private static final Log LOG = LogFactory.getLog(RegionMergeTransaction.class);
|
public interface RegionMergeTransaction {
|
||||||
|
|
||||||
// Merged region info
|
|
||||||
private HRegionInfo mergedRegionInfo;
|
|
||||||
// region_a sorts before region_b
|
|
||||||
private final HRegion region_a;
|
|
||||||
private final HRegion region_b;
|
|
||||||
// merges dir is under region_a
|
|
||||||
private final Path mergesdir;
|
|
||||||
// We only merge adjacent regions if forcible is false
|
|
||||||
private final boolean forcible;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Types to add to the transaction journal. Each enum is a step in the merge
|
* Each enum is a step in the merge transaction.
|
||||||
* transaction. Used to figure how much we need to rollback.
|
|
||||||
*/
|
*/
|
||||||
enum JournalEntry {
|
enum RegionMergeTransactionPhase {
|
||||||
|
STARTED,
|
||||||
|
/**
|
||||||
|
* Prepared
|
||||||
|
*/
|
||||||
|
PREPARED,
|
||||||
/**
|
/**
|
||||||
* Set region as in transition, set it into MERGING state.
|
* Set region as in transition, set it into MERGING state.
|
||||||
*/
|
*/
|
||||||
|
@ -122,95 +108,59 @@ public class RegionMergeTransaction {
|
||||||
* Point of no return. If we got here, then transaction is not recoverable
|
* Point of no return. If we got here, then transaction is not recoverable
|
||||||
* other than by crashing out the regionserver.
|
* other than by crashing out the regionserver.
|
||||||
*/
|
*/
|
||||||
PONR
|
PONR,
|
||||||
}
|
/**
|
||||||
|
* Completed
|
||||||
/*
|
*/
|
||||||
* Journal of how far the merge transaction has progressed.
|
COMPLETED
|
||||||
*/
|
|
||||||
private final List<JournalEntry> journal = new ArrayList<JournalEntry>();
|
|
||||||
|
|
||||||
private static IOException closedByOtherException = new IOException(
|
|
||||||
"Failed to close region: already closed by another thread");
|
|
||||||
|
|
||||||
private RegionServerCoprocessorHost rsCoprocessorHost = null;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Constructor
|
|
||||||
* @param a region a to merge
|
|
||||||
* @param b region b to merge
|
|
||||||
* @param forcible if false, we will only merge adjacent regions
|
|
||||||
*/
|
|
||||||
public RegionMergeTransaction(final Region a, final Region b,
|
|
||||||
final boolean forcible) {
|
|
||||||
if (a.getRegionInfo().compareTo(b.getRegionInfo()) <= 0) {
|
|
||||||
this.region_a = (HRegion)a;
|
|
||||||
this.region_b = (HRegion)b;
|
|
||||||
} else {
|
|
||||||
this.region_a = (HRegion)b;
|
|
||||||
this.region_b = (HRegion)a;
|
|
||||||
}
|
|
||||||
this.forcible = forcible;
|
|
||||||
this.mergesdir = region_a.getRegionFileSystem().getMergesDir();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Does checks on merge inputs.
|
* Split transaction journal entry
|
||||||
|
*/
|
||||||
|
public interface JournalEntry {
|
||||||
|
|
||||||
|
/** @return the completed phase marked by this journal entry */
|
||||||
|
RegionMergeTransactionPhase getPhase();
|
||||||
|
|
||||||
|
/** @return the time of phase completion */
|
||||||
|
long getTimeStamp();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Split transaction listener
|
||||||
|
*/
|
||||||
|
public interface TransactionListener {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Invoked when transitioning forward from one transaction phase to another
|
||||||
|
* @param transaction the transaction
|
||||||
|
* @param from the current phase
|
||||||
|
* @param to the next phase
|
||||||
|
* @throws IOException listener can throw this to abort
|
||||||
|
*/
|
||||||
|
void transition(RegionMergeTransaction transaction, RegionMergeTransactionPhase from,
|
||||||
|
RegionMergeTransactionPhase to) throws IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Invoked when rolling back a transaction from one transaction phase to the
|
||||||
|
* previous
|
||||||
|
* @param transaction the transaction
|
||||||
|
* @param from the current phase
|
||||||
|
* @param to the previous phase
|
||||||
|
*/
|
||||||
|
void rollback(RegionMergeTransaction transaction, RegionMergeTransactionPhase from,
|
||||||
|
RegionMergeTransactionPhase to);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check merge inputs and prepare the transaction.
|
||||||
* @param services
|
* @param services
|
||||||
* @return <code>true</code> if the regions are mergeable else
|
* @return <code>true</code> if the regions are mergeable else
|
||||||
* <code>false</code> if they are not (e.g. its already closed, etc.).
|
* <code>false</code> if they are not (e.g. its already closed, etc.).
|
||||||
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public boolean prepare(final RegionServerServices services) {
|
boolean prepare(RegionServerServices services) throws IOException;
|
||||||
if (!region_a.getTableDesc().getTableName()
|
|
||||||
.equals(region_b.getTableDesc().getTableName())) {
|
|
||||||
LOG.info("Can't merge regions " + region_a + "," + region_b
|
|
||||||
+ " because they do not belong to the same table");
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
if (region_a.getRegionInfo().equals(region_b.getRegionInfo())) {
|
|
||||||
LOG.info("Can't merge the same region " + region_a);
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
if (!forcible && !HRegionInfo.areAdjacent(region_a.getRegionInfo(),
|
|
||||||
region_b.getRegionInfo())) {
|
|
||||||
String msg = "Skip merging " + this.region_a.getRegionInfo().getRegionNameAsString()
|
|
||||||
+ " and " + this.region_b.getRegionInfo().getRegionNameAsString()
|
|
||||||
+ ", because they are not adjacent.";
|
|
||||||
LOG.info(msg);
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
if (!this.region_a.isMergeable() || !this.region_b.isMergeable()) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
try {
|
|
||||||
boolean regionAHasMergeQualifier = hasMergeQualifierInMeta(services,
|
|
||||||
region_a.getRegionInfo().getRegionName());
|
|
||||||
if (regionAHasMergeQualifier ||
|
|
||||||
hasMergeQualifierInMeta(services, region_b.getRegionInfo().getRegionName())) {
|
|
||||||
LOG.debug("Region " + (regionAHasMergeQualifier ?
|
|
||||||
region_a.getRegionInfo().getRegionNameAsString() :
|
|
||||||
region_b.getRegionInfo().getRegionNameAsString())
|
|
||||||
+ " is not mergeable because it has merge qualifier in META");
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
} catch (IOException e) {
|
|
||||||
LOG.warn("Failed judging whether merge transaction is available for "
|
|
||||||
+ region_a.getRegionInfo().getRegionNameAsString() + " and "
|
|
||||||
+ region_b.getRegionInfo().getRegionNameAsString(), e);
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
// WARN: make sure there is no parent region of the two merging regions in
|
|
||||||
// hbase:meta If exists, fixing up daughters would cause daughter regions(we
|
|
||||||
// have merged one) online again when we restart master, so we should clear
|
|
||||||
// the parent region to prevent the above case
|
|
||||||
// Since HBASE-7721, we don't need fix up daughters any more. so here do
|
|
||||||
// nothing
|
|
||||||
|
|
||||||
this.mergedRegionInfo = getMergedRegionInfo(region_a.getRegionInfo(),
|
|
||||||
region_b.getRegionInfo());
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Run the transaction.
|
* Run the transaction.
|
||||||
|
@ -222,325 +172,10 @@ public class RegionMergeTransaction {
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
* @see #rollback(Server, RegionServerServices)
|
* @see #rollback(Server, RegionServerServices)
|
||||||
*/
|
*/
|
||||||
public HRegion execute(final Server server,
|
Region execute(Server server, RegionServerServices services) throws IOException;
|
||||||
final RegionServerServices services) throws IOException {
|
|
||||||
if (rsCoprocessorHost == null) {
|
|
||||||
rsCoprocessorHost = server != null ?
|
|
||||||
((HRegionServer) server).getRegionServerCoprocessorHost() : null;
|
|
||||||
}
|
|
||||||
HRegion mergedRegion = createMergedRegion(server, services);
|
|
||||||
if (rsCoprocessorHost != null) {
|
|
||||||
rsCoprocessorHost.postMergeCommit(this.region_a, this.region_b, mergedRegion);
|
|
||||||
}
|
|
||||||
return stepsAfterPONR(server, services, mergedRegion);
|
|
||||||
}
|
|
||||||
|
|
||||||
public HRegion stepsAfterPONR(final Server server, final RegionServerServices services,
|
|
||||||
HRegion mergedRegion) throws IOException {
|
|
||||||
openMergedRegion(server, services, mergedRegion);
|
|
||||||
if (rsCoprocessorHost != null) {
|
|
||||||
rsCoprocessorHost.postMerge(this.region_a, this.region_b, mergedRegion);
|
|
||||||
}
|
|
||||||
return mergedRegion;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Prepare the merged region and region files.
|
|
||||||
* @param server Hosting server instance. Can be null when testing
|
|
||||||
* @param services Used to online/offline regions.
|
|
||||||
* @return merged region
|
|
||||||
* @throws IOException If thrown, transaction failed. Call
|
|
||||||
* {@link #rollback(Server, RegionServerServices)}
|
|
||||||
*/
|
|
||||||
HRegion createMergedRegion(final Server server,
|
|
||||||
final RegionServerServices services) throws IOException {
|
|
||||||
LOG.info("Starting merge of " + region_a + " and "
|
|
||||||
+ region_b.getRegionInfo().getRegionNameAsString() + ", forcible=" + forcible);
|
|
||||||
if ((server != null && server.isStopped())
|
|
||||||
|| (services != null && services.isStopping())) {
|
|
||||||
throw new IOException("Server is stopped or stopping");
|
|
||||||
}
|
|
||||||
|
|
||||||
if (rsCoprocessorHost != null) {
|
|
||||||
if (rsCoprocessorHost.preMerge(this.region_a, this.region_b)) {
|
|
||||||
throw new IOException("Coprocessor bypassing regions " + this.region_a + " "
|
|
||||||
+ this.region_b + " merge.");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// If true, no cluster to write meta edits to or to use coordination.
|
|
||||||
boolean testing = server == null ? true : server.getConfiguration()
|
|
||||||
.getBoolean("hbase.testing.nocluster", false);
|
|
||||||
|
|
||||||
HRegion mergedRegion = stepsBeforePONR(server, services, testing);
|
|
||||||
|
|
||||||
@MetaMutationAnnotation
|
|
||||||
List<Mutation> metaEntries = new ArrayList<Mutation>();
|
|
||||||
if (rsCoprocessorHost != null) {
|
|
||||||
if (rsCoprocessorHost.preMergeCommit(this.region_a, this.region_b, metaEntries)) {
|
|
||||||
throw new IOException("Coprocessor bypassing regions " + this.region_a + " "
|
|
||||||
+ this.region_b + " merge.");
|
|
||||||
}
|
|
||||||
try {
|
|
||||||
for (Mutation p : metaEntries) {
|
|
||||||
HRegionInfo.parseRegionName(p.getRow());
|
|
||||||
}
|
|
||||||
} catch (IOException e) {
|
|
||||||
LOG.error("Row key of mutation from coprocessor is not parsable as region name."
|
|
||||||
+ "Mutations from coprocessor should only be for hbase:meta table.", e);
|
|
||||||
throw e;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// This is the point of no return. Similar with SplitTransaction.
|
|
||||||
// IF we reach the PONR then subsequent failures need to crash out this
|
|
||||||
// regionserver
|
|
||||||
this.journal.add(JournalEntry.PONR);
|
|
||||||
|
|
||||||
// Add merged region and delete region_a and region_b
|
|
||||||
// as an atomic update. See HBASE-7721. This update to hbase:meta makes the region
|
|
||||||
// will determine whether the region is merged or not in case of failures.
|
|
||||||
// If it is successful, master will roll-forward, if not, master will
|
|
||||||
// rollback
|
|
||||||
if (services != null && !services.reportRegionStateTransition(TransitionCode.MERGE_PONR,
|
|
||||||
mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
|
|
||||||
// Passed PONR, let SSH clean it up
|
|
||||||
throw new IOException("Failed to notify master that merge passed PONR: "
|
|
||||||
+ region_a.getRegionInfo().getRegionNameAsString() + " and "
|
|
||||||
+ region_b.getRegionInfo().getRegionNameAsString());
|
|
||||||
}
|
|
||||||
return mergedRegion;
|
|
||||||
}
|
|
||||||
|
|
||||||
public void prepareMutationsForMerge(HRegionInfo mergedRegion, HRegionInfo regionA,
|
|
||||||
HRegionInfo regionB, ServerName serverName, List<Mutation> mutations) throws IOException {
|
|
||||||
HRegionInfo copyOfMerged = new HRegionInfo(mergedRegion);
|
|
||||||
|
|
||||||
// Put for parent
|
|
||||||
Put putOfMerged = MetaTableAccessor.makePutFromRegionInfo(copyOfMerged);
|
|
||||||
putOfMerged.add(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER, regionA.toByteArray());
|
|
||||||
putOfMerged.add(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER, regionB.toByteArray());
|
|
||||||
mutations.add(putOfMerged);
|
|
||||||
// Deletes for merging regions
|
|
||||||
Delete deleteA = MetaTableAccessor.makeDeleteFromRegionInfo(regionA);
|
|
||||||
Delete deleteB = MetaTableAccessor.makeDeleteFromRegionInfo(regionB);
|
|
||||||
mutations.add(deleteA);
|
|
||||||
mutations.add(deleteB);
|
|
||||||
// The merged is a new region, openSeqNum = 1 is fine.
|
|
||||||
addLocation(putOfMerged, serverName, 1);
|
|
||||||
}
|
|
||||||
|
|
||||||
public Put addLocation(final Put p, final ServerName sn, long openSeqNum) {
|
|
||||||
p.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER, Bytes
|
|
||||||
.toBytes(sn.getHostAndPort()));
|
|
||||||
p.add(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER, Bytes.toBytes(sn
|
|
||||||
.getStartcode()));
|
|
||||||
p.add(HConstants.CATALOG_FAMILY, HConstants.SEQNUM_QUALIFIER, Bytes.toBytes(openSeqNum));
|
|
||||||
return p;
|
|
||||||
}
|
|
||||||
|
|
||||||
public HRegion stepsBeforePONR(final Server server, final RegionServerServices services,
|
|
||||||
boolean testing) throws IOException {
|
|
||||||
if (services != null && !services.reportRegionStateTransition(TransitionCode.READY_TO_MERGE,
|
|
||||||
mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
|
|
||||||
throw new IOException("Failed to get ok from master to merge "
|
|
||||||
+ region_a.getRegionInfo().getRegionNameAsString() + " and "
|
|
||||||
+ region_b.getRegionInfo().getRegionNameAsString());
|
|
||||||
}
|
|
||||||
this.journal.add(JournalEntry.SET_MERGING);
|
|
||||||
|
|
||||||
this.region_a.getRegionFileSystem().createMergesDir();
|
|
||||||
this.journal.add(JournalEntry.CREATED_MERGE_DIR);
|
|
||||||
|
|
||||||
Map<byte[], List<StoreFile>> hstoreFilesOfRegionA = closeAndOfflineRegion(
|
|
||||||
services, this.region_a, true, testing);
|
|
||||||
Map<byte[], List<StoreFile>> hstoreFilesOfRegionB = closeAndOfflineRegion(
|
|
||||||
services, this.region_b, false, testing);
|
|
||||||
|
|
||||||
assert hstoreFilesOfRegionA != null && hstoreFilesOfRegionB != null;
|
|
||||||
|
|
||||||
|
|
||||||
//
|
|
||||||
// mergeStoreFiles creates merged region dirs under the region_a merges dir
|
|
||||||
// Nothing to unroll here if failure -- clean up of CREATE_MERGE_DIR will
|
|
||||||
// clean this up.
|
|
||||||
mergeStoreFiles(hstoreFilesOfRegionA, hstoreFilesOfRegionB);
|
|
||||||
|
|
||||||
// Log to the journal that we are creating merged region. We could fail
|
|
||||||
// halfway through. If we do, we could have left
|
|
||||||
// stuff in fs that needs cleanup -- a storefile or two. Thats why we
|
|
||||||
// add entry to journal BEFORE rather than AFTER the change.
|
|
||||||
this.journal.add(JournalEntry.STARTED_MERGED_REGION_CREATION);
|
|
||||||
HRegion mergedRegion = createMergedRegionFromMerges(this.region_a,
|
|
||||||
this.region_b, this.mergedRegionInfo);
|
|
||||||
return mergedRegion;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Create a merged region from the merges directory under region a. In order
|
|
||||||
* to mock it for tests, place it with a new method.
|
|
||||||
* @param a hri of region a
|
|
||||||
* @param b hri of region b
|
|
||||||
* @param mergedRegion hri of merged region
|
|
||||||
* @return merged HRegion.
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
HRegion createMergedRegionFromMerges(final HRegion a, final HRegion b,
|
|
||||||
final HRegionInfo mergedRegion) throws IOException {
|
|
||||||
return a.createMergedRegionFromMerges(mergedRegion, b);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Close the merging region and offline it in regionserver
|
|
||||||
* @param services
|
|
||||||
* @param region
|
|
||||||
* @param isRegionA true if it is merging region a, false if it is region b
|
|
||||||
* @param testing true if it is testing
|
|
||||||
* @return a map of family name to list of store files
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
private Map<byte[], List<StoreFile>> closeAndOfflineRegion(
|
|
||||||
final RegionServerServices services, final HRegion region,
|
|
||||||
final boolean isRegionA, final boolean testing) throws IOException {
|
|
||||||
Map<byte[], List<StoreFile>> hstoreFilesToMerge = null;
|
|
||||||
Exception exceptionToThrow = null;
|
|
||||||
try {
|
|
||||||
hstoreFilesToMerge = region.close(false);
|
|
||||||
} catch (Exception e) {
|
|
||||||
exceptionToThrow = e;
|
|
||||||
}
|
|
||||||
if (exceptionToThrow == null && hstoreFilesToMerge == null) {
|
|
||||||
// The region was closed by a concurrent thread. We can't continue
|
|
||||||
// with the merge, instead we must just abandon the merge. If we
|
|
||||||
// reopen or merge this could cause problems because the region has
|
|
||||||
// probably already been moved to a different server, or is in the
|
|
||||||
// process of moving to a different server.
|
|
||||||
exceptionToThrow = closedByOtherException;
|
|
||||||
}
|
|
||||||
if (exceptionToThrow != closedByOtherException) {
|
|
||||||
this.journal.add(isRegionA ? JournalEntry.CLOSED_REGION_A
|
|
||||||
: JournalEntry.CLOSED_REGION_B);
|
|
||||||
}
|
|
||||||
if (exceptionToThrow != null) {
|
|
||||||
if (exceptionToThrow instanceof IOException)
|
|
||||||
throw (IOException) exceptionToThrow;
|
|
||||||
throw new IOException(exceptionToThrow);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (!testing) {
|
|
||||||
services.removeFromOnlineRegions(region, null);
|
|
||||||
}
|
|
||||||
this.journal.add(isRegionA ? JournalEntry.OFFLINED_REGION_A
|
|
||||||
: JournalEntry.OFFLINED_REGION_B);
|
|
||||||
return hstoreFilesToMerge;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Get merged region info through the specified two regions
|
|
||||||
* @param a merging region A
|
|
||||||
* @param b merging region B
|
|
||||||
* @return the merged region info
|
|
||||||
*/
|
|
||||||
public static HRegionInfo getMergedRegionInfo(final HRegionInfo a,
|
|
||||||
final HRegionInfo b) {
|
|
||||||
long rid = EnvironmentEdgeManager.currentTime();
|
|
||||||
// Regionid is timestamp. Merged region's id can't be less than that of
|
|
||||||
// merging regions else will insert at wrong location in hbase:meta
|
|
||||||
if (rid < a.getRegionId() || rid < b.getRegionId()) {
|
|
||||||
LOG.warn("Clock skew; merging regions id are " + a.getRegionId()
|
|
||||||
+ " and " + b.getRegionId() + ", but current time here is " + rid);
|
|
||||||
rid = Math.max(a.getRegionId(), b.getRegionId()) + 1;
|
|
||||||
}
|
|
||||||
|
|
||||||
byte[] startKey = null;
|
|
||||||
byte[] endKey = null;
|
|
||||||
// Choose the smaller as start key
|
|
||||||
if (a.compareTo(b) <= 0) {
|
|
||||||
startKey = a.getStartKey();
|
|
||||||
} else {
|
|
||||||
startKey = b.getStartKey();
|
|
||||||
}
|
|
||||||
// Choose the bigger as end key
|
|
||||||
if (Bytes.equals(a.getEndKey(), HConstants.EMPTY_BYTE_ARRAY)
|
|
||||||
|| (!Bytes.equals(b.getEndKey(), HConstants.EMPTY_BYTE_ARRAY)
|
|
||||||
&& Bytes.compareTo(a.getEndKey(), b.getEndKey()) > 0)) {
|
|
||||||
endKey = a.getEndKey();
|
|
||||||
} else {
|
|
||||||
endKey = b.getEndKey();
|
|
||||||
}
|
|
||||||
|
|
||||||
// Merged region is sorted between two merging regions in META
|
|
||||||
HRegionInfo mergedRegionInfo = new HRegionInfo(a.getTable(), startKey,
|
|
||||||
endKey, false, rid);
|
|
||||||
return mergedRegionInfo;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Perform time consuming opening of the merged region.
|
|
||||||
* @param server Hosting server instance. Can be null when testing
|
|
||||||
* @param services Used to online/offline regions.
|
|
||||||
* @param merged the merged region
|
|
||||||
* @throws IOException If thrown, transaction failed. Call
|
|
||||||
* {@link #rollback(Server, RegionServerServices)}
|
|
||||||
*/
|
|
||||||
void openMergedRegion(final Server server,
|
|
||||||
final RegionServerServices services, HRegion merged) throws IOException {
|
|
||||||
boolean stopped = server != null && server.isStopped();
|
|
||||||
boolean stopping = services != null && services.isStopping();
|
|
||||||
if (stopped || stopping) {
|
|
||||||
LOG.info("Not opening merged region " + merged.getRegionInfo().getRegionNameAsString()
|
|
||||||
+ " because stopping=" + stopping + ", stopped=" + stopped);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
HRegionInfo hri = merged.getRegionInfo();
|
|
||||||
LoggingProgressable reporter = server == null ? null
|
|
||||||
: new LoggingProgressable(hri, server.getConfiguration().getLong(
|
|
||||||
"hbase.regionserver.regionmerge.open.log.interval", 10000));
|
|
||||||
merged.openHRegion(reporter);
|
|
||||||
|
|
||||||
if (services != null) {
|
|
||||||
if (!services.reportRegionStateTransition(TransitionCode.MERGED,
|
|
||||||
mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
|
|
||||||
throw new IOException("Failed to report merged region to master: "
|
|
||||||
+ mergedRegionInfo.getShortNameToLog());
|
|
||||||
}
|
|
||||||
services.addToOnlineRegions(merged);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Create reference file(s) of merging regions under the region_a merges dir
|
|
||||||
* @param hstoreFilesOfRegionA
|
|
||||||
* @param hstoreFilesOfRegionB
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
private void mergeStoreFiles(
|
|
||||||
Map<byte[], List<StoreFile>> hstoreFilesOfRegionA,
|
|
||||||
Map<byte[], List<StoreFile>> hstoreFilesOfRegionB)
|
|
||||||
throws IOException {
|
|
||||||
// Create reference file(s) of region A in mergdir
|
|
||||||
HRegionFileSystem fs_a = this.region_a.getRegionFileSystem();
|
|
||||||
for (Map.Entry<byte[], List<StoreFile>> entry : hstoreFilesOfRegionA
|
|
||||||
.entrySet()) {
|
|
||||||
String familyName = Bytes.toString(entry.getKey());
|
|
||||||
for (StoreFile storeFile : entry.getValue()) {
|
|
||||||
fs_a.mergeStoreFile(this.mergedRegionInfo, familyName, storeFile,
|
|
||||||
this.mergesdir);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// Create reference file(s) of region B in mergedir
|
|
||||||
HRegionFileSystem fs_b = this.region_b.getRegionFileSystem();
|
|
||||||
for (Map.Entry<byte[], List<StoreFile>> entry : hstoreFilesOfRegionB
|
|
||||||
.entrySet()) {
|
|
||||||
String familyName = Bytes.toString(entry.getKey());
|
|
||||||
for (StoreFile storeFile : entry.getValue()) {
|
|
||||||
fs_b.mergeStoreFile(this.mergedRegionInfo, familyName, storeFile,
|
|
||||||
this.mergesdir);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
* Roll back a failed transaction
|
||||||
* @param server Hosting server instance (May be null when testing).
|
* @param server Hosting server instance (May be null when testing).
|
||||||
* @param services Services of regionserver, used to online regions.
|
* @param services Services of regionserver, used to online regions.
|
||||||
* @throws IOException If thrown, rollback failed. Take drastic action.
|
* @throws IOException If thrown, rollback failed. Take drastic action.
|
||||||
|
@ -548,123 +183,37 @@ public class RegionMergeTransaction {
|
||||||
* of no return and so now need to abort the server to minimize
|
* of no return and so now need to abort the server to minimize
|
||||||
* damage.
|
* damage.
|
||||||
*/
|
*/
|
||||||
@SuppressWarnings("deprecation")
|
boolean rollback(Server server, RegionServerServices services) throws IOException;
|
||||||
public boolean rollback(final Server server,
|
|
||||||
final RegionServerServices services) throws IOException {
|
|
||||||
assert this.mergedRegionInfo != null;
|
|
||||||
// Coprocessor callback
|
|
||||||
if (rsCoprocessorHost != null) {
|
|
||||||
rsCoprocessorHost.preRollBackMerge(this.region_a, this.region_b);
|
|
||||||
}
|
|
||||||
|
|
||||||
boolean result = true;
|
|
||||||
ListIterator<JournalEntry> iterator = this.journal
|
|
||||||
.listIterator(this.journal.size());
|
|
||||||
// Iterate in reverse.
|
|
||||||
while (iterator.hasPrevious()) {
|
|
||||||
JournalEntry je = iterator.previous();
|
|
||||||
switch (je) {
|
|
||||||
|
|
||||||
case SET_MERGING:
|
|
||||||
if (services != null
|
|
||||||
&& !services.reportRegionStateTransition(TransitionCode.MERGE_REVERTED,
|
|
||||||
mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
|
|
||||||
case CREATED_MERGE_DIR:
|
|
||||||
this.region_a.writestate.writesEnabled = true;
|
|
||||||
this.region_b.writestate.writesEnabled = true;
|
|
||||||
this.region_a.getRegionFileSystem().cleanupMergesDir();
|
|
||||||
break;
|
|
||||||
|
|
||||||
case CLOSED_REGION_A:
|
|
||||||
try {
|
|
||||||
// So, this returns a seqid but if we just closed and then reopened,
|
|
||||||
// we should be ok. On close, we flushed using sequenceid obtained
|
|
||||||
// from hosting regionserver so no need to propagate the sequenceid
|
|
||||||
// returned out of initialize below up into regionserver as we
|
|
||||||
// normally do.
|
|
||||||
this.region_a.initialize();
|
|
||||||
} catch (IOException e) {
|
|
||||||
LOG.error("Failed rollbacking CLOSED_REGION_A of region "
|
|
||||||
+ this.region_a.getRegionInfo().getRegionNameAsString(), e);
|
|
||||||
throw new RuntimeException(e);
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
|
|
||||||
case OFFLINED_REGION_A:
|
|
||||||
if (services != null)
|
|
||||||
services.addToOnlineRegions(this.region_a);
|
|
||||||
break;
|
|
||||||
|
|
||||||
case CLOSED_REGION_B:
|
|
||||||
try {
|
|
||||||
this.region_b.initialize();
|
|
||||||
} catch (IOException e) {
|
|
||||||
LOG.error("Failed rollbacking CLOSED_REGION_A of region "
|
|
||||||
+ this.region_b.getRegionInfo().getRegionNameAsString(), e);
|
|
||||||
throw new RuntimeException(e);
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
|
|
||||||
case OFFLINED_REGION_B:
|
|
||||||
if (services != null)
|
|
||||||
services.addToOnlineRegions(this.region_b);
|
|
||||||
break;
|
|
||||||
|
|
||||||
case STARTED_MERGED_REGION_CREATION:
|
|
||||||
this.region_a.getRegionFileSystem().cleanupMergedRegion(
|
|
||||||
this.mergedRegionInfo);
|
|
||||||
break;
|
|
||||||
|
|
||||||
case PONR:
|
|
||||||
// We got to the point-of-no-return so we need to just abort. Return
|
|
||||||
// immediately. Do not clean up created merged regions.
|
|
||||||
return false;
|
|
||||||
|
|
||||||
default:
|
|
||||||
throw new RuntimeException("Unhandled journal entry: " + je);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// Coprocessor callback
|
|
||||||
if (rsCoprocessorHost != null) {
|
|
||||||
rsCoprocessorHost.postRollBackMerge(this.region_a, this.region_b);
|
|
||||||
}
|
|
||||||
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
HRegionInfo getMergedRegionInfo() {
|
|
||||||
return this.mergedRegionInfo;
|
|
||||||
}
|
|
||||||
|
|
||||||
// For unit testing.
|
|
||||||
Path getMergesDir() {
|
|
||||||
return this.mergesdir;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Checks if the given region has merge qualifier in hbase:meta
|
* Register a listener for transaction preparation, execution, and possibly
|
||||||
* @param services
|
* rollback phases.
|
||||||
* @param regionName name of specified region
|
* <p>A listener can abort a transaction by throwing an exception.
|
||||||
* @return true if the given region has merge qualifier in META.(It will be
|
* @param listener the listener
|
||||||
* cleaned by CatalogJanitor)
|
* @return 'this' for chaining
|
||||||
* @throws IOException
|
|
||||||
*/
|
*/
|
||||||
boolean hasMergeQualifierInMeta(final RegionServerServices services,
|
RegionMergeTransaction registerTransactionListener(TransactionListener listener);
|
||||||
final byte[] regionName) throws IOException {
|
|
||||||
if (services == null) return false;
|
/** @return merged region info */
|
||||||
// Get merge regions if it is a merged region and already has merge
|
HRegionInfo getMergedRegionInfo();
|
||||||
// qualifier
|
|
||||||
Pair<HRegionInfo, HRegionInfo> mergeRegions = MetaTableAccessor
|
/**
|
||||||
.getRegionsFromMergeQualifier(services.getConnection(), regionName);
|
* Get the journal for the transaction.
|
||||||
if (mergeRegions != null &&
|
* <p>Journal entries are an opaque type represented as JournalEntry. They can
|
||||||
(mergeRegions.getFirst() != null || mergeRegions.getSecond() != null)) {
|
* also provide useful debugging information via their toString method.
|
||||||
// It has merge qualifier
|
* @return the transaction journal
|
||||||
return true;
|
*/
|
||||||
}
|
List<JournalEntry> getJournal();
|
||||||
return false;
|
|
||||||
}
|
/**
|
||||||
|
* Get the Server running the transaction or rollback
|
||||||
|
* @return server instance
|
||||||
|
*/
|
||||||
|
Server getServer();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the RegonServerServices of the server running the transaction or rollback
|
||||||
|
* @return region server services
|
||||||
|
*/
|
||||||
|
RegionServerServices getRegionServerServices();
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,76 @@
|
||||||
|
/*
|
||||||
|
* 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.regionserver;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configurable;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||||
|
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A factory for creating RegionMergeTransactions, which execute region split as a "transaction".
|
||||||
|
* See {@link RegionMergeTransactionImpl}
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||||
|
@InterfaceStability.Evolving
|
||||||
|
public class RegionMergeTransactionFactory implements Configurable {
|
||||||
|
|
||||||
|
public static final String MERGE_TRANSACTION_IMPL_KEY =
|
||||||
|
"hbase.regionserver.merge.transaction.impl";
|
||||||
|
|
||||||
|
private Configuration conf;
|
||||||
|
|
||||||
|
public RegionMergeTransactionFactory(Configuration conf) {
|
||||||
|
this.conf = conf;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Configuration getConf() {
|
||||||
|
return conf;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setConf(Configuration conf) {
|
||||||
|
this.conf = conf;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a merge transaction
|
||||||
|
* @param a region a to merge
|
||||||
|
* @param b region b to merge
|
||||||
|
* @param forcible if false, we will only merge adjacent regions
|
||||||
|
* @return transaction instance
|
||||||
|
*/
|
||||||
|
public RegionMergeTransactionImpl create(final Region a, final Region b,
|
||||||
|
final boolean forcible) {
|
||||||
|
// The implementation class must extend RegionMergeTransactionImpl, not only
|
||||||
|
// implement the RegionMergeTransaction interface like you might expect,
|
||||||
|
// because various places such as AssignmentManager use static methods
|
||||||
|
// from RegionMergeTransactionImpl. Whatever we use for implementation must
|
||||||
|
// be compatible, so it's safest to require ? extends RegionMergeTransactionImpl.
|
||||||
|
// If not compatible we will throw a runtime exception from here.
|
||||||
|
return ReflectionUtils.instantiateWithCustomCtor(
|
||||||
|
conf.getClass(MERGE_TRANSACTION_IMPL_KEY, RegionMergeTransactionImpl.class,
|
||||||
|
RegionMergeTransactionImpl.class).getName(),
|
||||||
|
new Class[] { Region.class, Region.class, boolean.class },
|
||||||
|
new Object[] { a, b, forcible });
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,702 @@
|
||||||
|
/**
|
||||||
|
* Copyright The Apache Software Foundation
|
||||||
|
*
|
||||||
|
* 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 limitationsME
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.hbase.regionserver;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.ListIterator;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
|
import org.apache.hadoop.hbase.MetaMutationAnnotation;
|
||||||
|
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||||
|
import org.apache.hadoop.hbase.Server;
|
||||||
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
|
import org.apache.hadoop.hbase.client.Delete;
|
||||||
|
import org.apache.hadoop.hbase.client.Mutation;
|
||||||
|
import org.apache.hadoop.hbase.client.Put;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.SplitTransactionImpl.LoggingProgressable;
|
||||||
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
|
|
||||||
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public class RegionMergeTransactionImpl implements RegionMergeTransaction {
|
||||||
|
private static final Log LOG = LogFactory.getLog(RegionMergeTransactionImpl.class);
|
||||||
|
|
||||||
|
// Merged region info
|
||||||
|
private HRegionInfo mergedRegionInfo;
|
||||||
|
// region_a sorts before region_b
|
||||||
|
private final HRegion region_a;
|
||||||
|
private final HRegion region_b;
|
||||||
|
// merges dir is under region_a
|
||||||
|
private final Path mergesdir;
|
||||||
|
// We only merge adjacent regions if forcible is false
|
||||||
|
private final boolean forcible;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Transaction state for listener, only valid during execute and
|
||||||
|
* rollback
|
||||||
|
*/
|
||||||
|
private RegionMergeTransactionPhase currentPhase = RegionMergeTransactionPhase.STARTED;
|
||||||
|
private Server server;
|
||||||
|
private RegionServerServices rsServices;
|
||||||
|
|
||||||
|
public static class JournalEntryImpl implements JournalEntry {
|
||||||
|
private RegionMergeTransactionPhase type;
|
||||||
|
private long timestamp;
|
||||||
|
|
||||||
|
public JournalEntryImpl(RegionMergeTransactionPhase type) {
|
||||||
|
this(type, EnvironmentEdgeManager.currentTime());
|
||||||
|
}
|
||||||
|
|
||||||
|
public JournalEntryImpl(RegionMergeTransactionPhase type, long timestamp) {
|
||||||
|
this.type = type;
|
||||||
|
this.timestamp = timestamp;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
StringBuilder sb = new StringBuilder();
|
||||||
|
sb.append(type);
|
||||||
|
sb.append(" at ");
|
||||||
|
sb.append(timestamp);
|
||||||
|
return sb.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RegionMergeTransactionPhase getPhase() {
|
||||||
|
return type;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getTimeStamp() {
|
||||||
|
return timestamp;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Journal of how far the merge transaction has progressed.
|
||||||
|
*/
|
||||||
|
private final List<JournalEntry> journal = new ArrayList<JournalEntry>();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Listeners
|
||||||
|
*/
|
||||||
|
private final ArrayList<TransactionListener> listeners = new ArrayList<TransactionListener>();
|
||||||
|
|
||||||
|
private static IOException closedByOtherException = new IOException(
|
||||||
|
"Failed to close region: already closed by another thread");
|
||||||
|
|
||||||
|
private RegionServerCoprocessorHost rsCoprocessorHost = null;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructor
|
||||||
|
* @param a region a to merge
|
||||||
|
* @param b region b to merge
|
||||||
|
* @param forcible if false, we will only merge adjacent regions
|
||||||
|
*/
|
||||||
|
public RegionMergeTransactionImpl(final HRegion a, final HRegion b,
|
||||||
|
final boolean forcible) {
|
||||||
|
if (a.getRegionInfo().compareTo(b.getRegionInfo()) <= 0) {
|
||||||
|
this.region_a = a;
|
||||||
|
this.region_b = b;
|
||||||
|
} else {
|
||||||
|
this.region_a = b;
|
||||||
|
this.region_b = a;
|
||||||
|
}
|
||||||
|
this.forcible = forcible;
|
||||||
|
this.mergesdir = region_a.getRegionFileSystem().getMergesDir();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void transition(RegionMergeTransactionPhase nextPhase) throws IOException {
|
||||||
|
transition(nextPhase, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void transition(RegionMergeTransactionPhase nextPhase, boolean isRollback)
|
||||||
|
throws IOException {
|
||||||
|
if (!isRollback) {
|
||||||
|
// Add to the journal first, because if the listener throws an exception
|
||||||
|
// we need to roll back starting at 'nextPhase'
|
||||||
|
this.journal.add(new JournalEntryImpl(nextPhase));
|
||||||
|
}
|
||||||
|
for (int i = 0; i < listeners.size(); i++) {
|
||||||
|
TransactionListener listener = listeners.get(i);
|
||||||
|
if (!isRollback) {
|
||||||
|
listener.transition(this, currentPhase, nextPhase);
|
||||||
|
} else {
|
||||||
|
listener.rollback(this, currentPhase, nextPhase);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
currentPhase = nextPhase;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean prepare(final RegionServerServices services) throws IOException {
|
||||||
|
if (!region_a.getTableDesc().getTableName()
|
||||||
|
.equals(region_b.getTableDesc().getTableName())) {
|
||||||
|
LOG.info("Can't merge regions " + region_a + "," + region_b
|
||||||
|
+ " because they do not belong to the same table");
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (region_a.getRegionInfo().equals(region_b.getRegionInfo())) {
|
||||||
|
LOG.info("Can't merge the same region " + region_a);
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (!forcible && !HRegionInfo.areAdjacent(region_a.getRegionInfo(),
|
||||||
|
region_b.getRegionInfo())) {
|
||||||
|
String msg = "Skip merging " + region_a.getRegionInfo().getRegionNameAsString()
|
||||||
|
+ " and " + region_b.getRegionInfo().getRegionNameAsString()
|
||||||
|
+ ", because they are not adjacent.";
|
||||||
|
LOG.info(msg);
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (!this.region_a.isMergeable() || !this.region_b.isMergeable()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
boolean regionAHasMergeQualifier = hasMergeQualifierInMeta(services,
|
||||||
|
region_a.getRegionInfo().getRegionName());
|
||||||
|
if (regionAHasMergeQualifier ||
|
||||||
|
hasMergeQualifierInMeta(services, region_b.getRegionInfo().getRegionName())) {
|
||||||
|
LOG.debug("Region " + (regionAHasMergeQualifier ?
|
||||||
|
region_a.getRegionInfo().getRegionNameAsString()
|
||||||
|
: region_b.getRegionInfo().getRegionNameAsString())
|
||||||
|
+ " is not mergeable because it has merge qualifier in META");
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.warn("Failed judging whether merge transaction is available for "
|
||||||
|
+ region_a.getRegionInfo().getRegionNameAsString() + " and "
|
||||||
|
+ region_b.getRegionInfo().getRegionNameAsString(), e);
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
// WARN: make sure there is no parent region of the two merging regions in
|
||||||
|
// hbase:meta If exists, fixing up daughters would cause daughter regions(we
|
||||||
|
// have merged one) online again when we restart master, so we should clear
|
||||||
|
// the parent region to prevent the above case
|
||||||
|
// Since HBASE-7721, we don't need fix up daughters any more. so here do
|
||||||
|
// nothing
|
||||||
|
|
||||||
|
this.mergedRegionInfo = getMergedRegionInfo(region_a.getRegionInfo(),
|
||||||
|
region_b.getRegionInfo());
|
||||||
|
|
||||||
|
transition(RegionMergeTransactionPhase.PREPARED);
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Region execute(final Server server, final RegionServerServices services)
|
||||||
|
throws IOException {
|
||||||
|
this.server = server;
|
||||||
|
this.rsServices = services;
|
||||||
|
if (rsCoprocessorHost == null) {
|
||||||
|
rsCoprocessorHost = server != null ?
|
||||||
|
((HRegionServer) server).getRegionServerCoprocessorHost() : null;
|
||||||
|
}
|
||||||
|
HRegion mergedRegion = createMergedRegion(server, services);
|
||||||
|
if (rsCoprocessorHost != null) {
|
||||||
|
rsCoprocessorHost.postMergeCommit(this.region_a, this.region_b, mergedRegion);
|
||||||
|
}
|
||||||
|
stepsAfterPONR(server, services, mergedRegion);
|
||||||
|
|
||||||
|
transition(RegionMergeTransactionPhase.COMPLETED);
|
||||||
|
|
||||||
|
return mergedRegion;
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
public void stepsAfterPONR(final Server server, final RegionServerServices services,
|
||||||
|
HRegion mergedRegion) throws IOException {
|
||||||
|
openMergedRegion(server, services, mergedRegion);
|
||||||
|
if (rsCoprocessorHost != null) {
|
||||||
|
rsCoprocessorHost.postMerge(this.region_a, this.region_b, mergedRegion);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Prepare the merged region and region files.
|
||||||
|
* @param server Hosting server instance. Can be null when testing
|
||||||
|
* @param services Used to online/offline regions.
|
||||||
|
* @return merged region
|
||||||
|
* @throws IOException If thrown, transaction failed. Call
|
||||||
|
* {@link #rollback(Server, RegionServerServices)}
|
||||||
|
*/
|
||||||
|
private HRegion createMergedRegion(final Server server, final RegionServerServices services)
|
||||||
|
throws IOException {
|
||||||
|
LOG.info("Starting merge of " + region_a + " and "
|
||||||
|
+ region_b.getRegionInfo().getRegionNameAsString() + ", forcible=" + forcible);
|
||||||
|
if ((server != null && server.isStopped())
|
||||||
|
|| (services != null && services.isStopping())) {
|
||||||
|
throw new IOException("Server is stopped or stopping");
|
||||||
|
}
|
||||||
|
|
||||||
|
if (rsCoprocessorHost != null) {
|
||||||
|
if (rsCoprocessorHost.preMerge(this.region_a, this.region_b)) {
|
||||||
|
throw new IOException("Coprocessor bypassing regions " + this.region_a + " "
|
||||||
|
+ this.region_b + " merge.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// If true, no cluster to write meta edits to or to use coordination.
|
||||||
|
boolean testing = server == null ? true : server.getConfiguration()
|
||||||
|
.getBoolean("hbase.testing.nocluster", false);
|
||||||
|
|
||||||
|
HRegion mergedRegion = stepsBeforePONR(server, services, testing);
|
||||||
|
|
||||||
|
@MetaMutationAnnotation
|
||||||
|
List<Mutation> metaEntries = new ArrayList<Mutation>();
|
||||||
|
if (rsCoprocessorHost != null) {
|
||||||
|
if (rsCoprocessorHost.preMergeCommit(this.region_a, this.region_b, metaEntries)) {
|
||||||
|
throw new IOException("Coprocessor bypassing regions " + this.region_a + " "
|
||||||
|
+ this.region_b + " merge.");
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
for (Mutation p : metaEntries) {
|
||||||
|
HRegionInfo.parseRegionName(p.getRow());
|
||||||
|
}
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.error("Row key of mutation from coprocessor is not parsable as region name."
|
||||||
|
+ "Mutations from coprocessor should only be for hbase:meta table.", e);
|
||||||
|
throw e;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// This is the point of no return. Similar with SplitTransaction.
|
||||||
|
// IF we reach the PONR then subsequent failures need to crash out this
|
||||||
|
// regionserver
|
||||||
|
transition(RegionMergeTransactionPhase.PONR);
|
||||||
|
|
||||||
|
// Add merged region and delete region_a and region_b
|
||||||
|
// as an atomic update. See HBASE-7721. This update to hbase:meta makes the region
|
||||||
|
// will determine whether the region is merged or not in case of failures.
|
||||||
|
// If it is successful, master will roll-forward, if not, master will
|
||||||
|
// rollback
|
||||||
|
if (services != null && !services.reportRegionStateTransition(TransitionCode.MERGE_PONR,
|
||||||
|
mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
|
||||||
|
// Passed PONR, let SSH clean it up
|
||||||
|
throw new IOException("Failed to notify master that merge passed PONR: "
|
||||||
|
+ region_a.getRegionInfo().getRegionNameAsString() + " and "
|
||||||
|
+ region_b.getRegionInfo().getRegionNameAsString());
|
||||||
|
}
|
||||||
|
return mergedRegion;
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
public void prepareMutationsForMerge(HRegionInfo mergedRegion, HRegionInfo regionA,
|
||||||
|
HRegionInfo regionB, ServerName serverName, List<Mutation> mutations) throws IOException {
|
||||||
|
HRegionInfo copyOfMerged = new HRegionInfo(mergedRegion);
|
||||||
|
|
||||||
|
// Put for parent
|
||||||
|
Put putOfMerged = MetaTableAccessor.makePutFromRegionInfo(copyOfMerged);
|
||||||
|
putOfMerged.add(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER,
|
||||||
|
regionA.toByteArray());
|
||||||
|
putOfMerged.add(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER,
|
||||||
|
regionB.toByteArray());
|
||||||
|
mutations.add(putOfMerged);
|
||||||
|
// Deletes for merging regions
|
||||||
|
Delete deleteA = MetaTableAccessor.makeDeleteFromRegionInfo(regionA);
|
||||||
|
Delete deleteB = MetaTableAccessor.makeDeleteFromRegionInfo(regionB);
|
||||||
|
mutations.add(deleteA);
|
||||||
|
mutations.add(deleteB);
|
||||||
|
// The merged is a new region, openSeqNum = 1 is fine.
|
||||||
|
addLocation(putOfMerged, serverName, 1);
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
Put addLocation(final Put p, final ServerName sn, long openSeqNum) {
|
||||||
|
p.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER, Bytes
|
||||||
|
.toBytes(sn.getHostAndPort()));
|
||||||
|
p.add(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER, Bytes.toBytes(sn
|
||||||
|
.getStartcode()));
|
||||||
|
p.add(HConstants.CATALOG_FAMILY, HConstants.SEQNUM_QUALIFIER, Bytes.toBytes(openSeqNum));
|
||||||
|
return p;
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
public HRegion stepsBeforePONR(final Server server, final RegionServerServices services,
|
||||||
|
boolean testing) throws IOException {
|
||||||
|
if (services != null && !services.reportRegionStateTransition(TransitionCode.READY_TO_MERGE,
|
||||||
|
mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
|
||||||
|
throw new IOException("Failed to get ok from master to merge "
|
||||||
|
+ region_a.getRegionInfo().getRegionNameAsString() + " and "
|
||||||
|
+ region_b.getRegionInfo().getRegionNameAsString());
|
||||||
|
}
|
||||||
|
|
||||||
|
transition(RegionMergeTransactionPhase.SET_MERGING);
|
||||||
|
|
||||||
|
this.region_a.getRegionFileSystem().createMergesDir();
|
||||||
|
|
||||||
|
transition(RegionMergeTransactionPhase.CREATED_MERGE_DIR);
|
||||||
|
|
||||||
|
Map<byte[], List<StoreFile>> hstoreFilesOfRegionA = closeAndOfflineRegion(
|
||||||
|
services, this.region_a, true, testing);
|
||||||
|
Map<byte[], List<StoreFile>> hstoreFilesOfRegionB = closeAndOfflineRegion(
|
||||||
|
services, this.region_b, false, testing);
|
||||||
|
|
||||||
|
assert hstoreFilesOfRegionA != null && hstoreFilesOfRegionB != null;
|
||||||
|
|
||||||
|
// mergeStoreFiles creates merged region dirs under the region_a merges dir
|
||||||
|
// Nothing to unroll here if failure -- clean up of CREATE_MERGE_DIR will
|
||||||
|
// clean this up.
|
||||||
|
mergeStoreFiles(hstoreFilesOfRegionA, hstoreFilesOfRegionB);
|
||||||
|
|
||||||
|
// Log to the journal that we are creating merged region. We could fail
|
||||||
|
// halfway through. If we do, we could have left
|
||||||
|
// stuff in fs that needs cleanup -- a storefile or two. Thats why we
|
||||||
|
// add entry to journal BEFORE rather than AFTER the change.
|
||||||
|
|
||||||
|
transition(RegionMergeTransactionPhase.STARTED_MERGED_REGION_CREATION);
|
||||||
|
|
||||||
|
HRegion mergedRegion = createMergedRegionFromMerges(this.region_a,
|
||||||
|
this.region_b, this.mergedRegionInfo);
|
||||||
|
return mergedRegion;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a merged region from the merges directory under region a. In order
|
||||||
|
* to mock it for tests, place it with a new method.
|
||||||
|
* @param a hri of region a
|
||||||
|
* @param b hri of region b
|
||||||
|
* @param mergedRegion hri of merged region
|
||||||
|
* @return merged HRegion.
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
@VisibleForTesting
|
||||||
|
HRegion createMergedRegionFromMerges(final HRegion a, final HRegion b,
|
||||||
|
final HRegionInfo mergedRegion) throws IOException {
|
||||||
|
return a.createMergedRegionFromMerges(mergedRegion, b);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Close the merging region and offline it in regionserver
|
||||||
|
* @param services
|
||||||
|
* @param region
|
||||||
|
* @param isRegionA true if it is merging region a, false if it is region b
|
||||||
|
* @param testing true if it is testing
|
||||||
|
* @return a map of family name to list of store files
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
private Map<byte[], List<StoreFile>> closeAndOfflineRegion(
|
||||||
|
final RegionServerServices services, final HRegion region,
|
||||||
|
final boolean isRegionA, final boolean testing) throws IOException {
|
||||||
|
Map<byte[], List<StoreFile>> hstoreFilesToMerge = null;
|
||||||
|
Exception exceptionToThrow = null;
|
||||||
|
try {
|
||||||
|
hstoreFilesToMerge = region.close(false);
|
||||||
|
} catch (Exception e) {
|
||||||
|
exceptionToThrow = e;
|
||||||
|
}
|
||||||
|
if (exceptionToThrow == null && hstoreFilesToMerge == null) {
|
||||||
|
// The region was closed by a concurrent thread. We can't continue
|
||||||
|
// with the merge, instead we must just abandon the merge. If we
|
||||||
|
// reopen or merge this could cause problems because the region has
|
||||||
|
// probably already been moved to a different server, or is in the
|
||||||
|
// process of moving to a different server.
|
||||||
|
exceptionToThrow = closedByOtherException;
|
||||||
|
}
|
||||||
|
if (exceptionToThrow != closedByOtherException) {
|
||||||
|
transition(isRegionA ? RegionMergeTransactionPhase.CLOSED_REGION_A
|
||||||
|
: RegionMergeTransactionPhase.CLOSED_REGION_B);
|
||||||
|
}
|
||||||
|
if (exceptionToThrow != null) {
|
||||||
|
if (exceptionToThrow instanceof IOException)
|
||||||
|
throw (IOException) exceptionToThrow;
|
||||||
|
throw new IOException(exceptionToThrow);
|
||||||
|
}
|
||||||
|
if (!testing) {
|
||||||
|
services.removeFromOnlineRegions(region, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
transition(isRegionA ? RegionMergeTransactionPhase.OFFLINED_REGION_A
|
||||||
|
: RegionMergeTransactionPhase.OFFLINED_REGION_B);
|
||||||
|
|
||||||
|
return hstoreFilesToMerge;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get merged region info through the specified two regions
|
||||||
|
* @param a merging region A
|
||||||
|
* @param b merging region B
|
||||||
|
* @return the merged region info
|
||||||
|
*/
|
||||||
|
@VisibleForTesting
|
||||||
|
static HRegionInfo getMergedRegionInfo(final HRegionInfo a, final HRegionInfo b) {
|
||||||
|
long rid = EnvironmentEdgeManager.currentTime();
|
||||||
|
// Regionid is timestamp. Merged region's id can't be less than that of
|
||||||
|
// merging regions else will insert at wrong location in hbase:meta
|
||||||
|
if (rid < a.getRegionId() || rid < b.getRegionId()) {
|
||||||
|
LOG.warn("Clock skew; merging regions id are " + a.getRegionId()
|
||||||
|
+ " and " + b.getRegionId() + ", but current time here is " + rid);
|
||||||
|
rid = Math.max(a.getRegionId(), b.getRegionId()) + 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
byte[] startKey = null;
|
||||||
|
byte[] endKey = null;
|
||||||
|
// Choose the smaller as start key
|
||||||
|
if (a.compareTo(b) <= 0) {
|
||||||
|
startKey = a.getStartKey();
|
||||||
|
} else {
|
||||||
|
startKey = b.getStartKey();
|
||||||
|
}
|
||||||
|
// Choose the bigger as end key
|
||||||
|
if (Bytes.equals(a.getEndKey(), HConstants.EMPTY_BYTE_ARRAY)
|
||||||
|
|| (!Bytes.equals(b.getEndKey(), HConstants.EMPTY_BYTE_ARRAY)
|
||||||
|
&& Bytes.compareTo(a.getEndKey(), b.getEndKey()) > 0)) {
|
||||||
|
endKey = a.getEndKey();
|
||||||
|
} else {
|
||||||
|
endKey = b.getEndKey();
|
||||||
|
}
|
||||||
|
|
||||||
|
// Merged region is sorted between two merging regions in META
|
||||||
|
HRegionInfo mergedRegionInfo = new HRegionInfo(a.getTable(), startKey,
|
||||||
|
endKey, false, rid);
|
||||||
|
return mergedRegionInfo;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Perform time consuming opening of the merged region.
|
||||||
|
* @param server Hosting server instance. Can be null when testing
|
||||||
|
* @param services Used to online/offline regions.
|
||||||
|
* @param merged the merged region
|
||||||
|
* @throws IOException If thrown, transaction failed. Call
|
||||||
|
* {@link #rollback(Server, RegionServerServices)}
|
||||||
|
*/
|
||||||
|
@VisibleForTesting
|
||||||
|
void openMergedRegion(final Server server, final RegionServerServices services,
|
||||||
|
HRegion merged) throws IOException {
|
||||||
|
boolean stopped = server != null && server.isStopped();
|
||||||
|
boolean stopping = services != null && services.isStopping();
|
||||||
|
if (stopped || stopping) {
|
||||||
|
LOG.info("Not opening merged region " + merged.getRegionInfo().getRegionNameAsString()
|
||||||
|
+ " because stopping=" + stopping + ", stopped=" + stopped);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
HRegionInfo hri = merged.getRegionInfo();
|
||||||
|
LoggingProgressable reporter = server == null ? null
|
||||||
|
: new LoggingProgressable(hri, server.getConfiguration().getLong(
|
||||||
|
"hbase.regionserver.regionmerge.open.log.interval", 10000));
|
||||||
|
merged.openHRegion(reporter);
|
||||||
|
|
||||||
|
if (services != null) {
|
||||||
|
if (!services.reportRegionStateTransition(TransitionCode.MERGED,
|
||||||
|
mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
|
||||||
|
throw new IOException("Failed to report merged region to master: "
|
||||||
|
+ mergedRegionInfo.getShortNameToLog());
|
||||||
|
}
|
||||||
|
services.addToOnlineRegions(merged);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create reference file(s) of merging regions under the region_a merges dir
|
||||||
|
* @param hstoreFilesOfRegionA
|
||||||
|
* @param hstoreFilesOfRegionB
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
private void mergeStoreFiles(
|
||||||
|
Map<byte[], List<StoreFile>> hstoreFilesOfRegionA,
|
||||||
|
Map<byte[], List<StoreFile>> hstoreFilesOfRegionB)
|
||||||
|
throws IOException {
|
||||||
|
// Create reference file(s) of region A in mergdir
|
||||||
|
HRegionFileSystem fs_a = this.region_a.getRegionFileSystem();
|
||||||
|
for (Map.Entry<byte[], List<StoreFile>> entry : hstoreFilesOfRegionA
|
||||||
|
.entrySet()) {
|
||||||
|
String familyName = Bytes.toString(entry.getKey());
|
||||||
|
for (StoreFile storeFile : entry.getValue()) {
|
||||||
|
fs_a.mergeStoreFile(this.mergedRegionInfo, familyName, storeFile,
|
||||||
|
this.mergesdir);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// Create reference file(s) of region B in mergedir
|
||||||
|
HRegionFileSystem fs_b = this.region_b.getRegionFileSystem();
|
||||||
|
for (Map.Entry<byte[], List<StoreFile>> entry : hstoreFilesOfRegionB
|
||||||
|
.entrySet()) {
|
||||||
|
String familyName = Bytes.toString(entry.getKey());
|
||||||
|
for (StoreFile storeFile : entry.getValue()) {
|
||||||
|
fs_b.mergeStoreFile(this.mergedRegionInfo, familyName, storeFile,
|
||||||
|
this.mergesdir);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean rollback(final Server server,
|
||||||
|
final RegionServerServices services) throws IOException {
|
||||||
|
assert this.mergedRegionInfo != null;
|
||||||
|
this.server = server;
|
||||||
|
this.rsServices = services;
|
||||||
|
// Coprocessor callback
|
||||||
|
if (rsCoprocessorHost != null) {
|
||||||
|
rsCoprocessorHost.preRollBackMerge(this.region_a, this.region_b);
|
||||||
|
}
|
||||||
|
|
||||||
|
boolean result = true;
|
||||||
|
ListIterator<JournalEntry> iterator = this.journal
|
||||||
|
.listIterator(this.journal.size());
|
||||||
|
// Iterate in reverse.
|
||||||
|
while (iterator.hasPrevious()) {
|
||||||
|
JournalEntry je = iterator.previous();
|
||||||
|
|
||||||
|
transition(je.getPhase(), true);
|
||||||
|
|
||||||
|
switch (je.getPhase()) {
|
||||||
|
|
||||||
|
case SET_MERGING:
|
||||||
|
if (services != null
|
||||||
|
&& !services.reportRegionStateTransition(TransitionCode.MERGE_REVERTED,
|
||||||
|
mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
break;
|
||||||
|
|
||||||
|
case CREATED_MERGE_DIR:
|
||||||
|
this.region_a.writestate.writesEnabled = true;
|
||||||
|
this.region_b.writestate.writesEnabled = true;
|
||||||
|
this.region_a.getRegionFileSystem().cleanupMergesDir();
|
||||||
|
break;
|
||||||
|
|
||||||
|
case CLOSED_REGION_A:
|
||||||
|
try {
|
||||||
|
// So, this returns a seqid but if we just closed and then reopened,
|
||||||
|
// we should be ok. On close, we flushed using sequenceid obtained
|
||||||
|
// from hosting regionserver so no need to propagate the sequenceid
|
||||||
|
// returned out of initialize below up into regionserver as we
|
||||||
|
// normally do.
|
||||||
|
this.region_a.initialize();
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.error("Failed rollbacking CLOSED_REGION_A of region "
|
||||||
|
+ region_a.getRegionInfo().getRegionNameAsString(), e);
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
break;
|
||||||
|
|
||||||
|
case OFFLINED_REGION_A:
|
||||||
|
if (services != null)
|
||||||
|
services.addToOnlineRegions(this.region_a);
|
||||||
|
break;
|
||||||
|
|
||||||
|
case CLOSED_REGION_B:
|
||||||
|
try {
|
||||||
|
this.region_b.initialize();
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.error("Failed rollbacking CLOSED_REGION_A of region "
|
||||||
|
+ region_b.getRegionInfo().getRegionNameAsString(), e);
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
break;
|
||||||
|
|
||||||
|
case OFFLINED_REGION_B:
|
||||||
|
if (services != null)
|
||||||
|
services.addToOnlineRegions(this.region_b);
|
||||||
|
break;
|
||||||
|
|
||||||
|
case STARTED_MERGED_REGION_CREATION:
|
||||||
|
this.region_a.getRegionFileSystem().cleanupMergedRegion(
|
||||||
|
this.mergedRegionInfo);
|
||||||
|
break;
|
||||||
|
|
||||||
|
case PONR:
|
||||||
|
// We got to the point-of-no-return so we need to just abort. Return
|
||||||
|
// immediately. Do not clean up created merged regions.
|
||||||
|
return false;
|
||||||
|
|
||||||
|
// Informational states only
|
||||||
|
case STARTED:
|
||||||
|
case PREPARED:
|
||||||
|
case COMPLETED:
|
||||||
|
break;
|
||||||
|
|
||||||
|
default:
|
||||||
|
throw new RuntimeException("Unhandled journal entry: " + je);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// Coprocessor callback
|
||||||
|
if (rsCoprocessorHost != null) {
|
||||||
|
rsCoprocessorHost.postRollBackMerge(this.region_a, this.region_b);
|
||||||
|
}
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public HRegionInfo getMergedRegionInfo() {
|
||||||
|
return this.mergedRegionInfo;
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
Path getMergesDir() {
|
||||||
|
return this.mergesdir;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Checks if the given region has merge qualifier in hbase:meta
|
||||||
|
* @param services
|
||||||
|
* @param regionName name of specified region
|
||||||
|
* @return true if the given region has merge qualifier in META.(It will be
|
||||||
|
* cleaned by CatalogJanitor)
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
@VisibleForTesting
|
||||||
|
boolean hasMergeQualifierInMeta(final RegionServerServices services, final byte[] regionName)
|
||||||
|
throws IOException {
|
||||||
|
if (services == null) return false;
|
||||||
|
// Get merge regions if it is a merged region and already has merge
|
||||||
|
// qualifier
|
||||||
|
Pair<HRegionInfo, HRegionInfo> mergeRegions = MetaTableAccessor
|
||||||
|
.getRegionsFromMergeQualifier(services.getConnection(), regionName);
|
||||||
|
if (mergeRegions != null &&
|
||||||
|
(mergeRegions.getFirst() != null || mergeRegions.getSecond() != null)) {
|
||||||
|
// It has merge qualifier
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<JournalEntry> getJournal() {
|
||||||
|
return journal;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RegionMergeTransaction registerTransactionListener(TransactionListener listener) {
|
||||||
|
listeners.add(listener);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Server getServer() {
|
||||||
|
return server;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RegionServerServices getRegionServerServices() {
|
||||||
|
return rsServices;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,10 @@
|
||||||
|
--- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransactionImpl.java
|
||||||
|
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransactionImpl.java
|
||||||
|
@@ -41,6 +41,7 @@
|
||||||
|
import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
|
||||||
|
import org.apache.hadoop.hbase.coordination.RegionMergeCoordination.RegionMergeDetails;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
|
||||||
|
+import org.apache.hadoop.hbase.regionserver.SplitTransactionImpl.JournalEntryImpl;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.SplitTransactionImpl.LoggingProgressable;
|
||||||
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.hadoop.hbase.util.ConfigUtil;
|
|
@ -64,7 +64,7 @@ class SplitRequest implements Runnable {
|
||||||
boolean success = false;
|
boolean success = false;
|
||||||
server.metricsRegionServer.incrSplitRequest();
|
server.metricsRegionServer.incrSplitRequest();
|
||||||
long startTime = EnvironmentEdgeManager.currentTime();
|
long startTime = EnvironmentEdgeManager.currentTime();
|
||||||
SplitTransaction st = new SplitTransaction(parent, midKey);
|
SplitTransactionImpl st = new SplitTransactionImpl(parent, midKey);
|
||||||
try {
|
try {
|
||||||
//acquire a shared read lock on the table, so that table schema modifications
|
//acquire a shared read lock on the table, so that table schema modifications
|
||||||
//do not happen concurrently
|
//do not happen concurrently
|
||||||
|
|
|
@ -1,5 +1,4 @@
|
||||||
/**
|
/*
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
|
@ -19,93 +18,62 @@
|
||||||
package org.apache.hadoop.hbase.regionserver;
|
package org.apache.hadoop.hbase.regionserver;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InterruptedIOException;
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.ListIterator;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
import java.util.concurrent.ExecutionException;
|
|
||||||
import java.util.concurrent.Executors;
|
|
||||||
import java.util.concurrent.Future;
|
|
||||||
import java.util.concurrent.ThreadFactory;
|
|
||||||
import java.util.concurrent.ThreadPoolExecutor;
|
|
||||||
import java.util.concurrent.TimeUnit;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
|
||||||
import org.apache.hadoop.hbase.HRegionInfo;
|
|
||||||
import org.apache.hadoop.hbase.Server;
|
import org.apache.hadoop.hbase.Server;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.client.Mutation;
|
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.hbase.client.Put;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
|
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
|
||||||
import org.apache.hadoop.hbase.util.CancelableProgressable;
|
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
|
||||||
import org.apache.hadoop.hbase.util.HasThread;
|
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
|
||||||
import org.apache.hadoop.hbase.util.PairOfSameType;
|
import org.apache.hadoop.hbase.util.PairOfSameType;
|
||||||
import org.apache.zookeeper.KeeperException;
|
|
||||||
|
|
||||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Executes region split as a "transaction". Call {@link #prepare()} to setup
|
* Executes region split as a "transaction". Call {@link #prepare()} to setup
|
||||||
* the transaction, {@link #execute(Server, RegionServerServices)} to run the
|
* the transaction, {@link #execute(Server, RegionServerServices)} to run the
|
||||||
* transaction and {@link #rollback(Server, RegionServerServices)} to cleanup if execute fails.
|
* transaction and {@link #rollback(Server, RegionServerServices)} to cleanup if execute fails.
|
||||||
*
|
*
|
||||||
* <p>Here is an example of how you would use this class:
|
* <p>Here is an example of how you would use this interface:
|
||||||
* <pre>
|
* <pre>
|
||||||
* SplitTransaction st = new SplitTransaction(this.conf, parent, midKey)
|
* SplitTransactionFactory factory = new SplitTransactionFactory(conf);
|
||||||
|
* SplitTransaction st = factory.create(parent, midKey)
|
||||||
|
* .registerTransactionListener(new TransactionListener() {
|
||||||
|
* public void transition(SplitTransaction transaction, SplitTransactionPhase from,
|
||||||
|
* SplitTransactionPhase to) throws IOException {
|
||||||
|
* // ...
|
||||||
|
* }
|
||||||
|
* public void rollback(SplitTransaction transaction, SplitTransactionPhase from,
|
||||||
|
* SplitTransactionPhase to) {
|
||||||
|
* // ...
|
||||||
|
* }
|
||||||
|
* });
|
||||||
* if (!st.prepare()) return;
|
* if (!st.prepare()) return;
|
||||||
* try {
|
* try {
|
||||||
* st.execute(server, services);
|
* st.execute(server, services);
|
||||||
* } catch (IOException ioe) {
|
* } catch (IOException e) {
|
||||||
* try {
|
* try {
|
||||||
* st.rollback(server, services);
|
* st.rollback(server, services);
|
||||||
* return;
|
* return;
|
||||||
* } catch (RuntimeException e) {
|
* } catch (RuntimeException e) {
|
||||||
* myAbortable.abort("Failed split, abort");
|
* // abort the server
|
||||||
* }
|
* }
|
||||||
* }
|
* }
|
||||||
* </Pre>
|
* </Pre>
|
||||||
* <p>This class is not thread safe. Caller needs ensure split is run by
|
* <p>A split transaction is not thread safe. Callers must ensure a split is run by
|
||||||
* one thread only.
|
* one thread only.
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||||
public class SplitTransaction {
|
@InterfaceStability.Evolving
|
||||||
private static final Log LOG = LogFactory.getLog(SplitTransaction.class);
|
public interface SplitTransaction {
|
||||||
|
|
||||||
/*
|
|
||||||
* Region to split
|
|
||||||
*/
|
|
||||||
private final HRegion parent;
|
|
||||||
private HRegionInfo hri_a;
|
|
||||||
private HRegionInfo hri_b;
|
|
||||||
private long fileSplitTimeout = 30000;
|
|
||||||
|
|
||||||
/*
|
|
||||||
* Row to split around
|
|
||||||
*/
|
|
||||||
private final byte [] splitrow;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Types to add to the transaction journal.
|
* Each enum is a step in the split transaction.
|
||||||
* Each enum is a step in the split transaction. Used to figure how much
|
|
||||||
* we need to rollback.
|
|
||||||
*/
|
*/
|
||||||
static enum JournalEntryType {
|
public enum SplitTransactionPhase {
|
||||||
/**
|
/**
|
||||||
* Started
|
* Started
|
||||||
*/
|
*/
|
||||||
STARTED,
|
STARTED,
|
||||||
/**
|
/**
|
||||||
* Prepared (after table lock)
|
* Prepared
|
||||||
*/
|
*/
|
||||||
PREPARED,
|
PREPARED,
|
||||||
/**
|
/**
|
||||||
|
@ -148,6 +116,12 @@ public class SplitTransaction {
|
||||||
* Opened the second daughter region
|
* Opened the second daughter region
|
||||||
*/
|
*/
|
||||||
OPENED_REGION_B,
|
OPENED_REGION_B,
|
||||||
|
/**
|
||||||
|
* Point of no return.
|
||||||
|
* If we got here, then transaction is not recoverable other than by
|
||||||
|
* crashing out the regionserver.
|
||||||
|
*/
|
||||||
|
PONR,
|
||||||
/**
|
/**
|
||||||
* Before postSplit coprocessor hook
|
* Before postSplit coprocessor hook
|
||||||
*/
|
*/
|
||||||
|
@ -157,327 +131,60 @@ public class SplitTransaction {
|
||||||
*/
|
*/
|
||||||
AFTER_POST_SPLIT_HOOK,
|
AFTER_POST_SPLIT_HOOK,
|
||||||
/**
|
/**
|
||||||
* Point of no return.
|
* Completed
|
||||||
* If we got here, then transaction is not recoverable other than by
|
|
||||||
* crashing out the regionserver.
|
|
||||||
*/
|
*/
|
||||||
PONR
|
COMPLETED
|
||||||
}
|
|
||||||
|
|
||||||
static class JournalEntry {
|
|
||||||
private JournalEntryType type;
|
|
||||||
private long timestamp;
|
|
||||||
|
|
||||||
public JournalEntry(JournalEntryType type) {
|
|
||||||
this(type, EnvironmentEdgeManager.currentTime());
|
|
||||||
}
|
|
||||||
|
|
||||||
public JournalEntry(JournalEntryType type, long timestamp) {
|
|
||||||
this.type = type;
|
|
||||||
this.timestamp = timestamp;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String toString() {
|
|
||||||
StringBuilder sb = new StringBuilder();
|
|
||||||
sb.append(type);
|
|
||||||
sb.append(" at ");
|
|
||||||
sb.append(timestamp);
|
|
||||||
return sb.toString();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/*
|
|
||||||
* Journal of how far the split transaction has progressed.
|
|
||||||
*/
|
|
||||||
private final List<JournalEntry> journal = new ArrayList<JournalEntry>();
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Constructor
|
|
||||||
* @param r Region to split
|
|
||||||
* @param splitrow Row to split around
|
|
||||||
*/
|
|
||||||
public SplitTransaction(final Region r, final byte [] splitrow) {
|
|
||||||
this.parent = (HRegion)r;
|
|
||||||
this.splitrow = splitrow;
|
|
||||||
this.journal.add(new JournalEntry(JournalEntryType.STARTED));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Does checks on split inputs.
|
* Split transaction journal entry
|
||||||
|
*/
|
||||||
|
public interface JournalEntry {
|
||||||
|
|
||||||
|
/** @return the completed phase marked by this journal entry */
|
||||||
|
SplitTransactionPhase getPhase();
|
||||||
|
|
||||||
|
/** @return the time of phase completion */
|
||||||
|
long getTimeStamp();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Split transaction listener
|
||||||
|
*/
|
||||||
|
public interface TransactionListener {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Invoked when transitioning forward from one transaction phase to another
|
||||||
|
* @param transaction the transaction
|
||||||
|
* @param from the current phase
|
||||||
|
* @param to the next phase
|
||||||
|
* @throws IOException listener can throw this to abort
|
||||||
|
*/
|
||||||
|
void transition(SplitTransaction transaction, SplitTransactionPhase from,
|
||||||
|
SplitTransactionPhase to) throws IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Invoked when rolling back a transaction from one transaction phase to the
|
||||||
|
* previous
|
||||||
|
* @param transaction the transaction
|
||||||
|
* @param from the current phase
|
||||||
|
* @param to the previous phase
|
||||||
|
*/
|
||||||
|
void rollback(SplitTransaction transaction, SplitTransactionPhase from,
|
||||||
|
SplitTransactionPhase to);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check split inputs and prepare the transaction.
|
||||||
* @return <code>true</code> if the region is splittable else
|
* @return <code>true</code> if the region is splittable else
|
||||||
* <code>false</code> if it is not (e.g. its already closed, etc.).
|
* <code>false</code> if it is not (e.g. its already closed, etc.).
|
||||||
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public boolean prepare() {
|
boolean prepare() throws IOException;
|
||||||
if (!this.parent.isSplittable()) return false;
|
|
||||||
// Split key can be null if this region is unsplittable; i.e. has refs.
|
|
||||||
if (this.splitrow == null) return false;
|
|
||||||
HRegionInfo hri = this.parent.getRegionInfo();
|
|
||||||
parent.prepareToSplit();
|
|
||||||
// Check splitrow.
|
|
||||||
byte [] startKey = hri.getStartKey();
|
|
||||||
byte [] endKey = hri.getEndKey();
|
|
||||||
if (Bytes.equals(startKey, splitrow) ||
|
|
||||||
!this.parent.getRegionInfo().containsRow(splitrow)) {
|
|
||||||
LOG.info("Split row is not inside region key range or is equal to " +
|
|
||||||
"startkey: " + Bytes.toStringBinary(this.splitrow));
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
long rid = getDaughterRegionIdTimestamp(hri);
|
|
||||||
this.hri_a = new HRegionInfo(hri.getTable(), startKey, this.splitrow, false, rid);
|
|
||||||
this.hri_b = new HRegionInfo(hri.getTable(), this.splitrow, endKey, false, rid);
|
|
||||||
this.journal.add(new JournalEntry(JournalEntryType.PREPARED));
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Calculate daughter regionid to use.
|
|
||||||
* @param hri Parent {@link HRegionInfo}
|
|
||||||
* @return Daughter region id (timestamp) to use.
|
|
||||||
*/
|
|
||||||
private static long getDaughterRegionIdTimestamp(final HRegionInfo hri) {
|
|
||||||
long rid = EnvironmentEdgeManager.currentTime();
|
|
||||||
// Regionid is timestamp. Can't be less than that of parent else will insert
|
|
||||||
// at wrong location in hbase:meta (See HBASE-710).
|
|
||||||
if (rid < hri.getRegionId()) {
|
|
||||||
LOG.warn("Clock skew; parent regions id is " + hri.getRegionId() +
|
|
||||||
" but current time here is " + rid);
|
|
||||||
rid = hri.getRegionId() + 1;
|
|
||||||
}
|
|
||||||
return rid;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static IOException closedByOtherException = new IOException(
|
|
||||||
"Failed to close region: already closed by another thread");
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Prepare the regions and region files.
|
|
||||||
* @param server Hosting server instance. Can be null when testing (won't try
|
|
||||||
* and update in zk if a null server)
|
|
||||||
* @param services Used to online/offline regions.
|
|
||||||
* @throws IOException If thrown, transaction failed.
|
|
||||||
* Call {@link #rollback(Server, RegionServerServices)}
|
|
||||||
* @return Regions created
|
|
||||||
*/
|
|
||||||
/* package */PairOfSameType<Region> createDaughters(final Server server,
|
|
||||||
final RegionServerServices services) throws IOException {
|
|
||||||
LOG.info("Starting split of region " + this.parent);
|
|
||||||
if ((server != null && server.isStopped()) ||
|
|
||||||
(services != null && services.isStopping())) {
|
|
||||||
throw new IOException("Server is stopped or stopping");
|
|
||||||
}
|
|
||||||
assert !this.parent.lock.writeLock().isHeldByCurrentThread():
|
|
||||||
"Unsafe to hold write lock while performing RPCs";
|
|
||||||
|
|
||||||
journal.add(new JournalEntry(JournalEntryType.BEFORE_PRE_SPLIT_HOOK));
|
|
||||||
|
|
||||||
// Coprocessor callback
|
|
||||||
if (this.parent.getCoprocessorHost() != null) {
|
|
||||||
// TODO: Remove one of these
|
|
||||||
this.parent.getCoprocessorHost().preSplit();
|
|
||||||
this.parent.getCoprocessorHost().preSplit(this.splitrow);
|
|
||||||
}
|
|
||||||
|
|
||||||
journal.add(new JournalEntry(JournalEntryType.AFTER_PRE_SPLIT_HOOK));
|
|
||||||
|
|
||||||
// If true, no cluster to write meta edits to or to update znodes in.
|
|
||||||
boolean testing = server == null? true:
|
|
||||||
server.getConfiguration().getBoolean("hbase.testing.nocluster", false);
|
|
||||||
this.fileSplitTimeout = testing ? this.fileSplitTimeout :
|
|
||||||
server.getConfiguration().getLong("hbase.regionserver.fileSplitTimeout",
|
|
||||||
this.fileSplitTimeout);
|
|
||||||
|
|
||||||
PairOfSameType<Region> daughterRegions = stepsBeforePONR(server, services, testing);
|
|
||||||
|
|
||||||
List<Mutation> metaEntries = new ArrayList<Mutation>();
|
|
||||||
if (this.parent.getCoprocessorHost() != null) {
|
|
||||||
if (this.parent.getCoprocessorHost().
|
|
||||||
preSplitBeforePONR(this.splitrow, metaEntries)) {
|
|
||||||
throw new IOException("Coprocessor bypassing region "
|
|
||||||
+ this.parent.getRegionInfo().getRegionNameAsString() + " split.");
|
|
||||||
}
|
|
||||||
try {
|
|
||||||
for (Mutation p : metaEntries) {
|
|
||||||
HRegionInfo.parseRegionName(p.getRow());
|
|
||||||
}
|
|
||||||
} catch (IOException e) {
|
|
||||||
LOG.error("Row key of mutation from coprossor is not parsable as region name."
|
|
||||||
+ "Mutations from coprocessor should only for hbase:meta table.");
|
|
||||||
throw e;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// This is the point of no return. Adding subsequent edits to .META. as we
|
|
||||||
// do below when we do the daughter opens adding each to .META. can fail in
|
|
||||||
// various interesting ways the most interesting of which is a timeout
|
|
||||||
// BUT the edits all go through (See HBASE-3872). IF we reach the PONR
|
|
||||||
// then subsequent failures need to crash out this regionserver; the
|
|
||||||
// server shutdown processing should be able to fix-up the incomplete split.
|
|
||||||
// The offlined parent will have the daughters as extra columns. If
|
|
||||||
// we leave the daughter regions in place and do not remove them when we
|
|
||||||
// crash out, then they will have their references to the parent in place
|
|
||||||
// still and the server shutdown fixup of .META. will point to these
|
|
||||||
// regions.
|
|
||||||
// We should add PONR JournalEntry before offlineParentInMeta,so even if
|
|
||||||
// OfflineParentInMeta timeout,this will cause regionserver exit,and then
|
|
||||||
// master ServerShutdownHandler will fix daughter & avoid data loss. (See
|
|
||||||
// HBase-4562).
|
|
||||||
this.journal.add(new JournalEntry(JournalEntryType.PONR));
|
|
||||||
|
|
||||||
// Edit parent in meta. Offlines parent region and adds splita and splitb
|
|
||||||
// as an atomic update. See HBASE-7721. This update to META makes the region
|
|
||||||
// will determine whether the region is split or not in case of failures.
|
|
||||||
// If it is successful, master will roll-forward, if not, master will rollback
|
|
||||||
// and assign the parent region.
|
|
||||||
if (services != null && !services.reportRegionStateTransition(TransitionCode.SPLIT_PONR,
|
|
||||||
parent.getRegionInfo(), hri_a, hri_b)) {
|
|
||||||
// Passed PONR, let SSH clean it up
|
|
||||||
throw new IOException("Failed to notify master that split passed PONR: "
|
|
||||||
+ parent.getRegionInfo().getRegionNameAsString());
|
|
||||||
}
|
|
||||||
return daughterRegions;
|
|
||||||
}
|
|
||||||
|
|
||||||
public PairOfSameType<Region> stepsBeforePONR(final Server server,
|
|
||||||
final RegionServerServices services, boolean testing) throws IOException {
|
|
||||||
if (services != null && !services.reportRegionStateTransition(TransitionCode.READY_TO_SPLIT,
|
|
||||||
parent.getRegionInfo(), hri_a, hri_b)) {
|
|
||||||
throw new IOException("Failed to get ok from master to split "
|
|
||||||
+ parent.getRegionInfo().getRegionNameAsString());
|
|
||||||
}
|
|
||||||
this.journal.add(new JournalEntry(JournalEntryType.SET_SPLITTING));
|
|
||||||
|
|
||||||
this.parent.getRegionFileSystem().createSplitsDir();
|
|
||||||
this.journal.add(new JournalEntry(JournalEntryType.CREATE_SPLIT_DIR));
|
|
||||||
|
|
||||||
Map<byte[], List<StoreFile>> hstoreFilesToSplit = null;
|
|
||||||
Exception exceptionToThrow = null;
|
|
||||||
try{
|
|
||||||
hstoreFilesToSplit = this.parent.close(false);
|
|
||||||
} catch (Exception e) {
|
|
||||||
exceptionToThrow = e;
|
|
||||||
}
|
|
||||||
if (exceptionToThrow == null && hstoreFilesToSplit == null) {
|
|
||||||
// The region was closed by a concurrent thread. We can't continue
|
|
||||||
// with the split, instead we must just abandon the split. If we
|
|
||||||
// reopen or split this could cause problems because the region has
|
|
||||||
// probably already been moved to a different server, or is in the
|
|
||||||
// process of moving to a different server.
|
|
||||||
exceptionToThrow = closedByOtherException;
|
|
||||||
}
|
|
||||||
if (exceptionToThrow != closedByOtherException) {
|
|
||||||
this.journal.add(new JournalEntry(JournalEntryType.CLOSED_PARENT_REGION));
|
|
||||||
}
|
|
||||||
if (exceptionToThrow != null) {
|
|
||||||
if (exceptionToThrow instanceof IOException) throw (IOException)exceptionToThrow;
|
|
||||||
throw new IOException(exceptionToThrow);
|
|
||||||
}
|
|
||||||
if (!testing) {
|
|
||||||
services.removeFromOnlineRegions(this.parent, null);
|
|
||||||
}
|
|
||||||
this.journal.add(new JournalEntry(JournalEntryType.OFFLINED_PARENT));
|
|
||||||
|
|
||||||
// TODO: If splitStoreFiles were multithreaded would we complete steps in
|
|
||||||
// less elapsed time? St.Ack 20100920
|
|
||||||
//
|
|
||||||
// splitStoreFiles creates daughter region dirs under the parent splits dir
|
|
||||||
// Nothing to unroll here if failure -- clean up of CREATE_SPLIT_DIR will
|
|
||||||
// clean this up.
|
|
||||||
Pair<Integer, Integer> expectedReferences = splitStoreFiles(hstoreFilesToSplit);
|
|
||||||
|
|
||||||
// Log to the journal that we are creating region A, the first daughter
|
|
||||||
// region. We could fail halfway through. If we do, we could have left
|
|
||||||
// stuff in fs that needs cleanup -- a storefile or two. Thats why we
|
|
||||||
// add entry to journal BEFORE rather than AFTER the change.
|
|
||||||
this.journal.add(new JournalEntry(JournalEntryType.STARTED_REGION_A_CREATION));
|
|
||||||
assertReferenceFileCount(expectedReferences.getFirst(),
|
|
||||||
this.parent.getRegionFileSystem().getSplitsDir(this.hri_a));
|
|
||||||
Region a = this.parent.createDaughterRegionFromSplits(this.hri_a);
|
|
||||||
assertReferenceFileCount(expectedReferences.getFirst(),
|
|
||||||
new Path(this.parent.getRegionFileSystem().getTableDir(), this.hri_a.getEncodedName()));
|
|
||||||
|
|
||||||
// Ditto
|
|
||||||
this.journal.add(new JournalEntry(JournalEntryType.STARTED_REGION_B_CREATION));
|
|
||||||
assertReferenceFileCount(expectedReferences.getSecond(),
|
|
||||||
this.parent.getRegionFileSystem().getSplitsDir(this.hri_b));
|
|
||||||
Region b = this.parent.createDaughterRegionFromSplits(this.hri_b);
|
|
||||||
assertReferenceFileCount(expectedReferences.getSecond(),
|
|
||||||
new Path(this.parent.getRegionFileSystem().getTableDir(), this.hri_b.getEncodedName()));
|
|
||||||
|
|
||||||
return new PairOfSameType<Region>(a, b);
|
|
||||||
}
|
|
||||||
|
|
||||||
void assertReferenceFileCount(int expectedReferenceFileCount, Path dir)
|
|
||||||
throws IOException {
|
|
||||||
if (expectedReferenceFileCount != 0 &&
|
|
||||||
expectedReferenceFileCount != FSUtils.getRegionReferenceFileCount(this.parent.getFilesystem(), dir)) {
|
|
||||||
throw new IOException("Failing split. Expected reference file count isn't equal.");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Perform time consuming opening of the daughter regions.
|
|
||||||
* @param server Hosting server instance. Can be null when testing
|
|
||||||
* @param services Used to online/offline regions.
|
|
||||||
* @param a first daughter region
|
|
||||||
* @param a second daughter region
|
|
||||||
* @throws IOException If thrown, transaction failed.
|
|
||||||
* Call {@link #rollback(Server, RegionServerServices)}
|
|
||||||
*/
|
|
||||||
/* package */void openDaughters(final Server server,
|
|
||||||
final RegionServerServices services, Region a, Region b)
|
|
||||||
throws IOException {
|
|
||||||
boolean stopped = server != null && server.isStopped();
|
|
||||||
boolean stopping = services != null && services.isStopping();
|
|
||||||
// TODO: Is this check needed here?
|
|
||||||
if (stopped || stopping) {
|
|
||||||
LOG.info("Not opening daughters " +
|
|
||||||
b.getRegionInfo().getRegionNameAsString() +
|
|
||||||
" and " +
|
|
||||||
a.getRegionInfo().getRegionNameAsString() +
|
|
||||||
" because stopping=" + stopping + ", stopped=" + stopped);
|
|
||||||
} else {
|
|
||||||
// Open daughters in parallel.
|
|
||||||
DaughterOpener aOpener = new DaughterOpener(server, (HRegion)a);
|
|
||||||
DaughterOpener bOpener = new DaughterOpener(server, (HRegion)b);
|
|
||||||
aOpener.start();
|
|
||||||
bOpener.start();
|
|
||||||
try {
|
|
||||||
aOpener.join();
|
|
||||||
if (aOpener.getException() == null) {
|
|
||||||
journal.add(new JournalEntry(JournalEntryType.OPENED_REGION_A));
|
|
||||||
}
|
|
||||||
bOpener.join();
|
|
||||||
if (bOpener.getException() == null) {
|
|
||||||
journal.add(new JournalEntry(JournalEntryType.OPENED_REGION_B));
|
|
||||||
}
|
|
||||||
} catch (InterruptedException e) {
|
|
||||||
throw (InterruptedIOException)new InterruptedIOException().initCause(e);
|
|
||||||
}
|
|
||||||
if (aOpener.getException() != null) {
|
|
||||||
throw new IOException("Failed " +
|
|
||||||
aOpener.getName(), aOpener.getException());
|
|
||||||
}
|
|
||||||
if (bOpener.getException() != null) {
|
|
||||||
throw new IOException("Failed " +
|
|
||||||
bOpener.getName(), bOpener.getException());
|
|
||||||
}
|
|
||||||
if (services != null) {
|
|
||||||
if (!services.reportRegionStateTransition(TransitionCode.SPLIT,
|
|
||||||
parent.getRegionInfo(), hri_a, hri_b)) {
|
|
||||||
throw new IOException("Failed to report split region to master: "
|
|
||||||
+ parent.getRegionInfo().getShortNameToLog());
|
|
||||||
}
|
|
||||||
// Should add it to OnlineRegions
|
|
||||||
services.addToOnlineRegions(b);
|
|
||||||
services.addToOnlineRegions(a);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Run the transaction.
|
* Run the transaction.
|
||||||
* @param server Hosting server instance. Can be null when testing
|
* @param server Hosting server instance. Can be null when testing.
|
||||||
* @param services Used to online/offline regions.
|
* @param services Used to online/offline regions.
|
||||||
* @throws IOException If thrown, transaction failed.
|
* @throws IOException If thrown, transaction failed.
|
||||||
* Call {@link #rollback(Server, RegionServerServices)}
|
* Call {@link #rollback(Server, RegionServerServices)}
|
||||||
|
@ -485,325 +192,44 @@ public class SplitTransaction {
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
* @see #rollback(Server, RegionServerServices)
|
* @see #rollback(Server, RegionServerServices)
|
||||||
*/
|
*/
|
||||||
public PairOfSameType<Region> execute(final Server server,
|
PairOfSameType<Region> execute(Server server, RegionServerServices services) throws IOException;
|
||||||
final RegionServerServices services)
|
|
||||||
throws IOException {
|
|
||||||
PairOfSameType<Region> regions = createDaughters(server, services);
|
|
||||||
if (this.parent.getCoprocessorHost() != null) {
|
|
||||||
this.parent.getCoprocessorHost().preSplitAfterPONR();
|
|
||||||
}
|
|
||||||
return stepsAfterPONR(server, services, regions);
|
|
||||||
}
|
|
||||||
|
|
||||||
public PairOfSameType<Region> stepsAfterPONR(final Server server,
|
|
||||||
final RegionServerServices services, PairOfSameType<Region> regions)
|
|
||||||
throws IOException {
|
|
||||||
openDaughters(server, services, regions.getFirst(), regions.getSecond());
|
|
||||||
journal.add(new JournalEntry(JournalEntryType.BEFORE_POST_SPLIT_HOOK));
|
|
||||||
// Coprocessor callback
|
|
||||||
if (parent.getCoprocessorHost() != null) {
|
|
||||||
parent.getCoprocessorHost().postSplit(regions.getFirst(), regions.getSecond());
|
|
||||||
}
|
|
||||||
journal.add(new JournalEntry(JournalEntryType.AFTER_POST_SPLIT_HOOK));
|
|
||||||
return regions;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Put addLocation(final Put p, final ServerName sn, long openSeqNum) {
|
|
||||||
p.addImmutable(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
|
|
||||||
Bytes.toBytes(sn.getHostAndPort()));
|
|
||||||
p.addImmutable(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
|
|
||||||
Bytes.toBytes(sn.getStartcode()));
|
|
||||||
p.addImmutable(HConstants.CATALOG_FAMILY, HConstants.SEQNUM_QUALIFIER,
|
|
||||||
Bytes.toBytes(openSeqNum));
|
|
||||||
return p;
|
|
||||||
}
|
|
||||||
|
|
||||||
/*
|
|
||||||
* Open daughter region in its own thread.
|
|
||||||
* If we fail, abort this hosting server.
|
|
||||||
*/
|
|
||||||
class DaughterOpener extends HasThread {
|
|
||||||
private final Server server;
|
|
||||||
private final HRegion r;
|
|
||||||
private Throwable t = null;
|
|
||||||
|
|
||||||
DaughterOpener(final Server s, final HRegion r) {
|
|
||||||
super((s == null? "null-services": s.getServerName()) +
|
|
||||||
"-daughterOpener=" + r.getRegionInfo().getEncodedName());
|
|
||||||
setDaemon(true);
|
|
||||||
this.server = s;
|
|
||||||
this.r = r;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return Null if open succeeded else exception that causes us fail open.
|
|
||||||
* Call it after this thread exits else you may get wrong view on result.
|
|
||||||
*/
|
|
||||||
Throwable getException() {
|
|
||||||
return this.t;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void run() {
|
|
||||||
try {
|
|
||||||
openDaughterRegion(this.server, r);
|
|
||||||
} catch (Throwable t) {
|
|
||||||
this.t = t;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Open daughter regions, add them to online list and update meta.
|
|
||||||
* @param server
|
|
||||||
* @param daughter
|
|
||||||
* @throws IOException
|
|
||||||
* @throws KeeperException
|
|
||||||
*/
|
|
||||||
void openDaughterRegion(final Server server, final HRegion daughter)
|
|
||||||
throws IOException, KeeperException {
|
|
||||||
HRegionInfo hri = daughter.getRegionInfo();
|
|
||||||
LoggingProgressable reporter = server == null ? null
|
|
||||||
: new LoggingProgressable(hri, server.getConfiguration().getLong(
|
|
||||||
"hbase.regionserver.split.daughter.open.log.interval", 10000));
|
|
||||||
daughter.openHRegion(reporter);
|
|
||||||
}
|
|
||||||
|
|
||||||
static class LoggingProgressable implements CancelableProgressable {
|
|
||||||
private final HRegionInfo hri;
|
|
||||||
private long lastLog = -1;
|
|
||||||
private final long interval;
|
|
||||||
|
|
||||||
LoggingProgressable(final HRegionInfo hri, final long interval) {
|
|
||||||
this.hri = hri;
|
|
||||||
this.interval = interval;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean progress() {
|
|
||||||
long now = EnvironmentEdgeManager.currentTime();
|
|
||||||
if (now - lastLog > this.interval) {
|
|
||||||
LOG.info("Opening " + this.hri.getRegionNameAsString());
|
|
||||||
this.lastLog = now;
|
|
||||||
}
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Creates reference files for top and bottom half of the
|
|
||||||
* @param hstoreFilesToSplit map of store files to create half file references for.
|
|
||||||
* @return the number of reference files that were created.
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
private Pair<Integer, Integer> splitStoreFiles(
|
|
||||||
final Map<byte[], List<StoreFile>> hstoreFilesToSplit)
|
|
||||||
throws IOException {
|
|
||||||
if (hstoreFilesToSplit == null) {
|
|
||||||
// Could be null because close didn't succeed -- for now consider it fatal
|
|
||||||
throw new IOException("Close returned empty list of StoreFiles");
|
|
||||||
}
|
|
||||||
// The following code sets up a thread pool executor with as many slots as
|
|
||||||
// there's files to split. It then fires up everything, waits for
|
|
||||||
// completion and finally checks for any exception
|
|
||||||
int nbFiles = hstoreFilesToSplit.size();
|
|
||||||
if (nbFiles == 0) {
|
|
||||||
// no file needs to be splitted.
|
|
||||||
return new Pair<Integer, Integer>(0,0);
|
|
||||||
}
|
|
||||||
LOG.info("Preparing to split " + nbFiles + " storefiles for region " + this.parent);
|
|
||||||
ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
|
|
||||||
builder.setNameFormat("StoreFileSplitter-%1$d");
|
|
||||||
ThreadFactory factory = builder.build();
|
|
||||||
ThreadPoolExecutor threadPool =
|
|
||||||
(ThreadPoolExecutor) Executors.newFixedThreadPool(nbFiles, factory);
|
|
||||||
List<Future<Pair<Path,Path>>> futures = new ArrayList<Future<Pair<Path,Path>>> (nbFiles);
|
|
||||||
|
|
||||||
// Split each store file.
|
|
||||||
for (Map.Entry<byte[], List<StoreFile>> entry: hstoreFilesToSplit.entrySet()) {
|
|
||||||
for (StoreFile sf: entry.getValue()) {
|
|
||||||
StoreFileSplitter sfs = new StoreFileSplitter(entry.getKey(), sf);
|
|
||||||
futures.add(threadPool.submit(sfs));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// Shutdown the pool
|
|
||||||
threadPool.shutdown();
|
|
||||||
|
|
||||||
// Wait for all the tasks to finish
|
|
||||||
try {
|
|
||||||
boolean stillRunning = !threadPool.awaitTermination(
|
|
||||||
this.fileSplitTimeout, TimeUnit.MILLISECONDS);
|
|
||||||
if (stillRunning) {
|
|
||||||
threadPool.shutdownNow();
|
|
||||||
// wait for the thread to shutdown completely.
|
|
||||||
while (!threadPool.isTerminated()) {
|
|
||||||
Thread.sleep(50);
|
|
||||||
}
|
|
||||||
throw new IOException("Took too long to split the" +
|
|
||||||
" files and create the references, aborting split");
|
|
||||||
}
|
|
||||||
} catch (InterruptedException e) {
|
|
||||||
throw (InterruptedIOException)new InterruptedIOException().initCause(e);
|
|
||||||
}
|
|
||||||
|
|
||||||
int created_a = 0;
|
|
||||||
int created_b = 0;
|
|
||||||
// Look for any exception
|
|
||||||
for (Future<Pair<Path, Path>> future : futures) {
|
|
||||||
try {
|
|
||||||
Pair<Path, Path> p = future.get();
|
|
||||||
created_a += p.getFirst() != null ? 1 : 0;
|
|
||||||
created_b += p.getSecond() != null ? 1 : 0;
|
|
||||||
} catch (InterruptedException e) {
|
|
||||||
throw (InterruptedIOException) new InterruptedIOException().initCause(e);
|
|
||||||
} catch (ExecutionException e) {
|
|
||||||
throw new IOException(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (LOG.isDebugEnabled()) {
|
|
||||||
LOG.debug("Split storefiles for region " + this.parent + " Daugther A: " + created_a
|
|
||||||
+ " storefiles, Daugther B: " + created_b + " storefiles.");
|
|
||||||
}
|
|
||||||
return new Pair<Integer, Integer>(created_a, created_b);
|
|
||||||
}
|
|
||||||
|
|
||||||
private Pair<Path, Path> splitStoreFile(final byte[] family, final StoreFile sf) throws IOException {
|
|
||||||
HRegionFileSystem fs = this.parent.getRegionFileSystem();
|
|
||||||
String familyName = Bytes.toString(family);
|
|
||||||
Path path_a =
|
|
||||||
fs.splitStoreFile(this.hri_a, familyName, sf, this.splitrow, false,
|
|
||||||
this.parent.getSplitPolicy());
|
|
||||||
Path path_b =
|
|
||||||
fs.splitStoreFile(this.hri_b, familyName, sf, this.splitrow, true,
|
|
||||||
this.parent.getSplitPolicy());
|
|
||||||
return new Pair<Path,Path>(path_a, path_b);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Utility class used to do the file splitting / reference writing
|
|
||||||
* in parallel instead of sequentially.
|
|
||||||
*/
|
|
||||||
class StoreFileSplitter implements Callable<Pair<Path,Path>> {
|
|
||||||
private final byte[] family;
|
|
||||||
private final StoreFile sf;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Constructor that takes what it needs to split
|
|
||||||
* @param family Family that contains the store file
|
|
||||||
* @param sf which file
|
|
||||||
*/
|
|
||||||
public StoreFileSplitter(final byte[] family, final StoreFile sf) {
|
|
||||||
this.sf = sf;
|
|
||||||
this.family = family;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Pair<Path,Path> call() throws IOException {
|
|
||||||
return splitStoreFile(family, sf);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
* Roll back a failed transaction
|
||||||
* @param server Hosting server instance (May be null when testing).
|
* @param server Hosting server instance (May be null when testing).
|
||||||
* @param services
|
* @param services
|
||||||
* @throws IOException If thrown, rollback failed. Take drastic action.
|
* @throws IOException If thrown, rollback failed. Take drastic action.
|
||||||
* @return True if we successfully rolled back, false if we got to the point
|
* @return True if we successfully rolled back, false if we got to the point
|
||||||
* of no return and so now need to abort the server to minimize damage.
|
* of no return and so now need to abort the server to minimize damage.
|
||||||
*/
|
*/
|
||||||
@SuppressWarnings("deprecation")
|
boolean rollback(Server server, RegionServerServices services) throws IOException;
|
||||||
public boolean rollback(final Server server, final RegionServerServices services)
|
|
||||||
throws IOException {
|
|
||||||
// Coprocessor callback
|
|
||||||
if (this.parent.getCoprocessorHost() != null) {
|
|
||||||
this.parent.getCoprocessorHost().preRollBackSplit();
|
|
||||||
}
|
|
||||||
|
|
||||||
boolean result = true;
|
/**
|
||||||
ListIterator<JournalEntry> iterator =
|
* Register a listener for transaction preparation, execution, and possibly
|
||||||
this.journal.listIterator(this.journal.size());
|
* rollback phases.
|
||||||
// Iterate in reverse.
|
* <p>A listener can abort a transaction by throwing an exception.
|
||||||
while (iterator.hasPrevious()) {
|
* @param listener the listener
|
||||||
JournalEntry je = iterator.previous();
|
* @return 'this' for chaining
|
||||||
switch(je.type) {
|
*/
|
||||||
|
SplitTransaction registerTransactionListener(TransactionListener listener);
|
||||||
|
|
||||||
case SET_SPLITTING:
|
/**
|
||||||
if (services != null
|
* Get the journal for the transaction.
|
||||||
&& !services.reportRegionStateTransition(TransitionCode.SPLIT_REVERTED,
|
* <p>Journal entries are an opaque type represented as JournalEntry. They can
|
||||||
parent.getRegionInfo(), hri_a, hri_b)) {
|
* also provide useful debugging information via their toString method.
|
||||||
return false;
|
* @return the transaction journal
|
||||||
}
|
*/
|
||||||
break;
|
List<JournalEntry> getJournal();
|
||||||
|
|
||||||
case CREATE_SPLIT_DIR:
|
/**
|
||||||
this.parent.writestate.writesEnabled = true;
|
* Get the Server running the transaction or rollback
|
||||||
this.parent.getRegionFileSystem().cleanupSplitsDir();
|
* @return server instance
|
||||||
break;
|
*/
|
||||||
|
Server getServer();
|
||||||
|
|
||||||
case CLOSED_PARENT_REGION:
|
/**
|
||||||
try {
|
* Get the RegonServerServices of the server running the transaction or rollback
|
||||||
// So, this returns a seqid but if we just closed and then reopened, we
|
* @return region server services
|
||||||
// should be ok. On close, we flushed using sequenceid obtained from
|
*/
|
||||||
// hosting regionserver so no need to propagate the sequenceid returned
|
RegionServerServices getRegionServerServices();
|
||||||
// out of initialize below up into regionserver as we normally do.
|
|
||||||
// TODO: Verify.
|
|
||||||
this.parent.initialize();
|
|
||||||
} catch (IOException e) {
|
|
||||||
LOG.error("Failed rollbacking CLOSED_PARENT_REGION of region " +
|
|
||||||
this.parent.getRegionInfo().getRegionNameAsString(), e);
|
|
||||||
throw new RuntimeException(e);
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
|
|
||||||
case STARTED_REGION_A_CREATION:
|
|
||||||
this.parent.getRegionFileSystem().cleanupDaughterRegion(this.hri_a);
|
|
||||||
break;
|
|
||||||
|
|
||||||
case STARTED_REGION_B_CREATION:
|
|
||||||
this.parent.getRegionFileSystem().cleanupDaughterRegion(this.hri_b);
|
|
||||||
break;
|
|
||||||
|
|
||||||
case OFFLINED_PARENT:
|
|
||||||
if (services != null) services.addToOnlineRegions(this.parent);
|
|
||||||
break;
|
|
||||||
|
|
||||||
case PONR:
|
|
||||||
// We got to the point-of-no-return so we need to just abort. Return
|
|
||||||
// immediately. Do not clean up created daughter regions. They need
|
|
||||||
// to be in place so we don't delete the parent region mistakenly.
|
|
||||||
// See HBASE-3872.
|
|
||||||
return false;
|
|
||||||
|
|
||||||
// Informational only cases
|
|
||||||
case STARTED:
|
|
||||||
case PREPARED:
|
|
||||||
case BEFORE_PRE_SPLIT_HOOK:
|
|
||||||
case AFTER_PRE_SPLIT_HOOK:
|
|
||||||
case BEFORE_POST_SPLIT_HOOK:
|
|
||||||
case AFTER_POST_SPLIT_HOOK:
|
|
||||||
case OPENED_REGION_A:
|
|
||||||
case OPENED_REGION_B:
|
|
||||||
break;
|
|
||||||
|
|
||||||
default:
|
|
||||||
throw new RuntimeException("Unhandled journal entry: " + je);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// Coprocessor callback
|
|
||||||
if (this.parent.getCoprocessorHost() != null) {
|
|
||||||
this.parent.getCoprocessorHost().postRollBackSplit();
|
|
||||||
}
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
HRegionInfo getFirstDaughter() {
|
|
||||||
return hri_a;
|
|
||||||
}
|
|
||||||
|
|
||||||
HRegionInfo getSecondDaughter() {
|
|
||||||
return hri_b;
|
|
||||||
}
|
|
||||||
|
|
||||||
List<JournalEntry> getJournal() {
|
|
||||||
return journal;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,74 @@
|
||||||
|
/*
|
||||||
|
* 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.regionserver;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configurable;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||||
|
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A factory for creating SplitTransactions, which execute region split as a "transaction".
|
||||||
|
* See {@link SplitTransaction}
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||||
|
@InterfaceStability.Evolving
|
||||||
|
public class SplitTransactionFactory implements Configurable {
|
||||||
|
|
||||||
|
public static final String SPLIT_TRANSACTION_IMPL_KEY =
|
||||||
|
"hbase.regionserver.split.transaction.impl";
|
||||||
|
|
||||||
|
private Configuration conf;
|
||||||
|
|
||||||
|
public SplitTransactionFactory(Configuration conf) {
|
||||||
|
this.conf = conf;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Configuration getConf() {
|
||||||
|
return conf;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setConf(Configuration conf) {
|
||||||
|
this.conf = conf;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a split transaction
|
||||||
|
* @param r the region to split
|
||||||
|
* @param splitrow the split point in the keyspace
|
||||||
|
* @return transaction instance
|
||||||
|
*/
|
||||||
|
public SplitTransaction create(final Region r, final byte [] splitrow) {
|
||||||
|
return ReflectionUtils.instantiateWithCustomCtor(
|
||||||
|
// The implementation class must extend SplitTransactionImpl, not only
|
||||||
|
// implement the SplitTransaction interface like you might expect,
|
||||||
|
// because various places such as AssignmentManager use static methods
|
||||||
|
// from SplitTransactionImpl. Whatever we use for implementation must
|
||||||
|
// be compatible, so it's safest to require ? extends SplitTransactionImpl.
|
||||||
|
// If not compatible we will throw a runtime exception from here.
|
||||||
|
conf.getClass(SPLIT_TRANSACTION_IMPL_KEY, SplitTransactionImpl.class,
|
||||||
|
SplitTransactionImpl.class).getName(),
|
||||||
|
new Class[] { Region.class, byte[].class },
|
||||||
|
new Object[] { r, splitrow });
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,789 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.regionserver;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.InterruptedIOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.ListIterator;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.concurrent.Callable;
|
||||||
|
import java.util.concurrent.ExecutionException;
|
||||||
|
import java.util.concurrent.Executors;
|
||||||
|
import java.util.concurrent.Future;
|
||||||
|
import java.util.concurrent.ThreadFactory;
|
||||||
|
import java.util.concurrent.ThreadPoolExecutor;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
|
import org.apache.hadoop.hbase.Server;
|
||||||
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
|
import org.apache.hadoop.hbase.client.Mutation;
|
||||||
|
import org.apache.hadoop.hbase.client.Put;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
|
||||||
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.hadoop.hbase.util.CancelableProgressable;
|
||||||
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
|
import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
|
import org.apache.hadoop.hbase.util.HasThread;
|
||||||
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
|
import org.apache.hadoop.hbase.util.PairOfSameType;
|
||||||
|
import org.apache.zookeeper.KeeperException;
|
||||||
|
|
||||||
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||||
|
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public class SplitTransactionImpl implements SplitTransaction {
|
||||||
|
private static final Log LOG = LogFactory.getLog(SplitTransactionImpl.class);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Region to split
|
||||||
|
*/
|
||||||
|
private final HRegion parent;
|
||||||
|
private HRegionInfo hri_a;
|
||||||
|
private HRegionInfo hri_b;
|
||||||
|
private long fileSplitTimeout = 30000;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Row to split around
|
||||||
|
*/
|
||||||
|
private final byte [] splitrow;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Transaction state for listener, only valid during execute and
|
||||||
|
* rollback
|
||||||
|
*/
|
||||||
|
private SplitTransactionPhase currentPhase = SplitTransactionPhase.STARTED;
|
||||||
|
private Server server;
|
||||||
|
private RegionServerServices rsServices;
|
||||||
|
|
||||||
|
public static class JournalEntryImpl implements JournalEntry {
|
||||||
|
private SplitTransactionPhase type;
|
||||||
|
private long timestamp;
|
||||||
|
|
||||||
|
public JournalEntryImpl(SplitTransactionPhase type) {
|
||||||
|
this(type, EnvironmentEdgeManager.currentTime());
|
||||||
|
}
|
||||||
|
|
||||||
|
public JournalEntryImpl(SplitTransactionPhase type, long timestamp) {
|
||||||
|
this.type = type;
|
||||||
|
this.timestamp = timestamp;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
StringBuilder sb = new StringBuilder();
|
||||||
|
sb.append(type);
|
||||||
|
sb.append(" at ");
|
||||||
|
sb.append(timestamp);
|
||||||
|
return sb.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public SplitTransactionPhase getPhase() {
|
||||||
|
return type;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getTimeStamp() {
|
||||||
|
return timestamp;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Journal of how far the split transaction has progressed.
|
||||||
|
*/
|
||||||
|
private final ArrayList<JournalEntry> journal = new ArrayList<JournalEntry>();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Listeners
|
||||||
|
*/
|
||||||
|
private final ArrayList<TransactionListener> listeners = new ArrayList<TransactionListener>();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructor
|
||||||
|
* @param r Region to split
|
||||||
|
* @param splitrow Row to split around
|
||||||
|
*/
|
||||||
|
public SplitTransactionImpl(final Region r, final byte [] splitrow) {
|
||||||
|
this.parent = (HRegion)r;
|
||||||
|
this.splitrow = splitrow;
|
||||||
|
this.journal.add(new JournalEntryImpl(SplitTransactionPhase.STARTED));
|
||||||
|
}
|
||||||
|
|
||||||
|
private void transition(SplitTransactionPhase nextPhase) throws IOException {
|
||||||
|
transition(nextPhase, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void transition(SplitTransactionPhase nextPhase, boolean isRollback)
|
||||||
|
throws IOException {
|
||||||
|
if (!isRollback) {
|
||||||
|
// Add to the journal first, because if the listener throws an exception
|
||||||
|
// we need to roll back starting at 'nextPhase'
|
||||||
|
this.journal.add(new JournalEntryImpl(nextPhase));
|
||||||
|
}
|
||||||
|
for (int i = 0; i < listeners.size(); i++) {
|
||||||
|
TransactionListener listener = listeners.get(i);
|
||||||
|
if (!isRollback) {
|
||||||
|
listener.transition(this, currentPhase, nextPhase);
|
||||||
|
} else {
|
||||||
|
listener.rollback(this, currentPhase, nextPhase);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
currentPhase = nextPhase;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean prepare() throws IOException {
|
||||||
|
if (!this.parent.isSplittable()) return false;
|
||||||
|
// Split key can be null if this region is unsplittable; i.e. has refs.
|
||||||
|
if (this.splitrow == null) return false;
|
||||||
|
HRegionInfo hri = this.parent.getRegionInfo();
|
||||||
|
parent.prepareToSplit();
|
||||||
|
// Check splitrow.
|
||||||
|
byte [] startKey = hri.getStartKey();
|
||||||
|
byte [] endKey = hri.getEndKey();
|
||||||
|
if (Bytes.equals(startKey, splitrow) ||
|
||||||
|
!this.parent.getRegionInfo().containsRow(splitrow)) {
|
||||||
|
LOG.info("Split row is not inside region key range or is equal to " +
|
||||||
|
"startkey: " + Bytes.toStringBinary(this.splitrow));
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
long rid = getDaughterRegionIdTimestamp(hri);
|
||||||
|
this.hri_a = new HRegionInfo(hri.getTable(), startKey, this.splitrow, false, rid);
|
||||||
|
this.hri_b = new HRegionInfo(hri.getTable(), this.splitrow, endKey, false, rid);
|
||||||
|
|
||||||
|
transition(SplitTransactionPhase.PREPARED);
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Calculate daughter regionid to use.
|
||||||
|
* @param hri Parent {@link HRegionInfo}
|
||||||
|
* @return Daughter region id (timestamp) to use.
|
||||||
|
*/
|
||||||
|
private static long getDaughterRegionIdTimestamp(final HRegionInfo hri) {
|
||||||
|
long rid = EnvironmentEdgeManager.currentTime();
|
||||||
|
// Regionid is timestamp. Can't be less than that of parent else will insert
|
||||||
|
// at wrong location in hbase:meta (See HBASE-710).
|
||||||
|
if (rid < hri.getRegionId()) {
|
||||||
|
LOG.warn("Clock skew; parent regions id is " + hri.getRegionId() +
|
||||||
|
" but current time here is " + rid);
|
||||||
|
rid = hri.getRegionId() + 1;
|
||||||
|
}
|
||||||
|
return rid;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static IOException closedByOtherException = new IOException(
|
||||||
|
"Failed to close region: already closed by another thread");
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Prepare the regions and region files.
|
||||||
|
* @param server Hosting server instance. Can be null when testing (won't try
|
||||||
|
* and update in zk if a null server)
|
||||||
|
* @param services Used to online/offline regions.
|
||||||
|
* @throws IOException If thrown, transaction failed.
|
||||||
|
* Call {@link #rollback(Server, RegionServerServices)}
|
||||||
|
* @return Regions created
|
||||||
|
*/
|
||||||
|
@VisibleForTesting
|
||||||
|
PairOfSameType<Region> createDaughters(final Server server,
|
||||||
|
final RegionServerServices services) throws IOException {
|
||||||
|
LOG.info("Starting split of region " + this.parent);
|
||||||
|
if ((server != null && server.isStopped()) ||
|
||||||
|
(services != null && services.isStopping())) {
|
||||||
|
throw new IOException("Server is stopped or stopping");
|
||||||
|
}
|
||||||
|
assert !this.parent.lock.writeLock().isHeldByCurrentThread():
|
||||||
|
"Unsafe to hold write lock while performing RPCs";
|
||||||
|
|
||||||
|
transition(SplitTransactionPhase.BEFORE_PRE_SPLIT_HOOK);
|
||||||
|
|
||||||
|
// Coprocessor callback
|
||||||
|
if (this.parent.getCoprocessorHost() != null) {
|
||||||
|
// TODO: Remove one of these
|
||||||
|
this.parent.getCoprocessorHost().preSplit();
|
||||||
|
this.parent.getCoprocessorHost().preSplit(this.splitrow);
|
||||||
|
}
|
||||||
|
|
||||||
|
transition(SplitTransactionPhase.AFTER_PRE_SPLIT_HOOK);
|
||||||
|
|
||||||
|
// If true, no cluster to write meta edits to or to update znodes in.
|
||||||
|
boolean testing = server == null? true:
|
||||||
|
server.getConfiguration().getBoolean("hbase.testing.nocluster", false);
|
||||||
|
this.fileSplitTimeout = testing ? this.fileSplitTimeout :
|
||||||
|
server.getConfiguration().getLong("hbase.regionserver.fileSplitTimeout",
|
||||||
|
this.fileSplitTimeout);
|
||||||
|
|
||||||
|
PairOfSameType<Region> daughterRegions = stepsBeforePONR(server, services, testing);
|
||||||
|
|
||||||
|
List<Mutation> metaEntries = new ArrayList<Mutation>();
|
||||||
|
if (this.parent.getCoprocessorHost() != null) {
|
||||||
|
if (this.parent.getCoprocessorHost().
|
||||||
|
preSplitBeforePONR(this.splitrow, metaEntries)) {
|
||||||
|
throw new IOException("Coprocessor bypassing region "
|
||||||
|
+ parent.getRegionInfo().getRegionNameAsString() + " split.");
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
for (Mutation p : metaEntries) {
|
||||||
|
HRegionInfo.parseRegionName(p.getRow());
|
||||||
|
}
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.error("Row key of mutation from coprossor is not parsable as region name."
|
||||||
|
+ "Mutations from coprocessor should only for hbase:meta table.");
|
||||||
|
throw e;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// This is the point of no return. Adding subsequent edits to .META. as we
|
||||||
|
// do below when we do the daughter opens adding each to .META. can fail in
|
||||||
|
// various interesting ways the most interesting of which is a timeout
|
||||||
|
// BUT the edits all go through (See HBASE-3872). IF we reach the PONR
|
||||||
|
// then subsequent failures need to crash out this regionserver; the
|
||||||
|
// server shutdown processing should be able to fix-up the incomplete split.
|
||||||
|
// The offlined parent will have the daughters as extra columns. If
|
||||||
|
// we leave the daughter regions in place and do not remove them when we
|
||||||
|
// crash out, then they will have their references to the parent in place
|
||||||
|
// still and the server shutdown fixup of .META. will point to these
|
||||||
|
// regions.
|
||||||
|
// We should add PONR JournalEntry before offlineParentInMeta,so even if
|
||||||
|
// OfflineParentInMeta timeout,this will cause regionserver exit,and then
|
||||||
|
// master ServerShutdownHandler will fix daughter & avoid data loss. (See
|
||||||
|
// HBase-4562).
|
||||||
|
|
||||||
|
transition(SplitTransactionPhase.PONR);
|
||||||
|
|
||||||
|
// Edit parent in meta. Offlines parent region and adds splita and splitb
|
||||||
|
// as an atomic update. See HBASE-7721. This update to META makes the region
|
||||||
|
// will determine whether the region is split or not in case of failures.
|
||||||
|
// If it is successful, master will roll-forward, if not, master will rollback
|
||||||
|
// and assign the parent region.
|
||||||
|
if (services != null && !services.reportRegionStateTransition(TransitionCode.SPLIT_PONR,
|
||||||
|
parent.getRegionInfo(), hri_a, hri_b)) {
|
||||||
|
// Passed PONR, let SSH clean it up
|
||||||
|
throw new IOException("Failed to notify master that split passed PONR: "
|
||||||
|
+ parent.getRegionInfo().getRegionNameAsString());
|
||||||
|
}
|
||||||
|
return daughterRegions;
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
Put addLocation(final Put p, final ServerName sn, long openSeqNum) {
|
||||||
|
p.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER, Bytes
|
||||||
|
.toBytes(sn.getHostAndPort()));
|
||||||
|
p.add(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER, Bytes.toBytes(sn
|
||||||
|
.getStartcode()));
|
||||||
|
p.add(HConstants.CATALOG_FAMILY, HConstants.SEQNUM_QUALIFIER, Bytes.toBytes(openSeqNum));
|
||||||
|
return p;
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
public PairOfSameType<Region> stepsBeforePONR(final Server server,
|
||||||
|
final RegionServerServices services, boolean testing) throws IOException {
|
||||||
|
if (services != null && !services.reportRegionStateTransition(TransitionCode.READY_TO_SPLIT,
|
||||||
|
parent.getRegionInfo(), hri_a, hri_b)) {
|
||||||
|
throw new IOException("Failed to get ok from master to split "
|
||||||
|
+ parent.getRegionInfo().getRegionNameAsString());
|
||||||
|
}
|
||||||
|
|
||||||
|
transition(SplitTransactionPhase.SET_SPLITTING);
|
||||||
|
|
||||||
|
this.parent.getRegionFileSystem().createSplitsDir();
|
||||||
|
|
||||||
|
transition(SplitTransactionPhase.CREATE_SPLIT_DIR);
|
||||||
|
|
||||||
|
Map<byte[], List<StoreFile>> hstoreFilesToSplit = null;
|
||||||
|
Exception exceptionToThrow = null;
|
||||||
|
try{
|
||||||
|
hstoreFilesToSplit = this.parent.close(false);
|
||||||
|
} catch (Exception e) {
|
||||||
|
exceptionToThrow = e;
|
||||||
|
}
|
||||||
|
if (exceptionToThrow == null && hstoreFilesToSplit == null) {
|
||||||
|
// The region was closed by a concurrent thread. We can't continue
|
||||||
|
// with the split, instead we must just abandon the split. If we
|
||||||
|
// reopen or split this could cause problems because the region has
|
||||||
|
// probably already been moved to a different server, or is in the
|
||||||
|
// process of moving to a different server.
|
||||||
|
exceptionToThrow = closedByOtherException;
|
||||||
|
}
|
||||||
|
if (exceptionToThrow != closedByOtherException) {
|
||||||
|
transition(SplitTransactionPhase.CLOSED_PARENT_REGION);
|
||||||
|
}
|
||||||
|
if (exceptionToThrow != null) {
|
||||||
|
if (exceptionToThrow instanceof IOException) throw (IOException)exceptionToThrow;
|
||||||
|
throw new IOException(exceptionToThrow);
|
||||||
|
}
|
||||||
|
if (!testing) {
|
||||||
|
services.removeFromOnlineRegions(this.parent, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
transition(SplitTransactionPhase.OFFLINED_PARENT);
|
||||||
|
|
||||||
|
// TODO: If splitStoreFiles were multithreaded would we complete steps in
|
||||||
|
// less elapsed time? St.Ack 20100920
|
||||||
|
//
|
||||||
|
// splitStoreFiles creates daughter region dirs under the parent splits dir
|
||||||
|
// Nothing to unroll here if failure -- clean up of CREATE_SPLIT_DIR will
|
||||||
|
// clean this up.
|
||||||
|
Pair<Integer, Integer> expectedReferences = splitStoreFiles(hstoreFilesToSplit);
|
||||||
|
|
||||||
|
// Log to the journal that we are creating region A, the first daughter
|
||||||
|
// region. We could fail halfway through. If we do, we could have left
|
||||||
|
// stuff in fs that needs cleanup -- a storefile or two. Thats why we
|
||||||
|
// add entry to journal BEFORE rather than AFTER the change.
|
||||||
|
|
||||||
|
transition(SplitTransactionPhase.STARTED_REGION_A_CREATION);
|
||||||
|
|
||||||
|
assertReferenceFileCount(expectedReferences.getFirst(),
|
||||||
|
this.parent.getRegionFileSystem().getSplitsDir(this.hri_a));
|
||||||
|
HRegion a = this.parent.createDaughterRegionFromSplits(this.hri_a);
|
||||||
|
assertReferenceFileCount(expectedReferences.getFirst(),
|
||||||
|
new Path(this.parent.getRegionFileSystem().getTableDir(), this.hri_a.getEncodedName()));
|
||||||
|
|
||||||
|
// Ditto
|
||||||
|
|
||||||
|
transition(SplitTransactionPhase.STARTED_REGION_B_CREATION);
|
||||||
|
|
||||||
|
assertReferenceFileCount(expectedReferences.getSecond(),
|
||||||
|
this.parent.getRegionFileSystem().getSplitsDir(this.hri_b));
|
||||||
|
HRegion b = this.parent.createDaughterRegionFromSplits(this.hri_b);
|
||||||
|
assertReferenceFileCount(expectedReferences.getSecond(),
|
||||||
|
new Path(this.parent.getRegionFileSystem().getTableDir(), this.hri_b.getEncodedName()));
|
||||||
|
|
||||||
|
return new PairOfSameType<Region>(a, b);
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
void assertReferenceFileCount(int expectedReferenceFileCount, Path dir)
|
||||||
|
throws IOException {
|
||||||
|
if (expectedReferenceFileCount != 0 &&
|
||||||
|
expectedReferenceFileCount != FSUtils.getRegionReferenceFileCount(parent.getFilesystem(),
|
||||||
|
dir)) {
|
||||||
|
throw new IOException("Failing split. Expected reference file count isn't equal.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Perform time consuming opening of the daughter regions.
|
||||||
|
* @param server Hosting server instance. Can be null when testing
|
||||||
|
* @param services Used to online/offline regions.
|
||||||
|
* @param a first daughter region
|
||||||
|
* @param a second daughter region
|
||||||
|
* @throws IOException If thrown, transaction failed.
|
||||||
|
* Call {@link #rollback(Server, RegionServerServices)}
|
||||||
|
*/
|
||||||
|
@VisibleForTesting
|
||||||
|
void openDaughters(final Server server, final RegionServerServices services, Region a,
|
||||||
|
Region b) throws IOException {
|
||||||
|
boolean stopped = server != null && server.isStopped();
|
||||||
|
boolean stopping = services != null && services.isStopping();
|
||||||
|
// TODO: Is this check needed here?
|
||||||
|
if (stopped || stopping) {
|
||||||
|
LOG.info("Not opening daughters " +
|
||||||
|
b.getRegionInfo().getRegionNameAsString() +
|
||||||
|
" and " +
|
||||||
|
a.getRegionInfo().getRegionNameAsString() +
|
||||||
|
" because stopping=" + stopping + ", stopped=" + stopped);
|
||||||
|
} else {
|
||||||
|
// Open daughters in parallel.
|
||||||
|
DaughterOpener aOpener = new DaughterOpener(server, a);
|
||||||
|
DaughterOpener bOpener = new DaughterOpener(server, b);
|
||||||
|
aOpener.start();
|
||||||
|
bOpener.start();
|
||||||
|
try {
|
||||||
|
aOpener.join();
|
||||||
|
if (aOpener.getException() == null) {
|
||||||
|
transition(SplitTransactionPhase.OPENED_REGION_A);
|
||||||
|
}
|
||||||
|
bOpener.join();
|
||||||
|
if (bOpener.getException() == null) {
|
||||||
|
transition(SplitTransactionPhase.OPENED_REGION_B);
|
||||||
|
}
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
throw (InterruptedIOException)new InterruptedIOException().initCause(e);
|
||||||
|
}
|
||||||
|
if (aOpener.getException() != null) {
|
||||||
|
throw new IOException("Failed " +
|
||||||
|
aOpener.getName(), aOpener.getException());
|
||||||
|
}
|
||||||
|
if (bOpener.getException() != null) {
|
||||||
|
throw new IOException("Failed " +
|
||||||
|
bOpener.getName(), bOpener.getException());
|
||||||
|
}
|
||||||
|
if (services != null) {
|
||||||
|
if (!services.reportRegionStateTransition(TransitionCode.SPLIT,
|
||||||
|
parent.getRegionInfo(), hri_a, hri_b)) {
|
||||||
|
throw new IOException("Failed to report split region to master: "
|
||||||
|
+ parent.getRegionInfo().getShortNameToLog());
|
||||||
|
}
|
||||||
|
// Should add it to OnlineRegions
|
||||||
|
services.addToOnlineRegions(b);
|
||||||
|
services.addToOnlineRegions(a);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public PairOfSameType<Region> execute(final Server server, final RegionServerServices services)
|
||||||
|
throws IOException {
|
||||||
|
this.server = server;
|
||||||
|
this.rsServices = services;
|
||||||
|
PairOfSameType<Region> regions = createDaughters(server, services);
|
||||||
|
stepsAfterPONR(server, services, regions);
|
||||||
|
transition(SplitTransactionPhase.COMPLETED);
|
||||||
|
return regions;
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
void stepsAfterPONR(final Server server,
|
||||||
|
final RegionServerServices services, PairOfSameType<Region> regions)
|
||||||
|
throws IOException {
|
||||||
|
if (this.parent.getCoprocessorHost() != null) {
|
||||||
|
this.parent.getCoprocessorHost().preSplitAfterPONR();
|
||||||
|
}
|
||||||
|
|
||||||
|
openDaughters(server, services, regions.getFirst(), regions.getSecond());
|
||||||
|
|
||||||
|
transition(SplitTransactionPhase.BEFORE_POST_SPLIT_HOOK);
|
||||||
|
|
||||||
|
// Coprocessor callback
|
||||||
|
if (parent.getCoprocessorHost() != null) {
|
||||||
|
parent.getCoprocessorHost().postSplit(regions.getFirst(), regions.getSecond());
|
||||||
|
}
|
||||||
|
|
||||||
|
transition(SplitTransactionPhase.AFTER_POST_SPLIT_HOOK);
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Open daughter region in its own thread.
|
||||||
|
* If we fail, abort this hosting server.
|
||||||
|
*/
|
||||||
|
private class DaughterOpener extends HasThread {
|
||||||
|
private final Server server;
|
||||||
|
private final Region r;
|
||||||
|
private Throwable t = null;
|
||||||
|
|
||||||
|
DaughterOpener(final Server s, final Region r) {
|
||||||
|
super((s == null? "null-services": s.getServerName()) +
|
||||||
|
"-daughterOpener=" + r.getRegionInfo().getEncodedName());
|
||||||
|
setDaemon(true);
|
||||||
|
this.server = s;
|
||||||
|
this.r = r;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return Null if open succeeded else exception that causes us fail open.
|
||||||
|
* Call it after this thread exits else you may get wrong view on result.
|
||||||
|
*/
|
||||||
|
Throwable getException() {
|
||||||
|
return this.t;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
try {
|
||||||
|
openDaughterRegion(this.server, r);
|
||||||
|
} catch (Throwable t) {
|
||||||
|
this.t = t;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Open daughter regions, add them to online list and update meta.
|
||||||
|
* @param server
|
||||||
|
* @param daughter
|
||||||
|
* @throws IOException
|
||||||
|
* @throws KeeperException
|
||||||
|
*/
|
||||||
|
@VisibleForTesting
|
||||||
|
void openDaughterRegion(final Server server, final Region daughter)
|
||||||
|
throws IOException, KeeperException {
|
||||||
|
HRegionInfo hri = daughter.getRegionInfo();
|
||||||
|
LoggingProgressable reporter = server == null ? null
|
||||||
|
: new LoggingProgressable(hri, server.getConfiguration().getLong(
|
||||||
|
"hbase.regionserver.split.daughter.open.log.interval", 10000));
|
||||||
|
((HRegion)daughter).openHRegion(reporter);
|
||||||
|
}
|
||||||
|
|
||||||
|
static class LoggingProgressable implements CancelableProgressable {
|
||||||
|
private final HRegionInfo hri;
|
||||||
|
private long lastLog = -1;
|
||||||
|
private final long interval;
|
||||||
|
|
||||||
|
LoggingProgressable(final HRegionInfo hri, final long interval) {
|
||||||
|
this.hri = hri;
|
||||||
|
this.interval = interval;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean progress() {
|
||||||
|
long now = EnvironmentEdgeManager.currentTime();
|
||||||
|
if (now - lastLog > this.interval) {
|
||||||
|
LOG.info("Opening " + this.hri.getRegionNameAsString());
|
||||||
|
this.lastLog = now;
|
||||||
|
}
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates reference files for top and bottom half of the
|
||||||
|
* @param hstoreFilesToSplit map of store files to create half file references for.
|
||||||
|
* @return the number of reference files that were created.
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
private Pair<Integer, Integer> splitStoreFiles(
|
||||||
|
final Map<byte[], List<StoreFile>> hstoreFilesToSplit)
|
||||||
|
throws IOException {
|
||||||
|
if (hstoreFilesToSplit == null) {
|
||||||
|
// Could be null because close didn't succeed -- for now consider it fatal
|
||||||
|
throw new IOException("Close returned empty list of StoreFiles");
|
||||||
|
}
|
||||||
|
// The following code sets up a thread pool executor with as many slots as
|
||||||
|
// there's files to split. It then fires up everything, waits for
|
||||||
|
// completion and finally checks for any exception
|
||||||
|
int nbFiles = hstoreFilesToSplit.size();
|
||||||
|
if (nbFiles == 0) {
|
||||||
|
// no file needs to be splitted.
|
||||||
|
return new Pair<Integer, Integer>(0,0);
|
||||||
|
}
|
||||||
|
LOG.info("Preparing to split " + nbFiles + " storefiles for region " + this.parent);
|
||||||
|
ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
|
||||||
|
builder.setNameFormat("StoreFileSplitter-%1$d");
|
||||||
|
ThreadFactory factory = builder.build();
|
||||||
|
ThreadPoolExecutor threadPool =
|
||||||
|
(ThreadPoolExecutor) Executors.newFixedThreadPool(nbFiles, factory);
|
||||||
|
List<Future<Pair<Path,Path>>> futures = new ArrayList<Future<Pair<Path,Path>>> (nbFiles);
|
||||||
|
|
||||||
|
// Split each store file.
|
||||||
|
for (Map.Entry<byte[], List<StoreFile>> entry: hstoreFilesToSplit.entrySet()) {
|
||||||
|
for (StoreFile sf: entry.getValue()) {
|
||||||
|
StoreFileSplitter sfs = new StoreFileSplitter(entry.getKey(), sf);
|
||||||
|
futures.add(threadPool.submit(sfs));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// Shutdown the pool
|
||||||
|
threadPool.shutdown();
|
||||||
|
|
||||||
|
// Wait for all the tasks to finish
|
||||||
|
try {
|
||||||
|
boolean stillRunning = !threadPool.awaitTermination(
|
||||||
|
this.fileSplitTimeout, TimeUnit.MILLISECONDS);
|
||||||
|
if (stillRunning) {
|
||||||
|
threadPool.shutdownNow();
|
||||||
|
// wait for the thread to shutdown completely.
|
||||||
|
while (!threadPool.isTerminated()) {
|
||||||
|
Thread.sleep(50);
|
||||||
|
}
|
||||||
|
throw new IOException("Took too long to split the" +
|
||||||
|
" files and create the references, aborting split");
|
||||||
|
}
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
throw (InterruptedIOException)new InterruptedIOException().initCause(e);
|
||||||
|
}
|
||||||
|
|
||||||
|
int created_a = 0;
|
||||||
|
int created_b = 0;
|
||||||
|
// Look for any exception
|
||||||
|
for (Future<Pair<Path, Path>> future : futures) {
|
||||||
|
try {
|
||||||
|
Pair<Path, Path> p = future.get();
|
||||||
|
created_a += p.getFirst() != null ? 1 : 0;
|
||||||
|
created_b += p.getSecond() != null ? 1 : 0;
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
throw (InterruptedIOException) new InterruptedIOException().initCause(e);
|
||||||
|
} catch (ExecutionException e) {
|
||||||
|
throw new IOException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Split storefiles for region " + this.parent + " Daugther A: " + created_a
|
||||||
|
+ " storefiles, Daugther B: " + created_b + " storefiles.");
|
||||||
|
}
|
||||||
|
return new Pair<Integer, Integer>(created_a, created_b);
|
||||||
|
}
|
||||||
|
|
||||||
|
private Pair<Path, Path> splitStoreFile(final byte[] family, final StoreFile sf)
|
||||||
|
throws IOException {
|
||||||
|
HRegionFileSystem fs = this.parent.getRegionFileSystem();
|
||||||
|
String familyName = Bytes.toString(family);
|
||||||
|
Path path_a =
|
||||||
|
fs.splitStoreFile(this.hri_a, familyName, sf, this.splitrow, false,
|
||||||
|
this.parent.getSplitPolicy());
|
||||||
|
Path path_b =
|
||||||
|
fs.splitStoreFile(this.hri_b, familyName, sf, this.splitrow, true,
|
||||||
|
this.parent.getSplitPolicy());
|
||||||
|
return new Pair<Path,Path>(path_a, path_b);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Utility class used to do the file splitting / reference writing
|
||||||
|
* in parallel instead of sequentially.
|
||||||
|
*/
|
||||||
|
private class StoreFileSplitter implements Callable<Pair<Path,Path>> {
|
||||||
|
private final byte[] family;
|
||||||
|
private final StoreFile sf;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructor that takes what it needs to split
|
||||||
|
* @param family Family that contains the store file
|
||||||
|
* @param sf which file
|
||||||
|
*/
|
||||||
|
public StoreFileSplitter(final byte[] family, final StoreFile sf) {
|
||||||
|
this.sf = sf;
|
||||||
|
this.family = family;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Pair<Path,Path> call() throws IOException {
|
||||||
|
return splitStoreFile(family, sf);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean rollback(final Server server, final RegionServerServices services)
|
||||||
|
throws IOException {
|
||||||
|
this.server = server;
|
||||||
|
this.rsServices = services;
|
||||||
|
// Coprocessor callback
|
||||||
|
if (this.parent.getCoprocessorHost() != null) {
|
||||||
|
this.parent.getCoprocessorHost().preRollBackSplit();
|
||||||
|
}
|
||||||
|
|
||||||
|
boolean result = true;
|
||||||
|
ListIterator<JournalEntry> iterator =
|
||||||
|
this.journal.listIterator(this.journal.size());
|
||||||
|
// Iterate in reverse.
|
||||||
|
while (iterator.hasPrevious()) {
|
||||||
|
JournalEntry je = iterator.previous();
|
||||||
|
|
||||||
|
transition(je.getPhase(), true);
|
||||||
|
|
||||||
|
switch (je.getPhase()) {
|
||||||
|
|
||||||
|
case SET_SPLITTING:
|
||||||
|
if (services != null
|
||||||
|
&& !services.reportRegionStateTransition(TransitionCode.SPLIT_REVERTED,
|
||||||
|
parent.getRegionInfo(), hri_a, hri_b)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
break;
|
||||||
|
|
||||||
|
case CREATE_SPLIT_DIR:
|
||||||
|
this.parent.writestate.writesEnabled = true;
|
||||||
|
this.parent.getRegionFileSystem().cleanupSplitsDir();
|
||||||
|
break;
|
||||||
|
|
||||||
|
case CLOSED_PARENT_REGION:
|
||||||
|
try {
|
||||||
|
// So, this returns a seqid but if we just closed and then reopened, we
|
||||||
|
// should be ok. On close, we flushed using sequenceid obtained from
|
||||||
|
// hosting regionserver so no need to propagate the sequenceid returned
|
||||||
|
// out of initialize below up into regionserver as we normally do.
|
||||||
|
// TODO: Verify.
|
||||||
|
this.parent.initialize();
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.error("Failed rollbacking CLOSED_PARENT_REGION of region " +
|
||||||
|
parent.getRegionInfo().getRegionNameAsString(), e);
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
break;
|
||||||
|
|
||||||
|
case STARTED_REGION_A_CREATION:
|
||||||
|
this.parent.getRegionFileSystem().cleanupDaughterRegion(this.hri_a);
|
||||||
|
break;
|
||||||
|
|
||||||
|
case STARTED_REGION_B_CREATION:
|
||||||
|
this.parent.getRegionFileSystem().cleanupDaughterRegion(this.hri_b);
|
||||||
|
break;
|
||||||
|
|
||||||
|
case OFFLINED_PARENT:
|
||||||
|
if (services != null) services.addToOnlineRegions(this.parent);
|
||||||
|
break;
|
||||||
|
|
||||||
|
case PONR:
|
||||||
|
// We got to the point-of-no-return so we need to just abort. Return
|
||||||
|
// immediately. Do not clean up created daughter regions. They need
|
||||||
|
// to be in place so we don't delete the parent region mistakenly.
|
||||||
|
// See HBASE-3872.
|
||||||
|
return false;
|
||||||
|
|
||||||
|
// Informational only cases
|
||||||
|
case STARTED:
|
||||||
|
case PREPARED:
|
||||||
|
case BEFORE_PRE_SPLIT_HOOK:
|
||||||
|
case AFTER_PRE_SPLIT_HOOK:
|
||||||
|
case BEFORE_POST_SPLIT_HOOK:
|
||||||
|
case AFTER_POST_SPLIT_HOOK:
|
||||||
|
case OPENED_REGION_A:
|
||||||
|
case OPENED_REGION_B:
|
||||||
|
case COMPLETED:
|
||||||
|
break;
|
||||||
|
|
||||||
|
default:
|
||||||
|
throw new RuntimeException("Unhandled journal entry: " + je);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// Coprocessor callback
|
||||||
|
if (this.parent.getCoprocessorHost() != null) {
|
||||||
|
this.parent.getCoprocessorHost().postRollBackSplit();
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
/* package */ HRegionInfo getFirstDaughter() {
|
||||||
|
return hri_a;
|
||||||
|
}
|
||||||
|
|
||||||
|
/* package */ HRegionInfo getSecondDaughter() {
|
||||||
|
return hri_b;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<JournalEntry> getJournal() {
|
||||||
|
return journal;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public SplitTransaction registerTransactionListener(TransactionListener listener) {
|
||||||
|
listeners.add(listener);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Server getServer() {
|
||||||
|
return server;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RegionServerServices getRegionServerServices() {
|
||||||
|
return rsServices;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -59,6 +59,7 @@ import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
|
||||||
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
||||||
import org.apache.hadoop.hbase.regionserver.ScanType;
|
import org.apache.hadoop.hbase.regionserver.ScanType;
|
||||||
import org.apache.hadoop.hbase.regionserver.SplitTransaction;
|
import org.apache.hadoop.hbase.regionserver.SplitTransaction;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.SplitTransactionFactory;
|
||||||
import org.apache.hadoop.hbase.regionserver.Store;
|
import org.apache.hadoop.hbase.regionserver.Store;
|
||||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||||
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
|
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
|
||||||
|
@ -500,7 +501,8 @@ public class TestCoprocessorInterface {
|
||||||
private Region [] split(final Region r, final byte [] splitRow) throws IOException {
|
private Region [] split(final Region r, final byte [] splitRow) throws IOException {
|
||||||
Region[] regions = new Region[2];
|
Region[] regions = new Region[2];
|
||||||
|
|
||||||
SplitTransaction st = new SplitTransaction(r, splitRow);
|
SplitTransaction st = new SplitTransactionFactory(TEST_UTIL.getConfiguration())
|
||||||
|
.create(r, splitRow);
|
||||||
int i = 0;
|
int i = 0;
|
||||||
|
|
||||||
if (!st.prepare()) {
|
if (!st.prepare()) {
|
||||||
|
@ -509,8 +511,7 @@ public class TestCoprocessorInterface {
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
Server mockServer = Mockito.mock(Server.class);
|
Server mockServer = Mockito.mock(Server.class);
|
||||||
when(mockServer.getConfiguration()).thenReturn(
|
when(mockServer.getConfiguration()).thenReturn(TEST_UTIL.getConfiguration());
|
||||||
TEST_UTIL.getConfiguration());
|
|
||||||
PairOfSameType<Region> daughters = st.execute(mockServer, null);
|
PairOfSameType<Region> daughters = st.execute(mockServer, null);
|
||||||
for (Region each_daughter: daughters) {
|
for (Region each_daughter: daughters) {
|
||||||
regions[i] = each_daughter;
|
regions[i] = each_daughter;
|
||||||
|
|
|
@ -40,7 +40,8 @@ import org.apache.hadoop.hbase.client.Mutation;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||||
import org.apache.hadoop.hbase.regionserver.Region;
|
import org.apache.hadoop.hbase.regionserver.Region;
|
||||||
import org.apache.hadoop.hbase.regionserver.RegionMergeTransaction;
|
import org.apache.hadoop.hbase.regionserver.RegionMergeTransactionFactory;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.RegionMergeTransactionImpl;
|
||||||
import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
|
import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
|
||||||
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
|
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
|
@ -110,7 +111,7 @@ public class TestRegionServerObserver {
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class CPRegionServerObserver extends BaseRegionServerObserver {
|
public static class CPRegionServerObserver extends BaseRegionServerObserver {
|
||||||
private RegionMergeTransaction rmt = null;
|
private RegionMergeTransactionImpl rmt = null;
|
||||||
private HRegion mergedRegion = null;
|
private HRegion mergedRegion = null;
|
||||||
|
|
||||||
private boolean preMergeCalled;
|
private boolean preMergeCalled;
|
||||||
|
@ -143,7 +144,8 @@ public class TestRegionServerObserver {
|
||||||
HRegionServer rs = (HRegionServer) environment.getRegionServerServices();
|
HRegionServer rs = (HRegionServer) environment.getRegionServerServices();
|
||||||
List<Region> onlineRegions =
|
List<Region> onlineRegions =
|
||||||
rs.getOnlineRegions(TableName.valueOf("testRegionServerObserver_2"));
|
rs.getOnlineRegions(TableName.valueOf("testRegionServerObserver_2"));
|
||||||
rmt = new RegionMergeTransaction(onlineRegions.get(0), onlineRegions.get(1), true);
|
rmt = (RegionMergeTransactionImpl) new RegionMergeTransactionFactory(rs.getConfiguration())
|
||||||
|
.create(onlineRegions.get(0), onlineRegions.get(1), true);
|
||||||
if (!rmt.prepare(rs)) {
|
if (!rmt.prepare(rs)) {
|
||||||
LOG.error("Prepare for the region merge of table "
|
LOG.error("Prepare for the region merge of table "
|
||||||
+ onlineRegions.get(0).getTableDesc().getNameAsString()
|
+ onlineRegions.get(0).getTableDesc().getNameAsString()
|
||||||
|
|
|
@ -106,7 +106,7 @@ public class TestEndToEndSplitTransaction {
|
||||||
byte[] regionName = conn.getRegionLocator(tableName).getRegionLocation(splitRow)
|
byte[] regionName = conn.getRegionLocator(tableName).getRegionLocation(splitRow)
|
||||||
.getRegionInfo().getRegionName();
|
.getRegionInfo().getRegionName();
|
||||||
Region region = server.getRegion(regionName);
|
Region region = server.getRegion(regionName);
|
||||||
SplitTransaction split = new SplitTransaction(region, splitRow);
|
SplitTransactionImpl split = new SplitTransactionImpl((HRegion) region, splitRow);
|
||||||
split.prepare();
|
split.prepare();
|
||||||
|
|
||||||
// 1. phase I
|
// 1. phase I
|
||||||
|
|
|
@ -2490,7 +2490,7 @@ public class TestHRegion {
|
||||||
*/
|
*/
|
||||||
HRegion[] splitRegion(final HRegion parent, final byte[] midkey) throws IOException {
|
HRegion[] splitRegion(final HRegion parent, final byte[] midkey) throws IOException {
|
||||||
PairOfSameType<Region> result = null;
|
PairOfSameType<Region> result = null;
|
||||||
SplitTransaction st = new SplitTransaction(parent, midkey);
|
SplitTransactionImpl st = new SplitTransactionImpl(parent, midkey);
|
||||||
// If prepare does not return true, for some reason -- logged inside in
|
// If prepare does not return true, for some reason -- logged inside in
|
||||||
// the prepare call -- we are not ready to split just now. Just return.
|
// the prepare call -- we are not ready to split just now. Just return.
|
||||||
if (!st.prepare()) {
|
if (!st.prepare()) {
|
||||||
|
|
|
@ -60,7 +60,7 @@ import org.mockito.Mockito;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test the {@link RegionMergeTransaction} class against two HRegions (as
|
* Test the {@link RegionMergeTransactionImpl} class against two HRegions (as
|
||||||
* opposed to running cluster).
|
* opposed to running cluster).
|
||||||
*/
|
*/
|
||||||
@Category({RegionServerTests.class, SmallTests.class})
|
@Category({RegionServerTests.class, SmallTests.class})
|
||||||
|
@ -120,10 +120,10 @@ public class TestRegionMergeTransaction {
|
||||||
prepareOnGoodRegions();
|
prepareOnGoodRegions();
|
||||||
}
|
}
|
||||||
|
|
||||||
private RegionMergeTransaction prepareOnGoodRegions() throws IOException {
|
private RegionMergeTransactionImpl prepareOnGoodRegions() throws IOException {
|
||||||
RegionMergeTransaction mt = new RegionMergeTransaction(region_a, region_b,
|
RegionMergeTransactionImpl mt = new RegionMergeTransactionImpl(region_a, region_b,
|
||||||
false);
|
false);
|
||||||
RegionMergeTransaction spyMT = Mockito.spy(mt);
|
RegionMergeTransactionImpl spyMT = Mockito.spy(mt);
|
||||||
doReturn(false).when(spyMT).hasMergeQualifierInMeta(null,
|
doReturn(false).when(spyMT).hasMergeQualifierInMeta(null,
|
||||||
region_a.getRegionInfo().getRegionName());
|
region_a.getRegionInfo().getRegionName());
|
||||||
doReturn(false).when(spyMT).hasMergeQualifierInMeta(null,
|
doReturn(false).when(spyMT).hasMergeQualifierInMeta(null,
|
||||||
|
@ -137,7 +137,7 @@ public class TestRegionMergeTransaction {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
public void testPrepareWithSameRegion() throws IOException {
|
public void testPrepareWithSameRegion() throws IOException {
|
||||||
RegionMergeTransaction mt = new RegionMergeTransaction(this.region_a,
|
RegionMergeTransactionImpl mt = new RegionMergeTransactionImpl(this.region_a,
|
||||||
this.region_a, true);
|
this.region_a, true);
|
||||||
assertFalse("should not merge the same region even if it is forcible ",
|
assertFalse("should not merge the same region even if it is forcible ",
|
||||||
mt.prepare(null));
|
mt.prepare(null));
|
||||||
|
@ -148,7 +148,7 @@ public class TestRegionMergeTransaction {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
public void testPrepareWithRegionsNotAdjacent() throws IOException {
|
public void testPrepareWithRegionsNotAdjacent() throws IOException {
|
||||||
RegionMergeTransaction mt = new RegionMergeTransaction(this.region_a,
|
RegionMergeTransactionImpl mt = new RegionMergeTransactionImpl(this.region_a,
|
||||||
this.region_c, false);
|
this.region_c, false);
|
||||||
assertFalse("should not merge two regions if they are adjacent except it is forcible",
|
assertFalse("should not merge two regions if they are adjacent except it is forcible",
|
||||||
mt.prepare(null));
|
mt.prepare(null));
|
||||||
|
@ -160,9 +160,9 @@ public class TestRegionMergeTransaction {
|
||||||
@Test
|
@Test
|
||||||
public void testPrepareWithRegionsNotAdjacentUnderCompulsory()
|
public void testPrepareWithRegionsNotAdjacentUnderCompulsory()
|
||||||
throws IOException {
|
throws IOException {
|
||||||
RegionMergeTransaction mt = new RegionMergeTransaction(region_a, region_c,
|
RegionMergeTransactionImpl mt = new RegionMergeTransactionImpl(region_a, region_c,
|
||||||
true);
|
true);
|
||||||
RegionMergeTransaction spyMT = Mockito.spy(mt);
|
RegionMergeTransactionImpl spyMT = Mockito.spy(mt);
|
||||||
doReturn(false).when(spyMT).hasMergeQualifierInMeta(null,
|
doReturn(false).when(spyMT).hasMergeQualifierInMeta(null,
|
||||||
region_a.getRegionInfo().getRegionName());
|
region_a.getRegionInfo().getRegionName());
|
||||||
doReturn(false).when(spyMT).hasMergeQualifierInMeta(null,
|
doReturn(false).when(spyMT).hasMergeQualifierInMeta(null,
|
||||||
|
@ -181,7 +181,7 @@ public class TestRegionMergeTransaction {
|
||||||
when(storeMock.getFamily()).thenReturn(new HColumnDescriptor("cf"));
|
when(storeMock.getFamily()).thenReturn(new HColumnDescriptor("cf"));
|
||||||
when(storeMock.close()).thenReturn(ImmutableList.<StoreFile>of());
|
when(storeMock.close()).thenReturn(ImmutableList.<StoreFile>of());
|
||||||
this.region_a.stores.put(Bytes.toBytes(""), storeMock);
|
this.region_a.stores.put(Bytes.toBytes(""), storeMock);
|
||||||
RegionMergeTransaction mt = new RegionMergeTransaction(this.region_a,
|
RegionMergeTransactionImpl mt = new RegionMergeTransactionImpl(this.region_a,
|
||||||
this.region_b, false);
|
this.region_b, false);
|
||||||
assertFalse(
|
assertFalse(
|
||||||
"a region should not be mergeable if it has instances of store file references",
|
"a region should not be mergeable if it has instances of store file references",
|
||||||
|
@ -191,7 +191,7 @@ public class TestRegionMergeTransaction {
|
||||||
@Test
|
@Test
|
||||||
public void testPrepareWithClosedRegion() throws IOException {
|
public void testPrepareWithClosedRegion() throws IOException {
|
||||||
this.region_a.close();
|
this.region_a.close();
|
||||||
RegionMergeTransaction mt = new RegionMergeTransaction(this.region_a,
|
RegionMergeTransactionImpl mt = new RegionMergeTransactionImpl(this.region_a,
|
||||||
this.region_b, false);
|
this.region_b, false);
|
||||||
assertFalse(mt.prepare(null));
|
assertFalse(mt.prepare(null));
|
||||||
}
|
}
|
||||||
|
@ -202,9 +202,9 @@ public class TestRegionMergeTransaction {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
public void testPrepareWithRegionsWithMergeReference() throws IOException {
|
public void testPrepareWithRegionsWithMergeReference() throws IOException {
|
||||||
RegionMergeTransaction mt = new RegionMergeTransaction(region_a, region_b,
|
RegionMergeTransactionImpl mt = new RegionMergeTransactionImpl(region_a, region_b,
|
||||||
false);
|
false);
|
||||||
RegionMergeTransaction spyMT = Mockito.spy(mt);
|
RegionMergeTransactionImpl spyMT = Mockito.spy(mt);
|
||||||
doReturn(true).when(spyMT).hasMergeQualifierInMeta(null,
|
doReturn(true).when(spyMT).hasMergeQualifierInMeta(null,
|
||||||
region_a.getRegionInfo().getRegionName());
|
region_a.getRegionInfo().getRegionName());
|
||||||
doReturn(true).when(spyMT).hasMergeQualifierInMeta(null,
|
doReturn(true).when(spyMT).hasMergeQualifierInMeta(null,
|
||||||
|
@ -221,14 +221,14 @@ public class TestRegionMergeTransaction {
|
||||||
assertEquals(rowCountOfRegionB, countRows(this.region_b));
|
assertEquals(rowCountOfRegionB, countRows(this.region_b));
|
||||||
|
|
||||||
// Start transaction.
|
// Start transaction.
|
||||||
RegionMergeTransaction mt = prepareOnGoodRegions();
|
RegionMergeTransactionImpl mt = prepareOnGoodRegions();
|
||||||
|
|
||||||
// Run the execute. Look at what it returns.
|
// Run the execute. Look at what it returns.
|
||||||
TEST_UTIL.getConfiguration().setInt(HConstants.REGIONSERVER_PORT, 0);
|
TEST_UTIL.getConfiguration().setInt(HConstants.REGIONSERVER_PORT, 0);
|
||||||
CoordinatedStateManager cp = CoordinatedStateManagerFactory.getCoordinatedStateManager(
|
CoordinatedStateManager cp = CoordinatedStateManagerFactory.getCoordinatedStateManager(
|
||||||
TEST_UTIL.getConfiguration());
|
TEST_UTIL.getConfiguration());
|
||||||
Server mockServer = new HRegionServer(TEST_UTIL.getConfiguration(), cp);
|
Server mockServer = new HRegionServer(TEST_UTIL.getConfiguration(), cp);
|
||||||
HRegion mergedRegion = mt.execute(mockServer, null);
|
HRegion mergedRegion = (HRegion)mt.execute(mockServer, null);
|
||||||
// Do some assertions about execution.
|
// Do some assertions about execution.
|
||||||
assertTrue(this.fs.exists(mt.getMergesDir()));
|
assertTrue(this.fs.exists(mt.getMergesDir()));
|
||||||
// Assert region_a and region_b is closed.
|
// Assert region_a and region_b is closed.
|
||||||
|
@ -265,7 +265,7 @@ public class TestRegionMergeTransaction {
|
||||||
assertEquals(rowCountOfRegionB, countRows(this.region_b));
|
assertEquals(rowCountOfRegionB, countRows(this.region_b));
|
||||||
|
|
||||||
// Start transaction.
|
// Start transaction.
|
||||||
RegionMergeTransaction mt = prepareOnGoodRegions();
|
RegionMergeTransactionImpl mt = prepareOnGoodRegions();
|
||||||
|
|
||||||
when(mt.createMergedRegionFromMerges(region_a, region_b,
|
when(mt.createMergedRegionFromMerges(region_a, region_b,
|
||||||
mt.getMergedRegionInfo())).thenThrow(
|
mt.getMergedRegionInfo())).thenThrow(
|
||||||
|
@ -301,7 +301,7 @@ public class TestRegionMergeTransaction {
|
||||||
|
|
||||||
// Now retry the merge but do not throw an exception this time.
|
// Now retry the merge but do not throw an exception this time.
|
||||||
assertTrue(mt.prepare(null));
|
assertTrue(mt.prepare(null));
|
||||||
HRegion mergedRegion = mt.execute(mockServer, null);
|
HRegion mergedRegion = (HRegion)mt.execute(mockServer, null);
|
||||||
// Count rows. daughters are already open
|
// Count rows. daughters are already open
|
||||||
// Count rows. merged region are already open
|
// Count rows. merged region are already open
|
||||||
try {
|
try {
|
||||||
|
@ -325,7 +325,7 @@ public class TestRegionMergeTransaction {
|
||||||
assertEquals(rowCountOfRegionB, countRows(this.region_b));
|
assertEquals(rowCountOfRegionB, countRows(this.region_b));
|
||||||
|
|
||||||
// Start transaction.
|
// Start transaction.
|
||||||
RegionMergeTransaction mt = prepareOnGoodRegions();
|
RegionMergeTransactionImpl mt = prepareOnGoodRegions();
|
||||||
Mockito.doThrow(new MockedFailedMergedRegionOpen())
|
Mockito.doThrow(new MockedFailedMergedRegionOpen())
|
||||||
.when(mt)
|
.when(mt)
|
||||||
.openMergedRegion((Server) Mockito.anyObject(),
|
.openMergedRegion((Server) Mockito.anyObject(),
|
||||||
|
@ -365,31 +365,31 @@ public class TestRegionMergeTransaction {
|
||||||
byte[] z = Bytes.toBytes("z");
|
byte[] z = Bytes.toBytes("z");
|
||||||
HRegionInfo r1 = new HRegionInfo(tableName);
|
HRegionInfo r1 = new HRegionInfo(tableName);
|
||||||
HRegionInfo r2 = new HRegionInfo(tableName, a, z);
|
HRegionInfo r2 = new HRegionInfo(tableName, a, z);
|
||||||
HRegionInfo m = RegionMergeTransaction.getMergedRegionInfo(r1, r2);
|
HRegionInfo m = RegionMergeTransactionImpl.getMergedRegionInfo(r1, r2);
|
||||||
assertTrue(Bytes.equals(m.getStartKey(), r1.getStartKey())
|
assertTrue(Bytes.equals(m.getStartKey(), r1.getStartKey())
|
||||||
&& Bytes.equals(m.getEndKey(), r1.getEndKey()));
|
&& Bytes.equals(m.getEndKey(), r1.getEndKey()));
|
||||||
|
|
||||||
r1 = new HRegionInfo(tableName, null, a);
|
r1 = new HRegionInfo(tableName, null, a);
|
||||||
r2 = new HRegionInfo(tableName, a, z);
|
r2 = new HRegionInfo(tableName, a, z);
|
||||||
m = RegionMergeTransaction.getMergedRegionInfo(r1, r2);
|
m = RegionMergeTransactionImpl.getMergedRegionInfo(r1, r2);
|
||||||
assertTrue(Bytes.equals(m.getStartKey(), r1.getStartKey())
|
assertTrue(Bytes.equals(m.getStartKey(), r1.getStartKey())
|
||||||
&& Bytes.equals(m.getEndKey(), r2.getEndKey()));
|
&& Bytes.equals(m.getEndKey(), r2.getEndKey()));
|
||||||
|
|
||||||
r1 = new HRegionInfo(tableName, null, a);
|
r1 = new HRegionInfo(tableName, null, a);
|
||||||
r2 = new HRegionInfo(tableName, z, null);
|
r2 = new HRegionInfo(tableName, z, null);
|
||||||
m = RegionMergeTransaction.getMergedRegionInfo(r1, r2);
|
m = RegionMergeTransactionImpl.getMergedRegionInfo(r1, r2);
|
||||||
assertTrue(Bytes.equals(m.getStartKey(), r1.getStartKey())
|
assertTrue(Bytes.equals(m.getStartKey(), r1.getStartKey())
|
||||||
&& Bytes.equals(m.getEndKey(), r2.getEndKey()));
|
&& Bytes.equals(m.getEndKey(), r2.getEndKey()));
|
||||||
|
|
||||||
r1 = new HRegionInfo(tableName, a, z);
|
r1 = new HRegionInfo(tableName, a, z);
|
||||||
r2 = new HRegionInfo(tableName, z, null);
|
r2 = new HRegionInfo(tableName, z, null);
|
||||||
m = RegionMergeTransaction.getMergedRegionInfo(r1, r2);
|
m = RegionMergeTransactionImpl.getMergedRegionInfo(r1, r2);
|
||||||
assertTrue(Bytes.equals(m.getStartKey(), r1.getStartKey())
|
assertTrue(Bytes.equals(m.getStartKey(), r1.getStartKey())
|
||||||
&& Bytes.equals(m.getEndKey(), r2.getEndKey()));
|
&& Bytes.equals(m.getEndKey(), r2.getEndKey()));
|
||||||
|
|
||||||
r1 = new HRegionInfo(tableName, a, b);
|
r1 = new HRegionInfo(tableName, a, b);
|
||||||
r2 = new HRegionInfo(tableName, b, z);
|
r2 = new HRegionInfo(tableName, b, z);
|
||||||
m = RegionMergeTransaction.getMergedRegionInfo(r1, r2);
|
m = RegionMergeTransactionImpl.getMergedRegionInfo(r1, r2);
|
||||||
assertTrue(Bytes.equals(m.getStartKey(), r1.getStartKey())
|
assertTrue(Bytes.equals(m.getStartKey(), r1.getStartKey())
|
||||||
&& Bytes.equals(m.getEndKey(), r2.getEndKey()));
|
&& Bytes.equals(m.getEndKey(), r2.getEndKey()));
|
||||||
}
|
}
|
||||||
|
|
|
@ -81,7 +81,7 @@ import com.google.protobuf.ServiceException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Like {@link TestRegionMergeTransaction} in that we're testing
|
* Like {@link TestRegionMergeTransaction} in that we're testing
|
||||||
* {@link RegionMergeTransaction} only the below tests are against a running
|
* {@link RegionMergeTransactionImpl} only the below tests are against a running
|
||||||
* cluster where {@link TestRegionMergeTransaction} is tests against bare
|
* cluster where {@link TestRegionMergeTransaction} is tests against bare
|
||||||
* {@link HRegion}.
|
* {@link HRegion}.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -67,7 +67,7 @@ import org.mockito.Mockito;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test the {@link SplitTransaction} class against an HRegion (as opposed to
|
* Test the {@link SplitTransactionImpl} class against an HRegion (as opposed to
|
||||||
* running cluster).
|
* running cluster).
|
||||||
*/
|
*/
|
||||||
@Category({RegionServerTests.class, SmallTests.class})
|
@Category({RegionServerTests.class, SmallTests.class})
|
||||||
|
@ -120,8 +120,8 @@ public class TestSplitTransaction {
|
||||||
assertEquals(rowcount, parentRowCount);
|
assertEquals(rowcount, parentRowCount);
|
||||||
|
|
||||||
// Start transaction.
|
// Start transaction.
|
||||||
SplitTransaction st = prepareGOOD_SPLIT_ROW();
|
SplitTransactionImpl st = prepareGOOD_SPLIT_ROW();
|
||||||
SplitTransaction spiedUponSt = spy(st);
|
SplitTransactionImpl spiedUponSt = spy(st);
|
||||||
Mockito
|
Mockito
|
||||||
.doThrow(new MockedFailedDaughterOpen())
|
.doThrow(new MockedFailedDaughterOpen())
|
||||||
.when(spiedUponSt)
|
.when(spiedUponSt)
|
||||||
|
@ -161,12 +161,13 @@ public class TestSplitTransaction {
|
||||||
prepareGOOD_SPLIT_ROW();
|
prepareGOOD_SPLIT_ROW();
|
||||||
}
|
}
|
||||||
|
|
||||||
private SplitTransaction prepareGOOD_SPLIT_ROW() {
|
private SplitTransactionImpl prepareGOOD_SPLIT_ROW() throws IOException {
|
||||||
return prepareGOOD_SPLIT_ROW(this.parent);
|
return prepareGOOD_SPLIT_ROW(this.parent);
|
||||||
}
|
}
|
||||||
|
|
||||||
private SplitTransaction prepareGOOD_SPLIT_ROW(final HRegion parentRegion) {
|
private SplitTransactionImpl prepareGOOD_SPLIT_ROW(final HRegion parentRegion)
|
||||||
SplitTransaction st = new SplitTransaction(parentRegion, GOOD_SPLIT_ROW);
|
throws IOException {
|
||||||
|
SplitTransactionImpl st = new SplitTransactionImpl(parentRegion, GOOD_SPLIT_ROW);
|
||||||
assertTrue(st.prepare());
|
assertTrue(st.prepare());
|
||||||
return st;
|
return st;
|
||||||
}
|
}
|
||||||
|
@ -181,7 +182,7 @@ public class TestSplitTransaction {
|
||||||
when(storeMock.close()).thenReturn(ImmutableList.<StoreFile>of());
|
when(storeMock.close()).thenReturn(ImmutableList.<StoreFile>of());
|
||||||
this.parent.stores.put(Bytes.toBytes(""), storeMock);
|
this.parent.stores.put(Bytes.toBytes(""), storeMock);
|
||||||
|
|
||||||
SplitTransaction st = new SplitTransaction(this.parent, GOOD_SPLIT_ROW);
|
SplitTransactionImpl st = new SplitTransactionImpl(this.parent, GOOD_SPLIT_ROW);
|
||||||
|
|
||||||
assertFalse("a region should not be splittable if it has instances of store file references",
|
assertFalse("a region should not be splittable if it has instances of store file references",
|
||||||
st.prepare());
|
st.prepare());
|
||||||
|
@ -192,19 +193,19 @@ public class TestSplitTransaction {
|
||||||
*/
|
*/
|
||||||
@Test public void testPrepareWithBadSplitRow() throws IOException {
|
@Test public void testPrepareWithBadSplitRow() throws IOException {
|
||||||
// Pass start row as split key.
|
// Pass start row as split key.
|
||||||
SplitTransaction st = new SplitTransaction(this.parent, STARTROW);
|
SplitTransactionImpl st = new SplitTransactionImpl(this.parent, STARTROW);
|
||||||
assertFalse(st.prepare());
|
assertFalse(st.prepare());
|
||||||
st = new SplitTransaction(this.parent, HConstants.EMPTY_BYTE_ARRAY);
|
st = new SplitTransactionImpl(this.parent, HConstants.EMPTY_BYTE_ARRAY);
|
||||||
assertFalse(st.prepare());
|
assertFalse(st.prepare());
|
||||||
st = new SplitTransaction(this.parent, new byte [] {'A', 'A', 'A'});
|
st = new SplitTransactionImpl(this.parent, new byte [] {'A', 'A', 'A'});
|
||||||
assertFalse(st.prepare());
|
assertFalse(st.prepare());
|
||||||
st = new SplitTransaction(this.parent, ENDROW);
|
st = new SplitTransactionImpl(this.parent, ENDROW);
|
||||||
assertFalse(st.prepare());
|
assertFalse(st.prepare());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test public void testPrepareWithClosedRegion() throws IOException {
|
@Test public void testPrepareWithClosedRegion() throws IOException {
|
||||||
this.parent.close();
|
this.parent.close();
|
||||||
SplitTransaction st = new SplitTransaction(this.parent, GOOD_SPLIT_ROW);
|
SplitTransactionImpl st = new SplitTransactionImpl(this.parent, GOOD_SPLIT_ROW);
|
||||||
assertFalse(st.prepare());
|
assertFalse(st.prepare());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -220,7 +221,7 @@ public class TestSplitTransaction {
|
||||||
((LruBlockCache) cacheConf.getBlockCache()).clearCache();
|
((LruBlockCache) cacheConf.getBlockCache()).clearCache();
|
||||||
|
|
||||||
// Start transaction.
|
// Start transaction.
|
||||||
SplitTransaction st = prepareGOOD_SPLIT_ROW();
|
SplitTransactionImpl st = prepareGOOD_SPLIT_ROW();
|
||||||
|
|
||||||
// Run the execute. Look at what it returns.
|
// Run the execute. Look at what it returns.
|
||||||
Server mockServer = Mockito.mock(Server.class);
|
Server mockServer = Mockito.mock(Server.class);
|
||||||
|
@ -266,8 +267,8 @@ public class TestSplitTransaction {
|
||||||
|
|
||||||
// Start transaction.
|
// Start transaction.
|
||||||
HRegion spiedRegion = spy(this.parent);
|
HRegion spiedRegion = spy(this.parent);
|
||||||
SplitTransaction st = prepareGOOD_SPLIT_ROW(spiedRegion);
|
SplitTransactionImpl st = prepareGOOD_SPLIT_ROW(spiedRegion);
|
||||||
SplitTransaction spiedUponSt = spy(st);
|
SplitTransactionImpl spiedUponSt = spy(st);
|
||||||
doThrow(new IOException("Failing split. Expected reference file count isn't equal."))
|
doThrow(new IOException("Failing split. Expected reference file count isn't equal."))
|
||||||
.when(spiedUponSt).assertReferenceFileCount(anyInt(),
|
.when(spiedUponSt).assertReferenceFileCount(anyInt(),
|
||||||
eq(new Path(this.parent.getRegionFileSystem().getTableDir(),
|
eq(new Path(this.parent.getRegionFileSystem().getTableDir(),
|
||||||
|
@ -294,8 +295,8 @@ public class TestSplitTransaction {
|
||||||
|
|
||||||
// Start transaction.
|
// Start transaction.
|
||||||
HRegion spiedRegion = spy(this.parent);
|
HRegion spiedRegion = spy(this.parent);
|
||||||
SplitTransaction st = prepareGOOD_SPLIT_ROW(spiedRegion);
|
SplitTransactionImpl st = prepareGOOD_SPLIT_ROW(spiedRegion);
|
||||||
SplitTransaction spiedUponSt = spy(st);
|
SplitTransactionImpl spiedUponSt = spy(st);
|
||||||
doNothing().when(spiedUponSt).assertReferenceFileCount(anyInt(),
|
doNothing().when(spiedUponSt).assertReferenceFileCount(anyInt(),
|
||||||
eq(parent.getRegionFileSystem().getSplitsDir(st.getFirstDaughter())));
|
eq(parent.getRegionFileSystem().getSplitsDir(st.getFirstDaughter())));
|
||||||
when(spiedRegion.createDaughterRegionFromSplits(spiedUponSt.getSecondDaughter())).
|
when(spiedRegion.createDaughterRegionFromSplits(spiedUponSt.getSecondDaughter())).
|
||||||
|
|
|
@ -111,7 +111,7 @@ import com.google.protobuf.RpcController;
|
||||||
import com.google.protobuf.ServiceException;
|
import com.google.protobuf.ServiceException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Like TestSplitTransaction in that we're testing {@link SplitTransaction}
|
* Like TestSplitTransaction in that we're testing {@link SplitTransactionImpl}
|
||||||
* only the below tests are against a running cluster where TestSplitTransaction
|
* only the below tests are against a running cluster where TestSplitTransaction
|
||||||
* is tests against a bare {@link HRegion}.
|
* is tests against a bare {@link HRegion}.
|
||||||
*/
|
*/
|
||||||
|
@ -264,7 +264,7 @@ public class TestSplitTransactionOnCluster {
|
||||||
assertTrue(fileNum > store.getStorefiles().size());
|
assertTrue(fileNum > store.getStorefiles().size());
|
||||||
|
|
||||||
// 3, Split
|
// 3, Split
|
||||||
SplitTransaction st = new SplitTransaction(region, Bytes.toBytes("row3"));
|
SplitTransactionImpl st = new SplitTransactionImpl(region, Bytes.toBytes("row3"));
|
||||||
assertTrue(st.prepare());
|
assertTrue(st.prepare());
|
||||||
st.execute(regionServer, regionServer);
|
st.execute(regionServer, regionServer);
|
||||||
assertEquals(2, cluster.getRegions(tableName).size());
|
assertEquals(2, cluster.getRegions(tableName).size());
|
||||||
|
@ -474,7 +474,6 @@ public class TestSplitTransactionOnCluster {
|
||||||
|
|
||||||
@Test(timeout = 180000)
|
@Test(timeout = 180000)
|
||||||
public void testSplitShouldNotThrowNPEEvenARegionHasEmptySplitFiles() throws Exception {
|
public void testSplitShouldNotThrowNPEEvenARegionHasEmptySplitFiles() throws Exception {
|
||||||
Configuration conf = TESTING_UTIL.getConfiguration();
|
|
||||||
TableName userTableName =
|
TableName userTableName =
|
||||||
TableName.valueOf("testSplitShouldNotThrowNPEEvenARegionHasEmptySplitFiles");
|
TableName.valueOf("testSplitShouldNotThrowNPEEvenARegionHasEmptySplitFiles");
|
||||||
HTableDescriptor htd = new HTableDescriptor(userTableName);
|
HTableDescriptor htd = new HTableDescriptor(userTableName);
|
||||||
|
@ -649,7 +648,7 @@ public class TestSplitTransactionOnCluster {
|
||||||
assertEquals("The specified table should present.", true, tableExists);
|
assertEquals("The specified table should present.", true, tableExists);
|
||||||
final HRegion region = findSplittableRegion(regions);
|
final HRegion region = findSplittableRegion(regions);
|
||||||
assertTrue("not able to find a splittable region", region != null);
|
assertTrue("not able to find a splittable region", region != null);
|
||||||
SplitTransaction st = new SplitTransaction(region, Bytes.toBytes("row2"));
|
SplitTransactionImpl st = new SplitTransactionImpl(region, Bytes.toBytes("row2"));
|
||||||
try {
|
try {
|
||||||
st.prepare();
|
st.prepare();
|
||||||
st.createDaughters(regionServer, regionServer);
|
st.createDaughters(regionServer, regionServer);
|
||||||
|
@ -707,7 +706,7 @@ public class TestSplitTransactionOnCluster {
|
||||||
regionServerIndex = cluster.getServerWith(region.getRegionInfo().getRegionName());
|
regionServerIndex = cluster.getServerWith(region.getRegionInfo().getRegionName());
|
||||||
regionServer = cluster.getRegionServer(regionServerIndex);
|
regionServer = cluster.getRegionServer(regionServerIndex);
|
||||||
assertTrue("not able to find a splittable region", region != null);
|
assertTrue("not able to find a splittable region", region != null);
|
||||||
SplitTransaction st = new SplitTransaction(region, Bytes.toBytes("row2"));
|
SplitTransactionImpl st = new SplitTransactionImpl(region, Bytes.toBytes("row2"));
|
||||||
try {
|
try {
|
||||||
st.prepare();
|
st.prepare();
|
||||||
st.execute(regionServer, regionServer);
|
st.execute(regionServer, regionServer);
|
||||||
|
@ -807,7 +806,7 @@ public class TestSplitTransactionOnCluster {
|
||||||
assertTrue("not able to find a splittable region", region != null);
|
assertTrue("not able to find a splittable region", region != null);
|
||||||
|
|
||||||
// Now split.
|
// Now split.
|
||||||
SplitTransaction st = new MockedSplitTransaction(region, Bytes.toBytes("row2"));
|
SplitTransactionImpl st = new MockedSplitTransaction(region, Bytes.toBytes("row2"));
|
||||||
try {
|
try {
|
||||||
st.prepare();
|
st.prepare();
|
||||||
st.execute(regionServer, regionServer);
|
st.execute(regionServer, regionServer);
|
||||||
|
@ -882,7 +881,7 @@ public class TestSplitTransactionOnCluster {
|
||||||
assertTrue("not able to find a splittable region", region != null);
|
assertTrue("not able to find a splittable region", region != null);
|
||||||
|
|
||||||
// Now split.
|
// Now split.
|
||||||
SplitTransaction st = new SplitTransaction(region, Bytes.toBytes("row2"));
|
SplitTransactionImpl st = new SplitTransactionImpl(region, Bytes.toBytes("row2"));
|
||||||
try {
|
try {
|
||||||
st.prepare();
|
st.prepare();
|
||||||
st.execute(regionServer, regionServer);
|
st.execute(regionServer, regionServer);
|
||||||
|
@ -976,7 +975,7 @@ public class TestSplitTransactionOnCluster {
|
||||||
int serverWith = cluster.getServerWith(regions.get(0).getRegionInfo().getRegionName());
|
int serverWith = cluster.getServerWith(regions.get(0).getRegionInfo().getRegionName());
|
||||||
HRegionServer regionServer = cluster.getRegionServer(serverWith);
|
HRegionServer regionServer = cluster.getRegionServer(serverWith);
|
||||||
cluster.getServerWith(regions.get(0).getRegionInfo().getRegionName());
|
cluster.getServerWith(regions.get(0).getRegionInfo().getRegionName());
|
||||||
SplitTransaction st = new SplitTransaction(regions.get(0), Bytes.toBytes("r3"));
|
SplitTransactionImpl st = new SplitTransactionImpl(regions.get(0), Bytes.toBytes("r3"));
|
||||||
st.prepare();
|
st.prepare();
|
||||||
st.stepsBeforePONR(regionServer, regionServer, false);
|
st.stepsBeforePONR(regionServer, regionServer, false);
|
||||||
Path tableDir =
|
Path tableDir =
|
||||||
|
@ -1019,7 +1018,7 @@ public class TestSplitTransactionOnCluster {
|
||||||
HRegionServer regionServer = cluster.getRegionServer(regionServerIndex);
|
HRegionServer regionServer = cluster.getRegionServer(regionServerIndex);
|
||||||
final HRegion region = findSplittableRegion(regions);
|
final HRegion region = findSplittableRegion(regions);
|
||||||
assertTrue("not able to find a splittable region", region != null);
|
assertTrue("not able to find a splittable region", region != null);
|
||||||
SplitTransaction st = new MockedSplitTransaction(region, Bytes.toBytes("row2")) {
|
SplitTransactionImpl st = new MockedSplitTransaction(region, Bytes.toBytes("row2")) {
|
||||||
@Override
|
@Override
|
||||||
public PairOfSameType<Region> stepsBeforePONR(final Server server,
|
public PairOfSameType<Region> stepsBeforePONR(final Server server,
|
||||||
final RegionServerServices services, boolean testing) throws IOException {
|
final RegionServerServices services, boolean testing) throws IOException {
|
||||||
|
@ -1080,7 +1079,7 @@ public class TestSplitTransactionOnCluster {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class MockedSplitTransaction extends SplitTransaction {
|
public static class MockedSplitTransaction extends SplitTransactionImpl {
|
||||||
|
|
||||||
private HRegion currentRegion;
|
private HRegion currentRegion;
|
||||||
public MockedSplitTransaction(HRegion region, byte[] splitrow) {
|
public MockedSplitTransaction(HRegion region, byte[] splitrow) {
|
||||||
|
@ -1318,7 +1317,7 @@ public class TestSplitTransactionOnCluster {
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class MockedRegionObserver extends BaseRegionObserver {
|
public static class MockedRegionObserver extends BaseRegionObserver {
|
||||||
private SplitTransaction st = null;
|
private SplitTransactionImpl st = null;
|
||||||
private PairOfSameType<Region> daughterRegions = null;
|
private PairOfSameType<Region> daughterRegions = null;
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -1335,7 +1334,7 @@ public class TestSplitTransactionOnCluster {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
st = new SplitTransaction(region, splitKey);
|
st = new SplitTransactionImpl((HRegion) region, splitKey);
|
||||||
if (!st.prepare()) {
|
if (!st.prepare()) {
|
||||||
LOG.error("Prepare for the table " + region.getTableDesc().getNameAsString()
|
LOG.error("Prepare for the table " + region.getTableDesc().getNameAsString()
|
||||||
+ " failed. So returning null. ");
|
+ " failed. So returning null. ");
|
||||||
|
|
|
@ -100,7 +100,8 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
|
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||||
import org.apache.hadoop.hbase.regionserver.SplitTransaction;
|
import org.apache.hadoop.hbase.regionserver.SplitTransactionFactory;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.SplitTransactionImpl;
|
||||||
import org.apache.hadoop.hbase.regionserver.TestEndToEndSplitTransaction;
|
import org.apache.hadoop.hbase.regionserver.TestEndToEndSplitTransaction;
|
||||||
import org.apache.hadoop.hbase.security.access.AccessControlClient;
|
import org.apache.hadoop.hbase.security.access.AccessControlClient;
|
||||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||||
|
@ -1251,7 +1252,9 @@ public class TestHBaseFsck {
|
||||||
int serverWith = cluster.getServerWith(regions.get(0).getRegionInfo().getRegionName());
|
int serverWith = cluster.getServerWith(regions.get(0).getRegionInfo().getRegionName());
|
||||||
HRegionServer regionServer = cluster.getRegionServer(serverWith);
|
HRegionServer regionServer = cluster.getRegionServer(serverWith);
|
||||||
cluster.getServerWith(regions.get(0).getRegionInfo().getRegionName());
|
cluster.getServerWith(regions.get(0).getRegionInfo().getRegionName());
|
||||||
SplitTransaction st = new SplitTransaction(regions.get(0), Bytes.toBytes("r3"));
|
SplitTransactionImpl st = (SplitTransactionImpl)
|
||||||
|
new SplitTransactionFactory(TEST_UTIL.getConfiguration())
|
||||||
|
.create(regions.get(0), Bytes.toBytes("r3"));
|
||||||
st.prepare();
|
st.prepare();
|
||||||
st.stepsBeforePONR(regionServer, regionServer, false);
|
st.stepsBeforePONR(regionServer, regionServer, false);
|
||||||
AssignmentManager am = cluster.getMaster().getAssignmentManager();
|
AssignmentManager am = cluster.getMaster().getAssignmentManager();
|
||||||
|
|
Loading…
Reference in New Issue