You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@kvrocks.apache.org by GitBox <gi...@apache.org> on 2022/10/07 13:04:05 UTC

[GitHub] [incubator-kvrocks] tisonkun commented on a diff in pull request #944: Move TCL test unit/type/string to Go case

tisonkun commented on code in PR #944:
URL: https://github.com/apache/incubator-kvrocks/pull/944#discussion_r990061373


##########
tests/gocase/unit/type/strings/strings_test.go:
##########
@@ -0,0 +1,513 @@
+/*
+* 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 strings
+
+import (
+	"context"
+	"strconv"
+	"strings"
+	"testing"
+	"time"
+
+	"github.com/apache/incubator-kvrocks/tests/gocase/util"
+	"github.com/go-redis/redis/v9"
+	"github.com/stretchr/testify/require"
+)
+
+func TestString(t *testing.T) {
+	srv := util.StartServer(t, map[string]string{})
+	defer srv.Close()
+	ctx := context.Background()
+	rdb := srv.NewClient()
+	defer func() { require.NoError(t, rdb.Close()) }()
+
+	t.Run("SET and GET an item", func(t *testing.T) {
+		key := "x"
+		value := "foo"
+		require.NoError(t, rdb.Set(ctx, key, value, 0).Err())
+		require.Equal(t, value, rdb.Get(ctx, key).Val())
+	})
+
+	t.Run("SET and GET an empty item", func(t *testing.T) {
+		key := "x"
+		value := ""
+		require.NoError(t, rdb.Set(ctx, key, value, 0).Err())
+		require.Equal(t, value, rdb.Get(ctx, key).Val())
+	})
+
+	t.Run("Very big payload in GET/SET", func(t *testing.T) {
+		key := "x"
+		value := strings.Repeat("abcd", 1000000)
+		require.NoError(t, rdb.Set(ctx, key, value, 0).Err())
+		require.Equal(t, value, rdb.Get(ctx, key).Val())
+	})
+
+	t.Run("Very big payload random access", func(t *testing.T) {
+		var values []string
+		for i := 0; i < 100; i++ {
+			value := util.RandString(1, 100000, util.Alpha)
+			values = append(values, value)
+
+			require.NoError(t, rdb.Set(ctx, "key_"+strconv.Itoa(i), value, 0).Err())
+		}
+
+		for i := 0; i < 1000; i++ {
+			numElements := util.RandomInt(100)
+			key := "key_" + strconv.FormatInt(numElements, 10)
+			value := rdb.Get(ctx, key).Val()
+			require.Equal(t, values[numElements], value)
+		}
+	})
+
+	t.Run("SET 10000 numeric keys and access all them in reverse order", func(t *testing.T) {
+		for i := 0; i < 10000; i++ {
+			key := strconv.Itoa(i)
+			value := key
+			require.NoError(t, rdb.Set(ctx, key, value, 0).Err())
+		}
+
+		for i := 9999; i >= 0; i-- {
+			key := strconv.Itoa(i)
+			value := key
+			require.EqualValues(t, value, rdb.Get(ctx, key).Val())
+		}
+	})
+
+	t.Run("SETNX target key missing", func(t *testing.T) {
+		require.NoError(t, rdb.SetNX(ctx, "foo", "bar", 0).Err())
+		require.Equal(t, "bar", rdb.Get(ctx, "foo").Val())
+	})
+
+	t.Run("SETNX target key exists", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "bar", 0).Err())
+		require.NoError(t, rdb.SetNX(ctx, "foo", "bared", 0).Err())
+		require.Equal(t, "bar", rdb.Get(ctx, "foo").Val())
+	})
+
+	t.Run("SETNX against not-expired volatile key", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "x", "10", 10000*time.Second).Err())
+		require.NoError(t, rdb.SetNX(ctx, "x", "20", 0).Err())
+		require.Equal(t, "10", rdb.Get(ctx, "x").Val())
+	})
+
+	t.Run("SETNX against expired volatile key", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "x", "10", 1*time.Second).Err())
+		require.Eventually(t, func() bool {
+			require.NoError(t, rdb.SetNX(ctx, "x", "20", 0).Err())
+			return rdb.Get(ctx, "x").Val() == "20"
+		}, 5*time.Second, 100*time.Millisecond)
+	})
+
+	t.Run("GETDEL command", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "bar", 0).Err())
+		require.Equal(t, "bar", rdb.GetDel(ctx, "foo").Val())
+		require.Equal(t, "", rdb.GetDel(ctx, "foo").Val())
+	})
+
+	t.Run("MGET command", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "FOO", 0).Err())
+		require.NoError(t, rdb.Set(ctx, "bar", "BAR", 0).Err())
+		require.Equal(t, []interface{}{"FOO", "BAR"}, rdb.MGet(ctx, "foo", "bar").Val())
+	})
+
+	t.Run("MGET against non existing key", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "FOO", 0).Err())
+		require.NoError(t, rdb.Set(ctx, "bar", "BAR", 0).Err())

Review Comment:
   I think the original case is stateful so that you don't re-set these keys. Also, you may keep all `r flushdb` with `require.NoError(t, rdb.FlushDB(ctx).Err())`.



##########
tests/gocase/unit/type/strings/strings_test.go:
##########
@@ -0,0 +1,513 @@
+/*
+* 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 strings
+
+import (
+	"context"
+	"strconv"
+	"strings"
+	"testing"
+	"time"
+
+	"github.com/apache/incubator-kvrocks/tests/gocase/util"
+	"github.com/go-redis/redis/v9"
+	"github.com/stretchr/testify/require"
+)
+
+func TestString(t *testing.T) {
+	srv := util.StartServer(t, map[string]string{})
+	defer srv.Close()
+	ctx := context.Background()
+	rdb := srv.NewClient()
+	defer func() { require.NoError(t, rdb.Close()) }()
+
+	t.Run("SET and GET an item", func(t *testing.T) {
+		key := "x"
+		value := "foo"
+		require.NoError(t, rdb.Set(ctx, key, value, 0).Err())
+		require.Equal(t, value, rdb.Get(ctx, key).Val())
+	})
+
+	t.Run("SET and GET an empty item", func(t *testing.T) {
+		key := "x"
+		value := ""
+		require.NoError(t, rdb.Set(ctx, key, value, 0).Err())
+		require.Equal(t, value, rdb.Get(ctx, key).Val())
+	})
+
+	t.Run("Very big payload in GET/SET", func(t *testing.T) {
+		key := "x"
+		value := strings.Repeat("abcd", 1000000)
+		require.NoError(t, rdb.Set(ctx, key, value, 0).Err())
+		require.Equal(t, value, rdb.Get(ctx, key).Val())
+	})
+
+	t.Run("Very big payload random access", func(t *testing.T) {
+		var values []string
+		for i := 0; i < 100; i++ {
+			value := util.RandString(1, 100000, util.Alpha)
+			values = append(values, value)
+
+			require.NoError(t, rdb.Set(ctx, "key_"+strconv.Itoa(i), value, 0).Err())
+		}
+
+		for i := 0; i < 1000; i++ {
+			numElements := util.RandomInt(100)
+			key := "key_" + strconv.FormatInt(numElements, 10)
+			value := rdb.Get(ctx, key).Val()
+			require.Equal(t, values[numElements], value)
+		}
+	})
+
+	t.Run("SET 10000 numeric keys and access all them in reverse order", func(t *testing.T) {
+		for i := 0; i < 10000; i++ {
+			key := strconv.Itoa(i)
+			value := key
+			require.NoError(t, rdb.Set(ctx, key, value, 0).Err())
+		}
+
+		for i := 9999; i >= 0; i-- {
+			key := strconv.Itoa(i)
+			value := key
+			require.EqualValues(t, value, rdb.Get(ctx, key).Val())
+		}
+	})
+
+	t.Run("SETNX target key missing", func(t *testing.T) {
+		require.NoError(t, rdb.SetNX(ctx, "foo", "bar", 0).Err())
+		require.Equal(t, "bar", rdb.Get(ctx, "foo").Val())
+	})
+
+	t.Run("SETNX target key exists", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "bar", 0).Err())
+		require.NoError(t, rdb.SetNX(ctx, "foo", "bared", 0).Err())
+		require.Equal(t, "bar", rdb.Get(ctx, "foo").Val())
+	})
+
+	t.Run("SETNX against not-expired volatile key", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "x", "10", 10000*time.Second).Err())
+		require.NoError(t, rdb.SetNX(ctx, "x", "20", 0).Err())
+		require.Equal(t, "10", rdb.Get(ctx, "x").Val())
+	})
+
+	t.Run("SETNX against expired volatile key", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "x", "10", 1*time.Second).Err())
+		require.Eventually(t, func() bool {
+			require.NoError(t, rdb.SetNX(ctx, "x", "20", 0).Err())
+			return rdb.Get(ctx, "x").Val() == "20"
+		}, 5*time.Second, 100*time.Millisecond)
+	})

Review Comment:
   This case seems different from the original case. Can you explain a bit?



##########
tests/gocase/unit/type/strings/strings_test.go:
##########
@@ -0,0 +1,513 @@
+/*
+* 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 strings
+
+import (
+	"context"
+	"strconv"
+	"strings"
+	"testing"
+	"time"
+
+	"github.com/apache/incubator-kvrocks/tests/gocase/util"
+	"github.com/go-redis/redis/v9"
+	"github.com/stretchr/testify/require"
+)
+
+func TestString(t *testing.T) {
+	srv := util.StartServer(t, map[string]string{})
+	defer srv.Close()
+	ctx := context.Background()
+	rdb := srv.NewClient()
+	defer func() { require.NoError(t, rdb.Close()) }()
+
+	t.Run("SET and GET an item", func(t *testing.T) {
+		key := "x"
+		value := "foo"
+		require.NoError(t, rdb.Set(ctx, key, value, 0).Err())
+		require.Equal(t, value, rdb.Get(ctx, key).Val())
+	})
+
+	t.Run("SET and GET an empty item", func(t *testing.T) {
+		key := "x"
+		value := ""
+		require.NoError(t, rdb.Set(ctx, key, value, 0).Err())
+		require.Equal(t, value, rdb.Get(ctx, key).Val())
+	})
+
+	t.Run("Very big payload in GET/SET", func(t *testing.T) {
+		key := "x"
+		value := strings.Repeat("abcd", 1000000)
+		require.NoError(t, rdb.Set(ctx, key, value, 0).Err())
+		require.Equal(t, value, rdb.Get(ctx, key).Val())
+	})
+
+	t.Run("Very big payload random access", func(t *testing.T) {
+		var values []string
+		for i := 0; i < 100; i++ {
+			value := util.RandString(1, 100000, util.Alpha)
+			values = append(values, value)
+
+			require.NoError(t, rdb.Set(ctx, "key_"+strconv.Itoa(i), value, 0).Err())
+		}
+
+		for i := 0; i < 1000; i++ {
+			numElements := util.RandomInt(100)
+			key := "key_" + strconv.FormatInt(numElements, 10)
+			value := rdb.Get(ctx, key).Val()
+			require.Equal(t, values[numElements], value)
+		}
+	})
+
+	t.Run("SET 10000 numeric keys and access all them in reverse order", func(t *testing.T) {
+		for i := 0; i < 10000; i++ {
+			key := strconv.Itoa(i)
+			value := key
+			require.NoError(t, rdb.Set(ctx, key, value, 0).Err())
+		}
+
+		for i := 9999; i >= 0; i-- {
+			key := strconv.Itoa(i)
+			value := key
+			require.EqualValues(t, value, rdb.Get(ctx, key).Val())
+		}
+	})
+
+	t.Run("SETNX target key missing", func(t *testing.T) {
+		require.NoError(t, rdb.SetNX(ctx, "foo", "bar", 0).Err())
+		require.Equal(t, "bar", rdb.Get(ctx, "foo").Val())
+	})
+
+	t.Run("SETNX target key exists", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "bar", 0).Err())
+		require.NoError(t, rdb.SetNX(ctx, "foo", "bared", 0).Err())
+		require.Equal(t, "bar", rdb.Get(ctx, "foo").Val())
+	})
+
+	t.Run("SETNX against not-expired volatile key", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "x", "10", 10000*time.Second).Err())
+		require.NoError(t, rdb.SetNX(ctx, "x", "20", 0).Err())
+		require.Equal(t, "10", rdb.Get(ctx, "x").Val())
+	})
+
+	t.Run("SETNX against expired volatile key", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "x", "10", 1*time.Second).Err())
+		require.Eventually(t, func() bool {
+			require.NoError(t, rdb.SetNX(ctx, "x", "20", 0).Err())
+			return rdb.Get(ctx, "x").Val() == "20"
+		}, 5*time.Second, 100*time.Millisecond)
+	})
+
+	t.Run("GETDEL command", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "bar", 0).Err())
+		require.Equal(t, "bar", rdb.GetDel(ctx, "foo").Val())
+		require.Equal(t, "", rdb.GetDel(ctx, "foo").Val())
+	})
+
+	t.Run("MGET command", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "FOO", 0).Err())
+		require.NoError(t, rdb.Set(ctx, "bar", "BAR", 0).Err())
+		require.Equal(t, []interface{}{"FOO", "BAR"}, rdb.MGet(ctx, "foo", "bar").Val())
+	})
+
+	t.Run("MGET against non existing key", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "FOO", 0).Err())
+		require.NoError(t, rdb.Set(ctx, "bar", "BAR", 0).Err())
+		require.Equal(t, []interface{}{"FOO", nil, "BAR"}, rdb.MGet(ctx, "foo", "baazz", "bar").Val())
+	})
+
+	t.Run("MGET against non-string key", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "FOO", 0).Err())
+		require.NoError(t, rdb.Set(ctx, "bar", "BAR", 0).Err())
+		require.NoError(t, rdb.SAdd(ctx, "myset", "ciao", "bau").Err())
+		require.Equal(t, []interface{}{"FOO", nil, "BAR", nil}, rdb.MGet(ctx, "foo", "baazz", "bar", "myset").Val())
+	})
+
+	t.Run("GETSET set new value", func(t *testing.T) {
+		require.NoError(t, rdb.Del(ctx, "foo").Err())
+		oldValue := rdb.GetSet(ctx, "foo", "xyz").Val()
+		newValue := rdb.Get(ctx, "foo").Val()
+		require.Equal(t, "", oldValue)
+		require.Equal(t, "xyz", newValue)
+	})
+
+	t.Run("GETSET replace old value", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "bar", 0).Err())
+		oldValue := rdb.GetSet(ctx, "foo", "xyz").Val()
+		newValue := rdb.Get(ctx, "foo").Val()
+		require.Equal(t, "bar", oldValue)
+		require.Equal(t, "xyz", newValue)
+	})
+
+	t.Run("MSET base case", func(t *testing.T) {
+		require.NoError(t, rdb.MSet(ctx, map[string]interface{}{
+			"x": "10",
+			"y": "foo bar",
+			"z": "x x x x x x x\n\n\r\n",
+		}).Err())
+		require.Equal(t, []interface{}{"10", "foo bar", "x x x x x x x\n\n\r\n"}, rdb.MGet(ctx, "x", "y", "z").Val())
+	})
+
+	t.Run("MSET wrong number of args", func(t *testing.T) {
+		r := rdb.MSet(ctx, "x", "10", "y", "foo bar", "z")
+		require.ErrorContains(t, r.Err(), "wrong number")
+	})
+
+	t.Run("MSETNX with already existent key", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "x", "10", 0).Err())
+		require.EqualValues(t, 0, rdb.Exists(ctx, "x1").Val())
+		require.EqualValues(t, 0, rdb.Exists(ctx, "y2").Val())
+		require.EqualValues(t, 1, rdb.Exists(ctx, "x").Val())
+		r := rdb.MSetNX(ctx, map[string]interface{}{
+			"x1": "xxx",
+			"y2": "yyy",
+			"x":  "20",
+		})
+		require.Equal(t, false, r.Val())

Review Comment:
   ```suggestion
   		require.False(t, r.Val())
   ```



##########
tests/gocase/unit/type/strings/strings_test.go:
##########
@@ -0,0 +1,513 @@
+/*
+* 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 strings
+
+import (
+	"context"
+	"strconv"
+	"strings"
+	"testing"
+	"time"
+
+	"github.com/apache/incubator-kvrocks/tests/gocase/util"
+	"github.com/go-redis/redis/v9"
+	"github.com/stretchr/testify/require"
+)
+
+func TestString(t *testing.T) {
+	srv := util.StartServer(t, map[string]string{})
+	defer srv.Close()
+	ctx := context.Background()
+	rdb := srv.NewClient()
+	defer func() { require.NoError(t, rdb.Close()) }()
+
+	t.Run("SET and GET an item", func(t *testing.T) {
+		key := "x"
+		value := "foo"
+		require.NoError(t, rdb.Set(ctx, key, value, 0).Err())
+		require.Equal(t, value, rdb.Get(ctx, key).Val())
+	})
+
+	t.Run("SET and GET an empty item", func(t *testing.T) {
+		key := "x"
+		value := ""
+		require.NoError(t, rdb.Set(ctx, key, value, 0).Err())
+		require.Equal(t, value, rdb.Get(ctx, key).Val())
+	})
+
+	t.Run("Very big payload in GET/SET", func(t *testing.T) {
+		key := "x"
+		value := strings.Repeat("abcd", 1000000)
+		require.NoError(t, rdb.Set(ctx, key, value, 0).Err())
+		require.Equal(t, value, rdb.Get(ctx, key).Val())
+	})
+
+	t.Run("Very big payload random access", func(t *testing.T) {
+		var values []string
+		for i := 0; i < 100; i++ {
+			value := util.RandString(1, 100000, util.Alpha)
+			values = append(values, value)
+
+			require.NoError(t, rdb.Set(ctx, "key_"+strconv.Itoa(i), value, 0).Err())
+		}
+
+		for i := 0; i < 1000; i++ {
+			numElements := util.RandomInt(100)
+			key := "key_" + strconv.FormatInt(numElements, 10)
+			value := rdb.Get(ctx, key).Val()
+			require.Equal(t, values[numElements], value)
+		}
+	})
+
+	t.Run("SET 10000 numeric keys and access all them in reverse order", func(t *testing.T) {
+		for i := 0; i < 10000; i++ {
+			key := strconv.Itoa(i)
+			value := key
+			require.NoError(t, rdb.Set(ctx, key, value, 0).Err())
+		}
+
+		for i := 9999; i >= 0; i-- {
+			key := strconv.Itoa(i)
+			value := key
+			require.EqualValues(t, value, rdb.Get(ctx, key).Val())
+		}
+	})
+
+	t.Run("SETNX target key missing", func(t *testing.T) {
+		require.NoError(t, rdb.SetNX(ctx, "foo", "bar", 0).Err())
+		require.Equal(t, "bar", rdb.Get(ctx, "foo").Val())
+	})
+
+	t.Run("SETNX target key exists", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "bar", 0).Err())
+		require.NoError(t, rdb.SetNX(ctx, "foo", "bared", 0).Err())
+		require.Equal(t, "bar", rdb.Get(ctx, "foo").Val())
+	})
+
+	t.Run("SETNX against not-expired volatile key", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "x", "10", 10000*time.Second).Err())
+		require.NoError(t, rdb.SetNX(ctx, "x", "20", 0).Err())
+		require.Equal(t, "10", rdb.Get(ctx, "x").Val())
+	})
+
+	t.Run("SETNX against expired volatile key", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "x", "10", 1*time.Second).Err())
+		require.Eventually(t, func() bool {
+			require.NoError(t, rdb.SetNX(ctx, "x", "20", 0).Err())
+			return rdb.Get(ctx, "x").Val() == "20"
+		}, 5*time.Second, 100*time.Millisecond)
+	})
+
+	t.Run("GETDEL command", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "bar", 0).Err())
+		require.Equal(t, "bar", rdb.GetDel(ctx, "foo").Val())
+		require.Equal(t, "", rdb.GetDel(ctx, "foo").Val())
+	})
+
+	t.Run("MGET command", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "FOO", 0).Err())
+		require.NoError(t, rdb.Set(ctx, "bar", "BAR", 0).Err())
+		require.Equal(t, []interface{}{"FOO", "BAR"}, rdb.MGet(ctx, "foo", "bar").Val())
+	})
+
+	t.Run("MGET against non existing key", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "FOO", 0).Err())
+		require.NoError(t, rdb.Set(ctx, "bar", "BAR", 0).Err())
+		require.Equal(t, []interface{}{"FOO", nil, "BAR"}, rdb.MGet(ctx, "foo", "baazz", "bar").Val())
+	})
+
+	t.Run("MGET against non-string key", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "FOO", 0).Err())
+		require.NoError(t, rdb.Set(ctx, "bar", "BAR", 0).Err())

Review Comment:
   ditto



##########
tests/gocase/unit/type/strings/strings_test.go:
##########
@@ -0,0 +1,513 @@
+/*
+* 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 strings
+
+import (
+	"context"
+	"strconv"
+	"strings"
+	"testing"
+	"time"
+
+	"github.com/apache/incubator-kvrocks/tests/gocase/util"
+	"github.com/go-redis/redis/v9"
+	"github.com/stretchr/testify/require"
+)
+
+func TestString(t *testing.T) {
+	srv := util.StartServer(t, map[string]string{})
+	defer srv.Close()
+	ctx := context.Background()
+	rdb := srv.NewClient()
+	defer func() { require.NoError(t, rdb.Close()) }()
+
+	t.Run("SET and GET an item", func(t *testing.T) {
+		key := "x"
+		value := "foo"
+		require.NoError(t, rdb.Set(ctx, key, value, 0).Err())
+		require.Equal(t, value, rdb.Get(ctx, key).Val())
+	})
+
+	t.Run("SET and GET an empty item", func(t *testing.T) {
+		key := "x"
+		value := ""
+		require.NoError(t, rdb.Set(ctx, key, value, 0).Err())
+		require.Equal(t, value, rdb.Get(ctx, key).Val())
+	})
+
+	t.Run("Very big payload in GET/SET", func(t *testing.T) {
+		key := "x"
+		value := strings.Repeat("abcd", 1000000)
+		require.NoError(t, rdb.Set(ctx, key, value, 0).Err())
+		require.Equal(t, value, rdb.Get(ctx, key).Val())
+	})
+
+	t.Run("Very big payload random access", func(t *testing.T) {
+		var values []string
+		for i := 0; i < 100; i++ {
+			value := util.RandString(1, 100000, util.Alpha)
+			values = append(values, value)
+
+			require.NoError(t, rdb.Set(ctx, "key_"+strconv.Itoa(i), value, 0).Err())
+		}
+
+		for i := 0; i < 1000; i++ {
+			numElements := util.RandomInt(100)
+			key := "key_" + strconv.FormatInt(numElements, 10)
+			value := rdb.Get(ctx, key).Val()
+			require.Equal(t, values[numElements], value)
+		}
+	})
+
+	t.Run("SET 10000 numeric keys and access all them in reverse order", func(t *testing.T) {
+		for i := 0; i < 10000; i++ {
+			key := strconv.Itoa(i)
+			value := key
+			require.NoError(t, rdb.Set(ctx, key, value, 0).Err())
+		}
+
+		for i := 9999; i >= 0; i-- {
+			key := strconv.Itoa(i)
+			value := key
+			require.EqualValues(t, value, rdb.Get(ctx, key).Val())
+		}
+	})
+
+	t.Run("SETNX target key missing", func(t *testing.T) {
+		require.NoError(t, rdb.SetNX(ctx, "foo", "bar", 0).Err())
+		require.Equal(t, "bar", rdb.Get(ctx, "foo").Val())
+	})
+
+	t.Run("SETNX target key exists", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "bar", 0).Err())
+		require.NoError(t, rdb.SetNX(ctx, "foo", "bared", 0).Err())
+		require.Equal(t, "bar", rdb.Get(ctx, "foo").Val())
+	})
+
+	t.Run("SETNX against not-expired volatile key", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "x", "10", 10000*time.Second).Err())
+		require.NoError(t, rdb.SetNX(ctx, "x", "20", 0).Err())
+		require.Equal(t, "10", rdb.Get(ctx, "x").Val())
+	})
+
+	t.Run("SETNX against expired volatile key", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "x", "10", 1*time.Second).Err())
+		require.Eventually(t, func() bool {
+			require.NoError(t, rdb.SetNX(ctx, "x", "20", 0).Err())
+			return rdb.Get(ctx, "x").Val() == "20"
+		}, 5*time.Second, 100*time.Millisecond)
+	})
+
+	t.Run("GETDEL command", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "bar", 0).Err())
+		require.Equal(t, "bar", rdb.GetDel(ctx, "foo").Val())
+		require.Equal(t, "", rdb.GetDel(ctx, "foo").Val())
+	})
+
+	t.Run("MGET command", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "FOO", 0).Err())
+		require.NoError(t, rdb.Set(ctx, "bar", "BAR", 0).Err())
+		require.Equal(t, []interface{}{"FOO", "BAR"}, rdb.MGet(ctx, "foo", "bar").Val())
+	})
+
+	t.Run("MGET against non existing key", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "FOO", 0).Err())
+		require.NoError(t, rdb.Set(ctx, "bar", "BAR", 0).Err())
+		require.Equal(t, []interface{}{"FOO", nil, "BAR"}, rdb.MGet(ctx, "foo", "baazz", "bar").Val())
+	})
+
+	t.Run("MGET against non-string key", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "FOO", 0).Err())
+		require.NoError(t, rdb.Set(ctx, "bar", "BAR", 0).Err())
+		require.NoError(t, rdb.SAdd(ctx, "myset", "ciao", "bau").Err())
+		require.Equal(t, []interface{}{"FOO", nil, "BAR", nil}, rdb.MGet(ctx, "foo", "baazz", "bar", "myset").Val())
+	})
+
+	t.Run("GETSET set new value", func(t *testing.T) {
+		require.NoError(t, rdb.Del(ctx, "foo").Err())
+		oldValue := rdb.GetSet(ctx, "foo", "xyz").Val()
+		newValue := rdb.Get(ctx, "foo").Val()
+		require.Equal(t, "", oldValue)
+		require.Equal(t, "xyz", newValue)
+	})
+
+	t.Run("GETSET replace old value", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "bar", 0).Err())
+		oldValue := rdb.GetSet(ctx, "foo", "xyz").Val()
+		newValue := rdb.Get(ctx, "foo").Val()
+		require.Equal(t, "bar", oldValue)
+		require.Equal(t, "xyz", newValue)
+	})
+
+	t.Run("MSET base case", func(t *testing.T) {
+		require.NoError(t, rdb.MSet(ctx, map[string]interface{}{
+			"x": "10",
+			"y": "foo bar",
+			"z": "x x x x x x x\n\n\r\n",
+		}).Err())
+		require.Equal(t, []interface{}{"10", "foo bar", "x x x x x x x\n\n\r\n"}, rdb.MGet(ctx, "x", "y", "z").Val())
+	})
+
+	t.Run("MSET wrong number of args", func(t *testing.T) {
+		r := rdb.MSet(ctx, "x", "10", "y", "foo bar", "z")
+		require.ErrorContains(t, r.Err(), "wrong number")
+	})
+
+	t.Run("MSETNX with already existent key", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "x", "10", 0).Err())
+		require.EqualValues(t, 0, rdb.Exists(ctx, "x1").Val())
+		require.EqualValues(t, 0, rdb.Exists(ctx, "y2").Val())
+		require.EqualValues(t, 1, rdb.Exists(ctx, "x").Val())
+		r := rdb.MSetNX(ctx, map[string]interface{}{
+			"x1": "xxx",
+			"y2": "yyy",
+			"x":  "20",
+		})
+		require.Equal(t, false, r.Val())

Review Comment:
   This test is in the wrong order from the original one.



##########
tests/tcl/tests/unit/type/string.tcl:
##########
@@ -1,742 +0,0 @@
-# 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.
-
-# Copyright (c) 2006-2020, Salvatore Sanfilippo
-# See bundled license file licenses/LICENSE.redis for details.
-
-# This file is copied and modified from the Redis project,
-# which started out as: https://github.com/redis/redis/blob/dbcc0a8/tests/unit/type/string.tcl
-
-start_server {tags {"string"}} {
-    test {SET and GET an item} {
-        r set x foobar
-        r get x
-    } {foobar}
-
-    test {SET and GET an empty item} {
-        r set x {}
-        r get x
-    } {}
-
-    test {Very big payload in GET/SET} {
-        set buf [string repeat "abcd" 1000000]
-        r set foo $buf
-        r get foo
-    } [string repeat "abcd" 1000000]
-
-    tags {"slow"} {
-        test {Very big payload random access} {
-            set err {}
-            array set payload {}
-            for {set j 0} {$j < 100} {incr j} {
-                set size [expr 1+[randomInt 100000]]
-                set buf [string repeat "pl-$j" $size]
-                set payload($j) $buf
-                r set bigpayload_$j $buf
-            }
-            for {set j 0} {$j < 1000} {incr j} {
-                set index [randomInt 100]
-                set buf [r get bigpayload_$index]
-                if {$buf != $payload($index)} {
-                    set err "Values differ: I set '$payload($index)' but I read back '$buf'"
-                    break
-                }
-            }
-            unset payload
-            set _ $err
-        } {}
-
-        test {SET 10000 numeric keys and access all them in reverse order} {
-            r flushdb
-            set err {}
-            for {set x 0} {$x < 10000} {incr x} {
-                r set $x $x
-            }
-            set sum 0
-            for {set x 9999} {$x >= 0} {incr x -1} {
-                set val [r get $x]
-                if {$val ne $x} {
-                    set err "Element at position $x is $val instead of $x"
-                    break
-                }
-            }
-            set _ $err
-        } {}
-
-        # test {DBSIZE should be 10000 now} {
-        #     r dbsize
-        # } {10000}
-    }
-
-    test "SETNX target key missing" {
-        r del novar
-        assert_equal 1 [r setnx novar foobared]
-        assert_equal "foobared" [r get novar]
-    }
-
-    test "SETNX target key exists" {
-        r set novar foobared
-        assert_equal 0 [r setnx novar blabla]
-        assert_equal "foobared" [r get novar]
-    }
-
-    test "SETNX against not-expired volatile key" {
-        r set x 10
-        r expire x 10000
-        assert_equal 0 [r setnx x 20]
-        assert_equal 10 [r get x]
-    }
-
-    test "SETNX against expired volatile key" {
-        # Make it very unlikely for the key this test uses to be expired by the
-        # active expiry cycle. This is tightly coupled to the implementation of
-        # active expiry and dbAdd() but currently the only way to test that
-        # SETNX expires a key when it should have been.
-        for {set x 0} {$x < 9999} {incr x} {
-            r setex key-$x 3600 value
-        }
-
-        # This will be one of 10000 expiring keys. A cycle is executed every
-        # 100ms, sampling 10 keys for being expired or not.  This key will be
-        # expired for at most 1s when we wait 2s, resulting in a total sample
-        # of 100 keys. The probability of the success of this test being a
-        # false positive is therefore approx. 1%.
-        r set x 10
-        r expire x 1
-
-        # Wait for the key to expire
-        after 2000
-
-        assert_equal 1 [r setnx x 20]
-        assert_equal 20 [r get x]
-    }
-
-    # test "GETEX EX option" {

Review Comment:
   @git-hulk @ShooterIT @PragmaTwice do we simply not implement `GETEX`? Perhaps it worth an issue.



##########
tests/gocase/unit/type/strings/strings_test.go:
##########
@@ -0,0 +1,513 @@
+/*
+* 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 strings
+
+import (
+	"context"
+	"strconv"
+	"strings"
+	"testing"
+	"time"
+
+	"github.com/apache/incubator-kvrocks/tests/gocase/util"
+	"github.com/go-redis/redis/v9"
+	"github.com/stretchr/testify/require"
+)
+
+func TestString(t *testing.T) {
+	srv := util.StartServer(t, map[string]string{})
+	defer srv.Close()
+	ctx := context.Background()
+	rdb := srv.NewClient()
+	defer func() { require.NoError(t, rdb.Close()) }()
+
+	t.Run("SET and GET an item", func(t *testing.T) {
+		key := "x"
+		value := "foo"
+		require.NoError(t, rdb.Set(ctx, key, value, 0).Err())
+		require.Equal(t, value, rdb.Get(ctx, key).Val())
+	})
+
+	t.Run("SET and GET an empty item", func(t *testing.T) {
+		key := "x"
+		value := ""
+		require.NoError(t, rdb.Set(ctx, key, value, 0).Err())
+		require.Equal(t, value, rdb.Get(ctx, key).Val())
+	})
+
+	t.Run("Very big payload in GET/SET", func(t *testing.T) {
+		key := "x"
+		value := strings.Repeat("abcd", 1000000)
+		require.NoError(t, rdb.Set(ctx, key, value, 0).Err())
+		require.Equal(t, value, rdb.Get(ctx, key).Val())
+	})
+
+	t.Run("Very big payload random access", func(t *testing.T) {
+		var values []string
+		for i := 0; i < 100; i++ {
+			value := util.RandString(1, 100000, util.Alpha)
+			values = append(values, value)
+
+			require.NoError(t, rdb.Set(ctx, "key_"+strconv.Itoa(i), value, 0).Err())
+		}
+
+		for i := 0; i < 1000; i++ {
+			numElements := util.RandomInt(100)
+			key := "key_" + strconv.FormatInt(numElements, 10)
+			value := rdb.Get(ctx, key).Val()
+			require.Equal(t, values[numElements], value)
+		}
+	})
+
+	t.Run("SET 10000 numeric keys and access all them in reverse order", func(t *testing.T) {
+		for i := 0; i < 10000; i++ {
+			key := strconv.Itoa(i)
+			value := key
+			require.NoError(t, rdb.Set(ctx, key, value, 0).Err())
+		}
+
+		for i := 9999; i >= 0; i-- {
+			key := strconv.Itoa(i)
+			value := key
+			require.EqualValues(t, value, rdb.Get(ctx, key).Val())
+		}
+	})
+
+	t.Run("SETNX target key missing", func(t *testing.T) {
+		require.NoError(t, rdb.SetNX(ctx, "foo", "bar", 0).Err())
+		require.Equal(t, "bar", rdb.Get(ctx, "foo").Val())
+	})
+
+	t.Run("SETNX target key exists", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "bar", 0).Err())
+		require.NoError(t, rdb.SetNX(ctx, "foo", "bared", 0).Err())
+		require.Equal(t, "bar", rdb.Get(ctx, "foo").Val())
+	})
+
+	t.Run("SETNX against not-expired volatile key", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "x", "10", 10000*time.Second).Err())
+		require.NoError(t, rdb.SetNX(ctx, "x", "20", 0).Err())
+		require.Equal(t, "10", rdb.Get(ctx, "x").Val())
+	})
+
+	t.Run("SETNX against expired volatile key", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "x", "10", 1*time.Second).Err())
+		require.Eventually(t, func() bool {
+			require.NoError(t, rdb.SetNX(ctx, "x", "20", 0).Err())
+			return rdb.Get(ctx, "x").Val() == "20"
+		}, 5*time.Second, 100*time.Millisecond)
+	})
+
+	t.Run("GETDEL command", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "bar", 0).Err())
+		require.Equal(t, "bar", rdb.GetDel(ctx, "foo").Val())
+		require.Equal(t, "", rdb.GetDel(ctx, "foo").Val())
+	})
+
+	t.Run("MGET command", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "FOO", 0).Err())
+		require.NoError(t, rdb.Set(ctx, "bar", "BAR", 0).Err())
+		require.Equal(t, []interface{}{"FOO", "BAR"}, rdb.MGet(ctx, "foo", "bar").Val())
+	})
+
+	t.Run("MGET against non existing key", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "FOO", 0).Err())
+		require.NoError(t, rdb.Set(ctx, "bar", "BAR", 0).Err())
+		require.Equal(t, []interface{}{"FOO", nil, "BAR"}, rdb.MGet(ctx, "foo", "baazz", "bar").Val())
+	})
+
+	t.Run("MGET against non-string key", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "FOO", 0).Err())
+		require.NoError(t, rdb.Set(ctx, "bar", "BAR", 0).Err())
+		require.NoError(t, rdb.SAdd(ctx, "myset", "ciao", "bau").Err())
+		require.Equal(t, []interface{}{"FOO", nil, "BAR", nil}, rdb.MGet(ctx, "foo", "baazz", "bar", "myset").Val())
+	})
+
+	t.Run("GETSET set new value", func(t *testing.T) {
+		require.NoError(t, rdb.Del(ctx, "foo").Err())
+		oldValue := rdb.GetSet(ctx, "foo", "xyz").Val()
+		newValue := rdb.Get(ctx, "foo").Val()
+		require.Equal(t, "", oldValue)
+		require.Equal(t, "xyz", newValue)
+	})
+
+	t.Run("GETSET replace old value", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "foo", "bar", 0).Err())
+		oldValue := rdb.GetSet(ctx, "foo", "xyz").Val()
+		newValue := rdb.Get(ctx, "foo").Val()
+		require.Equal(t, "bar", oldValue)
+		require.Equal(t, "xyz", newValue)
+	})
+
+	t.Run("MSET base case", func(t *testing.T) {
+		require.NoError(t, rdb.MSet(ctx, map[string]interface{}{
+			"x": "10",
+			"y": "foo bar",
+			"z": "x x x x x x x\n\n\r\n",
+		}).Err())
+		require.Equal(t, []interface{}{"10", "foo bar", "x x x x x x x\n\n\r\n"}, rdb.MGet(ctx, "x", "y", "z").Val())
+	})
+
+	t.Run("MSET wrong number of args", func(t *testing.T) {
+		r := rdb.MSet(ctx, "x", "10", "y", "foo bar", "z")
+		require.ErrorContains(t, r.Err(), "wrong number")
+	})
+
+	t.Run("MSETNX with already existent key", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "x", "10", 0).Err())
+		require.EqualValues(t, 0, rdb.Exists(ctx, "x1").Val())
+		require.EqualValues(t, 0, rdb.Exists(ctx, "y2").Val())
+		require.EqualValues(t, 1, rdb.Exists(ctx, "x").Val())
+		r := rdb.MSetNX(ctx, map[string]interface{}{
+			"x1": "xxx",
+			"y2": "yyy",
+			"x":  "20",
+		})
+		require.Equal(t, false, r.Val())
+	})
+
+	t.Run("MSETNX with not existing keys", func(t *testing.T) {
+		require.EqualValues(t, 0, rdb.Exists(ctx, "x1").Val())
+		require.EqualValues(t, 0, rdb.Exists(ctx, "y2").Val())
+		r := rdb.MSetNX(ctx, map[string]interface{}{
+			"x1": "xxx",
+			"y2": "yyy",
+		})
+		require.Equal(t, true, r.Val())
+	})

Review Comment:
   ditto



-- 
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: issues-unsubscribe@kvrocks.apache.org

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