You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@helix.apache.org by hu...@apache.org on 2020/08/14 18:14:00 UTC

[helix] 05/12: Add HttpZkFallbackRoutingDataReader

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

hulee pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/helix.git

commit 0ff7e95c21c6a2d85009cfdf4c457a3d5b9b961d
Author: Hunter Lee <hu...@linkedin.com>
AuthorDate: Wed Jul 22 11:09:49 2020 -0700

    Add HttpZkFallbackRoutingDataReader
    
    Implement an Http-ZK fallback routing data reader. ZkRoutingDataReader will follow.
---
 .../routing/HttpZkFallbackRoutingDataReader.java   | 79 +++++++++++++++++++++-
 .../TestHttpZkFallbackRoutingDataReader.java       | 71 ++++++++++++++++++-
 2 files changed, 148 insertions(+), 2 deletions(-)

diff --git a/zookeeper-api/src/main/java/org/apache/helix/zookeeper/routing/HttpZkFallbackRoutingDataReader.java b/zookeeper-api/src/main/java/org/apache/helix/zookeeper/routing/HttpZkFallbackRoutingDataReader.java
index 1b4713c..add034f 100644
--- a/zookeeper-api/src/main/java/org/apache/helix/zookeeper/routing/HttpZkFallbackRoutingDataReader.java
+++ b/zookeeper-api/src/main/java/org/apache/helix/zookeeper/routing/HttpZkFallbackRoutingDataReader.java
@@ -1,4 +1,81 @@
 package org.apache.helix.zookeeper.routing;
 
-public class HttpZkFallbackRoutingDataReader {
+/*
+ * 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.
+ */
+
+import java.util.List;
+import java.util.Map;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.helix.zookeeper.constant.RoutingDataReaderType;
+import org.apache.helix.zookeeper.exception.MultiZkException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * HTTP and ZK-based RoutingDataReader that first tries an HTTP call to MSDS and upon failure,
+ * falls back to ZK for routing data.
+ * HttpZkFallbackRoutingDataReader does not maintain a ZK connection - it establishes for reading
+ * and closes it right away.
+ */
+public class HttpZkFallbackRoutingDataReader implements RoutingDataReader {
+  private static final Logger LOG = LoggerFactory.getLogger(HttpZkFallbackRoutingDataReader.class);
+
+  /**
+   * Returns a map form of metadata store routing data.
+   * The map fields stand for metadata store realm address (key), and a corresponding list of ZK
+   * path sharding keys (key).
+   * @param endpoint two comma-separated endpoints. <HTTP endpoint>,<ZK address>
+   * @return
+   */
+  @Override
+  public Map<String, List<String>> getRawRoutingData(String endpoint) {
+    Map<RoutingDataReaderType, String> endpointMap = parseEndpoint(endpoint);
+    try {
+      return new HttpRoutingDataReader()
+          .getRawRoutingData(endpointMap.get(RoutingDataReaderType.HTTP));
+    } catch (MultiZkException e) {
+      LOG.warn(
+          "HttpZkFallbackRoutingDataReader::getRawRoutingData: failed to read routing data via "
+              + "HTTP. Falling back to ZK!", e);
+      // TODO: increment failure count and emit as a metric
+      return new ZkRoutingDataReader().getRawRoutingData(endpointMap.get(RoutingDataReaderType.ZK));
+    }
+  }
+
+  /**
+   * For a fallback routing data reader, endpoints are given in a comma-separated string in the
+   * fallback order. This method parses the string and returns a map of routing data source type to
+   * its respective endpoint.
+   * @param endpointString
+   * @return
+   */
+  private Map<RoutingDataReaderType, String> parseEndpoint(String endpointString) {
+    String[] endpoints = endpointString.split(",");
+    if (endpoints.length != 2) {
+      throw new MultiZkException(
+          "HttpZkFallbackRoutingDataReader::parseEndpoint: endpoint string does not contain two "
+              + "proper comma-separated endpoints for HTTP and ZK! Endpoint string: "
+              + endpointString);
+    }
+    return ImmutableMap
+        .of(RoutingDataReaderType.HTTP, endpoints[0], RoutingDataReaderType.ZK, endpoints[1]);
+  }
 }
diff --git a/zookeeper-api/src/test/java/org/apache/helix/zookeeper/routing/TestHttpZkFallbackRoutingDataReader.java b/zookeeper-api/src/test/java/org/apache/helix/zookeeper/routing/TestHttpZkFallbackRoutingDataReader.java
index 85896af..d959ad2 100644
--- a/zookeeper-api/src/test/java/org/apache/helix/zookeeper/routing/TestHttpZkFallbackRoutingDataReader.java
+++ b/zookeeper-api/src/test/java/org/apache/helix/zookeeper/routing/TestHttpZkFallbackRoutingDataReader.java
@@ -1,4 +1,73 @@
 package org.apache.helix.zookeeper.routing;
 
-public class TestHttpZkFallbackRoutingDataReader {
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.helix.msdcommon.mock.MockMetadataStoreDirectoryServer;
+import org.apache.helix.zookeeper.constant.TestConstants;
+import org.apache.helix.zookeeper.exception.MultiZkException;
+import org.apache.helix.zookeeper.impl.ZkTestBase;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+
+public class TestHttpZkFallbackRoutingDataReader extends ZkTestBase {
+  private MockMetadataStoreDirectoryServer _msdsServer;
+  private static final String HOST = "localhost";
+  private static final int PORT = 1992;
+  private static final String NAMESPACE = "TestHttpZkFallbackRoutingDataReader";
+  private static final String MSDS_ENDPOINT =
+      "http://" + HOST + ":" + PORT + "/admin/v2/namespaces/" + NAMESPACE;
+
+  @BeforeClass
+  public void beforeClass() throws IOException {
+    // Start MockMSDS
+    _msdsServer = new MockMetadataStoreDirectoryServer(HOST, PORT, NAMESPACE,
+        TestConstants.FAKE_ROUTING_DATA);
+    _msdsServer.startServer();
+  }
+
+  @Test
+  public void testGetRawRoutingData() {
+    HttpZkFallbackRoutingDataReader reader = new HttpZkFallbackRoutingDataReader();
+
+    // This read should read from HTTP
+    String endpointString = MSDS_ENDPOINT + "," + ZK_ADDR;
+    Map<String, List<String>> rawRoutingData = reader.getRawRoutingData(endpointString);
+    TestConstants.FAKE_ROUTING_DATA.forEach((realm, keys) -> Assert
+        .assertEquals(new HashSet(rawRoutingData.get(realm)), new HashSet(keys)));
+
+    // Shut down MSDS so that it would read from ZK (fallback)
+    _msdsServer.stopServer();
+    try {
+      reader.getRawRoutingData(endpointString);
+      Assert.fail("Must encounter a MultiZkException since the path in ZK does not exist!");
+    } catch (MultiZkException e) {
+      // Check the exception message to ensure that it's reading from ZK
+      Assert.assertTrue(e.getMessage()
+          .contains("Routing data directory ZNode /METADATA_STORE_ROUTING_DATA does not exist"));
+    }
+  }
 }