You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by GitBox <gi...@apache.org> on 2021/05/27 08:34:01 UTC

[GitHub] [incubator-inlong] gosonzhang commented on a change in pull request #476: [INLONG-621][configuration] Introduce the ConfigConstants keys and default values references to replace related string constant values of configuration;refactor TLSConfig.java

gosonzhang commented on a change in pull request #476:
URL: https://github.com/apache/incubator-inlong/pull/476#discussion_r640385466



##########
File path: tubemq-core/src/main/java/org/apache/tubemq/corebase/configuration/ConfigOption.java
##########
@@ -0,0 +1,302 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.tubemq.corebase.configuration;

Review comment:
       tubemq-core/src/main/java/org/apache/tubemq/corebase/configuration/*
   
   I found that part of the content in this directory is directly copied from flink.
   
   If you want to use it directly, you need to declare its corresponding license and notice in LICENSE and NOTICE, such as https://github.com/apache/incubator-inlong/ Processing of section 1.3.1 in blob/master/LICENSE#L329 and https://github.com/apache/incubator-inlong/blob/master/NOTICE#L9
   
   By the way, can you digest this piece and implement it with your own ideas? You can also see that the LICENSE and NOTICE of most open source projects are very concise, indicating that most of them are written by themselves.
   
   It doesn't matter if you haven't considered it for the time being, we can improve it later.

##########
File path: tubemq-server/src/main/java/org/apache/tubemq/server/broker/web/BrokerAdminServlet.java
##########
@@ -436,7 +436,7 @@ public void adminQuerySnapshotMessageSet(HttpServletRequest req,
         }
         final String topicName = (String) result.retData1;
         if (!WebParameterUtils.getIntParamValue(req,
-                WebFieldDef.PARTITIONID, false, -1, 0, result)) {
+                WebFieldDef.PARTITIONID, true, -1, 0, result)) {

Review comment:
       You should rebase master codes before submitting the code. 
   
   Here, a modified bug was be rolled back: https://github.com/apache/incubator-inlong/pull/467

##########
File path: tubemq-core/src/main/java/org/apache/tubemq/corebase/configuration/ConfigConstants.java
##########
@@ -0,0 +1,940 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.tubemq.corebase.configuration;
+
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import org.apache.tubemq.corebase.TBaseConstants;
+import org.apache.tubemq.corerpc.RpcConstants;
+
+public interface ConfigConstants {
+
+    /**
+     * ==================================================
+     *  Keys constants.
+     * ==================================================
+     */
+
+    /**
+     * --------------------------------------------------
+     * Sections constants.
+     * --------------------------------------------------
+     */
+    String SECT_TOKEN_MASTER = "master";
+
+    String SECT_TOKEN_BROKER = "broker";
+
+    String SECT_TOKEN_BDB = "bdbStore";
+
+    String SECT_TOKEN_TLS = "tlsSetting";
+
+    String SECT_TOKEN_ZKEEPER = "zookeeper";
+
+    String SECT_TOKEN_REPLICATION = "replication";
+
+
+    /**
+     * ----------------------------------------------------
+     * Tls constants.
+     * ----------------------------------------------------
+     */
+    String TLS_ENABLE = "tlsEnable";
+
+    String TLS_PORT = "tlsPort";
+
+    String TLS_KEY_STORE_PATH = "tlsKeyStorePath";
+
+    String TLS_KEY_STORE_PASSWORD = "tlsKeyStorePassword";
+
+    String TLS_TWO_WAY_AUTH_ENABLE = "tlsTwoWayAuthEnable";
+
+    String TLS_TRUST_STORE_PATH = "tlsTrustStorePath";
+
+    String TLS_TRUST_STORE_PASSWORD = "tlsTrustStorePassword";
+
+
+    /**
+     * ----------------------------------------------------------
+     * Zookeeper constants.
+     * ----------------------------------------------------------
+     */
+
+    /**
+     * Address list of zookeeper cluster.
+     */
+    String ZK_SERVER_ADDR = "zkServerAddr";
+
+    /**
+     * Root path of tubeMq in zookeeper.
+     */
+    String ZK_NODE_ROOT = "zkNodeRoot";
+
+    /**
+     * Zookeeper session timeout setting, millis.
+     */
+    String ZK_SESSION_TIMEOUT_MS = "zkSessionTimeoutMs";
+
+    /**
+     * Zookeeper connection timeout setting. millis.
+     */
+    String ZK_CONNECTION_TIMEOUT_MS = "zkConnectionTimeoutMs";
+
+    String ZK_SYNC_TIME_MS = "zkSyncTimeMs";
+
+    String ZK_COMMIT_PERIOD_MS = "zkCommitPeriodMs";
+
+    String ZK_COMMIT_FAIL_RETRIES = "zkCommitFailRetries";
+
+
+    /**
+     * -------------------------------------------------------
+     * Common constants between broker and master components.
+     * -------------------------------------------------------
+     */
+    String PORT = "port";
+
+    String WEB_PORT = "webPort";
+
+    String HOST_NAME = "hostName";
+
+    String VISIT_NAME = "visitName";
+
+    String VISIT_PASSWORD = "visitPassword";
+
+    String AUTH_VALID_TIMESTAMP_PERIOD_MS = "authValidTimeStampPeriodMs";
+
+    String ROW_LOCK_WAIT_DUR_MS = "rowLockWaitDurMs";
+
+    String SOCKET_SEND_BUFFER = "socketSendBuffer";
+
+    String SOCKET_RECV_BUFFER = "socketRecvBuffer";
+
+    String RPC_READ_TIMEOUT_MS = "rpcReadTimeoutMs";
+
+    String NETTY_WRITE_BUFFER_HIGH_WATERMARK = "nettyWriteBufferHighWaterMark";
+
+    String NETTY_WRITE_BUFFER_LOW_WATERMARK = "nettyWriteBufferLowWaterMark";
+
+    /**
+     * --------------------------------------------------------------------------
+     * Broker constants.
+     * --------------------------------------------------------------------------
+     */
+    /**
+     * Broker id.
+     */
+    String BROKER_ID = "brokerId";
+
+    /**
+     * Broker port.
+     */
+    String BROKER_PORT = PORT;
+
+    /**
+     * Broker primaryPath.
+     */
+    String PRIMARY_PATH = "primaryPath";
+
+    /**
+     * Broker hostname.
+     */
+    String BROKER_HOST_NAME = HOST_NAME;
+
+    /**
+     * Default network interface.
+     */
+    String DEF_ETH_NAME = "defEthName";
+
+    /**
+     * Master service address.
+     */
+    String MASTER_ADDRESS_LIST = "masterAddressList";
+
+    /**
+     * Broker web service port.
+     */
+    String BROKER_WEB_PORT = WEB_PORT;
+
+    /**
+     * Max data segment size.
+     */
+    String MAX_SEGMENT_SIZE = "maxSegmentSize";
+
+    /**
+     * Transfer size.
+     */
+    String TRANSFER_SIZE = "transferSize";
+
+    /**
+     * Transfer index count.
+     */
+    String INDEX_TRANS_COUNT = "indexTransCount";
+
+    /**
+     * Log cleanup interval in milliseconds.
+     */
+    String LOG_CLEAR_UP_DURATION_MS = "logClearupDurationMs";
+
+    /**
+     * Log flush to disk interval in milliseconds.
+     */
+    String LOG_FLUSH_DISK_DUR_MS = "logFlushDiskDurMs";
+
+    /**
+     * Memory flush to disk interval in milliseconds.
+     */
+    String LOG_FLUSH_MEM_DUR_MS = "logFlushMemDurMs";
+
+    String BROKER_AUTH_VALID_TIMESTAMP_PERIOD_MS = AUTH_VALID_TIMESTAMP_PERIOD_MS;
+
+    String VISIT_TOKEN_CHECK_IN_VALID_TIME_MS = "visitTokenCheckInValidTimeMs";
+
+    /**
+     * Socket send buffer.
+     */
+    String BROKER_SOCKET_SEND_BUFFER = SOCKET_SEND_BUFFER;
+
+    /**
+     * Socket receive buffer.
+     */
+    String BROKER_SOCKET_RECV_BUFFER = SOCKET_RECV_BUFFER;
+
+    /**
+     * Max index segment size.
+     */
+    String MAX_INDEX_SEGMENT_SIZE = "maxIndexSegmentSize";
+
+    String UPDATE_CONSUMER_OFFSETS = "updateConsumerOffsets";
+
+    /**
+     * Rpc read timeout in milliseconds.
+     */
+    String BROKER_RPC_READ_TIMEOUT_MS = RPC_READ_TIMEOUT_MS;
+
+    /**
+     * Netty write buffer high water mark.
+     */
+    String BROKER_NETTY_WRITE_BUFFER_HIGH_WATERMARK = NETTY_WRITE_BUFFER_HIGH_WATERMARK;
+
+    /**
+     * Netty write buffer low water mark.
+     */
+    String BROKER_NETTY_WRITE_BUFFER_LOW_WATERMARK = NETTY_WRITE_BUFFER_LOW_WATERMARK;
+
+    /**
+     * Heartbeat interval in milliseconds.
+     */
+    String HEARTBEAT_PERIOD_MS = "heartbeatPeriodMs";
+
+    /**
+     * Tcp write service thread count.
+     */
+    String TCP_WRITE_SERVICE_THREAD = "tcpWriteServiceThread";
+
+    /**
+     * Tcp read service thread count.
+     */
+    String TCP_READ_SERVICE_THREAD = "tcpReadServiceThread";
+
+    /**
+     * TLS write service thread count.
+     */
+    String TLS_WRITE_SERVICE_THREAD = "tlsWriteServiceThread";
+
+    /**
+     * TLS read service thread count.
+     */
+    String TLS_READ_SERVICE_THREAD = "tlsReadServiceThread";
+
+    /**
+     * Consumer register timeout in milliseconds.
+     */
+    String CONSUMER_REG_TIMEOUT_MS = "consumerRegTimeoutMs";
+
+    String DEFAULT_DEDUCE_READ_SIZE = "defaultDeduceReadSize";
+
+    /**
+     * Row lock wait duration.
+     */
+    String BROKER_ROW_LOCK_WAIT_DUR_MS = ROW_LOCK_WAIT_DUR_MS;
+
+    /**
+     * Read io exception max count.
+     */
+    String ALLOWED_READ_IO_EXCEPTION_CNT = "allowedReadIOExcptCnt";
+
+    /**
+     * Write io exception max count.
+     */
+    String ALLOWED_WRITE_IO_EXCEPTION_CNT = "allowedWriteIOExcptCnt";
+
+    String IO_EXCEPTION_STATS_DURATION_MS = "ioExcptStatsDurationMs";
+
+    String VISIT_MASTER_AUTH = "visitMasterAuth";
+
+    String BROKER_VISIT_NAME = VISIT_NAME;
+
+    String BROKER_VISIT_PASSWORD = VISIT_PASSWORD;
+
+
+    /**
+     * -----------------------------------------------------------------
+     * Master constants.
+     * -----------------------------------------------------------------
+     */
+    String MASTER_PORT = PORT;
+
+    String MASTER_HOST_NAME = HOST_NAME;
+
+    String MASTER_WEB_PORT = WEB_PORT;
+
+    String WEB_RESOURCE_PATH = "webResourcePath";
+
+    String META_DATA_PATH = "metaDataPath";
+
+    String CONSUMER_BALANCE_PERIOD_MS = "consumerBalancePeriodMs";
+
+    String FIRST_BALANCE_DELAY_AFTER_START_MS = "firstBalanceDelayAfterStartMs";
+
+    String CONSUMER_HEARTBEAT_TIMEOUT_MS = "consumerHeartbeatTimeoutMs";
+
+    String PRODUCER_HEARTBEAT_TIMEOUT_MS = "producerHeartbeatTimeoutMs";
+
+    String BROKER_HEARTBEAT_TIMEOUT_MS = "brokerHeartbeatTimeoutMs";
+
+    String MASTER_SOCKET_SEND_BUFFER = SOCKET_SEND_BUFFER;
+
+    String MASTER_SOCKET_RECV_BUFFER = SOCKET_RECV_BUFFER;
+
+    String MASTER_RPC_READ_TIMEOUT_MS = RPC_READ_TIMEOUT_MS;
+
+    String MASTER_NETTY_WRITE_BUFFER_HIGH_WATERMARK = NETTY_WRITE_BUFFER_HIGH_WATERMARK;
+
+    String MASTER_NETTY_WRITE_BUFFER_LOW_WATERMARK = NETTY_WRITE_BUFFER_LOW_WATERMARK;
+
+    String ONLINE_ONLY_READ_TO_RW_PERIOD_MS = "onlineOnlyReadToRWPeriodMs";
+
+    String STEP_CHG_WAIT_PERIOD_MS = "stepChgWaitPeriodMs";
+
+    String OFFLINE_ONLY_READ_TO_RW_PERIOD_MS = "offlineOnlyReadToRWPeriodMs";
+
+    String CONF_MOD_AUTH_TOKEN = "confModAuthToken";
+
+    String MAX_GROUP_BROKER_CONSUME_RATE = "maxGroupBrokerConsumeRate";
+
+    String MAX_GROUP_REBALANCE_WAIT_PERIOD = "maxGroupRebalanceWaitPeriod";
+
+    String START_OFFSET_RESET_CHECK = "startOffsetResetCheck";
+
+    String MASTER_ROW_LOCK_WAIT_DUR_MS = ROW_LOCK_WAIT_DUR_MS;
+
+    String MAX_AUTO_FORBIDDEN_CNT = "maxAutoForbiddenCnt";
+
+    String VISIT_TOKEN_VALID_PERIOD_MS = "visitTokenValidPeriodMs";
+
+    String MASTER_AUTH_VALID_TIME_STAMP_PERIOD_MS = AUTH_VALID_TIMESTAMP_PERIOD_MS;
+
+    String START_VISIT_TOKEN_CHECK = "startVisitTokenCheck";
+
+    String START_PRODUCE_AUTHENTICATE = "startProduceAuthenticate";
+
+    String START_PRODUCE_AUTHORIZE = "startProduceAuthorize";
+
+    String USE_WEB_PROXY = "useWebProxy";
+
+    String START_CONSUME_AUTHENTICATE = "startConsumeAuthenticate";
+
+    String START_CONSUME_AUTHORIZE = "startConsumeAuthorize";
+
+    String NEED_BROKER_VISIT_AUTH = "needBrokerVisitAuth";
+
+    String MASTER_VISIT_NAME = VISIT_NAME;
+
+    String MASTER_VISIT_PASSWORD = VISIT_PASSWORD;
+
+    String REBALANCE_PARALLEL = "rebalanceParallel";
+
+    /**
+     * -------------------------------------------------------------------
+     * Bdb constants.
+     * -------------------------------------------------------------------
+     */
+    @Deprecated
+    String BDB_REP_GROUP_NAME = "bdbRepGroupName";
+
+    @Deprecated
+    String BDB_NODE_NAME = "bdbNodeName";
+
+    @Deprecated
+    String BDB_NODE_PORT = "bdbNodePort";
+
+    @Deprecated
+    String BDB_ENV_HOME = "bdbEnvHome";
+
+    @Deprecated
+    String BDB_HELPER_HOST = "bdbHelperHost";
+
+    @Deprecated
+    String BDB_LOCAL_SYNC = "bdbLocalSync";
+
+    @Deprecated
+    String BDB_REPLICA_SYNC = "bdbReplicaSync";
+
+    @Deprecated
+    String BDB_REPLICA_ACK = "bdbReplicaAck";
+
+    @Deprecated
+    String BDB_STATUS_CHECK_TIMEOUT_MS = "bdbStatusCheckTimeoutMs";
+
+
+    /**
+     * -------------------------------------------------------------
+     * Replication constants.
+     * -------------------------------------------------------------
+     */
+
+    String REP_GROUP_NAME = "repGroupName";
+
+    String REP_NODE_NAME = "repNodeName";
+
+    String REP_NODE_PORT = "repNodePort";
+
+    String REP_HELPER_HOST = "repHelperHost";
+
+    String META_LOCAL_SYNC_POLICY = "metaLocalSyncPolicy";
+
+    String META_REPLICA_SYNC_POLICY = "metaReplicaSyncPolicy";
+
+    String REP_REPLICA_ACK_POLICY = "repReplicaAckPolicy";
+
+    String REP_STATUS_CHECK_TIMEOUT_MS = "repStatusCheckTimeoutMs";
+
+
+    /**
+     * ==============================================================
+     * Default values.
+     * ==============================================================
+     */
+    Charset DEFAULT_CHARSET = StandardCharsets.UTF_8;
+
+    /**
+     * --------------------------------------------------------------
+     * Zookeeper default values.
+     * --------------------------------------------------------------
+     */
+
+    /**
+     * Default value of {@link #ZK_SERVER_ADDR}.
+     */
+    String DEFAULT_ZK_SERVER_ADDR = "localhost:2181";
+
+    /**
+     * Default value of {@link #ZK_NODE_ROOT}.
+     */
+    String DEFAULT_ZK_NODE_ROOT = "/tubemq";
+
+    /**
+     * Default value of {@link #ZK_SESSION_TIMEOUT_MS}.
+     */
+    int DEFAULT_ZK_SESSION_TIMEOUT_MS = 180000;
+
+    /**
+     * Default value of {@link #ZK_CONNECTION_TIMEOUT_MS}.
+     */
+    int DEFAULT_ZK_CONNECTION_TIMEOUT_MS = 600000;
+
+    /**
+     * Default value of {@link #ZK_SYNC_TIME_MS}.
+     */
+    int DEFAULT_ZK_SYNC_TIME_MS = 1000;
+
+    /**
+     * Default value of {@link #ZK_COMMIT_FAIL_RETRIES}.
+     */
+    int DEFAULT_ZK_COMMIT_FAIL_RETRIES = 10;
+
+    /**
+     * Default value of {@link #ZK_COMMIT_PERIOD_MS}.
+     */
+    long DEFAULT_ZK_COMMIT_PERIOD_MS = 5000L;
+
+    /**
+     * --------------------------------------------------------------
+     * TLS default values.
+     * --------------------------------------------------------------
+     */
+
+    /**
+     * Default value of {@link #TLS_ENABLE}.
+     */
+    boolean DEFAULT_TLS_ENABLE = false;
+
+    /**
+     * Default value of {@link #TLS_TWO_WAY_AUTH_ENABLE}.
+     */
+    boolean DEFAULT_TLS_TWO_WAY_AUTH_ENABLE = false;
+
+    /**
+     * Default value of {@link #TLS_PORT}.
+     */
+    int DEFAULT_TLS_PORT = 8124;
+
+    /**
+     * Default value of {@link #TLS_KEY_STORE_PATH}.
+     */
+    String DEFAULT_TLS_KEY_STORE_PATH = "";
+
+    /**
+     * Default value of {@link #TLS_KEY_STORE_PASSWORD}.
+     */
+    String DEFAULT_TLS_KEY_STORE_PASSWORD = "";
+
+    /**
+     * Default value of {@link #TLS_TRUST_STORE_PATH}.
+     */
+    String DEFAULT_TLS_TRUST_STORE_PATH = "";
+
+    /**
+     * Default value of {@link #TLS_TRUST_STORE_PASSWORD}.
+     */
+    String DEFAULT_TLS_TRUST_STORE_PASSWORD = "";
+
+    /**
+     * --------------------------------------------------------------
+     * Broker default values.
+     * --------------------------------------------------------------
+     */
+
+    /**
+     * Default value of {@link #BROKER_ID}.
+     */
+    int DEFAULT_BROKER_ID = 0;
+
+    /**
+     * Default value of {@link #BROKER_PORT}.
+     */
+    int DEFAULT_BROKER_PORT = TBaseConstants.META_DEFAULT_BROKER_PORT;
+
+    /**
+     * Default value of {@link #PRIMARY_PATH}.
+     */
+    String DEFAULT_PRIMARY_PATH = "";
+
+    /**
+     * Default value of {@link #BROKER_HOST_NAME}.
+     */
+    String DEFAULT_BROKER_HOST_NAME = "";
+
+    /**
+     * Default value of {@link #DEF_ETH_NAME}.
+     */
+    String DEFAULT_DEF_ETH_NAME = "eth1";
+
+    /**
+     * Default value of {@link #MASTER_ADDRESS_LIST}.
+     */
+    String DEFAULT_MASTER_ADDRESS_LIST = "";
+
+    /**
+     * Default value of {@link #BROKER_WEB_PORT}.
+     */
+    int DEFAULT_BROKER_WEB_PORT = 8081;
+
+    /**
+     * Default value of {@link #MAX_SEGMENT_SIZE}.
+     */
+    int DEFAULT_MAX_SEGMENT_SIZE = 1024 * 1024 * 1024;
+
+    /**
+     * Default value of {@link #TRANSFER_SIZE}.
+     */
+    int DEFAULT_TRANSFER_SIZE = 512 * 1024;
+
+    /**
+     * Default value of {@link #INDEX_TRANS_COUNT}.
+     */
+    int DEFAULT_INDEX_TRANS_COUNT = 1000;
+
+    /**
+     * Default value of {@link #LOG_CLEAR_UP_DURATION_MS}.
+     */
+    long DEFAULT_LOG_CLEAR_UP_DURATION_MS = 3 * 60 * 1000L;
+
+    /**
+     * Default value of {@link #LOG_FLUSH_DISK_DUR_MS}.
+     */
+    long DEFAULT_LOG_FLUSH_DISK_DUR_MS = 20 * 1000L;
+
+    /**
+     * Default value of {@link #LOG_FLUSH_MEM_DUR_MS}.
+     */
+    long DEFAULT_LOG_FLUSH_MEM_DUR_MS = 10 * 1000L;
+
+    /**
+     * Default value of {@link #BROKER_AUTH_VALID_TIMESTAMP_PERIOD_MS}.
+     */
+    long DEFAULT_BROKER_AUTH_VALID_TIMESTAMP_PERIOD_MS = TBaseConstants.CFG_DEFAULT_AUTH_TIMESTAMP_VALID_INTERVAL;
+
+    /**
+     * Default value of {@link #VISIT_TOKEN_CHECK_IN_VALID_TIME_MS}.
+     */
+    long DEFAULT_VISIT_TOKEN_CHECK_IN_VALID_TIME_MS = 120000L;
+
+    /**
+     * Default value of {@link #BROKER_SOCKET_SEND_BUFFER}.
+     */
+    long DEFAULT_BROKER_SOCKET_SEND_BUFFER = -1;
+
+    /**
+     * Default value of {@link #BROKER_SOCKET_RECV_BUFFER}.
+     */
+    long DEFAULT_BROKER_SOCKET_RECV_BUFFER = -1;
+
+
+    /**
+     * Default value of {@link #MAX_INDEX_SEGMENT_SIZE}.
+     */
+    // DataStoreUtils.STORE_INDEX_HEAD_LEN = 28
+    int DEFAULT_MAX_INDEX_SEGMENT_SIZE = 700000 * 28;

Review comment:
       The constant configurations of the server and the client should be separated, and constants should not be used here instead of DataStoreUtils.STORE_INDEX_HEAD_LEN, which may be changed anytime.




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