You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2020/04/29 17:29:24 UTC

[GitHub] [hbase] ndimiduk commented on a change in pull request #1600: HBASE-24274 `RESTApiClusterManager` attempts to deserialize response using serialization API

ndimiduk commented on a change in pull request #1600:
URL: https://github.com/apache/hbase/pull/1600#discussion_r417486316



##########
File path: hbase-it/src/test/java/org/apache/hadoop/hbase/RESTApiClusterManager.java
##########
@@ -238,16 +236,18 @@ private String getHostId(String hostname) throws IOException {
 
   // Execute GET against URI, returning a JsonNode object to be traversed.
   private JsonElement getJsonNodeFromURIGet(URI uri) throws IOException {
-    LOG.info("Executing GET against " + uri + "...");
-    WebTarget webTarget = client.target(uri);
-    Invocation.Builder invocationBuilder = webTarget.request(MediaType.APPLICATION_JSON);
-    Response response = invocationBuilder.get();
+    LOG.debug("Executing GET against " + uri + "...");
+    final Response response = client.target(uri)
+      .request(MediaType.APPLICATION_JSON_TYPE)
+      .get();
     int statusCode = response.getStatus();
     if (statusCode != Response.Status.OK.getStatusCode()) {
       throw new HTTPException(statusCode);
     }
     // This API folds information as the value to an "items" attribute.
-    return GSON.toJsonTree(response.readEntity(String.class)).getAsJsonObject().get("items");
+    return parser.parse(response.readEntity(String.class))

Review comment:
       This is the meat of the fix. `toJsonTree` is for serializing an object to json, not for reading an object from json. It was returning a `JsonPrimitive<String>` instead of parsing the string content.

##########
File path: hbase-it/pom.xml
##########
@@ -253,6 +253,21 @@
       <groupId>log4j</groupId>
       <artifactId>log4j</artifactId>
     </dependency>
+    <dependency>
+      <artifactId>javax.servlet-api</artifactId>
+      <groupId>javax.servlet</groupId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-server</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-util</artifactId>
+      <scope>test</scope>
+    </dependency>

Review comment:
       I gathered this list manually, looking at the classes imported in `MockHttpApiRule`. I could probably do without `jetty-util` and `RegexSet` if you prefer.

##########
File path: hbase-it/src/test/java/org/apache/hadoop/hbase/MockHttpApiRule.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.hadoop.hbase;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.net.URI;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.BiConsumer;
+import java.util.regex.Pattern;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.core.MediaType;
+import org.eclipse.jetty.server.Request;
+import org.eclipse.jetty.server.RequestLog;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.server.Slf4jRequestLog;
+import org.eclipse.jetty.server.handler.AbstractHandler;
+import org.eclipse.jetty.util.RegexSet;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link org.junit.Rule} that manages a simple http server. The caller registers request
+ * handlers to URI path regexp.
+ */
+public class MockHttpApiRule extends ExternalResource {
+  private static final Logger LOG = LoggerFactory.getLogger(MockHttpApiRule.class);
+
+  private MockHandler handler;
+  private Server server;
+
+  /**
+   * Register a callback handler for the specified path target.
+   */
+  public MockHttpApiRule addRegistration(
+    final String pathRegex,
+    final BiConsumer<String, HttpServletResponse> responder
+  ) {
+    handler.register(pathRegex, responder);
+    return this;
+  }
+
+  /**
+   * Shortcut method for calling {@link #addRegistration(String, BiConsumer)} with a 200 response.
+   */
+  public MockHttpApiRule registerOk(final String pathRegex, final String responseBody) {
+    return addRegistration(pathRegex, (target, resp) -> {
+      try {
+        resp.setStatus(HttpServletResponse.SC_OK);
+        resp.setCharacterEncoding("UTF-8");
+        resp.setContentType(MediaType.APPLICATION_JSON_TYPE.toString());
+        final PrintWriter writer = resp.getWriter();
+        writer.write(responseBody);
+        writer.flush();
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    });
+  }
+
+  public void clearRegistrations() {
+    handler.clearRegistrations();
+  }
+
+  /**
+   * Retrieve the service URI for this service.
+   */
+  public URI getURI() {
+    if (server == null || !server.isRunning()) {
+      throw new IllegalStateException("server is not running");
+    }
+    return server.getURI();
+  }
+
+  @Override
+  protected void before() throws Exception {
+    handler = new MockHandler();
+    server = new Server();
+    final ServerConnector http = new ServerConnector(server);
+    http.setHost("localhost");
+    server.addConnector(http);
+    server.setStopAtShutdown(true);
+    server.setHandler(handler);
+    server.setRequestLog(buildRequestLog());
+    server.start();
+  }
+
+  @Override
+  protected void after() {
+    try {
+      server.stop();
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static RequestLog buildRequestLog() {
+    final Slf4jRequestLog requestLog = new Slf4jRequestLog();
+    requestLog.setLoggerName(LOG.getName() + ".RequestLog");
+    requestLog.setExtended(true);
+    return requestLog;
+  }
+
+  private static class MockHandler extends AbstractHandler {
+
+    private final ReadWriteLock responseMappingLock = new ReentrantReadWriteLock();
+    private final Map<String, BiConsumer<String, HttpServletResponse>> responseMapping =
+      new HashMap<>();
+    private final RegexSet regexSet = new RegexSet();
+
+    void register(
+      final String pathRegex,
+      final BiConsumer<String, HttpServletResponse> responder
+    ) {
+      LOG.debug("Registering responder to '{}'", pathRegex);
+      responseMappingLock.writeLock().lock();
+      try {
+        responseMapping.put(pathRegex, responder);
+        regexSet.add(pathRegex);
+      } finally {
+        responseMappingLock.writeLock().unlock();
+      }
+    }
+
+    void clearRegistrations() {
+      LOG.debug("Clearing registrations");
+      responseMappingLock.writeLock().lock();
+      try {
+        responseMapping.clear();
+        regexSet.clear();
+      } finally {
+        responseMappingLock.writeLock().unlock();
+      }
+    }
+
+    @Override
+    public void handle(
+      final String target,
+      final Request baseRequest,
+      final HttpServletRequest request,
+      final HttpServletResponse response
+    ) {
+      responseMappingLock.readLock().lock();
+      try {
+        if (!regexSet.matches(target)) {
+          response.setStatus(HttpServletResponse.SC_NOT_FOUND);
+          return;
+        }
+        responseMapping.entrySet()
+          .stream()
+          .filter(e -> Pattern.matches(e.getKey(), target))
+          .findAny()
+          .map(Map.Entry::getValue)
+          .orElseThrow(() -> noMatchFound(target))
+          .accept(target, response);
+      } finally {
+        responseMappingLock.readLock().unlock();
+      }
+    }
+
+    private static RuntimeException noMatchFound(final String target) {
+      return new RuntimeException(
+        String.format("Target path '%s' matches no registered regex.", target));
+    }
+  }
+}

Review comment:
       Maybe this eventually gets moved into wherever else we have http clients that need testing.

##########
File path: hbase-it/src/test/java/org/apache/hadoop/hbase/RESTApiClusterManager.java
##########
@@ -334,7 +334,7 @@ public String toString() {
     roleServiceType.put(ServiceType.HBASE_REGIONSERVER, Service.HBASE);
   }
 
-  private enum Service {
+  enum Service {
     HBASE, HDFS, MAPREDUCE
   }
 }

Review comment:
       I suspect it worked before the switch to GSON.

##########
File path: hbase-it/src/test/java/org/apache/hadoop/hbase/TestRESTApiClusterManager.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.hadoop.hbase;
+
+import static org.junit.Assert.assertTrue;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ClusterManager.ServiceType;
+import org.apache.hadoop.hbase.RESTApiClusterManager.Service;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+@Category(SmallTests.class)
+public class TestRESTApiClusterManager {
+
+  @ClassRule
+  public static final HBaseClassTestRule testRule =
+    HBaseClassTestRule.forClass(TestRESTApiClusterManager.class);
+
+  @ClassRule
+  public static MockHttpApiRule mockHttpApi = new MockHttpApiRule();
+
+  @Rule
+  public final TestName testName = new TestName();
+
+  private static HBaseCommonTestingUtility testingUtility;
+  private ClusterManager clusterManager;
+
+  @BeforeClass
+  public static void beforeClass() {
+    testingUtility = new HBaseCommonTestingUtility();
+    configureClusterManager(testingUtility.getConfiguration());
+  }
+
+  @Before
+  public void before() {
+    mockHttpApi.clearRegistrations();
+    final Configuration methodConf = new Configuration(testingUtility.getConfiguration());
+    methodConf.set("hbase.it.clustermanager.restapi.clustername", testName.getMethodName());
+    clusterManager = new RESTApiClusterManager();
+    clusterManager.setConf(methodConf);
+  }
+
+  @Test
+  public void isRunningPositive() throws IOException {
+    final String clusterName = testName.getMethodName();
+    final String hostName = "somehost";
+    final String serviceName = "hbase";
+    final String hostId = "some-id";
+    registerServiceName(clusterName, Service.HBASE, serviceName);
+    registerHost(hostName, hostId);
+    final Map<String, String> hostProperties = new HashMap<>();
+    hostProperties.put("roleState", "STARTED");
+    hostProperties.put("healthSummary", "GOOD");
+    registerHostProperties(
+      clusterName, serviceName, hostId, ServiceType.HBASE_MASTER, hostProperties);
+    assertTrue(clusterManager.isRunning(ServiceType.HBASE_MASTER, hostName, -1));
+  }
+
+  private static void configureClusterManager(final Configuration conf) {
+    conf.set("hbase.it.clustermanager.restapi.hostname", mockHttpApi.getURI().toString());
+  }
+
+  private static void registerServiceName(
+    final String clusterName,
+    final Service service,
+    final String serviceName
+  ) {
+    final String target = String.format("^/api/v6/clusters/%s/services", clusterName);
+    final String response = String.format(
+      "{ \"items\": [ { \"type\": \"%s\", \"name\": \"%s\" } ] }", service, serviceName);

Review comment:
       Building these json strings by hand is awful. Maybe better to populate maps, or define POJOs? None of it is particularly elegant.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org