You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@knox.apache.org by pz...@apache.org on 2018/01/04 01:05:05 UTC

knox git commit: KNOX-1144 - Provider Configuration Reference Relationships Aren't Established For Existing Descriptors at Gateway Startup

Repository: knox
Updated Branches:
  refs/heads/v0.14.0 2ccb989c8 -> 7a2e7eab1


KNOX-1144 - Provider Configuration Reference Relationships Aren't Established For Existing Descriptors at Gateway Startup


Project: http://git-wip-us.apache.org/repos/asf/knox/repo
Commit: http://git-wip-us.apache.org/repos/asf/knox/commit/7a2e7eab
Tree: http://git-wip-us.apache.org/repos/asf/knox/tree/7a2e7eab
Diff: http://git-wip-us.apache.org/repos/asf/knox/diff/7a2e7eab

Branch: refs/heads/v0.14.0
Commit: 7a2e7eab16f28f5dce3a0f9e410e531186cc060b
Parents: 2ccb989
Author: Phil Zampino <pz...@apache.org>
Authored: Wed Jan 3 20:03:45 2018 -0500
Committer: Phil Zampino <pz...@apache.org>
Committed: Wed Jan 3 20:04:26 2018 -0500

----------------------------------------------------------------------
 .../topology/impl/DefaultTopologyService.java   | 40 +++++++++++++++-----
 .../DefaultRemoteConfigurationMonitor.java      | 22 ++++++++++-
 .../ZooKeeperConfigurationMonitorTest.java      | 17 ++++++++-
 3 files changed, 65 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/knox/blob/7a2e7eab/gateway-server/src/main/java/org/apache/hadoop/gateway/services/topology/impl/DefaultTopologyService.java
----------------------------------------------------------------------
diff --git a/gateway-server/src/main/java/org/apache/hadoop/gateway/services/topology/impl/DefaultTopologyService.java b/gateway-server/src/main/java/org/apache/hadoop/gateway/services/topology/impl/DefaultTopologyService.java
index aded6cd..398f3e9 100644
--- a/gateway-server/src/main/java/org/apache/hadoop/gateway/services/topology/impl/DefaultTopologyService.java
+++ b/gateway-server/src/main/java/org/apache/hadoop/gateway/services/topology/impl/DefaultTopologyService.java
@@ -52,6 +52,8 @@ import org.apache.hadoop.gateway.topology.builder.TopologyBuilder;
 import org.apache.hadoop.gateway.topology.discovery.ClusterConfigurationMonitor;
 import org.apache.hadoop.gateway.topology.monitor.RemoteConfigurationMonitor;
 import org.apache.hadoop.gateway.topology.monitor.RemoteConfigurationMonitorFactory;
+import org.apache.hadoop.gateway.topology.simple.SimpleDescriptor;
+import org.apache.hadoop.gateway.topology.simple.SimpleDescriptorFactory;
 import org.apache.hadoop.gateway.topology.simple.SimpleDescriptorHandler;
 import org.apache.hadoop.gateway.topology.validation.TopologyValidator;
 import org.apache.hadoop.gateway.topology.xml.AmbariFormatXmlTopologyRules;
@@ -592,18 +594,39 @@ public class DefaultTopologyService
       initListener(sharedProvidersDirectory, spm, spm);
       log.monitoringProviderConfigChangesInDirectory(sharedProvidersDirectory.getAbsolutePath());
 
