You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by "yunqingmoswu (via GitHub)" <gi...@apache.org> on 2023/01/29 03:35:52 UTC

[GitHub] [inlong] yunqingmoswu commented on a diff in pull request #7228: [INLONG-7060][Sort] Support write redis in sort-connector-redis

yunqingmoswu commented on code in PR #7228:
URL: https://github.com/apache/inlong/pull/7228#discussion_r1089858955


##########
inlong-sort/sort-connectors/redis/src/main/java/org/apache/inlong/sort/redis/common/config/RedisOptions.java:
##########
@@ -200,6 +200,81 @@ public class RedisOptions {
                     .defaultValue(false)
                     .withDescription("whether to set async lookup.");
 
+    public static final ConfigOption<String> EXPIRE_TIME =
+            ConfigOptions.key("expire-time")
+                    .stringType()
+                    .defaultValue("0s")
+                    .withDescription("The redis record expired time. If value set to " +
+                            "zero or negative, " +
+                            "record in redis will never expired.");
+
+    public static final ConfigOption<Integer> SINK_MAX_RETRIES =
+            ConfigOptions.key("sink.max-retries")
+                    .intType()
+                    .defaultValue(5)
+                    .withDescription("The maximum number of retries when an " +
+                            "exception is caught.");
+
+    public static final ConfigOption<Integer> CLUSTER_MAX_TOTAL =
+            ConfigOptions.key("sink.max-connections")
+                    .intType()
+                    .defaultValue(2)
+                    .withDescription("Set the value for connection instances created " +
+                            "in pool.");
+
+    public static final ConfigOption<Integer> CLUSTER_MAX_IDLE =
+            ConfigOptions.key("sink.max-idle-connections")
+                    .intType()
+                    .defaultValue(1)
+                    .withDescription("Set the value for max idle connection " +
+                            "instances created in pool.");
+
+    public static final ConfigOption<String> CLUSTER_MAX_WAIT =
+            ConfigOptions.key("sink.max-wait-time")
+                    .stringType()
+                    .defaultValue("10s")
+                    .withDescription("Set the value for max waiting time if there is " +
+                            "no connection resource.");
+
+    public static final ConfigOption<Integer> MAX_CACHE_SIZE =
+            ConfigOptions.key("sink.max-cache-size")
+                    .intType()
+                    .defaultValue(-1)
+                    .withDescription("The maximum number of results cached in the " +

Review Comment:
   describe error?



##########
inlong-sort/sort-connectors/redis/src/main/java/org/apache/inlong/sort/redis/common/config/SchemaMappingMode.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.inlong.sort.redis.common.config;
+
+/**
+ * The MappingMode between Sql Schema and {@link RedisDataType}.
+ */
+public enum SchemaMappingMode {
+
+    /**
+     * The DYNAMIC mode witch mapping a {@link java.util.Map} to {@link RedisDataType}.
+     * There are two members in DYNAMIC mode: <br/>
+     * the first member is Redis key. <br/>
+     * the second member is a {@link java.util.Map} object, witch will be iterated, <br/>

Review Comment:
   Spelling mistake for 'witch'?



##########
inlong-sort/sort-connectors/redis/src/main/java/org/apache/inlong/sort/redis/common/config/RedisOptions.java:
##########
@@ -200,6 +200,81 @@ public class RedisOptions {
                     .defaultValue(false)
                     .withDescription("whether to set async lookup.");
 
+    public static final ConfigOption<String> EXPIRE_TIME =
+            ConfigOptions.key("expire-time")
+                    .stringType()
+                    .defaultValue("0s")
+                    .withDescription("The redis record expired time. If value set to " +
+                            "zero or negative, " +
+                            "record in redis will never expired.");
+
+    public static final ConfigOption<Integer> SINK_MAX_RETRIES =
+            ConfigOptions.key("sink.max-retries")
+                    .intType()
+                    .defaultValue(5)
+                    .withDescription("The maximum number of retries when an " +
+                            "exception is caught.");
+
+    public static final ConfigOption<Integer> CLUSTER_MAX_TOTAL =
+            ConfigOptions.key("sink.max-connections")
+                    .intType()
+                    .defaultValue(2)
+                    .withDescription("Set the value for connection instances created " +
+                            "in pool.");
+
+    public static final ConfigOption<Integer> CLUSTER_MAX_IDLE =
+            ConfigOptions.key("sink.max-idle-connections")
+                    .intType()
+                    .defaultValue(1)
+                    .withDescription("Set the value for max idle connection " +
+                            "instances created in pool.");
+
+    public static final ConfigOption<String> CLUSTER_MAX_WAIT =
+            ConfigOptions.key("sink.max-wait-time")
+                    .stringType()
+                    .defaultValue("10s")
+                    .withDescription("Set the value for max waiting time if there is " +
+                            "no connection resource.");
+
+    public static final ConfigOption<Integer> MAX_CACHE_SIZE =
+            ConfigOptions.key("sink.max-cache-size")
+                    .intType()
+                    .defaultValue(-1)
+                    .withDescription("The maximum number of results cached in the " +
+                            "lookup source.");
+
+    public static final ConfigOption<String> MAX_CACHE_TIME =
+            ConfigOptions.key("sink.max-cache-time")
+                    .stringType()
+                    .defaultValue("60s")
+                    .withDescription("The maximum live time for cached results in " +
+                            "the lookup source.");
+
+    public static final ConfigOption<Long> SINK_BATCH_SIZE =
+            ConfigOptions.key("sink.batch-size")
+                    .longType()
+                    .defaultValue(100L)
+                    .withDescription("The batch size of the sink operator to send data.");
+
+    public static final ConfigOption<String> SINK_FLUSH_INTERVAL =
+            ConfigOptions.key("sink.flush-interval")
+                    .stringType()
+                    .defaultValue("10s")
+                    .withDescription("The maximum waiting time for batch data sent by the sink operator ");
+
+    public static final ConfigOption<RedisDataType> DATA_TYPE =
+            ConfigOptions.key("data-type")
+                    .enumType(RedisDataType.class)
+                    .defaultValue(RedisDataType.PLAIN)
+                    .withDescription("Defines the redis data type, valid types are: 'PLAIN', 'HASH'");

Review Comment:
   'LIST' is not supported, but i find it has defined in the 'RedisDataType'? 



-- 
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: commits-unsubscribe@inlong.apache.org

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