HBASE-12864 IntegrationTestTableSnapshotInputFormat fails
This commit is contained in:
parent
e87a69d9c0
commit
735fe20029
|
@ -52,24 +52,17 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.security.UserProvider;
|
||||
import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier;
|
||||
import org.apache.hadoop.hbase.security.token.AuthenticationTokenSelector;
|
||||
import org.apache.hadoop.hbase.security.token.TokenUtil;
|
||||
import org.apache.hadoop.hbase.util.Base64;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.io.WritableComparable;
|
||||
import org.apache.hadoop.mapreduce.InputFormat;
|
||||
import org.apache.hadoop.mapreduce.Job;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import com.yammer.metrics.core.MetricsRegistry;
|
||||
|
||||
/**
|
||||
* Utility for {@link TableMapper} and {@link TableReducer}
|
||||
|
@ -311,7 +304,7 @@ public class TableMapReduceUtil {
|
|||
conf.setFloat(
|
||||
HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, HConstants.HFILE_BLOCK_CACHE_SIZE_DEFAULT);
|
||||
conf.setFloat(HConstants.BUCKET_CACHE_SIZE_KEY, 0f);
|
||||
conf.setFloat("hbase.bucketcache.size", 0f);
|
||||
conf.unset(HConstants.BUCKET_CACHE_IOENGINE_KEY);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -343,6 +336,7 @@ public class TableMapReduceUtil {
|
|||
TableSnapshotInputFormat.setInput(job, snapshotName, tmpRestoreDir);
|
||||
initTableMapperJob(snapshotName, scan, mapper, outputKeyClass,
|
||||
outputValueClass, job, addDependencyJars, false, TableSnapshotInputFormat.class);
|
||||
addDependencyJars(job.getConfiguration(), MetricsRegistry.class);
|
||||
resetCacheConfig(job.getConfiguration());
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue