You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@gobblin.apache.org by su...@apache.org on 2020/04/21 22:23:45 UTC

[incubator-gobblin] branch master updated: [GOBBLIN-1121] Fix Issue that YarnService use the old token to acquire new container

This is an automated email from the ASF dual-hosted git repository.

suvasude pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-gobblin.git


The following commit(s) were added to refs/heads/master by this push:
     new c05b3cb  [GOBBLIN-1121] Fix Issue that YarnService use the old token to acquire new container
c05b3cb is described below

commit c05b3cb7d7acd5540d5182718f52a7fcf4216503
Author: Zihan Li <zi...@zihli-mn1.linkedin.biz>
AuthorDate: Tue Apr 21 15:23:37 2020 -0700

    [GOBBLIN-1121] Fix Issue that YarnService use the old token to acquire new container
    
    Closes #2961 from ZihanLi58/GOBBLIN-1121-new
---
 .../gobblin/yarn/GobblinApplicationMaster.java     |  4 +-
 .../gobblin/yarn/YarnAppMasterSecurityManager.java | 46 ++++++++++++++++++++++
 .../java/org/apache/gobblin/yarn/YarnService.java  |  6 ++-
 .../org/apache/gobblin/yarn/YarnServiceTest.java   |  1 -
 4 files changed, 53 insertions(+), 4 deletions(-)

diff --git a/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinApplicationMaster.java b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinApplicationMaster.java
index b1095bc..c47032e 100644
--- a/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinApplicationMaster.java
+++ b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinApplicationMaster.java
@@ -124,10 +124,10 @@ public class GobblinApplicationMaster extends GobblinClusterManager {
   }
 
   /**
-   * Build the {@link YarnContainerSecurityManager} for the Application Master.
+   * Build the {@link YarnAppMasterSecurityManager} for the Application Master.
    */
   private YarnContainerSecurityManager buildYarnContainerSecurityManager(Config config, FileSystem fs) {
-    return new YarnContainerSecurityManager(config, fs, this.eventBus);
+    return new YarnAppMasterSecurityManager(config, fs, this.eventBus, this.yarnService);
   }
 
   @Override
diff --git a/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/YarnAppMasterSecurityManager.java b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/YarnAppMasterSecurityManager.java
new file mode 100644
index 0000000..7c3be1c
--- /dev/null
+++ b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/YarnAppMasterSecurityManager.java
@@ -0,0 +1,46 @@
+/*
+ * 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.gobblin.yarn;
+
+import com.google.common.base.Throwables;
+import com.google.common.eventbus.EventBus;
+import com.typesafe.config.Config;
+import java.io.IOException;
+import org.apache.gobblin.yarn.event.DelegationTokenUpdatedEvent;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+
+public class YarnAppMasterSecurityManager extends YarnContainerSecurityManager{
+
+  private YarnService yarnService;
+  public YarnAppMasterSecurityManager(Config config, FileSystem fs, EventBus eventBus, YarnService yarnService) {
+    super(config, fs, eventBus);
+    this.yarnService = yarnService;
+  }
+
+  @Override
+  public void handleTokenFileUpdatedEvent(DelegationTokenUpdatedEvent delegationTokenUpdatedEvent) {
+    super.handleTokenFileUpdatedEvent(delegationTokenUpdatedEvent);
+    try {
+      yarnService.updateToken();
+    } catch (IOException ioe) {
+      throw Throwables.propagate(ioe);
+    }
+  }
+}
diff --git a/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/YarnService.java b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/YarnService.java
index 5f6da10..3f127eb 100644
--- a/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/YarnService.java
+++ b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/YarnService.java
@@ -165,7 +165,7 @@ public class YarnService extends AbstractIdleService {
   private volatile Optional<Resource> maxResourceCapacity = Optional.absent();
 
   // Security tokens for accessing HDFS
-  private final ByteBuffer tokens;
+  private ByteBuffer tokens;
 
   private final Closer closer = Closer.create();
 
@@ -390,6 +390,10 @@ public class YarnService extends AbstractIdleService {
     }
   }
 
+  public void updateToken() throws IOException{
+    this.tokens = getSecurityTokens();
+  }
+
   private GobblinMetrics buildGobblinMetrics() {
     // Create tags list
     ImmutableList.Builder<Tag<?>> tags = new ImmutableList.Builder<>();
diff --git a/gobblin-yarn/src/test/java/org/apache/gobblin/yarn/YarnServiceTest.java b/gobblin-yarn/src/test/java/org/apache/gobblin/yarn/YarnServiceTest.java
index 63e2cd5..3266298 100644
--- a/gobblin-yarn/src/test/java/org/apache/gobblin/yarn/YarnServiceTest.java
+++ b/gobblin-yarn/src/test/java/org/apache/gobblin/yarn/YarnServiceTest.java
@@ -271,7 +271,6 @@ public class YarnServiceTest {
     Config modifiedConfig = this.config
         .withValue(GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_OVERHEAD_MBS_KEY, ConfigValueFactory.fromAnyRef("10"))
         .withValue(GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_XMX_RATIO_KEY, ConfigValueFactory.fromAnyRef("0.8"));
-
     TestYarnService yarnService =
         new TestYarnService(modifiedConfig, "testApp2", "appId2",
             this.clusterConf, FileSystem.getLocal(new Configuration()), this.eventBus);