You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@pegasus.apache.org by GitBox <gi...@apache.org> on 2022/06/16 07:56:58 UTC

[GitHub] [incubator-pegasus] foreverneverer opened a new pull request, #1006: feat(admin-cli): support migrate table to another cluster using table-duplication

foreverneverer opened a new pull request, #1006:
URL: https://github.com/apache/incubator-pegasus/pull/1006

   # Related-Issue
   https://github.com/apache/incubator-pegasus/issues/962
   
   # Change
   This pr support migrate table online via duplication. the tool will auto-execute some command and check current progress to copy table to another cluster online.
   ## Note
   - you must add config such as follow to open the duplication:
   ```diff
   [replication]
   +  duplication_enabled = true
   [pegasus.clusters]
   + tjsrv-account = 127.0.0.1:51601,127.0.0.1:51601 # the target cluster info
   [duplication-group]
   + tjsrv-account = 1 # current cluster
   + bjsrv-account = 2 # target cluster
   ```
   - In some stage, current server will keep reject-write status for some minute to consume the remaining wal
   - If want to auto-switch the table meta addr, user need deploy `meta-proxy` and client connect table via the `meta-proxy`, otherwise, user need update client config to connect the new cluster
   - Only `replica data version = 1` support the feature, otherwise the tool will retuern error
   
   # Manual-Test
   When you use the tool, the info as follow(`-h` to get the args usage):
   ```bash
   Pegasus-AdminCli-1.2.0 ยป table-migrator -t duptable -c c4tst-function2 -m 127.0.0.1:22601,127.0.0.1:22601 -r /pegasus/c4srv-metaproxy
   INFO: set pendingMutationThreshold = 100000 means that server will reject all write and ready to switch cluster if the pending less the value
   INFO: check data version
   INFO: create table duplication
   successfully add duplication [dupid: 0]
   INFO: check pending mutation count if less 100000
   WARN: node1 has pending_mutations_count 201501
   WARN: node2 has pending_mutations_count 279044
   INFO: all the node pending_mutations_count has less 100000
   INFO: set env config deny write request
   INFO: check duplicate qps if equal 0
   WARN: node1[139.0] still sending pending mutation 970
   WARN:node2[139.0] still sending pending mutation 435
   INFO: all the node has stop duplicate the pending wal and wait 30s to switch cluster
   INFO: switch table addrs in metaproxy
   2022/06/16 15:31:22 connected to zk:11000
   2022/06/16 15:31:22 authenticated: id=178669580305488139, timeout=4000
   2022/06/16 15:31:22 re-submitting `0` credentials after reconnect
   duptable has updated metaproxy addr from &{c4tst-function1 127.0.0.1:12601,127.0.0.1:12601} to {"cluster_name": "c4tst-function2", "meta_addrs": "127.0.0.1:22601,127.0.1:22601"}, current table env is map[replica.deny_client_request:reconfig*all]
   2022/06/16 15:31:22 recv loop terminated: EOF
   2022/06/16 15:31:22 send loop terminated: <nil>
   
   ```


-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] foreverneverer commented on a diff in pull request #1006: feat(admin-cli): support online-migrate table to another cluster using table-duplication

Posted by GitBox <gi...@apache.org>.
foreverneverer commented on code in PR #1006:
URL: https://github.com/apache/incubator-pegasus/pull/1006#discussion_r902144739


##########
admin-cli/executor/server_config.go:
##########
@@ -20,26 +20,20 @@
 package executor

Review Comment:
   I use `batchHTTP` api in this patch, so I remove the api into `utils` package



##########
admin-cli/executor/table_version.go:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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 executor
+
+import (
+	"encoding/json"
+	"fmt"
+	"strconv"
+
+	"github.com/apache/incubator-pegasus/admin-cli/util"
+	"github.com/apache/incubator-pegasus/go-client/session"
+)
+
+type TableDataVersion struct {
+	DataVersion string `json:"data_version"`

Review Comment:
   as above



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] foreverneverer merged pull request #1006: feat(admin-cli): support online-migrate table to another cluster using table-duplication

