You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2022/11/03 23:47:55 UTC

[GitHub] [cassandra-sidecar] yifan-c commented on a diff in pull request #41: CASSANDRASC-47: Introduce JMX foundation in Sidecar

yifan-c commented on code in PR #41:
URL: https://github.com/apache/cassandra-sidecar/pull/41#discussion_r1012397866


##########
cassandra40/src/main/java/org/apache/cassandra/sidecar/cassandra40/Cassandra40StorageOperations.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.cassandra40;
+
+import java.util.Map;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.StorageOperations;
+
+/**
+ * An implementation of the {@link StorageOperations} that interfaces with Cassandra 4.0
+ */
+public class Cassandra40StorageOperations implements StorageOperations
+{
+    private static final String SS_OBJ_NAME = "org.apache.cassandra.db:type=StorageService";

Review Comment:
   nit: no need to abbreviate, use `STORAGE_SERVICE_OBJ_NAME`



##########
cassandra-integration-tests/src/testFixtures/java/org/apache/cassandra/sidecar/common/containers/CassandraContainer.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.common.containers;
+
+/**
+ * A Cassandra container that adds functionality to the testcontainers
+ * {@link org.testcontainers.containers.CassandraContainer} for JMX.
+ */
+public class CassandraContainer extends org.testcontainers.containers.CassandraContainer<CassandraContainer>
+{
+    public static final String RMI_SERVER_HOSTNAME = "127.0.0.1";
+    public static final Integer JMX_PORT = 7199;
+
+    /**
+     * Constructs a new {@link CassandraContainer} with the {@link #JMX_PORT JMX port} exposed
+     *
+     * @param dockerImageName the name of the docker image to use for the container
+     */
+    public CassandraContainer(String dockerImageName)
+    {
+        super(dockerImageName);
+//        addExposedPort(JMX_PORT);

Review Comment:
   nit: remove the commented line if it no longer needed. 



##########
common/src/main/java/org/apache/cassandra/sidecar/common/CassandraAdapterDelegate.java:
##########
@@ -174,6 +176,13 @@ public List<NodeStatus> getStatus()
         return adapter.getStatus();
     }
 
+    @Override
+    public StorageOperations storageOperations()
+    {
+        checkSession();

Review Comment:
   nit: remove the check. JMX operation does not require session to be established, although there is no harm to check here. 



##########
common/src/main/java/org/apache/cassandra/sidecar/common/ICassandraAdapter.java:
##########
@@ -28,6 +28,14 @@
  */
 public interface ICassandraAdapter
 {
+    /**
+     * @return a list of {@link NodeStatus} for the Cassandra cluster
+     */

Review Comment:
   TBH, I do not know why the return type is a list. The `ICassandraAdapter` is to interact with the local Cassandra node that pairs with the sidecar. Why returning the list of node status? Sidecar does not support bulk operation yet, and it is not even implemented. 
   Let not add the comment for the method. The comment might be no accurate. We might want to refactor the method later since it does not make too much sense. 



##########
common/src/main/java/org/apache/cassandra/sidecar/common/StorageOperations.java:
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.common;
+
+import java.util.Map;
+
+/**
+ * An interface that defines interactions with the storage system in Cassandra
+ */
+public interface StorageOperations

Review Comment:
   nit: should it be called `JmxOperations`, which interacts with different components in cassandra (not limited to storage) via JMX, or `StorageJmxOperations` if want to limit for storage service. 



##########
common/src/main/java/org/apache/cassandra/sidecar/common/JmxClient.java:
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.common;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.rmi.server.RMIClientSocketFactory;
+import java.rmi.server.RMISocketFactory;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import javax.management.JMX;
+import javax.management.MBeanServerConnection;
+import javax.management.MalformedObjectNameException;
+import javax.management.Notification;
+import javax.management.NotificationListener;
+import javax.management.ObjectName;
+import javax.management.remote.JMXConnectionNotification;
+import javax.management.remote.JMXConnector;
+import javax.management.remote.JMXConnectorFactory;
+import javax.management.remote.JMXServiceURL;
+import javax.rmi.ssl.SslRMIClientSocketFactory;
+
+import org.jetbrains.annotations.VisibleForTesting;
+
+/**
+ * A simple wrapper around a JMX connection that makes it easier to get proxy instances.
+ */
+public class JmxClient implements NotificationListener
+{
+    public static final String JMX_SERVICE_URL_FMT = "service:jmx:rmi://%s/jndi/rmi://%s:%d/jmxrmi";
+    public static final String REGISTRY_CONTEXT_SOCKET_FACTORY = "com.sun.jndi.rmi.factory.socket";
+    private final JMXServiceURL jmxServiceURL;
+    private MBeanServerConnection mBeanServerConnection;
+    private final Map<String, Object> jmxEnv;
+    private boolean connected = false;
+
+    /**
+     * Creates a new client with the provided {@code host} and {@code port}.
+     *
+     * @param host the host of the JMX service
+     * @param port the port of the JMX service
+     */
+    public JmxClient(String host, int port)
+    {
+        this(host, port, null, null, false);
+    }
+
+    /**
+     * Creates a new client with the provided parameters
+     *
+     * @param host      the host of the JMX service
+     * @param port      the port of the JMX service
+     * @param role      the JMX role used for authentication
+     * @param password  the JMX role password used for authentication
+     * @param enableSSl true if SSL is enabled for JMX, false otherwise
+     */
+    public JmxClient(String host, int port, String role, String password, boolean enableSSl)
+    {
+        this(buildJmxServiceURL(host, port), role, password, enableSSl);
+    }
+
+    @VisibleForTesting
+    JmxClient(JMXServiceURL jmxServiceURL)
+    {
+        this(jmxServiceURL, null, null, false);
+    }
+
+    @VisibleForTesting
+    JmxClient(JMXServiceURL jmxServiceURL, String role, String password)
+    {
+        this(jmxServiceURL, role, password, false);
+    }
+
+    private JmxClient(JMXServiceURL jmxServiceURL, String role, String password, boolean enableSsl)
+    {
+        this.jmxServiceURL = jmxServiceURL;
+
+        jmxEnv = new HashMap<>();
+        if (role != null && password != null)
+        {
+            String[] credentials = new String[]{ role, password };
+            jmxEnv.put(JMXConnector.CREDENTIALS, credentials);
+        }
+        jmxEnv.put(REGISTRY_CONTEXT_SOCKET_FACTORY, getRMIClientSocketFactory(enableSsl));
+    }
+
+    /**
+     * Applies a function through the JMX bean proxy
+     *
+     * @param clientClass the management interface that the MBean exports, which will
+     *                    also be implemented by the returned proxy
+     * @param remoteName  the name of the MBean within {@code connection} to forward to
+     * @param func        the remote function that will be executed through the proxy
+     * @param <TClient>   the type of the proxy client
+     * @param <TResult>   the type of the result returned by the proxy
+     * @return the result of the remote function after being executed via the proxy
+     */
+    public <TClient, TResult> TResult apply(Class<TClient> clientClass,

Review Comment:
   Generic types named in the form that looks like class names are misleading and confusing. Can you just do `C` for client and `R` for result? 



-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org