You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2022/03/01 11:18:41 UTC

[GitHub] [ignite-3] sashapolo commented on a change in pull request #690: IGNITE-16498 Add ability to add handlers to RestModule from other Ignite modules

sashapolo commented on a change in pull request #690:
URL: https://github.com/apache/ignite-3/pull/690#discussion_r816668817



##########
File path: modules/rest/src/main/java/org/apache/ignite/internal/rest/RestModule.java
##########
@@ -17,136 +17,73 @@
 
 package org.apache.ignite.internal.rest;
 
-import static io.netty.handler.codec.http.HttpHeaderValues.APPLICATION_JSON;
-import static io.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST;
-
 import io.netty.bootstrap.ServerBootstrap;
 import io.netty.channel.Channel;
 import io.netty.channel.ChannelFuture;
 import io.netty.handler.logging.LogLevel;
 import io.netty.handler.logging.LoggingHandler;
 import java.net.BindException;
-import java.nio.charset.StandardCharsets;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.CompletionException;
+import java.util.function.Consumer;
 import org.apache.ignite.configuration.schemas.rest.RestConfiguration;
 import org.apache.ignite.configuration.schemas.rest.RestView;
-import org.apache.ignite.configuration.validation.ConfigurationValidationException;
 import org.apache.ignite.internal.configuration.ConfigurationManager;
 import org.apache.ignite.internal.configuration.ConfigurationRegistry;
 import org.apache.ignite.internal.manager.IgniteComponent;
-import org.apache.ignite.internal.rest.netty.RestApiHttpRequest;
-import org.apache.ignite.internal.rest.netty.RestApiHttpResponse;
 import org.apache.ignite.internal.rest.netty.RestApiInitializer;
-import org.apache.ignite.internal.rest.presentation.ConfigurationPresentation;
-import org.apache.ignite.internal.rest.presentation.hocon.HoconPresentation;
 import org.apache.ignite.internal.rest.routes.Router;
+import org.apache.ignite.internal.rest.routes.SimpleRouter;
 import org.apache.ignite.lang.IgniteException;
 import org.apache.ignite.lang.IgniteLogger;
 import org.apache.ignite.network.NettyBootstrapFactory;
+import org.apache.ignite.rest.RestHandlersRegister;
+import org.apache.ignite.rest.Routes;
 
 /**
  * Rest module is responsible for starting a REST endpoints for accessing and managing configuration.
  *
- * <p>It is started on port 10300 by default but it is possible to change this in configuration itself. Refer to default config file in
+ * <p>It is started on port 10300 by default, but it is possible to change this in configuration itself. Refer to default config file in
  * resources for the example.
  */