Posted by GitBox <gi...@apache.org>.
foreverneverer merged PR #1006:
URL: https://github.com/apache/incubator-pegasus/pull/1006


-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] foreverneverer commented on a diff in pull request #1006: feat(admin-cli): support online-migrate table to another cluster using table-duplication

Posted by GitBox <gi...@apache.org>.
foreverneverer commented on code in PR #1006:
URL: https://github.com/apache/incubator-pegasus/pull/1006#discussion_r902139535


##########
admin-cli/cmd/table_migrator.go:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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 cmd
+
+import (
+	"github.com/apache/incubator-pegasus/admin-cli/executor/toolkits/tablemigrator"
+	"github.com/apache/incubator-pegasus/admin-cli/shell"
+	"github.com/desertbit/grumble"
+)
+
+func init() {
+	shell.AddCommand(&grumble.Command{
+		Name: "table-migrator",
+		Help: "migrate table from current cluster to another via table duplication and metaproxy",

Review Comment:
   migrate table actually contain two step:
   1. server side copy table from current to remote
   2. client switch connection with current cluster to remote
   
   if user access via metaproxy, server side can auto-control client's target cluster addr and client no need update config 
    and restart service.
   
   But using metaproxy and auto-switch is optional, user can just complete `step 1` and manual-switch table addr.
   
   The tool has little complex,  I will add `readme` to explain its usage



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] acelyc111 commented on a diff in pull request #1006: feat(admin-cli): support online-migrate table to another cluster using table-duplication

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on code in PR #1006:
URL: https://github.com/apache/incubator-pegasus/pull/1006#discussion_r902244757


##########
admin-cli/executor/toolkits/tablemigrator/migrator.go:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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 tablemigrator
+
+import (
+	"fmt"
+	"time"
+
+	"github.com/apache/incubator-pegasus/admin-cli/executor"
+	"github.com/apache/incubator-pegasus/admin-cli/executor/toolkits"
+	"github.com/apache/incubator-pegasus/admin-cli/util"
+	"github.com/apache/incubator-pegasus/go-client/session"
+	"github.com/pegasus-kv/collector/aggregate"
+)
+
+var pendingMutationThreshold = 100000.0
+
+func MigrateTable(client *executor.Client, table string, metaProxyZkAddrs string, metaProxyZkRoot string, targetCluster string, targetAddrs string, threshold float64) error {
+	pendingMutationThreshold = threshold
+	toolkits.LogInfo(fmt.Sprintf("set pendingMutationThreshold = %d means that server will reject all write "+
+		"and ready to switch cluster if the pending less the value", int64(pendingMutationThreshold)))
+	//1. check data version
+	toolkits.LogInfo("check the table data version")
+	version, err := executor.QueryReplicaDataVersion(client, table)
+	if err != nil {
+		return err
+	}
+	if version.DataVersion != "1" {
+		return fmt.Errorf("not support migrate table with data_version = %s by duplication", version.DataVersion)
+	}
+
+	//2. create table duplication
+	toolkits.LogInfo(fmt.Sprintf("create the table duplication to %s", targetCluster))
+	err = executor.AddDuplication(client, table, targetCluster, true)
+	if err != nil {
+		return err
+	}
+
+	//3. check pending mutation count if less `pendingMutationThreshold`
+	toolkits.LogInfo(fmt.Sprintf("check pending mutation count if less %d", int64(pendingMutationThreshold)))
+	nodes := client.Nodes.GetAllNodes(session.NodeTypeReplica)
+	perfSessions := make(map[string]*aggregate.PerfSession)
+	for _, n := range nodes {
+		perf := client.Nodes.GetPerfSession(n.TCPAddr(), session.NodeTypeReplica)
+		perfSessions[n.CombinedAddr()] = perf
+	}
+	err = checkPendingMutationCount(perfSessions)
+	if err != nil {
+		return err
+	}
+	//4. set env config deny write request
+	toolkits.LogInfo("set env config deny write request")
+	var envs = map[string]string{
+		"replica.deny_client_request": "timeout*write",
+	}
+	err = client.Meta.UpdateAppEnvs(table, envs)

Review Comment:
   I didn't see where to revert this config, is it needed?



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] foreverneverer commented on a diff in pull request #1006: feat(admin-cli): support online-migrate table to another cluster using table-duplication

Posted by GitBox <gi...@apache.org>.
foreverneverer commented on code in PR #1006:
URL: https://github.com/apache/incubator-pegasus/pull/1006#discussion_r902144363


##########
admin-cli/cmd/table_version.go:
##########
@@ -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 cmd
+
+import (
+	"github.com/apache/incubator-pegasus/admin-cli/executor"
+	"github.com/apache/incubator-pegasus/admin-cli/shell"
+	"github.com/desertbit/grumble"
+)
+
+func init() {
+	shell.AddCommand(&grumble.Command{
+		Name: "data-version",

Review Comment:
   pegasus replica data contain dataV1 and dataV2. Since duplication feature if after `dataV2` feature. so when creating `duplication table` in remote, the new table version just is `dataV2`. so only dataV2 support duplication 



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] foreverneverer commented on a diff in pull request #1006: feat(admin-cli): support online-migrate table to another cluster using table-duplication

Posted by GitBox <gi...@apache.org>.
foreverneverer commented on code in PR #1006:
URL: https://github.com/apache/incubator-pegasus/pull/1006#discussion_r902369624


##########
admin-cli/executor/toolkits/tablemigrator/migrator.go:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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 tablemigrator
+
+import (
+	"fmt"
+	"time"
+
+	"github.com/apache/incubator-pegasus/admin-cli/executor"
+	"github.com/apache/incubator-pegasus/admin-cli/executor/toolkits"
+	"github.com/apache/incubator-pegasus/admin-cli/util"
+	"github.com/apache/incubator-pegasus/go-client/session"
+	"github.com/pegasus-kv/collector/aggregate"
+)
+
+var pendingMutationThreshold = 100000.0
+
+func MigrateTable(client *executor.Client, table string, metaProxyZkAddrs string, metaProxyZkRoot string, targetCluster string, targetAddrs string, threshold float64) error {
+	pendingMutationThreshold = threshold
+	toolkits.LogInfo(fmt.Sprintf("set pendingMutationThreshold = %d means that server will reject all write "+
+		"and ready to switch cluster if the pending less the value", int64(pendingMutationThreshold)))
+	//1. check data version
+	toolkits.LogInfo("check the table data version")
+	version, err := executor.QueryReplicaDataVersion(client, table)
+	if err != nil {
+		return err
+	}
+	if version.DataVersion != "1" {
+		return fmt.Errorf("not support migrate table with data_version = %s by duplication", version.DataVersion)
+	}
+
+	//2. create table duplication
+	toolkits.LogInfo(fmt.Sprintf("create the table duplication to %s", targetCluster))
+	err = executor.AddDuplication(client, table, targetCluster, true)
+	if err != nil {
+		return err
+	}
+
+	//3. check pending mutation count if less `pendingMutationThreshold`
+	toolkits.LogInfo(fmt.Sprintf("check pending mutation count if less %d", int64(pendingMutationThreshold)))
+	nodes := client.Nodes.GetAllNodes(session.NodeTypeReplica)
+	perfSessions := make(map[string]*aggregate.PerfSession)
+	for _, n := range nodes {
+		perf := client.Nodes.GetPerfSession(n.TCPAddr(), session.NodeTypeReplica)
+		perfSessions[n.CombinedAddr()] = perf
+	}
+	err = checkPendingMutationCount(perfSessions)
+	if err != nil {
+		return err
+	}
+	//4. set env config deny write request
+	toolkits.LogInfo("set env config deny write request")
+	var envs = map[string]string{
+		"replica.deny_client_request": "timeout*write",
+	}
+	err = client.Meta.UpdateAppEnvs(table, envs)

Review Comment:
   Yeah, origin table `write request`will be prohibited and no need revert



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] acelyc111 commented on a diff in pull request #1006: feat(admin-cli): support online-migrate table to another cluster using table-duplication

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on code in PR #1006:
URL: https://github.com/apache/incubator-pegasus/pull/1006#discussion_r902129328


##########
admin-cli/cmd/table_migrator.go:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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 cmd
+
+import (
+	"github.com/apache/incubator-pegasus/admin-cli/executor/toolkits/tablemigrator"
+	"github.com/apache/incubator-pegasus/admin-cli/shell"
+	"github.com/desertbit/grumble"
+)
+
+func init() {
+	shell.AddCommand(&grumble.Command{
+		Name: "table-migrator",
+		Help: "migrate table from current cluster to another via table duplication and metaproxy",
+		Flags: func(f *grumble.Flags) {
+			f.String("t", "table", "", "table name")
+			f.String("n", "node", "", "zk node, addrs:port, default equal with peagsus "+
+				"cluster zk addrs, you can use `cluster-info` to show it")
+			f.String("r", "root", "", "zk root path. the tool will update table addrs in "+
+				"the path of meatproxy, if you don't specify it, that is means user need manual-switch the table addrs")
+			f.String("c", "cluster", "", "target cluster name")
+			f.String("m", "meta", "", "target meta list")
+			f.Float64("p", "threshold", 100000, "pending mutation throshold when server will reject all write request")

Review Comment:
   ```suggestion
   			f.Float64("p", "threshold", 100000, "pending mutation threshold when server will reject all write request")
   ```
   ```suggestion
   			f.Float64("p", "threshold", 100000, "pending mutation throshold when server will reject all write request")
   ```
   From the description, I still don't know what's it mean, and what's it effect. Could you plz give more description about it?



##########
admin-cli/cmd/table_migrator.go:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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 cmd
+
+import (
+	"github.com/apache/incubator-pegasus/admin-cli/executor/toolkits/tablemigrator"
+	"github.com/apache/incubator-pegasus/admin-cli/shell"
+	"github.com/desertbit/grumble"
+)
+
+func init() {
+	shell.AddCommand(&grumble.Command{
+		Name: "table-migrator",
+		Help: "migrate table from current cluster to another via table duplication and metaproxy",

Review Comment:
   Is the migrate tool bind to metaproxy? AFAIK, metaproxy is used for unify table access, how can it effect the migrate tool?



##########
admin-cli/cmd/table_version.go:
##########
@@ -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 cmd
+
+import (
+	"github.com/apache/incubator-pegasus/admin-cli/executor"
+	"github.com/apache/incubator-pegasus/admin-cli/shell"
+	"github.com/desertbit/grumble"
+)
+
+func init() {
+	shell.AddCommand(&grumble.Command{
+		Name: "data-version",

Review Comment:
   what's data version?



##########
admin-cli/executor/server_config.go:
##########
@@ -20,26 +20,20 @@
 package executor

Review Comment:
   Is this file modification related to this patch?



##########
admin-cli/cmd/table_migrator.go:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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 cmd
+
+import (
+	"github.com/apache/incubator-pegasus/admin-cli/executor/toolkits/tablemigrator"
+	"github.com/apache/incubator-pegasus/admin-cli/shell"
+	"github.com/desertbit/grumble"
+)
+
+func init() {
+	shell.AddCommand(&grumble.Command{
+		Name: "table-migrator",
+		Help: "migrate table from current cluster to another via table duplication and metaproxy",
+		Flags: func(f *grumble.Flags) {
+			f.String("t", "table", "", "table name")
+			f.String("n", "node", "", "zk node, addrs:port, default equal with peagsus "+
+				"cluster zk addrs, you can use `cluster-info` to show it")
+			f.String("r", "root", "", "zk root path. the tool will update table addrs in "+
+				"the path of meatproxy, if you don't specify it, that is means user need manual-switch the table addrs")
+			f.String("c", "cluster", "", "target cluster name")

Review Comment:
   is 'cluster name' used for duplication? Could you describe how to define and use it?



##########
admin-cli/executor/table_version.go:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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 executor
+
+import (
+	"encoding/json"
+	"fmt"
+	"strconv"
+
+	"github.com/apache/incubator-pegasus/admin-cli/util"
+	"github.com/apache/incubator-pegasus/go-client/session"
+)
+
+type TableDataVersion struct {
+	DataVersion string `json:"data_version"`

Review Comment:
   I think it would be necessary to describe what is data vesion firstly. :)



##########
admin-cli/util/http_client.go:
##########
@@ -0,0 +1,78 @@
+/*

Review Comment:
   Seems not highly related to this patch, would it be better to create another pr for it?



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] foreverneverer commented on a diff in pull request #1006: feat(admin-cli): support online-migrate table to another cluster using table-duplication

Posted by GitBox <gi...@apache.org>.
foreverneverer commented on code in PR #1006:
URL: https://github.com/apache/incubator-pegasus/pull/1006#discussion_r902142297


##########
admin-cli/cmd/table_migrator.go:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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 cmd
+
+import (
+	"github.com/apache/incubator-pegasus/admin-cli/executor/toolkits/tablemigrator"
+	"github.com/apache/incubator-pegasus/admin-cli/shell"
+	"github.com/desertbit/grumble"
+)
+
+func init() {
+	shell.AddCommand(&grumble.Command{
+		Name: "table-migrator",
+		Help: "migrate table from current cluster to another via table duplication and metaproxy",
+		Flags: func(f *grumble.Flags) {
+			f.String("t", "table", "", "table name")
+			f.String("n", "node", "", "zk node, addrs:port, default equal with peagsus "+
+				"cluster zk addrs, you can use `cluster-info` to show it")
+			f.String("r", "root", "", "zk root path. the tool will update table addrs in "+
+				"the path of meatproxy, if you don't specify it, that is means user need manual-switch the table addrs")
+			f.String("c", "cluster", "", "target cluster name")
+			f.String("m", "meta", "", "target meta list")
+			f.Float64("p", "threshold", 100000, "pending mutation throshold when server will reject all write request")

Review Comment:
   In the migration progess, the origin cluster must stop writing for a period of time (at the minute level) to synchronize all incremental data before the client can switch to the new cluster. This threshold is used to control the opportunity of stopping writing, that is to say, 100k pending incremental data are found, the writing of the client will be stopped.
   
   as above, I will add `readme`



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] foreverneverer commented on a diff in pull request #1006: feat(admin-cli): support online-migrate table to another cluster using table-duplication

Posted by GitBox <gi...@apache.org>.
foreverneverer commented on code in PR #1006:
URL: https://github.com/apache/incubator-pegasus/pull/1006#discussion_r902142297


##########
admin-cli/cmd/table_migrator.go:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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 cmd
+
+import (
+	"github.com/apache/incubator-pegasus/admin-cli/executor/toolkits/tablemigrator"
+	"github.com/apache/incubator-pegasus/admin-cli/shell"
+	"github.com/desertbit/grumble"
+)
+
+func init() {
+	shell.AddCommand(&grumble.Command{
+		Name: "table-migrator",
+		Help: "migrate table from current cluster to another via table duplication and metaproxy",
+		Flags: func(f *grumble.Flags) {
+			f.String("t", "table", "", "table name")
+			f.String("n", "node", "", "zk node, addrs:port, default equal with peagsus "+
+				"cluster zk addrs, you can use `cluster-info` to show it")
+			f.String("r", "root", "", "zk root path. the tool will update table addrs in "+
+				"the path of meatproxy, if you don't specify it, that is means user need manual-switch the table addrs")
+			f.String("c", "cluster", "", "target cluster name")
+			f.String("m", "meta", "", "target meta list")
+			f.Float64("p", "threshold", 100000, "pending mutation throshold when server will reject all write request")

Review Comment:
   In the migration progess, the origin cluster must stop writing for a period of time (at the minute level) to synchronize all incremental data before the client can switch to the new cluster. This threshold is used to control the timing of stopping writing, that is, 100k pending incremental data are found, the writing of the client will be stopped.
   
   as above, I will add `readme`



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] foreverneverer commented on a diff in pull request #1006: feat(admin-cli): support online-migrate table to another cluster using table-duplication

Posted by GitBox <gi...@apache.org>.
foreverneverer commented on code in PR #1006:
URL: https://github.com/apache/incubator-pegasus/pull/1006#discussion_r902139535


##########
admin-cli/cmd/table_migrator.go:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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 cmd
+
+import (
+	"github.com/apache/incubator-pegasus/admin-cli/executor/toolkits/tablemigrator"
+	"github.com/apache/incubator-pegasus/admin-cli/shell"
+	"github.com/desertbit/grumble"
+)
+
+func init() {
+	shell.AddCommand(&grumble.Command{
+		Name: "table-migrator",
+		Help: "migrate table from current cluster to another via table duplication and metaproxy",

Review Comment:
   migrate table actually contain two step:
   1. server side copy table from current to remote
   2. client switch connection with current cluster to remote
   
   if user access via metaproxy, server side can auto-control client's target cluster addr and client no need update config  and restart service.
   
   But using metaproxy and auto-switch is optional, user can just complete `step 1` and manual-switch table addr.
   
   The tool has little complex,  I will add `readme` to explain its usage



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] foreverneverer commented on a diff in pull request #1006: feat(admin-cli): support online-migrate table to another cluster using table-duplication

Posted by GitBox <gi...@apache.org>.
foreverneverer commented on code in PR #1006:
URL: https://github.com/apache/incubator-pegasus/pull/1006#discussion_r902143153


##########
admin-cli/cmd/table_migrator.go:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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 cmd
+
+import (
+	"github.com/apache/incubator-pegasus/admin-cli/executor/toolkits/tablemigrator"
+	"github.com/apache/incubator-pegasus/admin-cli/shell"
+	"github.com/desertbit/grumble"
+)
+
+func init() {
+	shell.AddCommand(&grumble.Command{
+		Name: "table-migrator",
+		Help: "migrate table from current cluster to another via table duplication and metaproxy",
+		Flags: func(f *grumble.Flags) {
+			f.String("t", "table", "", "table name")
+			f.String("n", "node", "", "zk node, addrs:port, default equal with peagsus "+
+				"cluster zk addrs, you can use `cluster-info` to show it")
+			f.String("r", "root", "", "zk root path. the tool will update table addrs in "+
+				"the path of meatproxy, if you don't specify it, that is means user need manual-switch the table addrs")
+			f.String("c", "cluster", "", "target cluster name")

Review Comment:
   Yeah, that is `duplication`  arguments



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] foreverneverer commented on a diff in pull request #1006: feat(admin-cli): support online-migrate table to another cluster using table-duplication

Posted by GitBox <gi...@apache.org>.
foreverneverer commented on code in PR #1006:
URL: https://github.com/apache/incubator-pegasus/pull/1006#discussion_r902144936


##########
admin-cli/util/http_client.go:
##########
@@ -0,0 +1,78 @@
+/*

Review Comment:
   https://github.com/apache/incubator-pegasus/pull/1006#discussion_r902144739



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org