-      // For all the descriptors currently in the descriptors dir at start-up time, trigger topology generation.
+      // For all the descriptors currently in the descriptors dir at start-up time, determine if topology regeneration
+      // is required.
       // This happens prior to the start-up loading of the topologies.
       String[] descriptorFilenames =  descriptorsDirectory.list();
       if (descriptorFilenames != null) {
         for (String descriptorFilename : descriptorFilenames) {
           if (DescriptorsMonitor.isDescriptorFile(descriptorFilename)) {
+            String topologyName = FilenameUtils.getBaseName(descriptorFilename);
+            File existingDescriptorFile = getExistingFile(descriptorsDirectory, topologyName);
+
             // If there isn't a corresponding topology file, or if the descriptor has been modified since the
             // corresponding topology file was generated, then trigger generation of one
-            File matchingTopologyFile = getExistingFile(topologiesDirectory, FilenameUtils.getBaseName(descriptorFilename));
-            if (matchingTopologyFile == null ||
-                    matchingTopologyFile.lastModified() < (new File(descriptorsDirectory, descriptorFilename)).lastModified()) {
-              descriptorsMonitor.onFileChange(new File(descriptorsDirectory, descriptorFilename));
+            File matchingTopologyFile = getExistingFile(topologiesDirectory, topologyName);
+            if (matchingTopologyFile == null || matchingTopologyFile.lastModified() < existingDescriptorFile.lastModified()) {
+              descriptorsMonitor.onFileChange(existingDescriptorFile);
+            } else {
+              // If regeneration is NOT required, then we at least need to report the provider configuration
+              // reference relationship (KNOX-1144)
+              String normalizedDescriptorPath = FilenameUtils.normalize(existingDescriptorFile.getAbsolutePath());
+
+              // Parse the descriptor to determine the provider config reference
+              SimpleDescriptor sd = SimpleDescriptorFactory.parse(normalizedDescriptorPath);
+              if (sd != null) {
+                File referencedProviderConfig =
+                           getExistingFile(sharedProvidersDirectory, FilenameUtils.getBaseName(sd.getProviderConfig()));
+                if (referencedProviderConfig != null) {
+                  List<String> references =
+                         descriptorsMonitor.getReferencingDescriptors(referencedProviderConfig.getAbsolutePath());
+                  if (!references.contains(normalizedDescriptorPath)) {
+                    references.add(normalizedDescriptorPath);
+                  }
+                }
+              }
             }
           }
         }
@@ -711,11 +734,8 @@ public class DefaultTopologyService
     }
 
     List<String> getReferencingDescriptors(String providerConfigPath) {
-      List<String> result = providerConfigReferences.get(FilenameUtils.normalize(providerConfigPath));
-      if (result == null) {
-        result = Collections.emptyList();
-      }
-      return result;
+      String normalizedPath = FilenameUtils.normalize(providerConfigPath);
+      return providerConfigReferences.computeIfAbsent(normalizedPath, p -> new ArrayList<>());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/knox/blob/7a2e7eab/gateway-server/src/main/java/org/apache/hadoop/gateway/topology/monitor/DefaultRemoteConfigurationMonitor.java
----------------------------------------------------------------------
diff --git a/gateway-server/src/main/java/org/apache/hadoop/gateway/topology/monitor/DefaultRemoteConfigurationMonitor.java b/gateway-server/src/main/java/org/apache/hadoop/gateway/topology/monitor/DefaultRemoteConfigurationMonitor.java
index af60058..3bf3330 100644
--- a/gateway-server/src/main/java/org/apache/hadoop/gateway/topology/monitor/DefaultRemoteConfigurationMonitor.java
+++ b/gateway-server/src/main/java/org/apache/hadoop/gateway/topology/monitor/DefaultRemoteConfigurationMonitor.java
@@ -29,6 +29,7 @@ import org.apache.zookeeper.ZooDefs;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
@@ -112,6 +113,19 @@ class DefaultRemoteConfigurationMonitor implements RemoteConfigurationMonitor {
         if (providerConfigs == null) {
             // Either the ZNode does not exist, or there is an authentication problem
             throw new IllegalStateException("Unable to access remote path: " + NODE_KNOX_PROVIDERS);
+        } else {
+            // Download any existing provider configs in the remote registry, which either do not exist locally, or have
+            // been modified, so that they are certain to be present when this monitor downloads any descriptors that
+            // reference them.
+            for (String providerConfig : providerConfigs) {
+                File localFile = new File(providersDir, providerConfig);
+
+                byte[] remoteContent = client.getEntryData(NODE_KNOX_PROVIDERS + "/" + providerConfig).getBytes();
+                if (!localFile.exists() || !Arrays.equals(remoteContent, FileUtils.readFileToByteArray(localFile))) {
+                    FileUtils.writeByteArrayToFile(localFile, remoteContent);
+                    log.downloadedRemoteConfigFile(providersDir.getName(), providerConfig);
+                }
+            }
         }
 
         // Confirm access to the remote descriptors directory znode
@@ -213,8 +227,12 @@ class DefaultRemoteConfigurationMonitor implements RemoteConfigurationMonitor {
             File localFile = new File(localDir, path.substring(path.lastIndexOf("/")));
             if (data != null) {
                 try {
-                    FileUtils.writeByteArrayToFile(localFile, data);
-                    log.downloadedRemoteConfigFile(localDir.getName(), localFile.getName());
+                    // If there is no corresponding local file, or the content is different from the existing local
+                    // file, write the data to the local file.
+                    if (!localFile.exists() || !Arrays.equals(FileUtils.readFileToByteArray(localFile), data)) {
+                        FileUtils.writeByteArrayToFile(localFile, data);
+                        log.downloadedRemoteConfigFile(localDir.getName(), localFile.getName());
+                    }
                 } catch (IOException e) {
                     log.errorDownloadingRemoteConfiguration(path, e);
                 }

http://git-wip-us.apache.org/repos/asf/knox/blob/7a2e7eab/gateway-server/src/test/java/org/apache/hadoop/gateway/topology/monitor/ZooKeeperConfigurationMonitorTest.java
----------------------------------------------------------------------
diff --git a/gateway-server/src/test/java/org/apache/hadoop/gateway/topology/monitor/ZooKeeperConfigurationMonitorTest.java b/gateway-server/src/test/java/org/apache/hadoop/gateway/topology/monitor/ZooKeeperConfigurationMonitorTest.java
index 1c4ed6e..ecf5b70 100644
--- a/gateway-server/src/test/java/org/apache/hadoop/gateway/topology/monitor/ZooKeeperConfigurationMonitorTest.java
+++ b/gateway-server/src/test/java/org/apache/hadoop/gateway/topology/monitor/ZooKeeperConfigurationMonitorTest.java
@@ -113,10 +113,10 @@ public class ZooKeeperConfigurationMonitorTest {
 
         client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT).withACL(acls).forPath(PATH_KNOX_DESCRIPTORS);
         assertNotNull("Failed to create node:" + PATH_KNOX_DESCRIPTORS,
-                client.checkExists().forPath(PATH_KNOX_DESCRIPTORS));
+                      client.checkExists().forPath(PATH_KNOX_DESCRIPTORS));
         client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT).withACL(acls).forPath(PATH_KNOX_PROVIDERS);
         assertNotNull("Failed to create node:" + PATH_KNOX_PROVIDERS,
-                client.checkExists().forPath(PATH_KNOX_PROVIDERS));
+                      client.checkExists().forPath(PATH_KNOX_PROVIDERS));
     }
 
     @AfterClass
@@ -164,12 +164,25 @@ public class ZooKeeperConfigurationMonitorTest {
 
         DefaultRemoteConfigurationMonitor cm = new DefaultRemoteConfigurationMonitor(gc, clientService);
 
+        // Create a provider configuration in the test ZK, prior to starting the monitor, to make sure that the monitor
+        // will download existing entries upon starting.
+        final String preExistingProviderConfig = getProviderPath("pre-existing-providers.xml");
+        client.create().withMode(CreateMode.PERSISTENT).forPath(preExistingProviderConfig,
+                                                                TEST_PROVIDERS_CONFIG_1.getBytes());
+        File preExistingProviderConfigLocalFile = new File(providersDir, "pre-existing-providers.xml");
+        assertFalse("This file should not exist locally prior to monitor starting.",
+                    preExistingProviderConfigLocalFile.exists());
+
         try {
             cm.start();
         } catch (Exception e) {
             fail("Failed to start monitor: " + e.getMessage());
         }
 
+        assertTrue("This file should exist locally immediately after monitor starting.",
+                    preExistingProviderConfigLocalFile.exists());
+
+
         try {
             final String pc_one_znode = getProviderPath("providers-config1.xml");
             final File pc_one         = new File(providersDir, "providers-config1.xml");