-public class RestModule implements IgniteComponent {
-    /** Default port. */
-    public static final int DFLT_PORT = 10300;
-
-    /** Node configuration route. */
-    private static final String NODE_CFG_URL = "/management/v1/configuration/node/";
-
-    /** Cluster configuration route. */
-    private static final String CLUSTER_CFG_URL = "/management/v1/configuration/cluster/";
-
-    /** Path parameter. */
-    private static final String PATH_PARAM = "selector";
-
+public class RestModule implements RestHandlersRegister, IgniteComponent {

Review comment:
       I would suggest to rename this class to `RestComponent` to be consistent with other components

##########
File path: modules/rest-api/src/main/java/org/apache/ignite/rest/ErrorResult.java
##########
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.rest;
+package org.apache.ignite.rest;

Review comment:
       Why did you remove the "internal" package?

##########
File path: modules/rest-api/src/main/java/org/apache/ignite/rest/Routes.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.ignite.rest;
+
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.util.AsciiString;
+
+/**
+ * Allows to configure REST handlers vs routes.
+ */
+public interface Routes {
+    /**
+     * Adds the route to router chain.
+     *
+     * @param route Route
+     */
+    void addRoute(Route route);
+
+    /**
+     * GET query helper.
+     *
+     * @param route      Route.
+     * @param acceptType Accept type.
+     * @param hnd        Actual handler of the request.
+     * @return Router
+     */
+    default Routes get(String route, AsciiString acceptType, RequestHandler hnd) {

Review comment:
       `AsciiString` is netty-specific, do we want to expose that?

##########
File path: modules/rest-api/src/main/java/org/apache/ignite/rest/Routes.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.ignite.rest;
+
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.util.AsciiString;
+
+/**
+ * Allows to configure REST handlers vs routes.
+ */
+public interface Routes {
+    /**
+     * Adds the route to router chain.
+     *
+     * @param route Route
+     */
+    void addRoute(Route route);
+
+    /**
+     * GET query helper.
+     *
+     * @param route      Route.
+     * @param acceptType Accept type.
+     * @param hnd        Actual handler of the request.
+     * @return Router
+     */
+    default Routes get(String route, AsciiString acceptType, RequestHandler hnd) {
+        addRoute(new Route(route, HttpMethod.GET, acceptType.toString(), hnd));
+        return this;
+    }
+
+    /**
+     * GET query helper.
+     *
+     * @param route Route.
+     * @param hnd   Actual handler of the request.
+     * @return Router
+     */
+    default Routes get(String route, RequestHandler hnd) {
+        addRoute(new Route(route, HttpMethod.GET, null, hnd));
+        return this;
+    }
+
+    /**
+     * PUT query helper.
+     *
+     * @param route      Route.
+     * @param acceptType Accept type.
+     * @param hnd        Actual handler of the request.
+     * @return Router
+     */
+    default Routes put(String route, AsciiString acceptType, RequestHandler hnd) {
+        addRoute(new Route(route, HttpMethod.PUT, acceptType.toString(), hnd));
+        return this;
+    }
+
+    /**
+     * Defines a PATCH route.

Review comment:
       This javadoc is inconsistent with other javadocs =)

##########
File path: modules/rest-api/src/main/java/org/apache/ignite/rest/Routes.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.ignite.rest;
+
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.util.AsciiString;
+
+/**
+ * Allows to configure REST handlers vs routes.
+ */
+public interface Routes {
+    /**
+     * Adds the route to router chain.

Review comment:
       ```suggestion
        * Adds the route to the router chain.
   ```

##########
File path: modules/rest-api/src/main/java/org/apache/ignite/rest/Routes.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.ignite.rest;
+
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.util.AsciiString;
+
+/**
+ * Allows to configure REST handlers vs routes.

Review comment:
       vs? Did you mean `via`?

##########
File path: modules/rest/src/main/java/org/apache/ignite/internal/rest/routes/Router.java
##########
@@ -17,113 +17,19 @@
 
 package org.apache.ignite.internal.rest.routes;
 
-import io.netty.handler.codec.http.HttpMethod;
 import io.netty.handler.codec.http.HttpRequest;
-import io.netty.util.AsciiString;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Optional;
+import org.apache.ignite.rest.Route;
 
 /**
  * Dispatcher of http requests.
- *
- * <p>Example:
- * <pre>
- * {@code
- * var router = new Router();
- * router.get("/user", (req, resp) -> {
- *     resp.status(HttpResponseStatus.OK);
- * });
- * }
- * </pre>
  */
-public class Router {
-    /** Routes. */
-    private final List<Route> routes;
-
-    /**
-     * Creates a new router with the given list of {@code routes}.
-     *
-     * @param routes Routes.
-     */
-    public Router(List<Route> routes) {
-        this.routes = routes;
-    }
-
-    /**
-     * Creates a new empty router.
-     */
-    public Router() {
-        routes = new ArrayList<>();
-    }
-
-    /**
-     * GET query helper.
-     *
-     * @param route      Route.
-     * @param acceptType Accept type.
-     * @param hnd        Actual handler of the request.
-     * @return Router
-     */
-    public Router get(String route, AsciiString acceptType, RequestHandler hnd) {
-        addRoute(new Route(route, HttpMethod.GET, acceptType.toString(), hnd));
-        return this;
-    }
-
-    /**
-     * GET query helper.
-     *
-     * @param route Route.
-     * @param hnd   Actual handler of the request.
-     * @return Router
-     */
-    public Router get(String route, RequestHandler hnd) {
-        addRoute(new Route(route, HttpMethod.GET, null, hnd));
-        return this;
-    }
-
-    /**
-     * PUT query helper.
-     *
-     * @param route      Route.
-     * @param acceptType Accept type.
-     * @param hnd        Actual handler of the request.
-     * @return Router
-     */
-    public Router put(String route, AsciiString acceptType, RequestHandler hnd) {
-        addRoute(new Route(route, HttpMethod.PUT, acceptType.toString(), hnd));
-        return this;
-    }
-
-    /**
-     * Defines a PATCH route.
-     *
-     * @param route      Route.
-     * @param acceptType Accept type.
-     * @param hnd        Actual handler of the request.
-     * @return Router
-     */
-    public Router patch(String route, AsciiString acceptType, RequestHandler hnd) {
-        addRoute(new Route(route, HttpMethod.PATCH, acceptType.toString(), hnd));
-        return this;
-    }
-
-    /**
-     * Adds the route to router chain.
-     *
-     * @param route Route
-     */
-    public void addRoute(Route route) {
-        routes.add(route);
-    }
-
+public interface Router {
     /**
      * Finds the route by request.
      *
      * @param req Request.
      * @return Route if founded.
      */
-    public Optional<Route> route(HttpRequest req) {
-        return routes.stream().filter(r -> r.match(req)).findFirst();
-    }
+    Optional<Route> route(HttpRequest req);

Review comment:
       we don't use Optionals anywhere, I would suggest to use `@Nullable` instead

##########
File path: modules/rest/src/main/java/org/apache/ignite/internal/rest/routes/Router.java
##########
@@ -17,113 +17,19 @@
 
 package org.apache.ignite.internal.rest.routes;
 
-import io.netty.handler.codec.http.HttpMethod;
 import io.netty.handler.codec.http.HttpRequest;
-import io.netty.util.AsciiString;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Optional;
+import org.apache.ignite.rest.Route;
 
 /**
  * Dispatcher of http requests.
- *
- * <p>Example:
- * <pre>
- * {@code
- * var router = new Router();
- * router.get("/user", (req, resp) -> {
- *     resp.status(HttpResponseStatus.OK);
- * });
- * }
- * </pre>
  */
-public class Router {
-    /** Routes. */
-    private final List<Route> routes;
-
-    /**
-     * Creates a new router with the given list of {@code routes}.
-     *
-     * @param routes Routes.
-     */
-    public Router(List<Route> routes) {
-        this.routes = routes;
-    }
-
-    /**
-     * Creates a new empty router.
-     */
-    public Router() {
-        routes = new ArrayList<>();
-    }
-
-    /**
-     * GET query helper.
-     *
-     * @param route      Route.
-     * @param acceptType Accept type.
-     * @param hnd        Actual handler of the request.
-     * @return Router
-     */
-    public Router get(String route, AsciiString acceptType, RequestHandler hnd) {
-        addRoute(new Route(route, HttpMethod.GET, acceptType.toString(), hnd));
-        return this;
-    }
-
-    /**
-     * GET query helper.
-     *
-     * @param route Route.
-     * @param hnd   Actual handler of the request.
-     * @return Router
-     */
-    public Router get(String route, RequestHandler hnd) {
-        addRoute(new Route(route, HttpMethod.GET, null, hnd));
-        return this;
-    }
-
-    /**
-     * PUT query helper.
-     *
-     * @param route      Route.
-     * @param acceptType Accept type.
-     * @param hnd        Actual handler of the request.
-     * @return Router
-     */
-    public Router put(String route, AsciiString acceptType, RequestHandler hnd) {
-        addRoute(new Route(route, HttpMethod.PUT, acceptType.toString(), hnd));
-        return this;
-    }
-
-    /**
-     * Defines a PATCH route.
-     *
-     * @param route      Route.
-     * @param acceptType Accept type.
-     * @param hnd        Actual handler of the request.
-     * @return Router
-     */
-    public Router patch(String route, AsciiString acceptType, RequestHandler hnd) {
-        addRoute(new Route(route, HttpMethod.PATCH, acceptType.toString(), hnd));
-        return this;
-    }
-
-    /**
-     * Adds the route to router chain.
-     *
-     * @param route Route
-     */
-    public void addRoute(Route route) {
-        routes.add(route);
-    }
-
+public interface Router {
     /**
      * Finds the route by request.
      *
      * @param req Request.
      * @return Route if founded.

Review comment:
       ```suggestion
        * @return Route if found.
   ```

##########
File path: modules/rest/src/main/java/org/apache/ignite/internal/rest/RestModule.java
##########
@@ -17,136 +17,73 @@
 
 package org.apache.ignite.internal.rest;
 
-import static io.netty.handler.codec.http.HttpHeaderValues.APPLICATION_JSON;
-import static io.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST;
-
 import io.netty.bootstrap.ServerBootstrap;
 import io.netty.channel.Channel;
 import io.netty.channel.ChannelFuture;
 import io.netty.handler.logging.LogLevel;
 import io.netty.handler.logging.LoggingHandler;
 import java.net.BindException;
-import java.nio.charset.StandardCharsets;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.CompletionException;
+import java.util.function.Consumer;
 import org.apache.ignite.configuration.schemas.rest.RestConfiguration;
 import org.apache.ignite.configuration.schemas.rest.RestView;
-import org.apache.ignite.configuration.validation.ConfigurationValidationException;
 import org.apache.ignite.internal.configuration.ConfigurationManager;
 import org.apache.ignite.internal.configuration.ConfigurationRegistry;
 import org.apache.ignite.internal.manager.IgniteComponent;
-import org.apache.ignite.internal.rest.netty.RestApiHttpRequest;
-import org.apache.ignite.internal.rest.netty.RestApiHttpResponse;
 import org.apache.ignite.internal.rest.netty.RestApiInitializer;
-import org.apache.ignite.internal.rest.presentation.ConfigurationPresentation;
-import org.apache.ignite.internal.rest.presentation.hocon.HoconPresentation;
 import org.apache.ignite.internal.rest.routes.Router;
+import org.apache.ignite.internal.rest.routes.SimpleRouter;
 import org.apache.ignite.lang.IgniteException;
 import org.apache.ignite.lang.IgniteLogger;
 import org.apache.ignite.network.NettyBootstrapFactory;
+import org.apache.ignite.rest.RestHandlersRegister;
+import org.apache.ignite.rest.Routes;
 
 /**
  * Rest module is responsible for starting a REST endpoints for accessing and managing configuration.
  *
- * <p>It is started on port 10300 by default but it is possible to change this in configuration itself. Refer to default config file in
+ * <p>It is started on port 10300 by default, but it is possible to change this in configuration itself. Refer to default config file in
  * resources for the example.
  */
-public class RestModule implements IgniteComponent {
-    /** Default port. */
-    public static final int DFLT_PORT = 10300;
-
-    /** Node configuration route. */
-    private static final String NODE_CFG_URL = "/management/v1/configuration/node/";
-
-    /** Cluster configuration route. */
-    private static final String CLUSTER_CFG_URL = "/management/v1/configuration/cluster/";
-
-    /** Path parameter. */
-    private static final String PATH_PARAM = "selector";
-
+public class RestModule implements RestHandlersRegister, IgniteComponent {
     /** Ignite logger. */
     private final IgniteLogger log = IgniteLogger.forClass(RestModule.class);
 
     /** Node configuration register. */
     private final ConfigurationRegistry nodeCfgRegistry;
 
-    /** Presentation of node configuration. */
-    private final ConfigurationPresentation<String> nodeCfgPresentation;
-
-    /** Presentation of cluster configuration. */
-    private final ConfigurationPresentation<String> clusterCfgPresentation;
-
     /** Netty bootstrap factory. */
     private final NettyBootstrapFactory bootstrapFactory;
 
+    private final SimpleRouter router = new SimpleRouter();
+
     /** Netty channel. */
     private volatile Channel channel;

Review comment:
       I would suggest to re-write this logic using the busy lock approach

##########
File path: modules/rest/src/main/java/org/apache/ignite/internal/rest/routes/SimpleRouter.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.ignite.internal.rest.routes;
+
+import io.netty.handler.codec.http.HttpRequest;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import org.apache.ignite.rest.Route;
+import org.apache.ignite.rest.Routes;
+
+/**
+ * Dispatcher of http requests.
+ *
+ * <p>Example:
+ * <pre>
+ * {@code
+ * var router = new SimpleRouter();
+ * router.get("/user", (req, resp) -> {
+ *     resp.status(HttpResponseStatus.OK);
+ * });
+ * }
+ * </pre>
+ */
+public class SimpleRouter implements Router, Routes {
+    /** Routes. */
+    private final List<Route> routes;
+
+    /**
+     * Creates a new router with the given list of {@code routes}.
+     *
+     * @param routes Routes.
+     */
+    public SimpleRouter(List<Route> routes) {
+        this.routes = List.copyOf(routes);

Review comment:
       1. This constructor is not used anywhere.
   2. `List.copyOf` creates an immutable list, `addRoute` which breaks the contract of this class (e.g. `addRoute` will fail)




-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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