|
@@ -22,9 +22,12 @@ import java.io.InputStream;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.Iterator;
|
|
|
|
|
|
+import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.hdds.client.ReplicationFactor;
|
|
|
import org.apache.hadoop.hdds.client.ReplicationType;
|
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
|
+import org.apache.hadoop.io.Text;
|
|
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
|
import org.apache.hadoop.ozone.client.ObjectStore;
|
|
|
import org.apache.hadoop.ozone.client.OzoneBucket;
|
|
@@ -35,6 +38,10 @@ import org.apache.hadoop.ozone.client.OzoneVolume;
|
|
|
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
|
|
|
|
|
import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
|
|
|
+
|
|
|
+import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;
|
|
|
+import org.apache.hadoop.security.token.Token;
|
|
|
+import org.apache.hadoop.security.token.TokenRenewer;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
@@ -251,8 +258,64 @@ public class OzoneClientAdapterImpl implements OzoneClientAdapter {
|
|
|
return new IteratorAdapter(bucket.listKeys(pathKey));
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
+ public Token<OzoneTokenIdentifier> getDelegationToken(String renewer)
|
|
|
+ throws IOException {
|
|
|
+ Token<OzoneTokenIdentifier> token =
|
|
|
+ ozoneClient.getObjectStore().getDelegationToken(new Text(renewer));
|
|
|
+ token.setKind(OzoneTokenIdentifier.KIND_NAME);
|
|
|
+ return token;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Ozone Delegation Token Renewer.
|
|
|
+ */
|
|
|
+ @InterfaceAudience.Private
|
|
|
+ public static class Renewer extends TokenRenewer {
|
|
|
+
|
|
|
+ //Ensure that OzoneConfiguration files are loaded before trying to use
|
|
|
+ // the renewer.
|
|
|
+ static {
|
|
|
+ OzoneConfiguration.activate();
|
|
|
+ }
|
|
|
+
|
|
|
+ public Text getKind() {
|
|
|
+ return OzoneTokenIdentifier.KIND_NAME;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public boolean handleKind(Text kind) {
|
|
|
+ return getKind().equals(kind);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public boolean isManaged(Token<?> token) throws IOException {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public long renew(Token<?> token, Configuration conf)
|
|
|
+ throws IOException, InterruptedException {
|
|
|
+ Token<OzoneTokenIdentifier> ozoneDt =
|
|
|
+ (Token<OzoneTokenIdentifier>) token;
|
|
|
+ OzoneClient ozoneClient =
|
|
|
+ OzoneClientFactory.getRpcClient(conf);
|
|
|
+ return ozoneClient.getObjectStore().renewDelegationToken(ozoneDt);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void cancel(Token<?> token, Configuration conf)
|
|
|
+ throws IOException, InterruptedException {
|
|
|
+ Token<OzoneTokenIdentifier> ozoneDt =
|
|
|
+ (Token<OzoneTokenIdentifier>) token;
|
|
|
+ OzoneClient ozoneClient =
|
|
|
+ OzoneClientFactory.getRpcClient(conf);
|
|
|
+ ozoneClient.getObjectStore().cancelDelegationToken(ozoneDt);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
- * Adapter to conver OzoneKey to a safe and simple Key implementation.
|
|
|
+ * Adapter to convert OzoneKey to a safe and simple Key implementation.
|
|
|
*/
|
|
|
public static class IteratorAdapter implements Iterator<BasicKeyInfo> {
|
|
|
|