HBASE-17132 Cleanup deprecated code for WAL

This commit is contained in:
zhangduo 2016-11-22 10:53:34 +08:00
parent 92b494f117
commit 47a4e3437b
40 changed files with 371 additions and 2262 deletions

View File

@ -19,6 +19,8 @@
package org.apache.hadoop.hbase.coprocessor;
import com.google.common.collect.ImmutableList;
import java.io.IOException;
import java.util.List;
import java.util.Map;
@ -58,13 +60,10 @@ import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileReader;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.wal.WALKey;
import com.google.common.collect.ImmutableList;
/**
* An abstract class that implements RegionObserver.
* By extending it, you can create your own region observer without
@ -485,12 +484,6 @@ public class BaseRegionObserver implements RegionObserver {
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
}
@Override
public void preWALRestore(ObserverContext<RegionCoprocessorEnvironment> env, HRegionInfo info,
HLogKey logKey, WALEdit logEdit) throws IOException {
preWALRestore(env, info, (WALKey)logKey, logEdit);
}
/**
* Implementers should override this version of the method and leave the deprecated one as-is.
*/
@ -499,12 +492,6 @@ public class BaseRegionObserver implements RegionObserver {
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
}
@Override
public void postWALRestore(ObserverContext<RegionCoprocessorEnvironment> env,
HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException {
postWALRestore(env, info, (WALKey)logKey, logEdit);
}
@Override
public void preBulkLoadHFile(final ObserverContext<RegionCoprocessorEnvironment> ctx,
List<Pair<byte[], String>> familyPaths) throws IOException {

View File

@ -21,14 +21,13 @@ package org.apache.hadoop.hbase.coprocessor;
import java.io.IOException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.wal.WALKey;
/**
* An abstract class that implements WALObserver.
@ -53,24 +52,12 @@ public class BaseWALObserver implements WALObserver {
return false;
}
@Override
public boolean preWALWrite(ObserverContext<WALCoprocessorEnvironment> ctx, HRegionInfo info,
HLogKey logKey, WALEdit logEdit) throws IOException {
return preWALWrite(ctx, info, (WALKey)logKey, logEdit);
}
/**
* Implementers should override this method and leave the deprecated version as-is.
*/
@Override
public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException { }
@Override
public void postWALWrite(ObserverContext<WALCoprocessorEnvironment> ctx, HRegionInfo info,
HLogKey logKey, WALEdit logEdit) throws IOException {
postWALWrite(ctx, info, (WALKey)logKey, logEdit);
}
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {}
@Override
public void preWALRoll(ObserverContext<? extends WALCoprocessorEnvironment> ctx,

View File

@ -19,6 +19,8 @@
package org.apache.hadoop.hbase.coprocessor;
import com.google.common.collect.ImmutableList;
import java.io.IOException;
import java.util.List;
import java.util.Map;
@ -59,13 +61,10 @@ import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileReader;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.wal.WALKey;
import com.google.common.collect.ImmutableList;
/**
* Coprocessors implement this interface to observe and mediate client actions
* on the region.
@ -1317,30 +1316,6 @@ public interface RegionObserver extends Coprocessor {
void preWALRestore(final ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException;
/**
* Called before a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
* replayed for this region.
*
* This method is left in place to maintain binary compatibility with older
* {@link RegionObserver}s. If an implementation directly overrides
* {@link #preWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit)} then this version
* won't be called at all, barring problems with the Security Manager. To work correctly
* in the presence of a strict Security Manager, or in the case of an implementation that
* relies on a parent class to implement preWALRestore, you should implement this method
* as a call to the non-deprecated version.
*
* Users of this method will see all edits that can be treated as HLogKey. If there are
* edits that can't be treated as HLogKey they won't be offered to coprocessors that rely
* on this method. If a coprocessor gets skipped because of this mechanism, a log message
* at ERROR will be generated per coprocessor on the logger for {@link CoprocessorHost} once per
* classloader.
*
* @deprecated use {@link #preWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit)}
*/
@Deprecated
void preWALRestore(final ObserverContext<RegionCoprocessorEnvironment> ctx,
HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException;
/**
* Called after a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
* replayed for this region.
@ -1348,30 +1323,6 @@ public interface RegionObserver extends Coprocessor {
void postWALRestore(final ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException;
/**
* Called after a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
* replayed for this region.
*
* This method is left in place to maintain binary compatibility with older
* {@link RegionObserver}s. If an implementation directly overrides
* {@link #postWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit)} then this version
* won't be called at all, barring problems with the Security Manager. To work correctly
* in the presence of a strict Security Manager, or in the case of an implementation that
* relies on a parent class to implement preWALRestore, you should implement this method
* as a call to the non-deprecated version.
*
* Users of this method will see all edits that can be treated as HLogKey. If there are
* edits that can't be treated as HLogKey they won't be offered to coprocessors that rely
* on this method. If a coprocessor gets skipped because of this mechanism, a log message
* at ERROR will be generated per coprocessor on the logger for {@link CoprocessorHost} once per
* classloader.
*
* @deprecated use {@link #postWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit)}
*/
@Deprecated
void postWALRestore(final ObserverContext<RegionCoprocessorEnvironment> ctx,
HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException;
/**
* Called before bulkLoadHFile. Users can create a StoreFile instance to
* access the contents of a HFile.

View File

@ -19,17 +19,16 @@
package org.apache.hadoop.hbase.coprocessor;
import java.io.IOException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import java.io.IOException;
import org.apache.hadoop.hbase.wal.WALKey;
/**
* It's provided to have a way for coprocessors to observe, rewrite,
@ -58,31 +57,6 @@ public interface WALObserver extends Coprocessor {
boolean preWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException;
/**
* Called before a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
* is writen to WAL.
*
* This method is left in place to maintain binary compatibility with older
* {@link WALObserver}s. If an implementation directly overrides
* {@link #preWALWrite(ObserverContext, HRegionInfo, WALKey, WALEdit)} then this version
* won't be called at all, barring problems with the Security Manager. To work correctly
* in the presence of a strict Security Manager, or in the case of an implementation that
* relies on a parent class to implement preWALWrite, you should implement this method
* as a call to the non-deprecated version.
*
* Users of this method will see all edits that can be treated as HLogKey. If there are
* edits that can't be treated as HLogKey they won't be offered to coprocessors that rely
* on this method. If a coprocessor gets skipped because of this mechanism, a log message
* at ERROR will be generated per coprocessor on the logger for {@link CoprocessorHost} once per
* classloader.
*
* @return true if default behavior should be bypassed, false otherwise
* @deprecated use {@link #preWALWrite(ObserverContext, HRegionInfo, WALKey, WALEdit)}
*/
@Deprecated
boolean preWALWrite(ObserverContext<WALCoprocessorEnvironment> ctx,
HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException;
/**
* Called after a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
* is writen to WAL.
@ -90,30 +64,6 @@ public interface WALObserver extends Coprocessor {
void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException;
/**
* Called after a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
* is writen to WAL.
*
* This method is left in place to maintain binary compatibility with older
* {@link WALObserver}s. If an implementation directly overrides
* {@link #postWALWrite(ObserverContext, HRegionInfo, WALKey, WALEdit)} then this version
* won't be called at all, barring problems with the Security Manager. To work correctly
* in the presence of a strict Security Manager, or in the case of an implementation that
* relies on a parent class to implement preWALWrite, you should implement this method
* as a call to the non-deprecated version.
*
* Users of this method will see all edits that can be treated as HLogKey. If there are
* edits that can't be treated as HLogKey they won't be offered to coprocessors that rely
* on this method. If a coprocessor gets skipped because of this mechanism, a log message
* at ERROR will be generated per coprocessor on the logger for {@link CoprocessorHost} once per
* classloader.
*
* @deprecated use {@link #postWALWrite(ObserverContext, HRegionInfo, WALKey, WALEdit)}
*/
@Deprecated
void postWALWrite(ObserverContext<WALCoprocessorEnvironment> ctx,
HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException;
/**
* Called before rolling the current WAL
* @param oldPath the path of the current wal that we are replacing

View File

@ -1,83 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.mapreduce;
import java.io.IOException;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.mapreduce.InputFormat;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
/**
* Simple {@link InputFormat} for {@link org.apache.hadoop.hbase.wal.WAL}
* files.
* @deprecated use {@link WALInputFormat}. Remove in hadoop 3.0
*/
@Deprecated
@InterfaceAudience.Public
public class HLogInputFormat extends InputFormat<HLogKey, WALEdit> {
private static final Log LOG = LogFactory.getLog(HLogInputFormat.class);
public static final String START_TIME_KEY = "hlog.start.time";
public static final String END_TIME_KEY = "hlog.end.time";
// Delegate to WALInputFormat for implementation.
private final WALInputFormat delegate = new WALInputFormat();
/**
* {@link RecordReader} that pulls out the legacy HLogKey format directly.
*/
static class HLogKeyRecordReader extends WALInputFormat.WALRecordReader<HLogKey> {
@Override
public HLogKey getCurrentKey() throws IOException, InterruptedException {
if (!(currentEntry.getKey() instanceof HLogKey)) {
final IllegalStateException exception = new IllegalStateException(
"HLogInputFormat only works when given entries that have HLogKey for keys. This" +
" one had '" + currentEntry.getKey().getClass() + "'");
LOG.error("The deprecated HLogInputFormat has to work with the deprecated HLogKey class, " +
" but HBase internals read the wal entry using some other class." +
" This is a bug; please file an issue or email the developer mailing list. It is " +
"likely that you would not have this problem if you updated to use WALInputFormat. " +
"You will need the following exception details when seeking help from the HBase " +
"community.",
exception);
throw exception;
}
return (HLogKey)currentEntry.getKey();
}
}
@Override
public List<InputSplit> getSplits(JobContext context) throws IOException,
InterruptedException {
return delegate.getSplits(context, START_TIME_KEY, END_TIME_KEY);
}
@Override
public RecordReader<HLogKey, WALEdit> createRecordReader(InputSplit split,
TaskAttemptContext context) throws IOException, InterruptedException {
return new HLogKeyRecordReader();
}
}

View File

@ -80,8 +80,6 @@ public class WALPlayer extends Configured implements Tool {
Configuration.addDeprecation("hlog.bulk.output", BULK_OUTPUT_CONF_KEY);
Configuration.addDeprecation("hlog.input.tables", TABLES_KEY);
Configuration.addDeprecation("hlog.input.tablesmap", TABLE_MAP_KEY);
Configuration.addDeprecation(HLogInputFormat.START_TIME_KEY, WALInputFormat.START_TIME_KEY);
Configuration.addDeprecation(HLogInputFormat.END_TIME_KEY, WALInputFormat.END_TIME_KEY);
}
private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
@ -241,11 +239,10 @@ public class WALPlayer extends Configured implements Tool {
* @return The newly created job.
* @throws IOException When setting up the job fails.
*/
public Job createSubmittableJob(String[] args)
throws IOException {
public Job createSubmittableJob(String[] args) throws IOException {
Configuration conf = getConf();
setupTime(conf, HLogInputFormat.START_TIME_KEY);
setupTime(conf, HLogInputFormat.END_TIME_KEY);
setupTime(conf, WALInputFormat.START_TIME_KEY);
setupTime(conf, WALInputFormat.END_TIME_KEY);
Path inputDir = new Path(args[0]);
String[] tables = args[1].split(",");
String[] tableMap;

View File

@ -19,6 +19,13 @@ package org.apache.hadoop.hbase.regionserver;
import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.io.Closeables;
import java.io.EOFException;
import java.io.FileNotFoundException;
import java.io.IOException;
@ -137,8 +144,6 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory;
import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.ReplayHLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
import org.apache.hadoop.hbase.security.User;
@ -182,13 +187,6 @@ import org.apache.hadoop.util.StringUtils;
import org.apache.htrace.Trace;
import org.apache.htrace.TraceScope;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.io.Closeables;
@SuppressWarnings("deprecation")
@InterfaceAudience.Private
@ -3258,7 +3256,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
long txid;
if (replay) {
// use wal key from the original
walKey = new ReplayHLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
walKey = new WALKey(this.getRegionInfo().getEncodedNameAsBytes(),
this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now,
mutation.getClusterIds(), currentNonceGroup, currentNonce, mvcc);
walKey.setOrigLogSeqNum(batchOp.getReplaySequenceId());
@ -3277,7 +3275,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
writeEntry = mvcc.begin();
}
// we use HLogKey here instead of WALKey directly to support legacy coprocessors.
walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
walKey = new WALKey(this.getRegionInfo().getEncodedNameAsBytes(),
this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now,
mutation.getClusterIds(), currentNonceGroup, currentNonce, mvcc,
this.getReplicationScope());

View File

@ -19,29 +19,26 @@
package org.apache.hadoop.hbase.regionserver;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.protobuf.Message;
import com.google.protobuf.Service;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.NavigableSet;
import java.util.Optional;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.regex.Matcher;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.protobuf.Message;
import com.google.protobuf.Service;
import org.apache.commons.collections.map.AbstractReferenceMap;
import org.apache.commons.collections.map.ReferenceMap;
import org.apache.commons.lang.ClassUtils;
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.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -53,6 +50,8 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability;
@ -79,13 +78,12 @@ import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.regionserver.Region.Operation;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CoprocessorClassLoader;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.wal.WALKey;
/**
* Implements the coprocessor environment and runtime support for coprocessors
@ -114,8 +112,6 @@ public class RegionCoprocessorHost
private Region region;
private RegionServerServices rsServices;
ConcurrentMap<String, Object> sharedData;
private final boolean useLegacyPre;
private final boolean useLegacyPost;
/**
* Constructor
@ -129,14 +125,6 @@ public class RegionCoprocessorHost
this.region = region;
this.rsServices = services;
this.sharedData = sharedData;
// Pick which version of the WAL related events we'll call.
// This way we avoid calling the new version on older RegionObservers so
// we can maintain binary compatibility.
// See notes in javadoc for RegionObserver
useLegacyPre = useLegacyMethod(impl.getClass(), "preWALRestore", ObserverContext.class,
HRegionInfo.class, WALKey.class, WALEdit.class);
useLegacyPost = useLegacyMethod(impl.getClass(), "postWALRestore", ObserverContext.class,
HRegionInfo.class, WALKey.class, WALEdit.class);
}
/** @return the region */
@ -421,31 +409,6 @@ public class RegionCoprocessorHost
rsServices, classData);
}
/**
* HBASE-4014 : This is used by coprocessor hooks which are not declared to throw exceptions.
*
* For example, {@link
* org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost#preOpen()} and
* {@link org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost#postOpen()} are such hooks.
*
* See also
* {@link org.apache.hadoop.hbase.master.MasterCoprocessorHost#handleCoprocessorThrowable(
* CoprocessorEnvironment, Throwable)}
* @param env The coprocessor that threw the exception.
* @param e The exception that was thrown.
*/
private void handleCoprocessorThrowableNoRethrow(
final CoprocessorEnvironment env, final Throwable e) {
try {
handleCoprocessorThrowable(env,e);
} catch (IOException ioe) {
// We cannot throw exceptions from the caller hook, so ignore.
LOG.warn(
"handleCoprocessorThrowable() threw an IOException while attempting to handle Throwable " +
e + ". Ignoring.",e);
}
}
/**
* Invoked before a region open.
*
@ -1470,32 +1433,11 @@ public class RegionCoprocessorHost
@Override
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
throws IOException {
// Once we don't need to support the legacy call, replace RegionOperation with a version
// that's ObserverContext<RegionEnvironment> and avoid this cast.
final RegionEnvironment env = (RegionEnvironment)ctx.getEnvironment();
if (env.useLegacyPre) {
if (logKey instanceof HLogKey) {
oserver.preWALRestore(ctx, info, (HLogKey)logKey, logEdit);
} else {
legacyWarning(oserver.getClass(), "There are wal keys present that are not HLogKey.");
}
} else {
oserver.preWALRestore(ctx, info, logKey, logEdit);
}
oserver.preWALRestore(ctx, info, logKey, logEdit);
}
});
}
/**
* @return true if default behavior should be bypassed, false otherwise
* @deprecated use {@link #preWALRestore(HRegionInfo, WALKey, WALEdit)}; as of 2.0, remove in 3.0
*/
@Deprecated
public boolean preWALRestore(final HRegionInfo info, final HLogKey logKey,
final WALEdit logEdit) throws IOException {
return preWALRestore(info, (WALKey)logKey, logEdit);
}
/**
* @param info
* @param logKey
@ -1508,31 +1450,11 @@ public class RegionCoprocessorHost
@Override
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
throws IOException {
// Once we don't need to support the legacy call, replace RegionOperation with a version
// that's ObserverContext<RegionEnvironment> and avoid this cast.
final RegionEnvironment env = (RegionEnvironment)ctx.getEnvironment();
if (env.useLegacyPost) {
if (logKey instanceof HLogKey) {
oserver.postWALRestore(ctx, info, (HLogKey)logKey, logEdit);
} else {
legacyWarning(oserver.getClass(), "There are wal keys present that are not HLogKey.");
}
} else {
oserver.postWALRestore(ctx, info, logKey, logEdit);
}
oserver.postWALRestore(ctx, info, logKey, logEdit);
}
});
}
/**
* @deprecated use {@link #postWALRestore(HRegionInfo, WALKey, WALEdit)}; as of 2.0, remove in 3.0
*/
@Deprecated
public void postWALRestore(final HRegionInfo info, final HLogKey logKey, final WALEdit logEdit)
throws IOException {
postWALRestore(info, (WALKey)logKey, logEdit);
}
/**
* @param familyPaths pairs of { CF, file path } submitted for bulk load
* @return true if the default operation should be bypassed

View File

@ -661,12 +661,8 @@ public class FSHLog extends AbstractFSWAL<Writer> {
} finally {
rollWriterLock.unlock();
}
try {
if (lowReplication || writer != null && writer.getLength() > logrollsize) {
requestLogRoll(lowReplication);
}
} catch (IOException e) {
LOG.warn("Writer.getLength() failed; continuing", e);
if (lowReplication || writer != null && writer.getLength() > logrollsize) {
requestLogRoll(lowReplication);
}
}

View File

@ -1,285 +0,0 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver.wal;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.EOFException;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import java.util.NavigableMap;
import java.util.UUID;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableUtils;
import com.google.common.annotations.VisibleForTesting;
/**
* A Key for an entry in the change log.
*
* The log intermingles edits to many tables and rows, so each log entry
* identifies the appropriate table and row. Within a table and row, they're
* also sorted.
*
* <p>Some Transactional edits (START, COMMIT, ABORT) will not have an
* associated row.
* @deprecated use WALKey. Deprecated as of 1.0 (HBASE-12522). Remove in 2.0
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
@Deprecated
public class HLogKey extends WALKey implements Writable {
private static final Log LOG = LogFactory.getLog(HLogKey.class);
public HLogKey() {
super();
}
@VisibleForTesting
public HLogKey(final byte[] encodedRegionName, final TableName tablename, long logSeqNum,
final long now, UUID clusterId) {
super(encodedRegionName, tablename, logSeqNum, now, clusterId);
}
public HLogKey(final byte[] encodedRegionName, final TableName tablename) {
super(encodedRegionName, tablename, null);
}
@VisibleForTesting
public HLogKey(final byte[] encodedRegionName, final TableName tablename, final long now) {
super(encodedRegionName, tablename, now);
}
@VisibleForTesting
public HLogKey(final byte[] encodedRegionName, final TableName tablename, final long now,
final NavigableMap<byte[], Integer> replicationScope) {
super(encodedRegionName, tablename, now, replicationScope);
}
public HLogKey(final byte[] encodedRegionName, final TableName tablename, final long now,
final MultiVersionConcurrencyControl mvcc, final NavigableMap<byte[], Integer> scopes) {
super(encodedRegionName, tablename, now, mvcc, scopes);
}
/**
* Create the log key for writing to somewhere.
* We maintain the tablename mainly for debugging purposes.
* A regionName is always a sub-table object.
* <p>Used by log splitting and snapshots.
*
* @param encodedRegionName Encoded name of the region as returned by
* <code>HRegionInfo#getEncodedNameAsBytes()</code>.
* @param tablename - name of table
* @param logSeqNum - log sequence number
* @param now Time at which this edit was written.
* @param clusterIds the clusters that have consumed the change(used in Replication)
*/
public HLogKey(
final byte[] encodedRegionName,
final TableName tablename,
long logSeqNum,
final long now,
List<UUID> clusterIds,
long nonceGroup,
long nonce,
MultiVersionConcurrencyControl mvcc) {
super(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce, mvcc);
}
/**
* Create the log key for writing to somewhere.
* We maintain the tablename mainly for debugging purposes.
* A regionName is always a sub-table object.
* <p>Used by log splitting and snapshots.
*
* @param encodedRegionName Encoded name of the region as returned by
* <code>HRegionInfo#getEncodedNameAsBytes()</code>.
* @param tablename - name of table
* @param logSeqNum - log sequence number
* @param now Time at which this edit was written.
* @param clusterIds the clusters that have consumed the change(used in Replication)
* @param nonceGroup the noncegroup
* @param nonce the nonce
* @param replicationScope the replicationScope of the non-default column families' of the region
*/
public HLogKey(
final byte[] encodedRegionName,
final TableName tablename,
long logSeqNum,
final long now,
List<UUID> clusterIds,
long nonceGroup,
long nonce,
MultiVersionConcurrencyControl mvcc, NavigableMap<byte[], Integer> replicationScope) {
super(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce, mvcc,
replicationScope);
}
/**
* Create the log key for writing to somewhere.
* We maintain the tablename mainly for debugging purposes.
* A regionName is always a sub-table object.
*
* @param encodedRegionName Encoded name of the region as returned by
* <code>HRegionInfo#getEncodedNameAsBytes()</code>.
* @param tablename
* @param now Time at which this edit was written.
* @param clusterIds the clusters that have consumed the change(used in Replication)
* @param nonceGroup
* @param nonce
*/
public HLogKey(final byte[] encodedRegionName,
final TableName tablename,
final long now,
List<UUID> clusterIds,
long nonceGroup,
long nonce,
final MultiVersionConcurrencyControl mvcc) {
super(encodedRegionName, tablename, now, clusterIds, nonceGroup, nonce, mvcc);
}
/**
* Create the log key for writing to somewhere.
* We maintain the tablename mainly for debugging purposes.
* A regionName is always a sub-table object.
*
* @param encodedRegionName Encoded name of the region as returned by
* <code>HRegionInfo#getEncodedNameAsBytes()</code>.
* @param tablename
* @param logSeqNum
* @param nonceGroup
* @param nonce
*/
public HLogKey(final byte [] encodedRegionName, final TableName tablename, long logSeqNum,
long nonceGroup, long nonce, MultiVersionConcurrencyControl mvcc) {
super(encodedRegionName, tablename, logSeqNum, nonceGroup, nonce, mvcc);
}
/**
* @deprecated Don't use these Writables methods. Use PB instead.
*/
@Override
@Deprecated
public void write(DataOutput out) throws IOException {
LOG.warn("HLogKey is being serialized to writable - only expected in test code");
WritableUtils.writeVInt(out, VERSION.code);
if (compressionContext == null) {
Bytes.writeByteArray(out, this.encodedRegionName);
Bytes.writeByteArray(out, this.tablename.getName());
} else {
Compressor.writeCompressed(this.encodedRegionName, 0,
this.encodedRegionName.length, out,
compressionContext.regionDict);
Compressor.writeCompressed(this.tablename.getName(), 0,
this.tablename.getName().length, out,
compressionContext.tableDict);
}
out.writeLong(getSequenceId());
out.writeLong(this.writeTime);
// Don't need to write the clusters information as we are using protobufs from 0.95
// Writing only the first clusterId for testing the legacy read
Iterator<UUID> iterator = clusterIds.iterator();
if(iterator.hasNext()){
out.writeBoolean(true);
UUID clusterId = iterator.next();
out.writeLong(clusterId.getMostSignificantBits());
out.writeLong(clusterId.getLeastSignificantBits());
} else {
out.writeBoolean(false);
}
}
@Override
public void readFields(DataInput in) throws IOException {
Version version = Version.UNVERSIONED;
// HLogKey was not versioned in the beginning.
// In order to introduce it now, we make use of the fact
// that encodedRegionName was written with Bytes.writeByteArray,
// which encodes the array length as a vint which is >= 0.
// Hence if the vint is >= 0 we have an old version and the vint
// encodes the length of encodedRegionName.
// If < 0 we just read the version and the next vint is the length.
// @see Bytes#readByteArray(DataInput)
serializeReplicationScope(false); // writable HLogKey does not contain scopes
int len = WritableUtils.readVInt(in);
byte[] tablenameBytes = null;
if (len < 0) {
// what we just read was the version
version = Version.fromCode(len);
// We only compress V2 of WALkey.
// If compression is on, the length is handled by the dictionary
if (compressionContext == null || !version.atLeast(Version.COMPRESSED)) {
len = WritableUtils.readVInt(in);
}
}
if (compressionContext == null || !version.atLeast(Version.COMPRESSED)) {
this.encodedRegionName = new byte[len];
in.readFully(this.encodedRegionName);
tablenameBytes = Bytes.readByteArray(in);
} else {
this.encodedRegionName = Compressor.readCompressed(in, compressionContext.regionDict);
tablenameBytes = Compressor.readCompressed(in, compressionContext.tableDict);
}
setSequenceId(in.readLong());
this.writeTime = in.readLong();
this.clusterIds.clear();
if (version.atLeast(Version.INITIAL)) {
if (in.readBoolean()) {
// read the older log
// Definitely is the originating cluster
clusterIds.add(new UUID(in.readLong(), in.readLong()));
}
} else {
try {
// dummy read (former byte cluster id)
in.readByte();
} catch(EOFException e) {
// Means it's a very old key, just continue
if (LOG.isTraceEnabled()) LOG.trace(e);
}
}
try {
this.tablename = TableName.valueOf(tablenameBytes);
} catch (IllegalArgumentException iae) {
if (Bytes.toString(tablenameBytes).equals(TableName.OLD_META_STR)) {
// It is a pre-namespace meta table edit, continue with new format.
LOG.info("Got an old .META. edit, continuing with new format ");
this.tablename = TableName.META_TABLE_NAME;
this.encodedRegionName = HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes();
} else if (Bytes.toString(tablenameBytes).equals(TableName.OLD_ROOT_STR)) {
this.tablename = TableName.OLD_ROOT_TABLE_NAME;
throw iae;
} else throw iae;
}
// Do not need to read the clusters information as we are using protobufs from 0.95
}
}

View File

@ -23,18 +23,18 @@ import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.util.LRUDictionary;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hbase.wal.WAL.Entry;
import org.apache.hadoop.hbase.wal.WALKey;
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
public abstract class ReaderBase implements AbstractFSWALProvider.Reader {
@ -91,9 +91,7 @@ public abstract class ReaderBase implements AbstractFSWALProvider.Reader {
public Entry next(Entry reuse) throws IOException {
Entry e = reuse;
if (e == null) {
// we use HLogKey here instead of WALKey directly to support legacy coprocessors,
// seqencefile based readers, and HLogInputFormat.
e = new Entry(new HLogKey(), new WALEdit());
e = new Entry(new WALKey(), new WALEdit());
}
if (compressionContext != null) {
e.setCompressionContext(compressionContext);

View File

@ -1,54 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver.wal;
import java.util.List;
import java.util.UUID;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
/**
* An HLogKey specific to WalEdits coming from replay.
*/
@InterfaceAudience.Private
public class ReplayHLogKey extends HLogKey {
public ReplayHLogKey(final byte [] encodedRegionName, final TableName tablename,
final long now, List<UUID> clusterIds, long nonceGroup, long nonce,
MultiVersionConcurrencyControl mvcc) {
super(encodedRegionName, tablename, now, clusterIds, nonceGroup, nonce, mvcc);
}
public ReplayHLogKey(final byte [] encodedRegionName, final TableName tablename,
long logSeqNum, final long now, List<UUID> clusterIds, long nonceGroup, long nonce,
MultiVersionConcurrencyControl mvcc) {
super(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce, mvcc);
}
/**
* Returns the original sequence id
* @return long the new assigned sequence number
*/
@Override
public long getSequenceId() {
return this.getOrigLogSeqNum();
}
}

View File

@ -1,309 +0,0 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver.wal;
import java.io.IOException;
import java.lang.reflect.Field;
import java.util.NavigableMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.wal.WAL.Entry;
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.SequenceFile.Metadata;
import org.apache.hadoop.io.Text;
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX,
HBaseInterfaceAudience.CONFIG})
public class SequenceFileLogReader extends ReaderBase {
private static final Log LOG = LogFactory.getLog(SequenceFileLogReader.class);
// Legacy stuff from pre-PB WAL metadata.
private static final Text WAL_VERSION_KEY = new Text("version");
// Let the version be 1. Let absence of a version meta tag be old, version 0.
// Set this version '1' to be the version that introduces compression,
// the COMPRESSION_VERSION.
private static final int COMPRESSION_VERSION = 1;
private static final Text WAL_COMPRESSION_TYPE_KEY = new Text("compression.type");
private static final Text DICTIONARY_COMPRESSION_TYPE = new Text("dictionary");
/**
* Hack just to set the correct file length up in SequenceFile.Reader.
* See HADOOP-6307. The below is all about setting the right length on the
* file we are reading. fs.getFileStatus(file).getLen() is passed down to
* a private SequenceFile.Reader constructor. This won't work. Need to do
* the available on the stream. The below is ugly. It makes getPos, the
* first time its called, return length of the file -- i.e. tell a lie -- just
* so this line up in SF.Reader's constructor ends up with right answer:
*
* this.end = in.getPos() + length;
*
*/
private static class WALReader extends SequenceFile.Reader {
WALReader(final FileSystem fs, final Path p, final Configuration c)
throws IOException {
super(fs, p, c);
}
@Override
protected FSDataInputStream openFile(FileSystem fs, Path file,
int bufferSize, long length)
throws IOException {
return new WALReaderFSDataInputStream(super.openFile(fs, file,
bufferSize, length), length);
}
/**
* Override just so can intercept first call to getPos.
*/
static class WALReaderFSDataInputStream extends FSDataInputStream {
private boolean firstGetPosInvocation = true;
private long length;
WALReaderFSDataInputStream(final FSDataInputStream is, final long l)
throws IOException {
super(is);
this.length = l;
}
// This section can be confusing. It is specific to how HDFS works.
// Let me try to break it down. This is the problem:
//
// 1. HDFS DataNodes update the NameNode about a filename's length
// on block boundaries or when a file is closed. Therefore,
// if an RS dies, then the NN's fs.getLength() can be out of date
// 2. this.in.available() would work, but it returns int &
// therefore breaks for files > 2GB (happens on big clusters)
// 3. DFSInputStream.getFileLength() gets the actual length from the DNs
// 4. DFSInputStream is wrapped 2 levels deep : this.in.in
//
// So, here we adjust getPos() using getFileLength() so the
// SequenceFile.Reader constructor (aka: first invocation) comes out
// with the correct end of the file:
// this.end = in.getPos() + length;
@Override
public long getPos() throws IOException {
if (this.firstGetPosInvocation) {
this.firstGetPosInvocation = false;
long adjust = 0;
HdfsDataInputStream hdfsDataInputStream = null;
try {
if (this.in.getClass().getName().endsWith("HdfsDataInputStream")
|| this.in.getClass().getName().endsWith("DFSInputStream")) {
hdfsDataInputStream = (HdfsDataInputStream) this.getWrappedStream();
long realLength = hdfsDataInputStream.getVisibleLength();
assert(realLength >= this.length);
adjust = realLength - this.length;
} else {
LOG.info(
"Input stream class: " + this.in.getClass().getName() + ", not adjusting length");
}
} catch (Exception e) {
LOG.warn("Error while trying to get accurate file length. "
+ "Truncation / data loss may occur if RegionServers die.",
e);
throw new IOException(e);
}
return adjust + super.getPos();
}
return super.getPos();
}
}
}
// Protected for tests.
protected SequenceFile.Reader reader;
long entryStart = 0; // needed for logging exceptions
public SequenceFileLogReader() {
super();
}
@Override
public void close() throws IOException {
try {
if (reader != null) {
this.reader.close();
this.reader = null;
}
} catch (IOException ioe) {
throw addFileInfoToException(ioe);
}
}
@Override
public long getPosition() throws IOException {
return reader != null ? reader.getPosition() : 0;
}
@Override
public void reset() throws IOException {
// Resetting the reader lets us see newly added data if the file is being written to
// We also keep the same compressionContext which was previously populated for this file
reader = new WALReader(fs, path, conf);
}
@Override
protected String initReader(FSDataInputStream stream) throws IOException {
// We don't use the stream because we have to have the magic stream above.
if (stream != null) {
stream.close();
}
reset();
return null;
}
@Override
protected void initAfterCompression(String cellCodecClsName) throws IOException {
// Nothing to do here
}
@Override
protected void initAfterCompression() throws IOException {
// Nothing to do here
}
@Override
protected boolean hasCompression() {
return isWALCompressionEnabled(reader.getMetadata());
}
@Override
protected boolean hasTagCompression() {
// Tag compression not supported with old SequenceFileLog Reader/Writer
return false;
}
/**
* Call this method after init() has been executed
* @return whether WAL compression is enabled
*/
static boolean isWALCompressionEnabled(final Metadata metadata) {
// Check version is >= VERSION?
Text txt = metadata.get(WAL_VERSION_KEY);
if (txt == null || Integer.parseInt(txt.toString()) < COMPRESSION_VERSION) {
return false;
}
// Now check that compression type is present. Currently only one value.
txt = metadata.get(WAL_COMPRESSION_TYPE_KEY);
return txt != null && txt.equals(DICTIONARY_COMPRESSION_TYPE);
}
/**
* fill in the passed entry with teh next key/value.
* Note that because this format deals with our legacy storage, the provided
* Entery MUST use an {@link HLogKey} for the key.
* @return boolean indicating if the contents of Entry have been filled in.
*/
@Override
protected boolean readNext(Entry e) throws IOException {
try {
if (!(e.getKey() instanceof HLogKey)) {
final IllegalArgumentException exception = new IllegalArgumentException(
"SequenceFileLogReader only works when given entries that have HLogKey for keys. This" +
" one had '" + e.getKey().getClass() + "'");
LOG.error("We need to use the legacy SequenceFileLogReader to handle a " +
" pre-0.96 style WAL, but HBase internals failed to use the deprecated HLogKey class." +
" This is a bug; please file an issue or email the developer mailing list. You will " +
"need the following exception details when seeking help from the HBase community.",
exception);
throw exception;
}
boolean hasNext = this.reader.next((HLogKey)e.getKey(), e.getEdit());
if (!hasNext) return false;
// Scopes are probably in WAL edit, move to key
NavigableMap<byte[], Integer> scopes = e.getEdit().getAndRemoveScopes();
if (scopes != null) {
e.getKey().readOlderScopes(scopes);
}
return true;
} catch (IOException ioe) {
throw addFileInfoToException(ioe);
}
}
@Override
protected void seekOnFs(long pos) throws IOException {
try {
reader.seek(pos);
} catch (IOException ioe) {
throw addFileInfoToException(ioe);
}
}
protected IOException addFileInfoToException(final IOException ioe)
throws IOException {
long pos = -1;
try {
pos = getPosition();
} catch (IOException e) {
LOG.warn("Failed getting position to add to throw", e);
}
// See what SequenceFile.Reader thinks is the end of the file
long end = Long.MAX_VALUE;
try {
Field fEnd = SequenceFile.Reader.class.getDeclaredField("end");
fEnd.setAccessible(true);
end = fEnd.getLong(this.reader);
} catch(NoSuchFieldException nfe) {
/* reflection failure, keep going */
if (LOG.isTraceEnabled()) LOG.trace(nfe);
} catch(IllegalAccessException iae) {
/* reflection failure, keep going */
if (LOG.isTraceEnabled()) LOG.trace(iae);
} catch(Exception e) {
/* All other cases. Should we handle it more aggressively? */
LOG.warn("Unexpected exception when accessing the end field", e);
}
String msg = (this.path == null? "": this.path.toString()) +
", entryStart=" + entryStart + ", pos=" + pos +
((end == Long.MAX_VALUE) ? "" : ", end=" + end) +
", edit=" + this.edit;
// Enhance via reflection so we don't change the original class type
try {
return (IOException) ioe.getClass()
.getConstructor(String.class)
.newInstance(msg)
.initCause(ioe);
} catch(NoSuchMethodException nfe) {
/* reflection failure, keep going */
if (LOG.isTraceEnabled()) LOG.trace(nfe);
} catch(IllegalAccessException iae) {
/* reflection failure, keep going */
if (LOG.isTraceEnabled()) LOG.trace(iae);
} catch(Exception e) {
/* All other cases. Should we handle it more aggressively? */
LOG.warn("Unexpected exception when accessing the end field", e);
}
return ioe;
}
}

View File

@ -23,12 +23,15 @@ package org.apache.hadoop.hbase.regionserver.wal;
import java.io.IOException;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.coprocessor.*;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.WALCoprocessorEnvironment;
import org.apache.hadoop.hbase.coprocessor.WALObserver;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALKey;
@ -48,9 +51,6 @@ public class WALCoprocessorHost
private final WAL wal;
final boolean useLegacyPre;
final boolean useLegacyPost;
@Override
public WAL getWAL() {
return wal;
@ -70,14 +70,6 @@ public class WALCoprocessorHost
final WAL wal) {
super(impl, priority, seq, conf);
this.wal = wal;
// Pick which version of the API we'll call.
// This way we avoid calling the new version on older WALObservers so
// we can maintain binary compatibility.
// See notes in javadoc for WALObserver
useLegacyPre = useLegacyMethod(impl.getClass(), "preWALWrite", ObserverContext.class,
HRegionInfo.class, WALKey.class, WALEdit.class);
useLegacyPost = useLegacyMethod(impl.getClass(), "postWALWrite", ObserverContext.class,
HRegionInfo.class, WALKey.class, WALEdit.class);
}
}
@ -131,16 +123,7 @@ public class WALCoprocessorHost
ClassLoader cl = currentThread.getContextClassLoader();
try {
currentThread.setContextClassLoader(env.getClassLoader());
if (env.useLegacyPre) {
if (logKey instanceof HLogKey) {
observer.preWALWrite(ctx, info, (HLogKey)logKey, logEdit);
} else {
legacyWarning(observer.getClass(),
"There are wal keys present that are not HLogKey.");
}
} else {
observer.preWALWrite(ctx, info, logKey, logEdit);
}
observer.preWALWrite(ctx, info, logKey, logEdit);
} catch (Throwable e) {
handleCoprocessorThrowable(env, e);
} finally {
@ -175,16 +158,7 @@ public class WALCoprocessorHost
ClassLoader cl = currentThread.getContextClassLoader();
try {
currentThread.setContextClassLoader(env.getClassLoader());
if (env.useLegacyPost) {
if (logKey instanceof HLogKey) {
observer.postWALWrite(ctx, info, (HLogKey)logKey, logEdit);
} else {
legacyWarning(observer.getClass(),
"There are wal keys present that are not HLogKey.");
}
} else {
observer.postWALWrite(ctx, info, logKey, logEdit);
}
observer.postWALWrite(ctx, info, logKey, logEdit);
} catch (Throwable e) {
handleCoprocessorThrowable(env, e);
} finally {

View File

@ -18,6 +18,10 @@
package org.apache.hadoop.hbase.security.access;
import com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.ListMultimap;
import com.google.common.collect.Lists;
import java.io.ByteArrayInputStream;
import java.io.DataInput;
import java.io.DataInputStream;
@ -39,7 +43,6 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
@ -70,10 +73,9 @@ import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.io.Text;
import com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.ListMultimap;
import com.google.common.collect.Lists;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableUtils;
/**
* Maintains lists of permission grants to users and groups to allow for
@ -596,13 +598,36 @@ public class AccessControlLists {
return ProtobufUtil.prependPBMagic(AccessControlUtil.toUserTablePermissions(perms).toByteArray());
}
// This is part of the old HbaseObjectWritableFor96Migration.
private static final int LIST_CODE = 61;
private static final int WRITABLE_CODE = 14;
private static final int WRITABLE_NOT_ENCODED = 0;
private static List<TablePermission> readWritablePermissions(DataInput in, Configuration conf)
throws IOException, ClassNotFoundException {
assert WritableUtils.readVInt(in) == LIST_CODE;
int length = in.readInt();
List<TablePermission> list = new ArrayList<>(length);
for (int i = 0; i < length; i++) {
assert WritableUtils.readVInt(in) == WRITABLE_CODE;
assert WritableUtils.readVInt(in) == WRITABLE_NOT_ENCODED;
String className = Text.readString(in);
Class<? extends Writable> clazz = conf.getClassByName(className).asSubclass(Writable.class);
Writable instance = WritableFactories.newInstance(clazz, conf);
instance.readFields(in);
list.add((TablePermission) instance);
}
return list;
}
/**
* Reads a set of permissions as {@link org.apache.hadoop.io.Writable} instances
* from the input stream.
* Reads a set of permissions as {@link org.apache.hadoop.io.Writable} instances from the input
* stream.
*/
public static ListMultimap<String, TablePermission> readPermissions(byte[] data,
Configuration conf)
throws DeserializationException {
Configuration conf) throws DeserializationException {
if (ProtobufUtil.isPBMagicPrefix(data)) {
int pblen = ProtobufUtil.lengthOfPBMagic();
try {
@ -614,17 +639,18 @@ public class AccessControlLists {
throw new DeserializationException(e);
}
} else {
ListMultimap<String,TablePermission> perms = ArrayListMultimap.create();
// TODO: We have to re-write non-PB data as PB encoded. Otherwise we will carry old Writables
// forever (here and a couple of other places).
ListMultimap<String, TablePermission> perms = ArrayListMultimap.create();
try {
DataInput in = new DataInputStream(new ByteArrayInputStream(data));
int length = in.readInt();
for (int i=0; i<length; i++) {
for (int i = 0; i < length; i++) {
String user = Text.readString(in);
List<TablePermission> userPerms =
(List)HbaseObjectWritableFor96Migration.readObject(in, conf);
List<TablePermission> userPerms = readWritablePermissions(in, conf);
perms.putAll(user, userPerms);
}
} catch (IOException e) {
} catch (IOException | ClassNotFoundException e) {
throw new DeserializationException(e);
}
return perms;

View File

@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
// imports we use from yet-to-be-moved regionsever.wal
import org.apache.hadoop.hbase.regionserver.wal.CompressionContext;
import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
import org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
@ -227,13 +226,11 @@ public interface WAL extends Closeable {
* Utility class that lets us keep track of the edit with it's key.
*/
class Entry {
private WALEdit edit;
private WALKey key;
private final WALEdit edit;
private final WALKey key;
public Entry() {
edit = new WALEdit();
// we use HLogKey here instead of WALKey directly to support legacy coprocessors.
key = new HLogKey();
this(new WALKey(), new WALEdit());
}
/**
@ -243,7 +240,6 @@ public interface WAL extends Closeable {
* @param key log's key
*/
public Entry(WALKey key, WALEdit edit) {
super();
this.key = key;
this.edit = edit;
}

View File

@ -20,32 +20,29 @@
package org.apache.hadoop.hbase.wal;
import com.google.common.annotations.VisibleForTesting;
import java.io.IOException;
import java.util.Arrays;
import java.io.InterruptedIOException;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.atomic.AtomicReference;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.wal.WAL.Reader;
import org.apache.hadoop.hbase.wal.WALProvider.Writer;
import org.apache.hadoop.hbase.util.CancelableProgressable;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.LeaseNotRecoveredException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
// imports for things that haven't moved from regionserver.wal yet.
import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
import org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogReader;
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
import org.apache.hadoop.hbase.util.CancelableProgressable;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.LeaseNotRecoveredException;
import org.apache.hadoop.hbase.wal.WAL.Reader;
import org.apache.hadoop.hbase.wal.WALProvider.Writer;
/**
* Entry point for users of the Write Ahead Log.
@ -285,7 +282,6 @@ public class WALFactory {
boolean allowCustom) throws IOException {
Class<? extends AbstractFSWALProvider.Reader> lrClass =
allowCustom ? logReaderClass : ProtobufLogReader.class;
try {
// A wal file could be under recovery, so it may take several
// tries to get it open. Instead of claiming it is corrupted, retry
@ -293,38 +289,13 @@ public class WALFactory {
long startWaiting = EnvironmentEdgeManager.currentTime();
long openTimeout = timeoutMillis + startWaiting;
int nbAttempt = 0;
FSDataInputStream stream = null;
AbstractFSWALProvider.Reader reader = null;
while (true) {
try {
if (lrClass != ProtobufLogReader.class) {
// User is overriding the WAL reader, let them.
reader = lrClass.newInstance();
reader.init(fs, path, conf, null);
return reader;
} else {
stream = fs.open(path);
// Note that zero-length file will fail to read PB magic, and attempt to create
// a non-PB reader and fail the same way existing code expects it to. If we get
// rid of the old reader entirely, we need to handle 0-size files differently from
// merely non-PB files.
byte[] magic = new byte[ProtobufLogReader.PB_WAL_MAGIC.length];
boolean isPbWal =
(stream.read(magic) == magic.length)
&& Arrays.equals(magic, ProtobufLogReader.PB_WAL_MAGIC);
reader = isPbWal ? new ProtobufLogReader() : new SequenceFileLogReader();
reader.init(fs, path, conf, stream);
return reader;
}
reader = lrClass.newInstance();
reader.init(fs, path, conf, null);
return reader;
} catch (IOException e) {
if (stream != null) {
try {
stream.close();
} catch (IOException exception) {
LOG.warn("Could not close AbstractFSWALProvider.Reader" + exception.getMessage());
LOG.debug("exception details", exception);
}
}
if (reader != null) {
try {
reader.close();

View File

@ -235,15 +235,6 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
HConstants.NO_NONCE, HConstants.NO_NONCE, null, null);
}
/**
* @deprecated Remove. Useless.
*/
@Deprecated // REMOVE
public WALKey(final byte[] encodedRegionName, final TableName tablename,
final NavigableMap<byte[], Integer> replicationScope) {
this(encodedRegionName, tablename, System.currentTimeMillis(), replicationScope);
}
// TODO: Fix being able to pass in sequenceid.
public WALKey(final byte[] encodedRegionName, final TableName tablename, final long now) {
init(encodedRegionName,
@ -687,7 +678,7 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
? UnsafeByteOperations.unsafeWrap(e.getKey())
: compressor.compress(e.getKey(), compressionContext.familyDict);
builder.addScopes(FamilyScope.newBuilder()
.setFamily(family).setScopeType(ScopeType.valueOf(e.getValue())));
.setFamily(family).setScopeType(ScopeType.forNumber(e.getValue())));
}
}
return builder;
@ -707,12 +698,6 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
this.tablename = TableName.valueOf(walKey.getTableName().toByteArray());
}
clusterIds.clear();
if (walKey.hasClusterId()) {
//When we are reading the older log (0.95.1 release)
//This is definitely the originating cluster
clusterIds.add(new UUID(walKey.getClusterId().getMostSigBits(), walKey.getClusterId()
.getLeastSigBits()));
}
for (HBaseProtos.UUID clusterId : walKey.getClusterIdsList()) {
clusterIds.add(new UUID(clusterId.getMostSigBits(), clusterId.getLeastSigBits()));
}

View File

@ -81,7 +81,7 @@ public interface WALProvider {
interface Writer extends Closeable {
void sync() throws IOException;
void append(WAL.Entry entry) throws IOException;
long getLength() throws IOException;
long getLength();
}
interface AsyncWriter extends Closeable {

View File

@ -18,6 +18,10 @@
*/
package org.apache.hadoop.hbase.wal;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import java.io.EOFException;
import java.io.FileNotFoundException;
import java.io.IOException;
@ -88,7 +92,6 @@ import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.LastSequenceId;
import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.regionserver.wal.WALEditsReplaySink;
@ -119,10 +122,6 @@ import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
import org.apache.hadoop.io.MultipleIOException;
import org.apache.hadoop.ipc.RemoteException;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
/**
* This class is responsible for splitting up a bunch of regionserver commit log
* files that are no longer being written to, into new files, one per region for
@ -298,13 +297,7 @@ public class WALSplitter {
progress_failed = true;
return false;
}
try {
in = getReader(logfile, skipErrors, reporter);
} catch (CorruptedLogFileException e) {
LOG.warn("Could not get reader, corrupted log file " + logPath, e);
ZKSplitLog.markCorrupted(rootDir, logfile.getPath().getName(), fs);
isCorrupted = true;
}
in = getReader(logfile, skipErrors, reporter);
if (in == null) {
LOG.warn("Nothing to split in log file " + logPath);
return true;
@ -388,6 +381,9 @@ public class WALSplitter {
// Some tests pass in a csm of null.
this.csm.getSplitLogWorkerCoordination().markCorrupted(rootDir,
logfile.getPath().getName(), fs);
} else {
// for tests only
ZKSplitLog.markCorrupted(rootDir, logfile.getPath().getName(), fs);
}
isCorrupted = true;
} catch (IOException e) {
@ -2375,8 +2371,7 @@ public class WALSplitter {
for (HBaseProtos.UUID uuid : entry.getKey().getClusterIdsList()) {
clusterIds.add(new UUID(uuid.getMostSigBits(), uuid.getLeastSigBits()));
}
// we use HLogKey here instead of WALKey directly to support legacy coprocessors.
key = new HLogKey(walKeyProto.getEncodedRegionName().toByteArray(), TableName.valueOf(
key = new WALKey(walKeyProto.getEncodedRegionName().toByteArray(), TableName.valueOf(
walKeyProto.getTableName().toByteArray()), replaySeqId, walKeyProto.getWriteTime(),
clusterIds, walKeyProto.getNonceGroup(), walKeyProto.getNonce(), null);
logEntry.setFirst(key);

View File

@ -23,28 +23,23 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import org.apache.hadoop.fs.Path;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.wal.WALKey;
/**
* Class for testing WALObserver coprocessor.
*
* It will monitor WAL writing and restoring, and modify passed-in WALEdit, i.e,
* ignore specified columns when writing, or add a KeyValue. On the other
* side, it checks whether the ignored column is still in WAL when Restoreed
* at region reconstruct.
* Class for testing WALObserver coprocessor. It will monitor WAL writing and restoring, and modify
* passed-in WALEdit, i.e, ignore specified columns when writing, or add a KeyValue. On the other
* side, it checks whether the ignored column is still in WAL when Restoreed at region reconstruct.
*/
public class SampleRegionWALObserver extends BaseRegionObserver
implements WALObserver {
public class SampleRegionWALObserver extends BaseRegionObserver implements WALObserver {
private static final Log LOG = LogFactory.getLog(SampleRegionWALObserver.class);
@ -64,12 +59,6 @@ implements WALObserver {
private boolean preWALRollCalled = false;
private boolean postWALRollCalled = false;
// Deprecated versions
private boolean preWALWriteDeprecatedCalled = false;
private boolean postWALWriteDeprecatedCalled = false;
private boolean preWALRestoreDeprecatedCalled = false;
private boolean postWALRestoreDeprecatedCalled = false;
/**
* Set values: with a table name, a column name which will be ignored, and
* a column name which will be added to WAL.
@ -88,10 +77,6 @@ implements WALObserver {
postWALWriteCalled = false;
preWALRestoreCalled = false;
postWALRestoreCalled = false;
preWALWriteDeprecatedCalled = false;
postWALWriteDeprecatedCalled = false;
preWALRestoreDeprecatedCalled = false;
postWALRestoreDeprecatedCalled = false;
preWALRollCalled = false;
postWALRollCalled = false;
}
@ -102,13 +87,6 @@ implements WALObserver {
postWALWriteCalled = true;
}
@Override
public void postWALWrite(ObserverContext<WALCoprocessorEnvironment> env,
HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException {
postWALWriteDeprecatedCalled = true;
postWALWrite(env, info, (WALKey)logKey, logEdit);
}
@Override
public boolean preWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> env,
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
@ -148,13 +126,6 @@ implements WALObserver {
return bypass;
}
@Override
public boolean preWALWrite(ObserverContext<WALCoprocessorEnvironment> env,
HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException {
preWALWriteDeprecatedCalled = true;
return preWALWrite(env, info, (WALKey)logKey, logEdit);
}
/**
* Triggered before {@link org.apache.hadoop.hbase.regionserver.HRegion} when WAL is
* Restoreed.
@ -165,13 +136,6 @@ implements WALObserver {
preWALRestoreCalled = true;
}
@Override
public void preWALRestore(ObserverContext<RegionCoprocessorEnvironment> env,
HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException {
preWALRestoreDeprecatedCalled = true;
preWALRestore(env, info, (WALKey)logKey, logEdit);
}
@Override
public void preWALRoll(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
Path oldPath, Path newPath) throws IOException {
@ -194,13 +158,6 @@ implements WALObserver {
postWALRestoreCalled = true;
}
@Override
public void postWALRestore(ObserverContext<RegionCoprocessorEnvironment> env,
HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException {
postWALRestoreDeprecatedCalled = true;
postWALRestore(env, info, (WALKey)logKey, logEdit);
}
public boolean isPreWALWriteCalled() {
return preWALWriteCalled;
}
@ -221,22 +178,6 @@ implements WALObserver {
return postWALRestoreCalled;
}
public boolean isPreWALWriteDeprecatedCalled() {
return preWALWriteDeprecatedCalled;
}
public boolean isPostWALWriteDeprecatedCalled() {
return postWALWriteDeprecatedCalled;
}
public boolean isPreWALRestoreDeprecatedCalled() {
return preWALRestoreDeprecatedCalled;
}
public boolean isPostWALRestoreDeprecatedCalled() {
return postWALRestoreDeprecatedCalled;
}
public boolean isPreWALRollCalled() {
return preWALRollCalled;
}
@ -244,11 +185,4 @@ implements WALObserver {
public boolean isPostWALRollCalled() {
return postWALRollCalled;
}
/**
* This class should trigger our legacy support since it does not directly implement the
* newer API methods.
*/
static class Legacy extends SampleRegionWALObserver {
}
}

View File

@ -24,6 +24,8 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import com.google.common.collect.ImmutableList;
import java.io.IOException;
import java.util.List;
import java.util.Map;
@ -31,8 +33,6 @@ import java.util.NavigableSet;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
@ -65,20 +65,16 @@ import org.apache.hadoop.hbase.regionserver.ScanType;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileReader;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.wal.WALKey;
import com.google.common.collect.ImmutableList;
/**
* A sample region observer that tests the RegionObserver interface.
* It works with TestRegionObserverInterface to provide the test case.
*/
public class SimpleRegionObserver extends BaseRegionObserver {
private static final Log LOG = LogFactory.getLog(TestRegionObserverInterface.class);
final AtomicInteger ctBeforeDelete = new AtomicInteger(1);
final AtomicInteger ctPreOpen = new AtomicInteger(0);
@ -130,8 +126,6 @@ public class SimpleRegionObserver extends BaseRegionObserver {
final AtomicInteger ctPostReplayWALs = new AtomicInteger(0);
final AtomicInteger ctPreWALRestore = new AtomicInteger(0);
final AtomicInteger ctPostWALRestore = new AtomicInteger(0);
final AtomicInteger ctPreWALRestoreDeprecated = new AtomicInteger(0);
final AtomicInteger ctPostWALRestoreDeprecated = new AtomicInteger(0);
final AtomicInteger ctPreSplitBeforePONR = new AtomicInteger(0);
final AtomicInteger ctPreSplitAfterPONR = new AtomicInteger(0);
final AtomicInteger ctPreStoreFileReaderOpen = new AtomicInteger(0);
@ -677,26 +671,12 @@ public class SimpleRegionObserver extends BaseRegionObserver {
ctPreWALRestore.incrementAndGet();
}
@Override
public void preWALRestore(ObserverContext<RegionCoprocessorEnvironment> env, HRegionInfo info,
HLogKey logKey, WALEdit logEdit) throws IOException {
preWALRestore(env, info, (WALKey)logKey, logEdit);
ctPreWALRestoreDeprecated.incrementAndGet();
}
@Override
public void postWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> env,
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
ctPostWALRestore.incrementAndGet();
}
@Override
public void postWALRestore(ObserverContext<RegionCoprocessorEnvironment> env,
HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException {
postWALRestore(env, info, (WALKey)logKey, logEdit);
ctPostWALRestoreDeprecated.incrementAndGet();
}
@Override
public StoreFileReader preStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx,
FileSystem fs, Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf,
@ -970,22 +950,7 @@ public class SimpleRegionObserver extends BaseRegionObserver {
return ctPostWALRestore.get();
}
public int getCtPreWALRestoreDeprecated() {
return ctPreWALRestoreDeprecated.get();
}
public int getCtPostWALRestoreDeprecated() {
return ctPostWALRestoreDeprecated.get();
}
public boolean wasStoreFileReaderOpenCalled() {
return ctPreStoreFileReaderOpen.get() > 0 && ctPostStoreFileReaderOpen.get() > 0;
}
/**
* This implementation should trigger our legacy support because it does not directly
* implement the newer API calls.
*/
public static class Legacy extends SimpleRegionObserver {
}
}

View File

@ -81,11 +81,10 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.util.Threads;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category({CoprocessorTests.class, MediumTests.class})
@Category({ CoprocessorTests.class, MediumTests.class })
public class TestRegionObserverInterface {
private static final Log LOG = LogFactory.getLog(TestRegionObserverInterface.class);
@ -104,8 +103,7 @@ public class TestRegionObserverInterface {
Configuration conf = util.getConfiguration();
conf.setBoolean("hbase.master.distributed.log.replay", true);
conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
"org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver",
"org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver$Legacy");
SimpleRegionObserver.class.getName());
util.startMiniCluster();
cluster = util.getMiniHBaseCluster();
@ -116,17 +114,18 @@ public class TestRegionObserverInterface {
util.shutdownMiniCluster();
}
@Test (timeout=300000)
@Test(timeout = 300000)
public void testRegionObserver() throws IOException {
TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testRegionObserver");
// recreate table every time in order to reset the status of the
// coprocessor.
Table table = util.createTable(tableName, new byte[][] {A, B, C});
Table table = util.createTable(tableName, new byte[][] { A, B, C });
try {
verifyMethodResult(SimpleRegionObserver.class, new String[] { "hadPreGet", "hadPostGet",
"hadPrePut", "hadPostPut", "hadDelete", "hadPostStartRegionOperation",
"hadPostCloseRegionOperation", "hadPostBatchMutateIndispensably" }, tableName,
new Boolean[] { false, false, false, false, false, false, false, false });
verifyMethodResult(SimpleRegionObserver.class,
new String[] { "hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut", "hadDelete",
"hadPostStartRegionOperation", "hadPostCloseRegionOperation",
"hadPostBatchMutateIndispensably" },
tableName, new Boolean[] { false, false, false, false, false, false, false, false });
Put put = new Put(ROW);
put.addColumn(A, A, A);
@ -134,15 +133,16 @@ public class TestRegionObserverInterface {
put.addColumn(C, C, C);
table.put(put);
verifyMethodResult(SimpleRegionObserver.class, new String[] { "hadPreGet", "hadPostGet",
"hadPrePut", "hadPostPut", "hadPreBatchMutate", "hadPostBatchMutate", "hadDelete",
"hadPostStartRegionOperation", "hadPostCloseRegionOperation",
"hadPostBatchMutateIndispensably" }, TEST_TABLE, new Boolean[] { false, false, true,
true, true, true, false, true, true, true });
verifyMethodResult(SimpleRegionObserver.class,
new String[] { "hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut", "hadPreBatchMutate",
"hadPostBatchMutate", "hadDelete", "hadPostStartRegionOperation",
"hadPostCloseRegionOperation", "hadPostBatchMutateIndispensably" },
TEST_TABLE,
new Boolean[] { false, false, true, true, true, true, false, true, true, true });
verifyMethodResult(SimpleRegionObserver.class,
new String[] { "getCtPreOpen", "getCtPostOpen", "getCtPreClose", "getCtPostClose" },
tableName, new Integer[] { 1, 1, 0, 0 });
new String[] { "getCtPreOpen", "getCtPostOpen", "getCtPreClose", "getCtPostClose" },
tableName, new Integer[] { 1, 1, 0, 0 });
Get get = new Get(ROW);
get.addColumn(A, A);
@ -151,9 +151,9 @@ public class TestRegionObserverInterface {
table.get(get);
verifyMethodResult(SimpleRegionObserver.class,
new String[] { "hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut", "hadDelete",
"hadPrePreparedDeleteTS" }, tableName,
new Boolean[] { true, true, true, true, false, false });
new String[] { "hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut", "hadDelete",
"hadPrePreparedDeleteTS" },
tableName, new Boolean[] { true, true, true, true, false, false });
Delete delete = new Delete(ROW);
delete.addColumn(A, A);
@ -162,31 +162,26 @@ public class TestRegionObserverInterface {
table.delete(delete);
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
"hadPreBatchMutate", "hadPostBatchMutate", "hadDelete", "hadPrePreparedDeleteTS"},
tableName,
new Boolean[] {true, true, true, true, true, true, true, true}
);
new String[] { "hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut", "hadPreBatchMutate",
"hadPostBatchMutate", "hadDelete", "hadPrePreparedDeleteTS" },
tableName, new Boolean[] { true, true, true, true, true, true, true, true });
} finally {
util.deleteTable(tableName);
table.close();
}
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"getCtPreOpen", "getCtPostOpen", "getCtPreClose", "getCtPostClose"},
tableName,
new Integer[] {1, 1, 1, 1});
new String[] { "getCtPreOpen", "getCtPostOpen", "getCtPreClose", "getCtPostClose" },
tableName, new Integer[] { 1, 1, 1, 1 });
}
@Test (timeout=300000)
@Test(timeout = 300000)
public void testRowMutation() throws IOException {
TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testRowMutation");
Table table = util.createTable(tableName, new byte[][] { A, B, C });
try {
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
"hadDeleted"},
tableName,
new Boolean[] {false, false, false, false, false});
new String[] { "hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut", "hadDeleted" },
tableName, new Boolean[] { false, false, false, false, false });
Put put = new Put(ROW);
put.addColumn(A, A, A);
put.addColumn(B, B, B);
@ -203,18 +198,15 @@ public class TestRegionObserverInterface {
table.mutateRow(arm);
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
"hadDeleted"},
tableName,
new Boolean[] {false, false, true, true, true}
);
new String[] { "hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut", "hadDeleted" },
tableName, new Boolean[] { false, false, true, true, true });
} finally {
util.deleteTable(tableName);
table.close();
}
}
@Test (timeout=300000)
@Test(timeout = 300000)
public void testIncrementHook() throws IOException {
TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testIncrementHook");
Table table = util.createTable(tableName, new byte[][] { A, B, C });
@ -223,80 +215,68 @@ public class TestRegionObserverInterface {
inc.addColumn(A, A, 1);
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreIncrement", "hadPostIncrement", "hadPreIncrementAfterRowLock"},
tableName,
new Boolean[] {false, false, false}
);
new String[] { "hadPreIncrement", "hadPostIncrement", "hadPreIncrementAfterRowLock" },
tableName, new Boolean[] { false, false, false });
table.increment(inc);
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreIncrement", "hadPostIncrement", "hadPreIncrementAfterRowLock"},
tableName,
new Boolean[] {true, true, true}
);
new String[] { "hadPreIncrement", "hadPostIncrement", "hadPreIncrementAfterRowLock" },
tableName, new Boolean[] { true, true, true });
} finally {
util.deleteTable(tableName);
table.close();
}
}
@Test (timeout=300000)
@Test(timeout = 300000)
public void testCheckAndPutHooks() throws IOException {
TableName tableName =
TableName.valueOf(TEST_TABLE.getNameAsString() + ".testCheckAndPutHooks");
try (Table table = util.createTable(tableName, new byte[][] {A, B, C})) {
TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testCheckAndPutHooks");
try (Table table = util.createTable(tableName, new byte[][] { A, B, C })) {
Put p = new Put(Bytes.toBytes(0));
p.addColumn(A, A, A);
table.put(p);
p = new Put(Bytes.toBytes(0));
p.addColumn(A, A, A);
verifyMethodResult(SimpleRegionObserver.class,
new String[] { "hadPreCheckAndPut", "hadPreCheckAndPutAfterRowLock",
"hadPostCheckAndPut" }, tableName, new Boolean[] { false, false, false });
new String[] { "hadPreCheckAndPut", "hadPreCheckAndPutAfterRowLock", "hadPostCheckAndPut" },
tableName, new Boolean[] { false, false, false });
table.checkAndPut(Bytes.toBytes(0), A, A, A, p);
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreCheckAndPut",
"hadPreCheckAndPutAfterRowLock", "hadPostCheckAndPut"},
tableName,
new Boolean[] {true, true, true}
);
new String[] { "hadPreCheckAndPut", "hadPreCheckAndPutAfterRowLock", "hadPostCheckAndPut" },
tableName, new Boolean[] { true, true, true });
} finally {
util.deleteTable(tableName);
}
}
@Test (timeout=300000)
@Test(timeout = 300000)
public void testCheckAndDeleteHooks() throws IOException {
TableName tableName =
TableName.valueOf(TEST_TABLE.getNameAsString() + ".testCheckAndDeleteHooks");
Table table = util.createTable(tableName, new byte[][] {A, B, C});
Table table = util.createTable(tableName, new byte[][] { A, B, C });
try {
Put p = new Put(Bytes.toBytes(0));
p.addColumn(A, A, A);
table.put(p);
Delete d = new Delete(Bytes.toBytes(0));
table.delete(d);
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreCheckAndDelete",
"hadPreCheckAndDeleteAfterRowLock", "hadPostCheckAndDelete"},
tableName,
new Boolean[] {false, false, false}
);
verifyMethodResult(
SimpleRegionObserver.class, new String[] { "hadPreCheckAndDelete",
"hadPreCheckAndDeleteAfterRowLock", "hadPostCheckAndDelete" },
tableName, new Boolean[] { false, false, false });
table.checkAndDelete(Bytes.toBytes(0), A, A, A, d);
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreCheckAndDelete",
"hadPreCheckAndDeleteAfterRowLock", "hadPostCheckAndDelete"},
tableName,
new Boolean[] {true, true, true}
);
verifyMethodResult(
SimpleRegionObserver.class, new String[] { "hadPreCheckAndDelete",
"hadPreCheckAndDeleteAfterRowLock", "hadPostCheckAndDelete" },
tableName, new Boolean[] { true, true, true });
} finally {
util.deleteTable(tableName);
table.close();
}
}
@Test (timeout=300000)
@Test(timeout = 300000)
public void testAppendHook() throws IOException {
TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testAppendHook");
Table table = util.createTable(tableName, new byte[][] { A, B, C });
@ -305,35 +285,30 @@ public class TestRegionObserverInterface {
app.add(A, A, A);
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreAppend", "hadPostAppend", "hadPreAppendAfterRowLock"},
tableName,
new Boolean[] {false, false, false}
);
new String[] { "hadPreAppend", "hadPostAppend", "hadPreAppendAfterRowLock" }, tableName,
new Boolean[] { false, false, false });
table.append(app);
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreAppend", "hadPostAppend", "hadPreAppendAfterRowLock"},
tableName,
new Boolean[] {true, true, true}
);
new String[] { "hadPreAppend", "hadPostAppend", "hadPreAppendAfterRowLock" }, tableName,
new Boolean[] { true, true, true });
} finally {
util.deleteTable(tableName);
table.close();
}
}
@Test (timeout=300000)
@Test(timeout = 300000)
// HBase-3583
public void testHBase3583() throws IOException {
TableName tableName =
TableName.valueOf("testHBase3583");
util.createTable(tableName, new byte[][] {A, B, C});
TableName tableName = TableName.valueOf("testHBase3583");
util.createTable(tableName, new byte[][] { A, B, C });
util.waitUntilAllRegionsAssigned(tableName);
verifyMethodResult(SimpleRegionObserver.class,
new String[] { "hadPreGet", "hadPostGet", "wasScannerNextCalled", "wasScannerCloseCalled" },
tableName, new Boolean[] { false, false, false, false });
new String[] { "hadPreGet", "hadPostGet", "wasScannerNextCalled", "wasScannerCloseCalled" },
tableName, new Boolean[] { false, false, false, false });
Table table = util.getConnection().getTable(tableName);
Put put = new Put(ROW);
@ -347,11 +322,8 @@ public class TestRegionObserverInterface {
// verify that scannerNext and scannerClose upcalls won't be invoked
// when we perform get().
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreGet", "hadPostGet", "wasScannerNextCalled",
"wasScannerCloseCalled"},
tableName,
new Boolean[] {true, true, false, false}
);
new String[] { "hadPreGet", "hadPostGet", "wasScannerNextCalled", "wasScannerCloseCalled" },
tableName, new Boolean[] { true, true, false, false });
Scan s = new Scan();
ResultScanner scanner = table.getScanner(s);
@ -364,15 +336,13 @@ public class TestRegionObserverInterface {
// now scanner hooks should be invoked.
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"wasScannerNextCalled", "wasScannerCloseCalled"},
tableName,
new Boolean[] {true, true}
);
new String[] { "wasScannerNextCalled", "wasScannerCloseCalled" }, tableName,
new Boolean[] { true, true });
util.deleteTable(tableName);
table.close();
}
@Test (timeout=300000)
@Test(timeout = 300000)
public void testHBASE14489() throws IOException {
TableName tableName = TableName.valueOf("testHBASE14489");
Table table = util.createTable(tableName, new byte[][] { A });
@ -396,18 +366,15 @@ public class TestRegionObserverInterface {
}
@Test (timeout=300000)
@Test(timeout = 300000)
// HBase-3758
public void testHBase3758() throws IOException {
TableName tableName =
TableName.valueOf("testHBase3758");
util.createTable(tableName, new byte[][] {A, B, C});
TableName tableName = TableName.valueOf("testHBase3758");
util.createTable(tableName, new byte[][] { A, B, C });
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadDeleted", "wasScannerOpenCalled"},
tableName,
new Boolean[] {false, false}
);
new String[] { "hadDeleted", "wasScannerOpenCalled" }, tableName,
new Boolean[] { false, false });
Table table = util.getConnection().getTable(tableName);
Put put = new Put(ROW);
@ -418,10 +385,8 @@ public class TestRegionObserverInterface {
table.delete(delete);
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadDeleted", "wasScannerOpenCalled"},
tableName,
new Boolean[] {true, false}
);
new String[] { "hadDeleted", "wasScannerOpenCalled" }, tableName,
new Boolean[] { true, false });
Scan s = new Scan();
ResultScanner scanner = table.getScanner(s);
@ -433,11 +398,8 @@ public class TestRegionObserverInterface {
}
// now scanner hooks should be invoked.
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"wasScannerOpenCalled"},
tableName,
new Boolean[] {true}
);
verifyMethodResult(SimpleRegionObserver.class, new String[] { "wasScannerOpenCalled" },
tableName, new Boolean[] { true });
util.deleteTable(tableName);
table.close();
}
@ -448,8 +410,8 @@ public class TestRegionObserverInterface {
long lastFlush;
@Override
public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> e,
Store store, final InternalScanner scanner, final ScanType scanType) {
public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> e, Store store,
final InternalScanner scanner, final ScanType scanType) {
return new InternalScanner() {
@Override
public boolean next(List<Cell> results) throws IOException {
@ -457,8 +419,7 @@ public class TestRegionObserverInterface {
}
@Override
public boolean next(List<Cell> results, ScannerContext scannerContext)
throws IOException {
public boolean next(List<Cell> results, ScannerContext scannerContext) throws IOException {
List<Cell> internalResults = new ArrayList<Cell>();
boolean hasMore;
do {
@ -488,8 +449,8 @@ public class TestRegionObserverInterface {
}
@Override
public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e,
Store store, StoreFile resultFile) {
public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e, Store store,
StoreFile resultFile) {
lastCompaction = EnvironmentEdgeManager.currentTime();
}
@ -498,14 +459,15 @@ public class TestRegionObserverInterface {
lastFlush = EnvironmentEdgeManager.currentTime();
}
}
/**
* Tests overriding compaction handling via coprocessor hooks
* @throws Exception
*/
@Test (timeout=300000)
@Test(timeout = 300000)
public void testCompactionOverride() throws Exception {
TableName compactTable = TableName.valueOf("TestCompactionOverride");
Admin admin = util.getHBaseAdmin();
Admin admin = util.getAdmin();
if (admin.tableExists(compactTable)) {
admin.disableTable(compactTable);
admin.deleteTable(compactTable);
@ -517,7 +479,7 @@ public class TestRegionObserverInterface {
admin.createTable(htd);
Table table = util.getConnection().getTable(compactTable);
for (long i=1; i<=10; i++) {
for (long i = 1; i <= 10; i++) {
byte[] iBytes = Bytes.toBytes(i);
Put put = new Put(iBytes);
put.setDurability(Durability.SKIP_WAL);
@ -526,16 +488,16 @@ public class TestRegionObserverInterface {
}
HRegion firstRegion = cluster.getRegions(compactTable).get(0);
Coprocessor cp = firstRegion.getCoprocessorHost().findCoprocessor(
EvenOnlyCompactor.class.getName());
Coprocessor cp =
firstRegion.getCoprocessorHost().findCoprocessor(EvenOnlyCompactor.class.getName());
assertNotNull("EvenOnlyCompactor coprocessor should be loaded", cp);
EvenOnlyCompactor compactor = (EvenOnlyCompactor)cp;
EvenOnlyCompactor compactor = (EvenOnlyCompactor) cp;
// force a compaction
long ts = System.currentTimeMillis();
admin.flush(compactTable);
// wait for flush
for (int i=0; i<10; i++) {
for (int i = 0; i < 10; i++) {
if (compactor.lastFlush >= ts) {
break;
}
@ -547,25 +509,25 @@ public class TestRegionObserverInterface {
ts = compactor.lastFlush;
admin.majorCompact(compactTable);
// wait for compaction
for (int i=0; i<30; i++) {
for (int i = 0; i < 30; i++) {
if (compactor.lastCompaction >= ts) {
break;
}
Thread.sleep(1000);
}
LOG.debug("Last compaction was at "+compactor.lastCompaction);
LOG.debug("Last compaction was at " + compactor.lastCompaction);
assertTrue("Compaction didn't complete", compactor.lastCompaction >= ts);
// only even rows should remain
ResultScanner scanner = table.getScanner(new Scan());
try {
for (long i=2; i<=10; i+=2) {
for (long i = 2; i <= 10; i += 2) {
Result r = scanner.next();
assertNotNull(r);
assertFalse(r.isEmpty());
byte[] iBytes = Bytes.toBytes(i);
assertArrayEquals("Row should be "+i, r.getRow(), iBytes);
assertArrayEquals("Value should be "+i, r.getValue(A, A), iBytes);
assertArrayEquals("Row should be " + i, r.getRow(), iBytes);
assertArrayEquals("Value should be " + i, r.getValue(A, A), iBytes);
}
} finally {
scanner.close();
@ -573,18 +535,16 @@ public class TestRegionObserverInterface {
table.close();
}
@Test (timeout=300000)
@Test(timeout = 300000)
public void bulkLoadHFileTest() throws Exception {
String testName = TestRegionObserverInterface.class.getName()+".bulkLoadHFileTest";
String testName = TestRegionObserverInterface.class.getName() + ".bulkLoadHFileTest";
TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".bulkLoadHFileTest");
Configuration conf = util.getConfiguration();
Table table = util.createTable(tableName, new byte[][] {A, B, C});
Table table = util.createTable(tableName, new byte[][] { A, B, C });
try (RegionLocator locator = util.getConnection().getRegionLocator(tableName)) {
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreBulkLoadHFile", "hadPostBulkLoadHFile"},
tableName,
new Boolean[] {false, false}
);
new String[] { "hadPreBulkLoadHFile", "hadPostBulkLoadHFile" }, tableName,
new Boolean[] { false, false });
FileSystem fs = util.getTestFileSystem();
final Path dir = util.getDataTestDirOnTestFS(testName).makeQualified(fs);
@ -593,32 +553,30 @@ public class TestRegionObserverInterface {
createHFile(util.getConfiguration(), fs, new Path(familyDir, Bytes.toString(A)), A, A);
// Bulk load
new LoadIncrementalHFiles(conf).doBulkLoad(dir, util.getHBaseAdmin(), table, locator);
new LoadIncrementalHFiles(conf).doBulkLoad(dir, util.getAdmin(), table, locator);
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreBulkLoadHFile", "hadPostBulkLoadHFile"},
tableName,
new Boolean[] {true, true}
);
new String[] { "hadPreBulkLoadHFile", "hadPostBulkLoadHFile" }, tableName,
new Boolean[] { true, true });
} finally {
util.deleteTable(tableName);
table.close();
}
}
@Test (timeout=300000)
@Test(timeout = 300000)
public void testRecovery() throws Exception {
LOG.info(TestRegionObserverInterface.class.getName() +".testRecovery");
LOG.info(TestRegionObserverInterface.class.getName() + ".testRecovery");
TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testRecovery");
Table table = util.createTable(tableName, new byte[][] {A, B, C});
Table table = util.createTable(tableName, new byte[][] { A, B, C });
try (RegionLocator locator = util.getConnection().getRegionLocator(tableName)) {
JVMClusterUtil.RegionServerThread rs1 = cluster.startRegionServer();
ServerName sn2 = rs1.getRegionServer().getServerName();
String regEN = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
util.getHBaseAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes());
while (!sn2.equals(locator.getAllRegionLocations().get(0).getServerName())){
util.getAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes());
while (!sn2.equals(locator.getAllRegionLocations().get(0).getServerName())) {
Thread.sleep(100);
}
@ -632,18 +590,14 @@ public class TestRegionObserverInterface {
table.put(put);
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
"hadPreBatchMutate", "hadPostBatchMutate", "hadDelete"},
tableName,
new Boolean[] {false, false, true, true, true, true, false}
);
new String[] { "hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut", "hadPreBatchMutate",
"hadPostBatchMutate", "hadDelete" },
tableName, new Boolean[] { false, false, true, true, true, true, false });
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"getCtPreReplayWALs", "getCtPostReplayWALs", "getCtPreWALRestore",
"getCtPostWALRestore", "getCtPrePut", "getCtPostPut",
"getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"},
tableName,
new Integer[] {0, 0, 0, 0, 2, 2, 0, 0});
new String[] { "getCtPreReplayWALs", "getCtPostReplayWALs", "getCtPreWALRestore",
"getCtPostWALRestore", "getCtPrePut", "getCtPostPut" },
tableName, new Integer[] { 0, 0, 0, 0, 2, 2 });
cluster.killRegionServer(rs1.getRegionServer().getServerName());
Threads.sleep(1000); // Let the kill soak in.
@ -651,75 +605,16 @@ public class TestRegionObserverInterface {
LOG.info("All regions assigned");
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"getCtPreReplayWALs", "getCtPostReplayWALs", "getCtPreWALRestore",
"getCtPostWALRestore", "getCtPrePut", "getCtPostPut",
"getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"},
tableName,
new Integer[]{1, 1, 2, 2, 0, 0, 0, 0});
new String[] { "getCtPreReplayWALs", "getCtPostReplayWALs", "getCtPreWALRestore",
"getCtPostWALRestore", "getCtPrePut", "getCtPostPut" },
tableName, new Integer[] { 1, 1, 2, 2, 0, 0 });
} finally {
util.deleteTable(tableName);
table.close();
}
}
@Test (timeout=300000)
public void testLegacyRecovery() throws Exception {
LOG.info(TestRegionObserverInterface.class.getName() +".testLegacyRecovery");
TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testLegacyRecovery");
Table table = util.createTable(tableName, new byte[][] {A, B, C});
try {
try (RegionLocator locator = util.getConnection().getRegionLocator(tableName)) {
JVMClusterUtil.RegionServerThread rs1 = cluster.startRegionServer();
ServerName sn2 = rs1.getRegionServer().getServerName();
String regEN = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
util.getHBaseAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes());
while (!sn2.equals(locator.getAllRegionLocations().get(0).getServerName())) {
Thread.sleep(100);
}
Put put = new Put(ROW);
put.addColumn(A, A, A);
put.addColumn(B, B, B);
put.addColumn(C, C, C);
table.put(put);
// put two times
table.put(put);
verifyMethodResult(SimpleRegionObserver.Legacy.class,
new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
"hadPreBatchMutate", "hadPostBatchMutate", "hadDelete"},
tableName,
new Boolean[] {false, false, true, true, true, true, false}
);
verifyMethodResult(SimpleRegionObserver.Legacy.class,
new String[] {"getCtPreReplayWALs", "getCtPostReplayWALs", "getCtPreWALRestore",
"getCtPostWALRestore", "getCtPrePut", "getCtPostPut",
"getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"},
tableName,
new Integer[] {0, 0, 0, 0, 2, 2, 0, 0});
cluster.killRegionServer(rs1.getRegionServer().getServerName());
Threads.sleep(1000); // Let the kill soak in.
util.waitUntilAllRegionsAssigned(tableName);
LOG.info("All regions assigned");
verifyMethodResult(SimpleRegionObserver.Legacy.class,
new String[] {"getCtPreReplayWALs", "getCtPostReplayWALs", "getCtPreWALRestore",
"getCtPostWALRestore", "getCtPrePut", "getCtPostPut",
"getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"},
tableName,
new Integer[]{1, 1, 2, 2, 0, 0, 2, 2});
}
} finally {
util.deleteTable(tableName);
table.close();
}
}
@Test (timeout=300000)
@Test(timeout = 300000)
public void testPreWALRestoreSkip() throws Exception {
LOG.info(TestRegionObserverInterface.class.getName() + ".testPreWALRestoreSkip");
TableName tableName = TableName.valueOf(SimpleRegionObserver.TABLE_SKIPPED);
@ -730,7 +625,7 @@ public class TestRegionObserverInterface {
ServerName sn2 = rs1.getRegionServer().getServerName();
String regEN = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
util.getHBaseAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes());
util.getAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes());
while (!sn2.equals(locator.getAllRegionLocations().get(0).getServerName())) {
Thread.sleep(100);
}
@ -746,10 +641,9 @@ public class TestRegionObserverInterface {
util.waitUntilAllRegionsAssigned(tableName);
}
verifyMethodResult(SimpleRegionObserver.class, new String[] { "getCtPreWALRestore",
"getCtPostWALRestore", "getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"},
tableName,
new Integer[] {0, 0, 0, 0});
verifyMethodResult(SimpleRegionObserver.class,
new String[] { "getCtPreWALRestore", "getCtPostWALRestore", }, tableName,
new Integer[] { 0, 0 });
util.deleteTable(tableName);
table.close();
@ -757,27 +651,28 @@ public class TestRegionObserverInterface {
// check each region whether the coprocessor upcalls are called or not.
private void verifyMethodResult(Class<?> c, String methodName[], TableName tableName,
Object value[]) throws IOException {
Object value[]) throws IOException {
try {
for (JVMClusterUtil.RegionServerThread t : cluster.getRegionServerThreads()) {
if (!t.isAlive() || t.getRegionServer().isAborted() || t.getRegionServer().isStopping()){
if (!t.isAlive() || t.getRegionServer().isAborted() || t.getRegionServer().isStopping()) {
continue;
}
for (HRegionInfo r : ProtobufUtil.getOnlineRegions(t.getRegionServer().getRSRpcServices())) {
for (HRegionInfo r : ProtobufUtil
.getOnlineRegions(t.getRegionServer().getRSRpcServices())) {
if (!r.getTable().equals(tableName)) {
continue;
}
RegionCoprocessorHost cph = t.getRegionServer().getOnlineRegion(r.getRegionName()).
getCoprocessorHost();
RegionCoprocessorHost cph =
t.getRegionServer().getOnlineRegion(r.getRegionName()).getCoprocessorHost();
Coprocessor cp = cph.findCoprocessor(c.getName());
assertNotNull(cp);
for (int i = 0; i < methodName.length; ++i) {
Method m = c.getMethod(methodName[i]);
Object o = m.invoke(cp);
assertTrue("Result of " + c.getName() + "." + methodName[i]
+ " is expected to be " + value[i].toString()
+ ", while we get " + o.toString(), o.equals(value[i]));
assertTrue("Result of " + c.getName() + "." + methodName[i] + " is expected to be "
+ value[i].toString() + ", while we get " + o.toString(),
o.equals(value[i]));
}
}
}
@ -786,19 +681,16 @@ public class TestRegionObserverInterface {
}
}
private static void createHFile(
Configuration conf,
FileSystem fs, Path path,
byte[] family, byte[] qualifier) throws IOException {
private static void createHFile(Configuration conf, FileSystem fs, Path path, byte[] family,
byte[] qualifier) throws IOException {
HFileContext context = new HFileContextBuilder().build();
HFile.Writer writer = HFile.getWriterFactory(conf, new CacheConfig(conf))
.withPath(fs, path)
.withFileContext(context)
.create();
HFile.Writer writer = HFile.getWriterFactory(conf, new CacheConfig(conf)).withPath(fs, path)
.withFileContext(context).create();
long now = System.currentTimeMillis();
try {
for (int i =1;i<=9;i++) {
KeyValue kv = new KeyValue(Bytes.toBytes(i+""), family, qualifier, now, Bytes.toBytes(i+""));
for (int i = 1; i <= 9; i++) {
KeyValue kv =
new KeyValue(Bytes.toBytes(i + ""), family, qualifier, now, Bytes.toBytes(i + ""));
writer.append(kv);
}
} finally {

View File

@ -51,7 +51,6 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.security.User;
@ -110,7 +109,7 @@ public class TestWALObserver {
public static void setupBeforeClass() throws Exception {
Configuration conf = TEST_UTIL.getConfiguration();
conf.setStrings(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY,
SampleRegionWALObserver.class.getName(), SampleRegionWALObserver.Legacy.class.getName());
SampleRegionWALObserver.class.getName());
conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
SampleRegionWALObserver.class.getName());
conf.setInt("dfs.client.block.recovery.retries", 2);
@ -169,17 +168,6 @@ public class TestWALObserver {
verifyWritesSeen(log, getCoprocessor(log, SampleRegionWALObserver.class), false);
}
/**
* Test WAL write behavior with WALObserver. The coprocessor monitors a
* WALEdit written to WAL, and ignore, modify, and add KeyValue's for the
* WALEdit.
*/
@Test
public void testLegacyWALObserverWriteToWAL() throws Exception {
final WAL log = wals.getWAL(UNSPECIFIED_REGION, null);
verifyWritesSeen(log, getCoprocessor(log, SampleRegionWALObserver.Legacy.class), true);
}
private void verifyWritesSeen(final WAL log, final SampleRegionWALObserver cp,
final boolean seesLegacy) throws Exception {
HRegionInfo hri = createBasic3FamilyHRegionInfo(Bytes.toString(TEST_TABLE));
@ -202,8 +190,6 @@ public class TestWALObserver {
assertFalse(cp.isPreWALWriteCalled());
assertFalse(cp.isPostWALWriteCalled());
assertFalse(cp.isPreWALWriteDeprecatedCalled());
assertFalse(cp.isPostWALWriteDeprecatedCalled());
// TEST_FAMILY[2] is not in the put, however it shall be added by the tested
// coprocessor.
@ -241,7 +227,7 @@ public class TestWALObserver {
long now = EnvironmentEdgeManager.currentTime();
// we use HLogKey here instead of WALKey directly to support legacy coprocessors.
long txid = log.append(hri,
new HLogKey(hri.getEncodedNameAsBytes(), hri.getTable(), now, scopes), edit, true);
new WALKey(hri.getEncodedNameAsBytes(), hri.getTable(), now, scopes), edit, true);
log.sync(txid);
// the edit shall have been change now by the coprocessor.
@ -267,88 +253,6 @@ public class TestWALObserver {
assertTrue(cp.isPreWALWriteCalled());
assertTrue(cp.isPostWALWriteCalled());
assertEquals(seesLegacy, cp.isPreWALWriteDeprecatedCalled());
assertEquals(seesLegacy, cp.isPostWALWriteDeprecatedCalled());
}
@Test
public void testNonLegacyWALKeysDoNotExplode() throws Exception {
TableName tableName = TableName.valueOf(TEST_TABLE);
final HTableDescriptor htd = createBasic3FamilyHTD(Bytes
.toString(TEST_TABLE));
final HRegionInfo hri = new HRegionInfo(tableName, null, null);
MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
fs.mkdirs(new Path(FSUtils.getTableDir(hbaseRootDir, tableName), hri.getEncodedName()));
final Configuration newConf = HBaseConfiguration.create(this.conf);
final WAL wal = wals.getWAL(UNSPECIFIED_REGION, null);
final SampleRegionWALObserver newApi = getCoprocessor(wal, SampleRegionWALObserver.class);
newApi.setTestValues(TEST_TABLE, TEST_ROW, null, null, null, null, null, null);
final SampleRegionWALObserver oldApi = getCoprocessor(wal,
SampleRegionWALObserver.Legacy.class);
oldApi.setTestValues(TEST_TABLE, TEST_ROW, null, null, null, null, null, null);
LOG.debug("ensuring wal entries haven't happened before we start");
assertFalse(newApi.isPreWALWriteCalled());
assertFalse(newApi.isPostWALWriteCalled());
assertFalse(newApi.isPreWALWriteDeprecatedCalled());
assertFalse(newApi.isPostWALWriteDeprecatedCalled());
assertFalse(oldApi.isPreWALWriteCalled());
assertFalse(oldApi.isPostWALWriteCalled());
assertFalse(oldApi.isPreWALWriteDeprecatedCalled());
assertFalse(oldApi.isPostWALWriteDeprecatedCalled());
LOG.debug("writing to WAL with non-legacy keys.");
NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
Bytes.BYTES_COMPARATOR);
for (HColumnDescriptor hcd : htd.getFamilies()) {
scopes.put(hcd.getName(), 0);
}
final int countPerFamily = 5;
for (HColumnDescriptor hcd : htd.getFamilies()) {
addWALEdits(tableName, hri, TEST_ROW, hcd.getName(), countPerFamily,
EnvironmentEdgeManager.getDelegate(), wal, scopes, mvcc);
}
LOG.debug("Verify that only the non-legacy CP saw edits.");
assertTrue(newApi.isPreWALWriteCalled());
assertTrue(newApi.isPostWALWriteCalled());
assertFalse(newApi.isPreWALWriteDeprecatedCalled());
assertFalse(newApi.isPostWALWriteDeprecatedCalled());
// wish we could test that the log message happened :/
assertFalse(oldApi.isPreWALWriteCalled());
assertFalse(oldApi.isPostWALWriteCalled());
assertFalse(oldApi.isPreWALWriteDeprecatedCalled());
assertFalse(oldApi.isPostWALWriteDeprecatedCalled());
LOG.debug("reseting cp state.");
newApi.setTestValues(TEST_TABLE, TEST_ROW, null, null, null, null, null, null);
oldApi.setTestValues(TEST_TABLE, TEST_ROW, null, null, null, null, null, null);
LOG.debug("write a log edit that supports legacy cps.");
final long now = EnvironmentEdgeManager.currentTime();
final WALKey legacyKey = new HLogKey(hri.getEncodedNameAsBytes(), hri.getTable(), now);
final WALEdit edit = new WALEdit();
final byte[] nonce = Bytes.toBytes("1772");
edit.add(new KeyValue(TEST_ROW, TEST_FAMILY[0], nonce, now, nonce));
final long txid = wal.append(hri, legacyKey, edit, true);
wal.sync(txid);
LOG.debug("Make sure legacy cps can see supported edits after having been skipped.");
assertTrue("non-legacy WALObserver didn't see pre-write.", newApi.isPreWALWriteCalled());
assertTrue("non-legacy WALObserver didn't see post-write.", newApi.isPostWALWriteCalled());
assertFalse("non-legacy WALObserver shouldn't have seen legacy pre-write.",
newApi.isPreWALWriteDeprecatedCalled());
assertFalse("non-legacy WALObserver shouldn't have seen legacy post-write.",
newApi.isPostWALWriteDeprecatedCalled());
assertTrue("legacy WALObserver didn't see pre-write.", oldApi.isPreWALWriteCalled());
assertTrue("legacy WALObserver didn't see post-write.", oldApi.isPostWALWriteCalled());
assertTrue("legacy WALObserver didn't see legacy pre-write.",
oldApi.isPreWALWriteDeprecatedCalled());
assertTrue("legacy WALObserver didn't see legacy post-write.",
oldApi.isPostWALWriteDeprecatedCalled());
}
/**
@ -431,8 +335,8 @@ public class TestWALObserver {
User user = HBaseTestingUtility.getDifferentUser(newConf,
".replay.wal.secondtime");
user.runAs(new PrivilegedExceptionAction() {
public Object run() throws Exception {
user.runAs(new PrivilegedExceptionAction<Void>() {
public Void run() throws Exception {
Path p = runWALSplit(newConf);
LOG.info("WALSplit path == " + p);
FileSystem newFS = FileSystem.get(newConf);
@ -450,8 +354,6 @@ public class TestWALObserver {
assertNotNull(cp2);
assertTrue(cp2.isPreWALRestoreCalled());
assertTrue(cp2.isPostWALRestoreCalled());
assertFalse(cp2.isPreWALRestoreDeprecatedCalled());
assertFalse(cp2.isPostWALRestoreDeprecatedCalled());
region.close();
wals2.close();
return null;

View File

@ -1,46 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.mapreduce;
import org.apache.hadoop.hbase.mapreduce.WALInputFormat.WALRecordReader;
import java.util.NavigableMap;
import org.apache.hadoop.hbase.mapreduce.HLogInputFormat.HLogKeyRecordReader;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.testclassification.MapReduceTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.junit.experimental.categories.Category;
/**
* JUnit tests for the record reader in HLogInputFormat
*/
@Category({MapReduceTests.class, MediumTests.class})
public class TestHLogRecordReader extends TestWALRecordReader {
@Override
protected WALKey getWalKey(final long time, NavigableMap<byte[], Integer> scopes) {
return new HLogKey(info.getEncodedNameAsBytes(), tableName, time, mvcc, scopes);
}
@Override
protected WALRecordReader getReader() {
return new HLogKeyRecordReader();
}
}

View File

@ -89,7 +89,6 @@ import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.testclassification.LargeTests;
@ -101,9 +100,9 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hbase.wal.FSHLogProvider;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.wal.WALSplitter;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@ -1308,8 +1307,8 @@ public class TestDistributedLogSplitting {
WALEdit e = new WALEdit();
value++;
e.add(new KeyValue(row, family, qualifier, timeStamp, Bytes.toBytes(value)));
wal.append(curRegionInfo, new HLogKey(curRegionInfo.getEncodedNameAsBytes(), tableName,
System.currentTimeMillis(), null), e, true);
wal.append(curRegionInfo, new WALKey(curRegionInfo.getEncodedNameAsBytes(), tableName,
System.currentTimeMillis()), e, true);
}
wal.sync();
wal.shutdown();
@ -1403,7 +1402,7 @@ public class TestDistributedLogSplitting {
WALEdit e = new WALEdit();
value++;
e.add(new KeyValue(row, family, qualifier, timeStamp, Bytes.toBytes(value)));
wal.append(curRegionInfo, new HLogKey(curRegionInfo.getEncodedNameAsBytes(),
wal.append(curRegionInfo, new WALKey(curRegionInfo.getEncodedNameAsBytes(),
tableName, System.currentTimeMillis()), e, true);
}
wal.sync();
@ -1617,7 +1616,7 @@ public class TestDistributedLogSplitting {
byte[] qualifier = Bytes.toBytes("c" + Integer.toString(i));
e.add(new KeyValue(row, family, qualifier, System.currentTimeMillis(), value));
log.append(curRegionInfo,
new HLogKey(curRegionInfo.getEncodedNameAsBytes(), fullTName,
new WALKey(curRegionInfo.getEncodedNameAsBytes(), fullTName,
System.currentTimeMillis()), e, true);
if (0 == i % syncEvery) {
log.sync();

View File

@ -150,7 +150,7 @@ public class TestFailedAppendAndSync {
}
@Override
public long getLength() throws IOException {
public long getLength() {
return w.getLength();
}
};

View File

@ -43,6 +43,10 @@ import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.security.PrivilegedExceptionAction;
@ -127,19 +131,11 @@ import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
import org.apache.hadoop.hbase.regionserver.Region.RowLock;
import org.apache.hadoop.hbase.regionserver.TestStore.FaultyFileSystem;
import org.apache.hadoop.hbase.regionserver.handler.FinishRegionRecoveringHandler;
import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
import org.apache.hadoop.hbase.regionserver.wal.MetricsWALSource;
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
@ -147,6 +143,13 @@ import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
import org.apache.hadoop.hbase.test.MetricsAssertHelper;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests;
@ -181,10 +184,6 @@ import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
/**
* Basic stand-alone testing of HRegion. No clusters!
*
@ -695,7 +694,7 @@ public class TestHRegion {
WALEdit edit = new WALEdit();
edit.add(new KeyValue(row, family, Bytes.toBytes(i), time, KeyValue.Type.Put, Bytes
.toBytes(i)));
writer.append(new WAL.Entry(new HLogKey(regionName, tableName, i, time,
writer.append(new WAL.Entry(new WALKey(regionName, tableName, i, time,
HConstants.DEFAULT_CLUSTER_ID), edit));
writer.close();
@ -746,7 +745,7 @@ public class TestHRegion {
WALEdit edit = new WALEdit();
edit.add(new KeyValue(row, family, Bytes.toBytes(i), time, KeyValue.Type.Put, Bytes
.toBytes(i)));
writer.append(new WAL.Entry(new HLogKey(regionName, tableName, i, time,
writer.append(new WAL.Entry(new WALKey(regionName, tableName, i, time,
HConstants.DEFAULT_CLUSTER_ID), edit));
writer.close();
@ -849,7 +848,7 @@ public class TestHRegion {
edit.add(new KeyValue(row, family, Bytes.toBytes(i), time, KeyValue.Type.Put, Bytes
.toBytes(i)));
}
writer.append(new WAL.Entry(new HLogKey(regionName, tableName, i, time,
writer.append(new WAL.Entry(new WALKey(regionName, tableName, i, time,
HConstants.DEFAULT_CLUSTER_ID), edit));
writer.close();
}
@ -946,7 +945,7 @@ public class TestHRegion {
long time = System.nanoTime();
writer.append(new WAL.Entry(new HLogKey(regionName, tableName, 10, time,
writer.append(new WAL.Entry(new WALKey(regionName, tableName, 10, time,
HConstants.DEFAULT_CLUSTER_ID), WALEdit.createCompaction(region.getRegionInfo(),
compactionDescriptor)));
writer.close();
@ -1187,7 +1186,7 @@ public class TestHRegion {
}
@Override
public long getLength() throws IOException {
public long getLength() {
return w.getLength();
}
};

View File

@ -190,7 +190,7 @@ public class TestWALLockup {
}
@Override
public long getLength() throws IOException {
public long getLength() {
return w.getLength();
}
};
@ -228,7 +228,7 @@ public class TestWALLockup {
Put put = new Put(bytes);
put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("1"), bytes);
WALKey key = new WALKey(region.getRegionInfo().getEncodedNameAsBytes(), htd.getTableName(),
scopes);
System.currentTimeMillis(), scopes);
WALEdit edit = new WALEdit();
CellScanner CellScanner = put.cellScanner();
assertTrue(CellScanner.advance());
@ -349,7 +349,7 @@ public class TestWALLockup {
}
@Override
public long getLength() throws IOException {
public long getLength() {
return w.getLength();
}
};
@ -403,8 +403,8 @@ public class TestWALLockup {
try {
Put put = new Put(bytes);
put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("1"), bytes);
WALKey key = new WALKey(region.getRegionInfo().getEncodedNameAsBytes(),
htd.getTableName(), scopes);
WALKey key = new WALKey(region.getRegionInfo().getEncodedNameAsBytes(), htd.getTableName(),
System.currentTimeMillis(), scopes);
WALEdit edit = new WALEdit();
CellScanner CellScanner = put.cellScanner();
assertTrue(CellScanner.advance());
@ -435,8 +435,8 @@ public class TestWALLockup {
// make RingBufferEventHandler sleep 1s, so the following sync
// endOfBatch=false
key = new WALKey(region.getRegionInfo().getEncodedNameAsBytes(),
TableName.valueOf("sleep"), scopes);
key = new WALKey(region.getRegionInfo().getEncodedNameAsBytes(), TableName.valueOf("sleep"),
System.currentTimeMillis(), scopes);
dodgyWAL2.append(region.getRegionInfo(), key, edit, true);
Thread t = new Thread("Sync") {
@ -460,7 +460,7 @@ public class TestWALLockup {
}
// make append throw DamagedWALException
key = new WALKey(region.getRegionInfo().getEncodedNameAsBytes(),
TableName.valueOf("DamagedWALException"), scopes);
TableName.valueOf("DamagedWALException"), System.currentTimeMillis(), scopes);
dodgyWAL2.append(region.getRegionInfo(), key, edit, true);
while (latch.getCount() > 0) {

View File

@ -1,5 +1,4 @@
/**
*
* 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
@ -16,7 +15,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver.wal;
import java.io.IOException;
@ -24,8 +22,9 @@ import java.util.LinkedList;
import java.util.Queue;
import org.apache.hadoop.hbase.wal.WAL.Entry;
import org.apache.hadoop.hbase.wal.WALKey;
public class FaultySequenceFileLogReader extends SequenceFileLogReader {
public class FaultyProtobufLogReader extends ProtobufLogReader {
// public until class relocates to o.a.h.h.wal
public enum FailureType {
@ -36,34 +35,31 @@ public class FaultySequenceFileLogReader extends SequenceFileLogReader {
int numberOfFileEntries = 0;
FailureType getFailureType() {
return FailureType.valueOf(conf.get("faultysequencefilelogreader.failuretype", "NONE"));
return FailureType.valueOf(conf.get("faultyprotobuflogreader.failuretype", "NONE"));
}
@Override
public Entry next(Entry reuse) throws IOException {
this.entryStart = this.getPosition();
boolean b = true;
if (nextQueue.isEmpty()) { // Read the whole thing at once and fake reading
while (b == true) {
Entry e = new Entry(new HLogKey(), new WALEdit());
boolean b;
do {
Entry e = new Entry(new WALKey(), new WALEdit());
if (compressionContext != null) {
e.setCompressionContext(compressionContext);
}
b = readNext(e);
nextQueue.offer(e);
numberOfFileEntries++;
}
} while (b);
}
if (nextQueue.size() == this.numberOfFileEntries
&& getFailureType() == FailureType.BEGINNING) {
throw this.addFileInfoToException(new IOException("fake Exception"));
if (nextQueue.size() == this.numberOfFileEntries && getFailureType() == FailureType.BEGINNING) {
throw new IOException("fake Exception");
} else if (nextQueue.size() == this.numberOfFileEntries / 2
&& getFailureType() == FailureType.MIDDLE) {
throw this.addFileInfoToException(new IOException("fake Exception"));
throw new IOException("fake Exception");
} else if (nextQueue.size() == 1 && getFailureType() == FailureType.END) {
throw this.addFileInfoToException(new IOException("fake Exception"));
throw new IOException("fake Exception");
}
if (nextQueue.peek() != null) {

View File

@ -1,239 +0,0 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver.wal;
import java.io.IOException;
import java.lang.reflect.Field;
import java.lang.reflect.InvocationTargetException;
import java.util.TreeMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.util.LRUDictionary;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.wal.FSHLogProvider;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.SequenceFile.CompressionType;
import org.apache.hadoop.io.SequenceFile.Metadata;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.compress.DefaultCodec;
/**
* Implementation of {@link WALProvider.Writer} that delegates to
* SequenceFile.Writer. Legacy implementation only used for compat tests.
*
* Note that because this class writes to the legacy hadoop-specific SequenceFile
* format, users of it must write {@link HLogKey} keys and not arbitrary
* {@link WALKey}s because the latter are not Writables (nor made to work with
* Hadoop serialization).
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
public class SequenceFileLogWriter implements FSHLogProvider.Writer {
private static final Log LOG = LogFactory.getLog(SequenceFileLogWriter.class);
// The sequence file we delegate to.
private SequenceFile.Writer writer;
// This is the FSDataOutputStream instance that is the 'out' instance
// in the SequenceFile.Writer 'writer' instance above.
private FSDataOutputStream writer_out;
private CompressionContext compressionContext;
// Legacy stuff from pre-PB WAL metadata.
private static final Text WAL_VERSION_KEY = new Text("version");
private static final Text WAL_COMPRESSION_TYPE_KEY = new Text("compression.type");
private static final Text DICTIONARY_COMPRESSION_TYPE = new Text("dictionary");
/**
* Default constructor.
*/
public SequenceFileLogWriter() {
super();
}
/**
* Create sequence file Metadata for our WAL file with version and compression
* type (if any).
* @param conf
* @param compress
* @return Metadata instance.
*/
private static Metadata createMetadata(final Configuration conf,
final boolean compress) {
TreeMap<Text, Text> metaMap = new TreeMap<Text, Text>();
metaMap.put(WAL_VERSION_KEY, new Text("1"));
if (compress) {
// Currently we only do one compression type.
metaMap.put(WAL_COMPRESSION_TYPE_KEY, DICTIONARY_COMPRESSION_TYPE);
}
return new Metadata(metaMap);
}
private boolean initializeCompressionContext(Configuration conf, Path path) throws IOException {
boolean doCompress = conf.getBoolean(HConstants.ENABLE_WAL_COMPRESSION, false);
if (doCompress) {
try {
this.compressionContext = new CompressionContext(LRUDictionary.class,
FSUtils.isRecoveredEdits(path), conf.getBoolean(
CompressionContext.ENABLE_WAL_TAGS_COMPRESSION, true));
} catch (Exception e) {
throw new IOException("Failed to initiate CompressionContext", e);
}
}
return doCompress;
}
@Override
public void init(FileSystem fs, Path path, Configuration conf, boolean overwritable)
throws IOException {
boolean compress = initializeCompressionContext(conf, path);
// Create a SF.Writer instance.
try {
// reflection for a version of SequenceFile.createWriter that doesn't
// automatically create the parent directory (see HBASE-2312)
this.writer = (SequenceFile.Writer) SequenceFile.class
.getMethod("createWriter", new Class[] {FileSystem.class,
Configuration.class, Path.class, Class.class, Class.class,
Integer.TYPE, Short.TYPE, Long.TYPE, Boolean.TYPE,
CompressionType.class, CompressionCodec.class, Metadata.class})
.invoke(null, new Object[] {fs, conf, path, HLogKey.class, WALEdit.class,
Integer.valueOf(FSUtils.getDefaultBufferSize(fs)),
Short.valueOf((short)
conf.getInt("hbase.regionserver.hlog.replication",
FSUtils.getDefaultReplication(fs, path))),
Long.valueOf(conf.getLong("hbase.regionserver.hlog.blocksize",
FSUtils.getDefaultBlockSize(fs, path))),
Boolean.valueOf(false) /*createParent*/,
SequenceFile.CompressionType.NONE, new DefaultCodec(),
createMetadata(conf, compress)
});
} catch (InvocationTargetException ite) {
// function was properly called, but threw it's own exception
throw new IOException(ite.getCause());
} catch (Exception e) {
// ignore all other exceptions. related to reflection failure
}
// if reflection failed, use the old createWriter
if (this.writer == null) {
LOG.debug("new createWriter -- HADOOP-6840 -- not available");
this.writer = SequenceFile.createWriter(fs, conf, path,
HLogKey.class, WALEdit.class,
FSUtils.getDefaultBufferSize(fs),
(short) conf.getInt("hbase.regionserver.hlog.replication",
FSUtils.getDefaultReplication(fs, path)),
conf.getLong("hbase.regionserver.hlog.blocksize",
FSUtils.getDefaultBlockSize(fs, path)),
SequenceFile.CompressionType.NONE,
new DefaultCodec(),
null,
createMetadata(conf, compress));
} else {
if (LOG.isTraceEnabled()) LOG.trace("Using new createWriter -- HADOOP-6840");
}
this.writer_out = getSequenceFilePrivateFSDataOutputStreamAccessible();
if (LOG.isTraceEnabled()) LOG.trace("Path=" + path + ", compression=" + compress);
}
// Get at the private FSDataOutputStream inside in SequenceFile so we can
// call sync on it. Make it accessible.
private FSDataOutputStream getSequenceFilePrivateFSDataOutputStreamAccessible()
throws IOException {
FSDataOutputStream out = null;
final Field fields [] = this.writer.getClass().getDeclaredFields();
final String fieldName = "out";
for (int i = 0; i < fields.length; ++i) {
if (fieldName.equals(fields[i].getName())) {
try {
// Make the 'out' field up in SF.Writer accessible.
fields[i].setAccessible(true);
out = (FSDataOutputStream)fields[i].get(this.writer);
break;
} catch (IllegalAccessException ex) {
throw new IOException("Accessing " + fieldName, ex);
} catch (SecurityException e) {
LOG.warn("Does not have access to out field from FSDataOutputStream",
e);
}
}
}
return out;
}
@Override
public void append(WAL.Entry entry) throws IOException {
entry.setCompressionContext(compressionContext);
try {
this.writer.append(entry.getKey(), entry.getEdit());
} catch (NullPointerException npe) {
// Concurrent close...
throw new IOException(npe);
}
}
@Override
public void close() throws IOException {
if (this.writer != null) {
try {
this.writer.close();
} catch (NullPointerException npe) {
// Can get a NPE coming up from down in DFSClient$DFSOutputStream#close
LOG.warn(npe);
}
this.writer = null;
}
}
@Override
public void sync() throws IOException {
try {
this.writer.syncFs();
} catch (NullPointerException npe) {
// Concurrent close...
throw new IOException(npe);
}
}
@Override
public long getLength() throws IOException {
try {
return this.writer.getLength();
} catch (NullPointerException npe) {
// Concurrent close...
throw new IOException(npe);
}
}
/**
* @return The dfsclient out stream up inside SF.Writer made accessible, or
* null if not available.
*/
public FSDataOutputStream getWriterFSDataOutputStream() {
return this.writer_out;
}
}

View File

@ -1,161 +0,0 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver.wal;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.wal.WALProvider;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Tests to read old ROOT, Meta edits.
*/
@Category({RegionServerTests.class, MediumTests.class})
public class TestReadOldRootAndMetaEdits {
private final static Log LOG = LogFactory.getLog(TestReadOldRootAndMetaEdits.class);
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private static Configuration conf;
private static FileSystem fs;
private static Path dir;
@BeforeClass
public static void setupBeforeClass() throws Exception {
conf = TEST_UTIL.getConfiguration();
conf.setClass("hbase.regionserver.hlog.writer.impl",
SequenceFileLogWriter.class, WALProvider.Writer.class);
fs = TEST_UTIL.getTestFileSystem();
dir = new Path(TEST_UTIL.createRootDir(), "testReadOldRootAndMetaEdits");
fs.mkdirs(dir);
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
}
/**
* Inserts three waledits in the wal file, and reads them back. The first edit is of a regular
* table, second waledit is for the ROOT table (it will be ignored while reading),
* and last waledit is for the hbase:meta table, which will be linked to the new system:meta table.
* @throws IOException
*/
@Test
public void testReadOldRootAndMetaEdits() throws IOException {
LOG.debug("testReadOldRootAndMetaEdits");
// kv list to be used for all WALEdits.
byte[] row = Bytes.toBytes("row");
KeyValue kv = new KeyValue(row, row, row, row);
List<KeyValue> kvs = new ArrayList<KeyValue>();
kvs.add(kv);
WALProvider.Writer writer = null;
WAL.Reader reader = null;
// a regular table
TableName t = TableName.valueOf("t");
HRegionInfo tRegionInfo = null;
int logCount = 0;
long timestamp = System.currentTimeMillis();
Path path = new Path(dir, "t");
try {
tRegionInfo = new HRegionInfo(t, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
WAL.Entry tEntry = createAEntry(new HLogKey(tRegionInfo.getEncodedNameAsBytes(), t,
++logCount, timestamp, HConstants.DEFAULT_CLUSTER_ID), kvs);
// create a old root edit (-ROOT-).
WAL.Entry rootEntry = createAEntry(new HLogKey(Bytes.toBytes(TableName.OLD_ROOT_STR),
TableName.OLD_ROOT_TABLE_NAME, ++logCount, timestamp,
HConstants.DEFAULT_CLUSTER_ID), kvs);
// create a old meta edit (hbase:meta).
WAL.Entry oldMetaEntry = createAEntry(new HLogKey(Bytes.toBytes(TableName.OLD_META_STR),
TableName.OLD_META_TABLE_NAME, ++logCount, timestamp,
HConstants.DEFAULT_CLUSTER_ID), kvs);
// write above entries
writer = WALFactory.createWALWriter(fs, path, conf);
writer.append(tEntry);
writer.append(rootEntry);
writer.append(oldMetaEntry);
// sync/close the writer
writer.sync();
writer.close();
// read the log and see things are okay.
reader = WALFactory.createReader(fs, path, conf);
WAL.Entry entry = reader.next();
assertNotNull(entry);
assertTrue(entry.getKey().getTablename().equals(t));
assertEquals(Bytes.toString(entry.getKey().getEncodedRegionName()),
Bytes.toString(tRegionInfo.getEncodedNameAsBytes()));
// read the ROOT waledit, but that will be ignored, and hbase:meta waledit will be read instead.
entry = reader.next();
assertEquals(entry.getKey().getTablename(), TableName.META_TABLE_NAME);
// should reach end of log
assertNull(reader.next());
} finally {
if (writer != null) {
writer.close();
}
if (reader != null) {
reader.close();
}
}
}
/**
* Creates a WALEdit for the passed KeyValues and returns a WALProvider.Entry instance composed of
* the WALEdit and passed WALKey.
* @return WAL.Entry instance for the passed WALKey and KeyValues
*/
private WAL.Entry createAEntry(WALKey walKey, List<KeyValue> kvs) {
WALEdit edit = new WALEdit();
for (KeyValue kv : kvs )
edit.add(kv);
return new WAL.Entry(walKey, edit);
}
}

View File

@ -18,6 +18,12 @@
package org.apache.hadoop.hbase.replication;
import static org.junit.Assert.assertNull;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import com.google.common.collect.Lists;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@ -30,22 +36,17 @@ import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.wal.WAL.Entry;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Assert;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import com.google.common.collect.Lists;
import static org.junit.Assert.*;
import static org.mockito.Mockito.*;
@Category({ReplicationTests.class, SmallTests.class})
public class TestReplicationWALEntryFilters {
@ -59,20 +60,21 @@ public class TestReplicationWALEntryFilters {
SystemTableWALEntryFilter filter = new SystemTableWALEntryFilter();
// meta
WALKey key1 = new WALKey( HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
TableName.META_TABLE_NAME, null);
WALKey key1 = new WALKey(HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
TableName.META_TABLE_NAME, System.currentTimeMillis());
Entry metaEntry = new Entry(key1, null);
assertNull(filter.filter(metaEntry));
// ns table
WALKey key2 = new WALKey(new byte[] {}, TableName.NAMESPACE_TABLE_NAME, null);
WALKey key2 =
new WALKey(new byte[0], TableName.NAMESPACE_TABLE_NAME, System.currentTimeMillis());
Entry nsEntry = new Entry(key2, null);
assertNull(filter.filter(nsEntry));
// user table
WALKey key3 = new WALKey(new byte[] {}, TableName.valueOf("foo"), null);
WALKey key3 = new WALKey(new byte[0], TableName.valueOf("foo"), System.currentTimeMillis());
Entry userEntry = new Entry(key3, null);
assertEquals(userEntry, filter.filter(userEntry));
@ -298,7 +300,8 @@ public class TestReplicationWALEntryFilters {
}
private Entry createEntry(TreeMap<byte[], Integer> scopes, byte[]... kvs) {
WALKey key1 = new WALKey(new byte[] {}, TableName.valueOf("foo"), scopes);
WALKey key1 =
new WALKey(new byte[0], TableName.valueOf("foo"), System.currentTimeMillis(), scopes);
WALEdit edit1 = new WALEdit();
for (byte[] kv : kvs) {

View File

@ -18,6 +18,9 @@
package org.apache.hadoop.hbase.security.access;
import com.google.protobuf.Message;
import com.google.protobuf.RpcController;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInput;
@ -38,7 +41,6 @@ import java.util.NavigableSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
@ -48,6 +50,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Action;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
@ -83,25 +86,20 @@ import org.apache.hadoop.hbase.filter.SkipFilter;
import org.apache.hadoop.hbase.filter.ValueFilter;
import org.apache.hadoop.hbase.filter.WhileMatchFilter;
import org.apache.hadoop.hbase.io.WritableWithSize;
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.wal.WAL.Entry;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ProtoUtil;
import org.apache.hadoop.hbase.wal.WAL.Entry;
import org.apache.hadoop.io.DataOutputOutputStream;
import org.apache.hadoop.io.MapWritable;
import org.apache.hadoop.io.ObjectWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableUtils;
import com.google.protobuf.Message;
import com.google.protobuf.RpcController;
/**
* <p>This is a customized version of the polymorphic hadoop
* {@link ObjectWritable}. It removes UTF8 (HADOOP-414).
@ -232,8 +230,9 @@ class HbaseObjectWritableFor96Migration implements Writable, WritableWithSize, C
addToMap(Entry.class, code++);
addToMap(Entry[].class, code++);
addToMap(HLogKey.class, code++);
// For HLogKey
code++;
addToMap(List.class, code++);
addToMap(NavigableSet.class, code++);
@ -392,7 +391,7 @@ class HbaseObjectWritableFor96Migration implements Writable, WritableWithSize, C
}
/**
* @return the next object code in the list. Used in testing to verify that additional fields are not added
* @return the next object code in the list. Used in testing to verify that additional fields are not added
*/
static int getNextClassCode(){
return NEXT_CLASS_CODE;
@ -541,26 +540,6 @@ class HbaseObjectWritableFor96Migration implements Writable, WritableWithSize, C
byte [] scanBytes = ProtobufUtil.toScan(scan).toByteArray();
out.writeInt(scanBytes.length);
out.write(scanBytes);
} else if (Entry.class.isAssignableFrom(declClass)) {
// Entry is no longer Writable, maintain compatible serialization.
// Writables write their exact runtime class
Class <?> c = instanceObj.getClass();
Integer code = CLASS_TO_CODE.get(c);
if (code == null) {
out.writeByte(NOT_ENCODED);
Text.writeString(out, c.getName());
} else {
writeClassCode(out, c);
}
final Entry entry = (Entry)instanceObj;
// We only support legacy HLogKey
WALKey key = entry.getKey();
if (!(key instanceof HLogKey)) {
throw new IOException("Can't write Entry '" + instanceObj + "' due to key class '" +
key.getClass() + "'");
}
((HLogKey)key).write(out);
entry.getEdit().write(out);
} else {
throw new IOException("Can't write: "+instanceObj+" as "+declClass);
}
@ -721,13 +700,6 @@ class HbaseObjectWritableFor96Migration implements Writable, WritableWithSize, C
declaredClass = ((NullInstance)instance).declaredClass;
instance = null;
}
} else if (Entry.class.isAssignableFrom(instanceClass)) {
// Entry stopped being Writable; maintain serialization support.
final HLogKey key = new HLogKey();
final WALEdit edit = new WALEdit();
key.readFields(in);
edit.readFields(in);
instance = new Entry(key, edit);
} else {
int length = in.readInt();
byte[] objectBytes = new byte[length];

View File

@ -162,8 +162,7 @@ public class TestTablePermissions {
* @throws IOException
*/
public static void writePermissions(DataOutput out,
ListMultimap<String,? extends Permission> perms, Configuration conf)
throws IOException {
ListMultimap<String, ? extends Permission> perms, Configuration conf) throws IOException {
Set<String> keys = perms.keySet();
out.writeInt(keys.size());
for (String key : keys) {
@ -172,7 +171,6 @@ public class TestTablePermissions {
}
}
@Test
public void testBasicWrite() throws Exception {
Configuration conf = UTIL.getConfiguration();

View File

@ -1,37 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.wal;
import java.util.NavigableMap;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.junit.experimental.categories.Category;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
@Category({RegionServerTests.class, LargeTests.class})
public class TestDefaultWALProviderWithHLogKey extends TestFSHLogProvider {
@Override
WALKey getWalKey(final byte[] info, final TableName tableName, final long timestamp,
final NavigableMap<byte[], Integer> scopes) {
return new HLogKey(info, tableName, timestamp, mvcc, scopes);
}
}

View File

@ -29,7 +29,6 @@ import java.util.NavigableMap;
import java.util.Random;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -207,7 +206,6 @@ public class TestFSHLogProvider {
final Configuration localConf = new Configuration(conf);
localConf.set(WALFactory.WAL_PROVIDER, FSHLogProvider.class.getName());
final WALFactory wals = new WALFactory(localConf, null, currentTest.getMethodName());
final AtomicLong sequenceId = new AtomicLong(1);
try {
HRegionInfo hri = new HRegionInfo(htd.getTableName(),
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);

View File

@ -18,11 +18,9 @@
*/
package org.apache.hadoop.hbase.wal;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@ -54,9 +52,6 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver;
import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogReader;
import org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogWriter;
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
import org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
@ -347,9 +342,9 @@ public class TestWALFactory {
if (previousRegion != null) {
assertEquals(previousRegion, region);
}
LOG.info("oldseqno=" + seqno + ", newseqno=" + key.getLogSeqNum());
assertTrue(seqno < key.getLogSeqNum());
seqno = key.getLogSeqNum();
LOG.info("oldseqno=" + seqno + ", newseqno=" + key.getSequenceId());
assertTrue(seqno < key.getSequenceId());
seqno = key.getSequenceId();
previousRegion = region;
count++;
}
@ -675,74 +670,6 @@ public class TestWALFactory {
assertNotNull(c);
}
/**
* @throws IOException
*/
@Test
public void testReadLegacyLog() throws IOException {
final int columnCount = 5;
final int recordCount = 5;
final TableName tableName =
TableName.valueOf("tablename");
final byte[] row = Bytes.toBytes("row");
long timestamp = System.currentTimeMillis();
Path path = new Path(dir, "tempwal");
SequenceFileLogWriter sflw = null;
WAL.Reader reader = null;
try {
HRegionInfo hri = new HRegionInfo(tableName,
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
HTableDescriptor htd = new HTableDescriptor(tableName);
fs.mkdirs(dir);
// Write log in pre-PB format.
sflw = new SequenceFileLogWriter();
sflw.init(fs, path, conf, false);
for (int i = 0; i < recordCount; ++i) {
WALKey key = new HLogKey(
hri.getEncodedNameAsBytes(), tableName, i, timestamp, HConstants.DEFAULT_CLUSTER_ID);
WALEdit edit = new WALEdit();
for (int j = 0; j < columnCount; ++j) {
if (i == 0) {
htd.addFamily(new HColumnDescriptor("column" + j));
}
String value = i + "" + j;
edit.add(new KeyValue(row, row, row, timestamp, Bytes.toBytes(value)));
}
sflw.append(new WAL.Entry(key, edit));
}
sflw.sync();
sflw.close();
// Now read the log using standard means.
reader = wals.createReader(fs, path);
assertTrue(reader instanceof SequenceFileLogReader);
for (int i = 0; i < recordCount; ++i) {
WAL.Entry entry = reader.next();
assertNotNull(entry);
assertEquals(columnCount, entry.getEdit().size());
assertArrayEquals(hri.getEncodedNameAsBytes(), entry.getKey().getEncodedRegionName());
assertEquals(tableName, entry.getKey().getTablename());
int idx = 0;
for (Cell val : entry.getEdit().getCells()) {
assertTrue(Bytes.equals(row, 0, row.length, val.getRowArray(), val.getRowOffset(),
val.getRowLength()));
String value = i + "" + idx;
assertArrayEquals(Bytes.toBytes(value), CellUtil.cloneValue(val));
idx++;
}
}
WAL.Entry entry = reader.next();
assertNull(entry);
} finally {
if (sflw != null) {
sflw.close();
}
if (reader != null) {
reader.close();
}
}
}
static class DumbWALActionsListener extends WALActionsListener.Base {
int increments = 0;

View File

@ -18,7 +18,11 @@
*/
package org.apache.hadoop.hbase.wal;
import static org.junit.Assert.*;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNotSame;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.NavigableSet;
@ -27,21 +31,22 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
import org.apache.hadoop.hbase.wal.WALSplitter.EntryBuffers;
import org.apache.hadoop.hbase.wal.WALSplitter.PipelineController;
import org.apache.hadoop.hbase.wal.WALSplitter.RegionEntryBuffer;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValueTestUtil;
import org.apache.hadoop.hbase.TableName;
// imports for things that haven't moved from regionserver.wal yet.
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.wal.WALSplitter.EntryBuffers;
import org.apache.hadoop.hbase.wal.WALSplitter.PipelineController;
import org.apache.hadoop.hbase.wal.WALSplitter.RegionEntryBuffer;
import org.junit.Test;
import org.junit.experimental.categories.Category;
// imports for things that haven't moved from regionserver.wal yet.
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
/**
* Simple testing of a few WAL methods.
*/
@ -119,10 +124,6 @@ public class TestWALMethods {
@Test
public void testEntrySink() throws Exception {
Configuration conf = new Configuration();
RecoveryMode mode = (conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false) ?
RecoveryMode.LOG_REPLAY : RecoveryMode.LOG_SPLITTING);
EntryBuffers sink = new EntryBuffers(new PipelineController(), 1*1024*1024);
for (int i = 0; i < 1000; i++) {
WAL.Entry entry = createTestLogEntry(i);

View File

@ -43,6 +43,7 @@ import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -62,16 +63,16 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.wal.FaultySequenceFileLogReader;
import org.apache.hadoop.hbase.regionserver.wal.FaultyProtobufLogReader;
import org.apache.hadoop.hbase.regionserver.wal.InstrumentedLogWriter;
import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -229,7 +230,7 @@ public class TestWALSplit {
while (startCount == counter.get()) Threads.sleep(1);
// Give it a second to write a few appends.
Threads.sleep(1000);
final Configuration conf2 = HBaseConfiguration.create(this.conf);
final Configuration conf2 = HBaseConfiguration.create(conf);
final User robber = User.createUserForTesting(conf2, ROBBER, GROUP);
int count = robber.runAs(new PrivilegedExceptionAction<Integer>() {
@Override
@ -571,11 +572,13 @@ public class TestWALSplit {
REGIONS.size() * (goodEntries + firstHalfEntries) <= allRegionsCount);
}
@Test (timeout=300000)
@Test(timeout = 300000)
public void testCorruptedFileGetsArchivedIfSkipErrors() throws IOException {
conf.setBoolean(HBASE_SKIP_ERRORS, true);
for (FaultySequenceFileLogReader.FailureType failureType :
FaultySequenceFileLogReader.FailureType.values()) {
List<FaultyProtobufLogReader.FailureType> failureTypes = Arrays
.asList(FaultyProtobufLogReader.FailureType.values()).stream()
.filter(x -> x != FaultyProtobufLogReader.FailureType.NONE).collect(Collectors.toList());
for (FaultyProtobufLogReader.FailureType failureType : failureTypes) {
final Set<String> walDirContents = splitCorruptWALs(failureType);
final Set<String> archivedLogs = new HashSet<String>();
final StringBuilder archived = new StringBuilder("Archived logs in CORRUPTDIR:");
@ -585,7 +588,7 @@ public class TestWALSplit {
}
LOG.debug(archived.toString());
assertEquals(failureType.name() + ": expected to find all of our wals corrupt.",
walDirContents, archivedLogs);
walDirContents, archivedLogs);
}
}
@ -593,16 +596,16 @@ public class TestWALSplit {
* @return set of wal names present prior to split attempt.
* @throws IOException if the split process fails
*/
private Set<String> splitCorruptWALs(final FaultySequenceFileLogReader.FailureType failureType)
private Set<String> splitCorruptWALs(final FaultyProtobufLogReader.FailureType failureType)
throws IOException {
Class<?> backupClass = conf.getClass("hbase.regionserver.hlog.reader.impl",
Reader.class);
InstrumentedLogWriter.activateFailure = false;
try {
conf.setClass("hbase.regionserver.hlog.reader.impl",
FaultySequenceFileLogReader.class, Reader.class);
conf.set("faultysequencefilelogreader.failuretype", failureType.name());
conf.setClass("hbase.regionserver.hlog.reader.impl", FaultyProtobufLogReader.class,
Reader.class);
conf.set("faultyprotobuflogreader.failuretype", failureType.name());
// Clean up from previous tests or previous loop
try {
wals.shutdown();
@ -639,7 +642,7 @@ public class TestWALSplit {
public void testTrailingGarbageCorruptionLogFileSkipErrorsFalseThrows()
throws IOException {
conf.setBoolean(HBASE_SKIP_ERRORS, false);
splitCorruptWALs(FaultySequenceFileLogReader.FailureType.BEGINNING);
splitCorruptWALs(FaultyProtobufLogReader.FailureType.BEGINNING);
}
@Test (timeout=300000)
@ -647,7 +650,7 @@ public class TestWALSplit {
throws IOException {
conf.setBoolean(HBASE_SKIP_ERRORS, false);
try {
splitCorruptWALs(FaultySequenceFileLogReader.FailureType.BEGINNING);
splitCorruptWALs(FaultyProtobufLogReader.FailureType.BEGINNING);
} catch (IOException e) {
LOG.debug("split with 'skip errors' set to 'false' correctly threw");
}
@ -1396,11 +1399,12 @@ public class TestWALSplit {
HConstants.DEFAULT_CLUSTER_ID), edit);
}
private void injectEmptyFile(String suffix, boolean closeFile)
throws IOException {
Writer writer = wals.createWALWriter(fs, new Path(WALDIR, WAL_FILE_PREFIX + suffix),
conf);
if (closeFile) writer.close();
private void injectEmptyFile(String suffix, boolean closeFile) throws IOException {
Writer writer =
WALFactory.createWALWriter(fs, new Path(WALDIR, WAL_FILE_PREFIX + suffix), conf);
if (closeFile) {
writer.close();
}
}
private boolean logsAreEqual(Path p1, Path p2) throws IOException {