You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by GitBox <gi...@apache.org> on 2020/02/11 17:52:15 UTC

[GitHub] [hive] aasha opened a new pull request #900: HIVE-22860 Support metadata only replication for external tables

aasha opened a new pull request #900: HIVE-22860 Support metadata only replication for external tables
URL: https://github.com/apache/hive/pull/900
 
 
   

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] maheshk114 commented on a change in pull request #900: HIVE-22860 Support metadata only replication for external tables

Posted by GitBox <gi...@apache.org>.
maheshk114 commented on a change in pull request #900: HIVE-22860 Support metadata only replication for external tables
URL: https://github.com/apache/hive/pull/900#discussion_r379995894
 
 

 ##########
 File path: itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestMetadataReplicationScenariosExternalTables.java
 ##########
 @@ -0,0 +1,647 @@
+/*
+ * 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.hadoop.hive.ql.parse;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.InjectableBehaviourObjectStore;
+import org.apache.hadoop.hive.metastore.InjectableBehaviourObjectStore.BehaviourInjection;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.messaging.json.gzip.GzipJSONMessageEncoder;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hive.metastore.ReplChangeManager.SOURCE_OF_REPLICATION;
+import static org.apache.hadoop.hive.ql.exec.repl.ReplExternalTables.FILE_NAME;
+import static org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils.INC_BOOTSTRAP_ROOT_DIR_NAME;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * TestMetadataReplicationScenariosExternalTables - Test metadata only replication.
+ * for external tables.
+ */
+public class TestMetadataReplicationScenariosExternalTables extends BaseReplicationAcrossInstances {
+
+  private static final String REPLICA_EXTERNAL_BASE = "/replica_external_base";
+  String extraPrimaryDb;
+
+  @BeforeClass
+  public static void classLevelSetup() throws Exception {
+    Map<String, String> overrides = new HashMap<>();
+    overrides.put(MetastoreConf.ConfVars.EVENT_MESSAGE_FACTORY.getHiveName(),
+        GzipJSONMessageEncoder.class.getCanonicalName());
+    overrides.put(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY.varname, "false");
+    overrides.put(HiveConf.ConfVars.REPL_INCLUDE_EXTERNAL_TABLES.varname, "true");
+    overrides.put(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY_FOR_EXTERNAL_TABLE.varname, "true");
+    overrides.put(HiveConf.ConfVars.HIVE_DISTCP_DOAS_USER.varname,
+        UserGroupInformation.getCurrentUser().getUserName());
+
+    internalBeforeClassSetup(overrides, TestReplicationScenarios.class);
+  }
+
+  @Before
+  public void setup() throws Throwable {
+    super.setup();
+    extraPrimaryDb = "extra_" + primaryDbName;
+  }
+
+  @After
+  public void tearDown() throws Throwable {
+    primary.run("drop database if exists " + extraPrimaryDb + " cascade");
+    super.tearDown();
+  }
+
+  @Test
+  public void replicationWithoutExternalTables() throws Throwable {
+    List<String> loadWithClause = externalTableBasePathWithClause();
+    List<String> dumpWithClause
+            = Arrays.asList("'" + HiveConf.ConfVars.REPL_INCLUDE_EXTERNAL_TABLES.varname + "'='false'");
+
+
+
+    WarehouseInstance.Tuple tuple = primary
+        .run("use " + primaryDbName)
+        .run("create external table t1 (id int)")
+        .run("insert into table t1 values (1)")
+        .run("insert into table t1 values (2)")
+        .run("create external table t2 (place string) partitioned by (country string)")
+        .run("insert into table t2 partition(country='india') values ('bangalore')")
+        .run("insert into table t2 partition(country='us') values ('austin')")
+        .run("insert into table t2 partition(country='france') values ('paris')")
+        .dump(primaryDbName, null, dumpWithClause);
+
+    // the _external_tables_file info only should be created if external tables are to be replicated not otherwise
+    assertFalse(primary.miniDFSCluster.getFileSystem()
+        .exists(new Path(new Path(tuple.dumpLocation, primaryDbName.toLowerCase()), FILE_NAME)));
+
+    replica.load(replicatedDbName, tuple.dumpLocation, loadWithClause)
+        .run("repl status " + replicatedDbName)
+        .verifyResult(tuple.lastReplicationId)
+        .run("use " + replicatedDbName)
+        .run("show tables like 't1'")
+        .verifyFailure(new String[] {"t1"})
+        .run("show tables like 't2'")
+        .verifyFailure(new String[] {"t2"})
+        .verifyReplTargetProperty(replicatedDbName);
+
+    tuple = primary.run("use " + primaryDbName)
+        .run("create external table t3 (id int)")
+        .run("insert into table t3 values (10)")
+        .run("insert into table t3 values (20)")
+        .dump(primaryDbName, tuple.lastReplicationId, dumpWithClause);
+
+    // the _external_tables_file info only should be created if external tables are to be replicated not otherwise
 
 Review comment:
   external tables "data" are to be replicated

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] maheshk114 commented on a change in pull request #900: HIVE-22860 Support metadata only replication for external tables

Posted by GitBox <gi...@apache.org>.
maheshk114 commented on a change in pull request #900: HIVE-22860 Support metadata only replication for external tables
URL: https://github.com/apache/hive/pull/900#discussion_r378093845
 
 

 ##########
 File path: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
 ##########
 @@ -478,6 +478,10 @@ private static void populateLlapDaemonVarsSet(Set<String> llapDaemonVarsSetLocal
     REPL_DUMP_METADATA_ONLY("hive.repl.dump.metadata.only", false,
         "Indicates whether replication dump only metadata information or data + metadata. \n"
           + "This config makes hive.repl.include.external.tables config ineffective."),
+    REPL_DUMP_METADATA_ONLY_FOR_EXTERNAL_TABLE("hive.repl.dump.metadata.only.for.external.table",
+            false,
+            "Indicates whether replication dump only metadata information or data + metadata. \n"
 
 Review comment:
   why in-effective ..what should be the behavior if this is set to false (default) and hive.repl.dump.metadata.only is set to true?

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] maheshk114 commented on a change in pull request #900: HIVE-22860 Support metadata only replication for external tables

Posted by GitBox <gi...@apache.org>.
maheshk114 commented on a change in pull request #900: HIVE-22860 Support metadata only replication for external tables
URL: https://github.com/apache/hive/pull/900#discussion_r378094418
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
 ##########
 @@ -172,7 +172,8 @@ private boolean shouldExamineTablesToDump() {
    */
   private boolean shouldDumpExternalTableLocation() {
     return conf.getBoolVar(HiveConf.ConfVars.REPL_INCLUDE_EXTERNAL_TABLES)
-            && !conf.getBoolVar(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY);
+            && (!conf.getBoolVar(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY) &&
 
 Review comment:
   what if user wants only external table data + metadata  ?

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] maheshk114 commented on a change in pull request #900: HIVE-22860 Support metadata only replication for external tables

Posted by GitBox <gi...@apache.org>.
maheshk114 commented on a change in pull request #900: HIVE-22860 Support metadata only replication for external tables
URL: https://github.com/apache/hive/pull/900#discussion_r378100685
 
 

 ##########
 File path: itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestMetadataReplicationScenariosExternalTables.java
 ##########
 @@ -0,0 +1,807 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
 
 Review comment:
   All these are new test cases ? some are not related to this feature i think 

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] maheshk114 commented on a change in pull request #900: HIVE-22860 Support metadata only replication for external tables

Posted by GitBox <gi...@apache.org>.
maheshk114 commented on a change in pull request #900: HIVE-22860 Support metadata only replication for external tables
URL: https://github.com/apache/hive/pull/900#discussion_r379991688
 
 

 ##########
 File path: itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestMetadataReplicationScenariosExternalTables.java
 ##########
 @@ -0,0 +1,647 @@
+/*
+ * 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.hadoop.hive.ql.parse;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.InjectableBehaviourObjectStore;
+import org.apache.hadoop.hive.metastore.InjectableBehaviourObjectStore.BehaviourInjection;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.messaging.json.gzip.GzipJSONMessageEncoder;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hive.metastore.ReplChangeManager.SOURCE_OF_REPLICATION;
+import static org.apache.hadoop.hive.ql.exec.repl.ReplExternalTables.FILE_NAME;
+import static org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils.INC_BOOTSTRAP_ROOT_DIR_NAME;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * TestMetadataReplicationScenariosExternalTables - Test metadata only replication.
+ * for external tables.
+ */
+public class TestMetadataReplicationScenariosExternalTables extends BaseReplicationAcrossInstances {
+
+  private static final String REPLICA_EXTERNAL_BASE = "/replica_external_base";
+  String extraPrimaryDb;
+
+  @BeforeClass
+  public static void classLevelSetup() throws Exception {
+    Map<String, String> overrides = new HashMap<>();
+    overrides.put(MetastoreConf.ConfVars.EVENT_MESSAGE_FACTORY.getHiveName(),
+        GzipJSONMessageEncoder.class.getCanonicalName());
+    overrides.put(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY.varname, "false");
+    overrides.put(HiveConf.ConfVars.REPL_INCLUDE_EXTERNAL_TABLES.varname, "true");
+    overrides.put(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY_FOR_EXTERNAL_TABLE.varname, "true");
+    overrides.put(HiveConf.ConfVars.HIVE_DISTCP_DOAS_USER.varname,
+        UserGroupInformation.getCurrentUser().getUserName());
+
+    internalBeforeClassSetup(overrides, TestReplicationScenarios.class);
+  }
+
+  @Before
+  public void setup() throws Throwable {
+    super.setup();
+    extraPrimaryDb = "extra_" + primaryDbName;
+  }
+
+  @After
+  public void tearDown() throws Throwable {
+    primary.run("drop database if exists " + extraPrimaryDb + " cascade");
+    super.tearDown();
+  }
+
+  @Test
+  public void replicationWithoutExternalTables() throws Throwable {
+    List<String> loadWithClause = externalTableBasePathWithClause();
+    List<String> dumpWithClause
+            = Arrays.asList("'" + HiveConf.ConfVars.REPL_INCLUDE_EXTERNAL_TABLES.varname + "'='false'");
+
+
 
 Review comment:
   remove extra spaces 

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] maheshk114 commented on a change in pull request #900: HIVE-22860 Support metadata only replication for external tables

Posted by GitBox <gi...@apache.org>.
maheshk114 commented on a change in pull request #900: HIVE-22860 Support metadata only replication for external tables
URL: https://github.com/apache/hive/pull/900#discussion_r379991400
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
 ##########
 @@ -493,7 +494,8 @@ Long bootStrapDump(Path dumpRoot, DumpMetaData dmd, Path cmRoot, Hive hiveDb)
       Exception caught = null;
       boolean shouldWriteExternalTableLocationInfo =
               conf.getBoolVar(HiveConf.ConfVars.REPL_INCLUDE_EXTERNAL_TABLES)
-                      && !conf.getBoolVar(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY);
+                      && (!conf.getBoolVar(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY) &&
 
 Review comment:
   why not use shouldDumpExternalTableLocation to check ?

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] aasha commented on a change in pull request #900: HIVE-22860 Support metadata only replication for external tables

Posted by GitBox <gi...@apache.org>.
aasha commented on a change in pull request #900: HIVE-22860 Support metadata only replication for external tables
URL: https://github.com/apache/hive/pull/900#discussion_r378408986
 
 

 ##########
 File path: itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestMetadataReplicationScenariosExternalTables.java
 ##########
 @@ -0,0 +1,807 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
 
 Review comment:
   Its existing cases, should n't be any regression because of the flag. Thats why added 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] aasha commented on a change in pull request #900: HIVE-22860 Support metadata only replication for external tables

Posted by GitBox <gi...@apache.org>.
aasha commented on a change in pull request #900: HIVE-22860 Support metadata only replication for external tables
URL: https://github.com/apache/hive/pull/900#discussion_r378400562
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
 ##########
 @@ -172,7 +172,8 @@ private boolean shouldExamineTablesToDump() {
    */
   private boolean shouldDumpExternalTableLocation() {
     return conf.getBoolVar(HiveConf.ConfVars.REPL_INCLUDE_EXTERNAL_TABLES)
-            && !conf.getBoolVar(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY);
+            && (!conf.getBoolVar(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY) &&
 
 Review comment:
   Then they will set REPL_INCLUDE_EXTERNAL_TABLES to true, REPL_DUMP_METADATA_ONLY and REPL_DUMP_METADATA_ONLY_FOR_EXTERNAL_TABLE to false. 
   If they set REPL_DUMP_METADATA_ONLY_FOR_EXTERNAL_TABLE to true and REPL_DUMP_METADATA_ONLY to false, then managed table data + metadata will be dumped, but external table metadata only.
   If they set REPL_DUMP_METADATA_ONLY_FOR_EXTERNAL_TABLE to false but REPL_DUMP_METADATA_ONLY to true then only metadata will be dumped for both external and managed tables

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] maheshk114 commented on a change in pull request #900: HIVE-22860 Support metadata only replication for external tables

Posted by GitBox <gi...@apache.org>.
maheshk114 commented on a change in pull request #900: HIVE-22860 Support metadata only replication for external tables
URL: https://github.com/apache/hive/pull/900#discussion_r378096951
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AlterTableHandler.java
 ##########
 @@ -229,7 +230,9 @@ public void handle(Context withinContext) throws Exception {
       // If we are not dumping metadata about a table, we shouldn't be dumping basic statistics
       // as well, since that won't be accurate. So reset them to what they would look like for an
       // empty table.
-      if (withinContext.hiveConf.getBoolVar(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY)) {
+      if (withinContext.hiveConf.getBoolVar(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY) ||
+              (qlMdTableAfter.getTableType().equals(TableType.EXTERNAL_TABLE)
+              && withinContext.hiveConf.getBoolVar(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY_FOR_EXTERNAL_TABLE))) {
 
 Review comment:
   This check can be put as an util method ..taking table object/type and config object

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] maheshk114 commented on a change in pull request #900: HIVE-22860 Support metadata only replication for external tables

Posted by GitBox <gi...@apache.org>.
maheshk114 commented on a change in pull request #900: HIVE-22860 Support metadata only replication for external tables
URL: https://github.com/apache/hive/pull/900#discussion_r379991523
 
 

 ##########
 File path: itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestMetadataReplicationScenariosExternalTables.java
 ##########
 @@ -0,0 +1,647 @@
+/*
+ * 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.hadoop.hive.ql.parse;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.InjectableBehaviourObjectStore;
+import org.apache.hadoop.hive.metastore.InjectableBehaviourObjectStore.BehaviourInjection;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.messaging.json.gzip.GzipJSONMessageEncoder;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hive.metastore.ReplChangeManager.SOURCE_OF_REPLICATION;
+import static org.apache.hadoop.hive.ql.exec.repl.ReplExternalTables.FILE_NAME;
+import static org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils.INC_BOOTSTRAP_ROOT_DIR_NAME;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * TestMetadataReplicationScenariosExternalTables - Test metadata only replication.
+ * for external tables.
+ */
+public class TestMetadataReplicationScenariosExternalTables extends BaseReplicationAcrossInstances {
 
 Review comment:
   change the name to TestReplicationScenariosExternalTablesMetaDataOnly.java 

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] aasha commented on a change in pull request #900: HIVE-22860 Support metadata only replication for external tables

Posted by GitBox <gi...@apache.org>.
aasha commented on a change in pull request #900: HIVE-22860 Support metadata only replication for external tables
URL: https://github.com/apache/hive/pull/900#discussion_r378395993
 
 

 ##########
 File path: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
 ##########
 @@ -478,6 +478,10 @@ private static void populateLlapDaemonVarsSet(Set<String> llapDaemonVarsSetLocal
     REPL_DUMP_METADATA_ONLY("hive.repl.dump.metadata.only", false,
         "Indicates whether replication dump only metadata information or data + metadata. \n"
           + "This config makes hive.repl.include.external.tables config ineffective."),
+    REPL_DUMP_METADATA_ONLY_FOR_EXTERNAL_TABLE("hive.repl.dump.metadata.only.for.external.table",
+            false,
+            "Indicates whether replication dump only metadata information or data + metadata. \n"
 
 Review comment:
   It was a typo. Fixed 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] maheshk114 commented on a change in pull request #900: HIVE-22860 Support metadata only replication for external tables

Posted by GitBox <gi...@apache.org>.
maheshk114 commented on a change in pull request #900: HIVE-22860 Support metadata only replication for external tables
URL: https://github.com/apache/hive/pull/900#discussion_r378096604
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/CreateTableHandler.java
 ##########
 @@ -66,7 +67,10 @@ public void handle(Context withinContext) throws Exception {
     // If we are not dumping data about a table, we shouldn't be dumping basic statistics
     // as well, since that won't be accurate. So reset them to what they would look like for an
     // empty table.
-    if (withinContext.hiveConf.getBoolVar(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY)) {
+    if (withinContext.hiveConf.getBoolVar(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY) ||
 
 Review comment:
   This configs should be from source ..target side ..this may not be set. 

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org