CASSANDRASC-91 Returning when AbstractHandler encounters exception while extracting params from the request.
Patch by Venkata Harikrishna Nukala; reviewed by Francisco Guerrero and Yifan Cai for CASSANDRASC-91
diff --git a/CHANGES.txt b/CHANGES.txt
index ad51d93..830f4cf 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,5 +1,6 @@
1.0.0
-----
+ * AbstractHandler is handling the request even when it fails to extract params (CASSANDRASC-91)
* Fix Sidecar TokenRangeReplicas endpoint to unwrap the token-range by partitioner's range for a single node clusters (CASSANDRASC-90)
* Expose TTL option for the create snapshot endpoint (CASSANDRASC-85)
* Allow DriverUtils to be pluggable (CASSANDRASC-88)
diff --git a/src/main/java/org/apache/cassandra/sidecar/routes/AbstractHandler.java b/src/main/java/org/apache/cassandra/sidecar/routes/AbstractHandler.java
index 04b3071..880398c 100644
--- a/src/main/java/org/apache/cassandra/sidecar/routes/AbstractHandler.java
+++ b/src/main/java/org/apache/cassandra/sidecar/routes/AbstractHandler.java
@@ -89,13 +89,12 @@
requestParams = extractParamsOrThrow(context);
logger.debug("{} received request={}, remoteAddress={}, instance={}",
this.getClass().getSimpleName(), requestParams, remoteAddress, host);
+ handleInternal(context, request, host, remoteAddress, requestParams);
}
catch (Exception exception)
{
- processFailure(exception, context, host, remoteAddress, null);
+ processFailure(exception, context, host, remoteAddress, requestParams);
}
-
- handleInternal(context, request, host, remoteAddress, requestParams);
}
/**
@@ -165,7 +164,7 @@
}
/**
- * Processes the failure with the given parameters
+ * Processes the failure while handling the request.
*
* @param cause the cause
* @param context the routing context
diff --git a/src/test/java/org/apache/cassandra/sidecar/routes/AbstractHandlerTest.java b/src/test/java/org/apache/cassandra/sidecar/routes/AbstractHandlerTest.java
new file mode 100644
index 0000000..a5ad93e
--- /dev/null
+++ b/src/test/java/org/apache/cassandra/sidecar/routes/AbstractHandlerTest.java
@@ -0,0 +1,132 @@
+/*
+ * 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.cassandra.sidecar.routes;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+import com.google.inject.util.Modules;
+import io.vertx.core.Vertx;
+import io.vertx.core.http.HttpServerRequest;
+import io.vertx.core.net.SocketAddress;
+import io.vertx.ext.web.Router;
+import io.vertx.ext.web.RoutingContext;
+import io.vertx.ext.web.client.WebClient;
+import io.vertx.junit5.VertxExtension;
+import io.vertx.junit5.VertxTestContext;
+import org.apache.cassandra.sidecar.TestModule;
+import org.apache.cassandra.sidecar.concurrent.ExecutorPools;
+import org.apache.cassandra.sidecar.server.MainModule;
+import org.apache.cassandra.sidecar.server.Server;
+import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+
+/**
+ * Unit tests for {@link AbstractHandler}
+ */
+@ExtendWith(VertxExtension.class)
+class AbstractHandlerTest
+{
+ private static final Logger LOGGER = LoggerFactory.getLogger(AbstractHandlerTest.class);
+ Vertx vertx;
+ Server server;
+ Injector injector;
+
+ @AfterEach
+ void after() throws InterruptedException
+ {
+ CountDownLatch closeLatch = new CountDownLatch(1);
+ server.close().onSuccess(res -> closeLatch.countDown());
+ if (closeLatch.await(60, TimeUnit.SECONDS))
+ LOGGER.info("Close event received before timeout.");
+ else
+ LOGGER.error("Close event timed out.");
+ }
+
+ @Test
+ void testDoNotHandleReqWhenParsingParamsFails(VertxTestContext context) throws InterruptedException
+ {
+ injector = Guice.createInjector(Modules.override(new MainModule())
+ .with(new TestModule()));
+ vertx = injector.getInstance(Vertx.class);
+
+ Router router = injector.getInstance(Router.class);
+ InstanceMetadataFetcher metadataFetcher = injector.getInstance(InstanceMetadataFetcher.class);
+ ExecutorPools executorPools = injector.getInstance(ExecutorPools.class);
+ router.get("/fail/parsing/params")
+ .handler(new FailsOnParameterParsing(context, metadataFetcher, executorPools));
+
+ VertxTestContext serverStartContext = new VertxTestContext();
+ server = injector.getInstance(Server.class);
+ server.start()
+ .onSuccess(s -> serverStartContext.completeNow())
+ .onFailure(serverStartContext::failNow);
+ serverStartContext.awaitCompletion(5, TimeUnit.SECONDS);
+
+ WebClient client = WebClient.create(vertx);
+ String testRoute = "/fail/parsing/params";
+ client.get(server.actualPort(), "127.0.0.1", testRoute)
+ .send(context.succeeding(response -> {
+ assertThat(response.statusCode()).isEqualTo(INTERNAL_SERVER_ERROR.code());
+ vertx.setTimer(1000, tid -> context.completeNow());
+ }));
+ }
+
+ /**
+ * A handler that will simulate a failure in the {@link #extractParamsOrThrow(RoutingContext)} method
+ */
+ static class FailsOnParameterParsing extends AbstractHandler<Object>
+ {
+ private final VertxTestContext testContext;
+
+ protected FailsOnParameterParsing(VertxTestContext testContext,
+ InstanceMetadataFetcher metadataFetcher,
+ ExecutorPools executorPools)
+ {
+ super(metadataFetcher, executorPools, null);
+ this.testContext = testContext;
+ }
+
+ @Override
+ protected Object extractParamsOrThrow(RoutingContext context)
+ {
+ throw new RuntimeException("Simulated Exception");
+ }
+
+ @Override
+ protected void handleInternal(RoutingContext context,
+ HttpServerRequest httpRequest,
+ String host,
+ SocketAddress remoteAddress,
+ Object request)
+ {
+ testContext.failNow("Should never reach here");
+ }
+ }
+}
diff --git a/src/test/java/org/apache/cassandra/sidecar/routes/StreamSSTableComponentHandlerTest.java b/src/test/java/org/apache/cassandra/sidecar/routes/StreamSSTableComponentHandlerTest.java
index 00f7c65..3dc938b 100644
--- a/src/test/java/org/apache/cassandra/sidecar/routes/StreamSSTableComponentHandlerTest.java
+++ b/src/test/java/org/apache/cassandra/sidecar/routes/StreamSSTableComponentHandlerTest.java
@@ -221,7 +221,8 @@
void failsWhenTableNameContainsInvalidCharacters(VertxTestContext context)
{
WebClient client = WebClient.create(vertx);
- String testRoute = "/keyspaces/" + TEST_KEYSPACE + "/tables/i_❤_u/snapshots/snap/components/component-Data.db";
+ String testRoute = "/keyspaces/" + TEST_KEYSPACE +
+ "/tables/i_❤_u/snapshots/snap/components/component-Data.db";
client.get(server.actualPort(), "localhost", "/api/v1" + testRoute)
.send(context.succeeding(response -> context.verify(() -> {
assertThat(response.statusCode()).isEqualTo(BAD_REQUEST.code());