HBASE-18452 VerifyReplication by Snapshot should cache HDFS token before submit job for kerberos env
This commit is contained in:
parent
8119acfca7
commit
5195435941
|
@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hbase.Abortable;
|
import org.apache.hadoop.hbase.Abortable;
|
||||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||||
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.client.Connection;
|
import org.apache.hadoop.hbase.client.Connection;
|
||||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||||
|
@ -428,7 +429,7 @@ public class VerifyReplication extends Configured implements Tool {
|
||||||
conf.set(NAME + ".peerHBaseRootAddress", peerHBaseRootAddress);
|
conf.set(NAME + ".peerHBaseRootAddress", peerHBaseRootAddress);
|
||||||
|
|
||||||
// This is to create HDFS delegation token for peer cluster in case of secured
|
// This is to create HDFS delegation token for peer cluster in case of secured
|
||||||
conf.setStrings(MRJobConfig.JOB_NAMENODES, peerFSAddress);
|
conf.setStrings(MRJobConfig.JOB_NAMENODES, peerFSAddress, conf.get(HConstants.HBASE_DIR));
|
||||||
}
|
}
|
||||||
|
|
||||||
Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + tableName));
|
Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + tableName));
|
||||||
|
|
Loading…
Reference in New Issue