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/14 04:03:25 UTC

[GitHub] [hive] pkumarsinha opened a new pull request #911: HIVE-22865 Include data in replication staging directory

pkumarsinha opened a new pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911
 
 
   

----------------------------------------------------------------
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 #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
aasha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r384348340
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
 ##########
 @@ -582,10 +592,20 @@ void dumpTable(String dbName, String tblName, String validTxnList, Path dbRoot,
     }
     MmContext mmCtx = MmContext.createIfNeeded(tableSpec.tableHandle);
     tuple.replicationSpec.setRepl(true);
-    new TableExport(
-        exportPaths, tableSpec, tuple.replicationSpec, hiveDb, distCpDoAsUser, conf, mmCtx).write();
-
+    Path replDataDir = new Path(dumproot, EximUtil.DATA_PATH_NAME);
+    new TableExport(exportPaths, tableSpec, tuple.replicationSpec, hiveDb, distCpDoAsUser, conf, mmCtx).write();
+    if (conf.getBoolVar(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY) ||
+            (TableType.EXTERNAL_TABLE.equals(tuple.object.getTableType())
+                    && (!conf.getBoolVar(HiveConf.ConfVars.REPL_INCLUDE_EXTERNAL_TABLES)))) {
+          return;
+    }
     replLogger.tableLog(tblName, tableSpec.tableHandle.getTableType());
+    Path dumpDataDir = new Path(dumproot, EximUtil.DATA_PATH_NAME);
+    Path tblCopyPath = new Path(dumpDataDir, dbName);
+    tblCopyPath = new Path(tblCopyPath, tblName);
+    Task<?> copyTask = ReplCopyTask.getLoadCopyTask(
+            tuple.replicationSpec, tuple.object.getPath(), tblCopyPath, conf, false);
+    this.addDependentTask(copyTask);
 
 Review comment:
   We are not using dynamic DAG traversal for this?

----------------------------------------------------------------
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 #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
aasha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r386129824
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
 ##########
 @@ -582,10 +586,19 @@ void dumpTable(String dbName, String tblName, String validTxnList, Path dbRoot,
     }
     MmContext mmCtx = MmContext.createIfNeeded(tableSpec.tableHandle);
     tuple.replicationSpec.setRepl(true);
-    new TableExport(
-        exportPaths, tableSpec, tuple.replicationSpec, hiveDb, distCpDoAsUser, conf, mmCtx).write();
-
+    List<ReplPathMapping> replPathMappings = new TableExport(
+            exportPaths, tableSpec, tuple.replicationSpec, hiveDb, distCpDoAsUser, conf, mmCtx).write(false);
     replLogger.tableLog(tblName, tableSpec.tableHandle.getTableType());
+    if (Utils.shouldDumpMetaDataOnly(tuple.object, conf)) {
+      return;
+    }
+    for (ReplPathMapping replPathMapping: replPathMappings) {
+      Task<?> copyTask = ReplCopyTask.getLoadCopyTask(
+              tuple.replicationSpec, replPathMapping.getSrcPath(), replPathMapping.getTargetPath(), conf, false);
+      this.addDependentTask(copyTask);
 
 Review comment:
   Dynamic DAG generation needed here

----------------------------------------------------------------
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 #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
aasha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r383789102
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/TableExport.java
 ##########
 @@ -160,8 +160,10 @@ private void writeData(PartitionIterable partitions) throws SemanticException {
       } else {
         List<Path> dataPathList = Utils.getDataPathList(tableSpec.tableHandle.getDataLocation(),
                 replicationSpec, conf);
-        new FileOperations(dataPathList, paths.dataExportDir(), distCpDoAsUser, conf, mmCtx)
-                .export(replicationSpec);
+        // this is the data copy
+        if (!replicationSpec.isLazy()) {
 
 Review comment:
   Is the check 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.
 
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 #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
aasha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r383789019
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/PartitionExport.java
 ##########
 @@ -115,8 +115,9 @@ void write(final ReplicationSpec forReplicationSpec) throws InterruptedException
           List<Path> dataPathList = Utils.getDataPathList(partition.getDataLocation(),
                   forReplicationSpec, hiveConf);
           Path rootDataDumpDir = paths.partitionExportDir(partitionName);
-          new FileOperations(dataPathList, rootDataDumpDir, distCpDoAsUser, hiveConf, mmCtx)
-                  .export(forReplicationSpec);
+          if (!forReplicationSpec.isLazy()) {
 
 Review comment:
   Is the check 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.
 
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 #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
aasha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r386039609
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
 ##########
 @@ -305,7 +309,7 @@ private Long incrementalDump(Path dumpRoot, DumpMetaData dmd, Path cmRoot, Hive
       NotificationEvent ev = evIter.next();
       lastReplId = ev.getEventId();
       Path evRoot = new Path(dumpRoot, String.valueOf(lastReplId));
-      dumpEvent(ev, evRoot, cmRoot, hiveDb);
+      dumpEvent(ev, evRoot, dumpRoot, cmRoot, hiveDb);
 
 Review comment:
   Why is this dumpRoot and not hiveDumpRoot

----------------------------------------------------------------
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 #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
aasha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r383776695
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
 ##########
 @@ -344,8 +347,14 @@ private Long incrementalDump(Path dumpRoot, DumpMetaData dmd, Path cmRoot, Hive
             // Dump the table to be bootstrapped if required.
             if (shouldBootstrapDumpTable(table)) {
               HiveWrapper.Tuple<Table> tableTuple = new HiveWrapper(hiveDb, dbName).table(table);
-              dumpTable(dbName, tableName, validTxnList, dbRoot, bootDumpBeginReplId, hiveDb,
+              dumpTable(dbName, tableName, validTxnList, dbRoot, dumpRoot, bootDumpBeginReplId, hiveDb,
                       tableTuple);
+              Path tableRoot = new Path(dbRoot, tableName);
+              Path dumpDataDir = new Path(dumpRoot, EximUtil.DATA_PATH_NAME);
+              Path tblCopyPath = new Path(dumpDataDir, dbName);
+              tblCopyPath = new Path(tblCopyPath, tableName);
+              Task<?> copyTask = ReplCopyTask.getLoadCopyTask(new ReplicationSpec(), tableRoot, tblCopyPath, conf);
 
 Review comment:
   There is a copytask inside the dumpTable method as well

----------------------------------------------------------------
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 #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
aasha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r383671544
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplLoadTask.java
 ##########
 @@ -278,7 +278,7 @@ a database ( directory )
       }
       this.childTasks = scope.rootTasks;
       /*
-      Since there can be multiple rounds of this run all of which will be tied to the same
+      Since there can be multiple rounds rcof this run all of which will be tied to the same
 
 Review comment:
   typo?

----------------------------------------------------------------
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] pkumarsinha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
pkumarsinha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r386348243
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
 ##########
 @@ -582,10 +586,19 @@ void dumpTable(String dbName, String tblName, String validTxnList, Path dbRoot,
     }
     MmContext mmCtx = MmContext.createIfNeeded(tableSpec.tableHandle);
     tuple.replicationSpec.setRepl(true);
-    new TableExport(
-        exportPaths, tableSpec, tuple.replicationSpec, hiveDb, distCpDoAsUser, conf, mmCtx).write();
-
+    List<ReplPathMapping> replPathMappings = new TableExport(
+            exportPaths, tableSpec, tuple.replicationSpec, hiveDb, distCpDoAsUser, conf, mmCtx).write(false);
     replLogger.tableLog(tblName, tableSpec.tableHandle.getTableType());
+    if (Utils.shouldDumpMetaDataOnly(tuple.object, conf)) {
+      return;
+    }
+    for (ReplPathMapping replPathMapping: replPathMappings) {
+      Task<?> copyTask = ReplCopyTask.getLoadCopyTask(
+              tuple.replicationSpec, replPathMapping.getSrcPath(), replPathMapping.getTargetPath(), conf, false);
+      this.addDependentTask(copyTask);
 
 Review comment:
   Will track it as part of another JIRA

----------------------------------------------------------------
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 #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
aasha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r384345535
 
 

 ##########
 File path: itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosWithCopyData.java
 ##########
 @@ -0,0 +1,422 @@
+/*
+ * 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.commons.io.FileUtils;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.cli.CliSessionState;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
+import org.apache.hadoop.hive.metastore.PersistenceManagerProvider;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.messaging.json.gzip.GzipJSONMessageEncoder;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
+import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.exec.repl.ReplDumpWork;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorException;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.shims.Utils;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.hive.metastore.ReplChangeManager.SOURCE_OF_REPLICATION;
+import static org.junit.Assert.assertEquals;;
+
+public class TestReplicationScenariosWithCopyData {
+
+  @Rule
+  public final TestName testName = new TestName();
+
+  private final static String DBNOTIF_LISTENER_CLASSNAME =
+      "org.apache.hive.hcatalog.listener.DbNotificationListener";
+      // FIXME : replace with hive copy once that is copied
+  private final static String tid =
+              TestReplicationScenariosWithCopyData.class.getCanonicalName().toLowerCase().replace('.','_') + "_" + System.currentTimeMillis();
+  private final static String TEST_PATH =
+      System.getProperty("test.warehouse.dir", "/tmp") + Path.SEPARATOR + tid;
+
+  static HiveConf hconf;
+  static HiveMetaStoreClient metaStoreClient;
+  private static IDriver driver;
+  private static String proxySettingName;
+  private static HiveConf hconfMirror;
+  private static IDriver driverMirror;
+  private static HiveMetaStoreClient metaStoreClientMirror;
+  private static boolean isMigrationTest;
+
+  // Make sure we skip backward-compat checking for those tests that don't generate events
+
+  protected static final Logger LOG = LoggerFactory.getLogger(TestReplicationScenariosWithCopyData.class);
+  private ArrayList<String> lastResults;
+
+  private final boolean VERIFY_SETUP_STEPS = false;
+  // if verifySetup is set to true, all the test setup we do will perform additional
+  // verifications as well, which is useful to verify that our setup occurred
+  // correctly when developing and debugging tests. These verifications, however
+  // do not test any new functionality for replication, and thus, are not relevant
+  // for testing replication itself. For steady state, we want this to be false.
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    HashMap<String, String> overrideProperties = new HashMap<>();
+    overrideProperties.put(MetastoreConf.ConfVars.EVENT_MESSAGE_FACTORY.getHiveName(),
+        GzipJSONMessageEncoder.class.getCanonicalName());
+    internalBeforeClassSetup(overrideProperties, false);
+  }
+
+  static void internalBeforeClassSetup(Map<String, String> additionalProperties, boolean forMigration)
+      throws Exception {
+    hconf = new HiveConf(TestReplicationScenariosWithCopyData.class);
+    String metastoreUri = System.getProperty("test."+MetastoreConf.ConfVars.THRIFT_URIS.getHiveName());
+    if (metastoreUri != null) {
+      hconf.set(MetastoreConf.ConfVars.THRIFT_URIS.getHiveName(), metastoreUri);
+      return;
+    }
+    isMigrationTest = forMigration;
+
+    hconf.set(MetastoreConf.ConfVars.TRANSACTIONAL_EVENT_LISTENERS.getHiveName(),
+        DBNOTIF_LISTENER_CLASSNAME); // turn on db notification listener on metastore
+    hconf.setBoolVar(HiveConf.ConfVars.REPLCMENABLED, true);
+    hconf.setBoolVar(HiveConf.ConfVars.FIRE_EVENTS_FOR_DML, true);
+    hconf.setVar(HiveConf.ConfVars.REPLCMDIR, TEST_PATH + "/cmroot/");
+    proxySettingName = "hadoop.proxyuser." + Utils.getUGI().getShortUserName() + ".hosts";
+    hconf.set(proxySettingName, "*");
+    hconf.setVar(HiveConf.ConfVars.REPLDIR,TEST_PATH + "/hrepl/");
+    hconf.set(MetastoreConf.ConfVars.THRIFT_CONNECTION_RETRIES.getHiveName(), "3");
+    hconf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, "");
+    hconf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, "");
+    hconf.set(HiveConf.ConfVars.HIVE_IN_TEST_REPL.varname, "true");
+    hconf.setBoolVar(HiveConf.ConfVars.HIVE_IN_TEST, true);
+    hconf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
+    hconf.set(HiveConf.ConfVars.HIVE_TXN_MANAGER.varname,
+        "org.apache.hadoop.hive.ql.lockmgr.DummyTxnManager");
+    hconf.set(HiveConf.ConfVars.METASTORE_RAW_STORE_IMPL.varname,
+        "org.apache.hadoop.hive.metastore.InjectableBehaviourObjectStore");
+    hconf.setBoolVar(HiveConf.ConfVars.HIVEOPTIMIZEMETADATAQUERIES, true);
+    hconf.setBoolVar(HiveConf.ConfVars.HIVESTATSAUTOGATHER, true);
+    hconf.setBoolVar(HiveConf.ConfVars.HIVE_STATS_RELIABLE, true);
+    System.setProperty(HiveConf.ConfVars.PREEXECHOOKS.varname, " ");
+    System.setProperty(HiveConf.ConfVars.POSTEXECHOOKS.varname, " ");
+
+    additionalProperties.forEach((key, value) -> {
+      hconf.set(key, value);
+    });
+
+    MetaStoreTestUtils.startMetaStoreWithRetry(hconf);
+    // re set the WAREHOUSE property to the test dir, as the previous command added a random port to it
+    hconf.set(MetastoreConf.ConfVars.WAREHOUSE.getVarname(), System.getProperty("test.warehouse.dir", "/tmp"));
+
+    Path testPath = new Path(TEST_PATH);
+    FileSystem fs = FileSystem.get(testPath.toUri(),hconf);
+    fs.mkdirs(testPath);
+    driver = DriverFactory.newDriver(hconf);
+    SessionState.start(new CliSessionState(hconf));
+    metaStoreClient = new HiveMetaStoreClient(hconf);
+
+    FileUtils.deleteDirectory(new File("metastore_db2"));
+    HiveConf hconfMirrorServer = new HiveConf();
+    hconfMirrorServer.set(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname, "jdbc:derby:;databaseName=metastore_db2;create=true");
+    MetaStoreTestUtils.startMetaStoreWithRetry(hconfMirrorServer);
+    hconfMirror = new HiveConf(hconf);
+    String thriftUri = MetastoreConf.getVar(hconfMirrorServer, MetastoreConf.ConfVars.THRIFT_URIS);
+    MetastoreConf.setVar(hconfMirror, MetastoreConf.ConfVars.THRIFT_URIS, thriftUri);
+
+    if (forMigration) {
+      hconfMirror.setBoolVar(HiveConf.ConfVars.HIVE_STRICT_MANAGED_TABLES, true);
+      hconfMirror.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, true);
+      hconfMirror.set(HiveConf.ConfVars.HIVE_TXN_MANAGER.varname,
+              "org.apache.hadoop.hive.ql.lockmgr.DbTxnManager");
+    }
+    driverMirror = DriverFactory.newDriver(hconfMirror);
+    metaStoreClientMirror = new HiveMetaStoreClient(hconfMirror);
+
+    PersistenceManagerProvider.setTwoMetastoreTesting(true);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass(){
+  }
+
+  @Before
+  public void setUp(){
+    // before each test
+    SessionState.get().setCurrentDatabase("default");
+  }
+
+  @After
+  public void tearDown(){
+    // after each test
+  }
+
+  private static  int next = 0;
+  private synchronized void advanceDumpDir() {
+    next++;
+    ReplDumpWork.injectNextDumpDirForTest(String.valueOf(next));
+  }
+
+ static class Tuple {
+    final String dumpLocation;
+    final String lastReplId;
+
+    Tuple(String dumpLocation, String lastReplId) {
+      this.dumpLocation = dumpLocation;
+      this.lastReplId = lastReplId;
+    }
+  }
+
+  private Tuple bootstrapLoadAndVerify(String dbName, String replDbName) throws IOException {
+    return incrementalLoadAndVerify(dbName, null, replDbName);
+  }
+
+  private Tuple incrementalLoadAndVerify(String dbName, String fromReplId, String replDbName) throws IOException {
+    Tuple dump = replDumpDb(dbName, fromReplId, null, null);
+    loadAndVerify(replDbName, dump.dumpLocation, dump.lastReplId);
+    return dump;
+  }
+
+  private Tuple replDumpDb(String dbName, String fromReplID, String toReplID, String limit) throws IOException {
+    advanceDumpDir();
+    String dumpCmd = "REPL DUMP " + dbName;
+    if (null != fromReplID) {
+      dumpCmd = dumpCmd + " FROM " + fromReplID;
+    }
+    if (null != toReplID) {
+      dumpCmd = dumpCmd + " TO " + toReplID;
+    }
+    if (null != limit) {
+      dumpCmd = dumpCmd + " LIMIT " + limit;
+    }
+    run(dumpCmd, driver);
+    String dumpLocation = getResult(0, 0, driver);
+    String lastReplId = getResult(0, 1, true, driver);
+    LOG.info("Dumped to {} with id {} for command: {}", dumpLocation, lastReplId, dumpCmd);
+    return new Tuple(dumpLocation, lastReplId);
+  }
+
+  private void loadAndVerify(String replDbName, String dumpLocation, String lastReplId) throws IOException {
+    run("REPL LOAD " + replDbName + " FROM '" + dumpLocation + "'", driverMirror);
+    verifyRun("REPL STATUS " + replDbName, lastReplId, driverMirror);
+    return;
+  }
+
+  private abstract class checkTaskPresent {
+    public boolean hasTask(Task rootTask) {
+      if (rootTask == null) {
+        return false;
+      }
+      if (validate(rootTask)) {
+        return true;
+      }
+      List<Task<?>> childTasks = rootTask.getChildTasks();
+      if (childTasks == null) {
+        return false;
+      }
+      for (Task<?> childTask : childTasks) {
+        if (hasTask(childTask)) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    public abstract boolean validate(Task task);
+  }
+
+  @Test
+  public void testIncrementalLoadWithDumpData() throws IOException {
 
 Review comment:
   Are these new test scenarios?

----------------------------------------------------------------
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 #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
aasha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r384345859
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
 ##########
 @@ -119,18 +121,19 @@ public String getName() {
   public int execute() {
     try {
       Hive hiveDb = getHive();
-      Path dumpRoot = new Path(conf.getVar(HiveConf.ConfVars.REPLDIR), getNextDumpDir());
-      DumpMetaData dmd = new DumpMetaData(dumpRoot, conf);
+      Path dumpBaseDir = new Path(conf.getVar(HiveConf.ConfVars.REPLDIR), getNextDumpDir());
+      Path hiveDumpRoot = new Path(dumpBaseDir, ReplUtils.REPL_HIVE_BASE_DIR);
+      DumpMetaData dmd = new DumpMetaData(hiveDumpRoot, conf);
       // Initialize ReplChangeManager instance since we will require it to encode file URI.
       ReplChangeManager.getInstance(conf);
       Path cmRoot = new Path(conf.getVar(HiveConf.ConfVars.REPLCMDIR));
       Long lastReplId;
       if (work.isBootStrapDump()) {
-        lastReplId = bootStrapDump(dumpRoot, dmd, cmRoot, hiveDb);
+        lastReplId = bootStrapDump(hiveDumpRoot, dmd, cmRoot, hiveDb);
 
 Review comment:
   Are we still keeping the cm?

----------------------------------------------------------------
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] pkumarsinha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
pkumarsinha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r386348059
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
 ##########
 @@ -582,10 +586,19 @@ void dumpTable(String dbName, String tblName, String validTxnList, Path dbRoot,
     }
     MmContext mmCtx = MmContext.createIfNeeded(tableSpec.tableHandle);
     tuple.replicationSpec.setRepl(true);
-    new TableExport(
-        exportPaths, tableSpec, tuple.replicationSpec, hiveDb, distCpDoAsUser, conf, mmCtx).write();
-
+    List<ReplPathMapping> replPathMappings = new TableExport(
+            exportPaths, tableSpec, tuple.replicationSpec, hiveDb, distCpDoAsUser, conf, mmCtx).write(false);
     replLogger.tableLog(tblName, tableSpec.tableHandle.getTableType());
+    if (Utils.shouldDumpMetaDataOnly(tuple.object, conf)) {
 
 Review comment:
   No, it is not done at this level. The checks have been done inside some other method calls.

----------------------------------------------------------------
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] pkumarsinha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
pkumarsinha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r386348502
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AbstractEventHandler.java
 ##########
 @@ -71,4 +85,31 @@ public long fromEventId() {
   public long toEventId() {
     return event.getEventId();
   }
+
+  public void writeFileEntry(String dbName, Table table, String file, BufferedWriter fileListWriter,
+                             Context withinContext) throws IOException, LoginException {
+    HiveConf hiveConf = withinContext.hiveConf;
+    String distCpDoAsUser = hiveConf.getVar(HiveConf.ConfVars.HIVE_DISTCP_DOAS_USER);
+    if (Utils.shouldDumpMetaDataOnly(table, withinContext.hiveConf)) {
 
 Review comment:
   Will fix this.

----------------------------------------------------------------
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 #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
aasha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r386039239
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/exec/ExportTask.java
 ##########
 @@ -53,9 +53,7 @@ public int execute() {
       work.acidPostProcess(db);
       TableExport tableExport = new TableExport(exportPaths, work.getTableSpec(),
           work.getReplicationSpec(), db, null, conf, work.getMmContext());
-      if (!tableExport.write()) {
 
 Review comment:
   Why is the check removed?

----------------------------------------------------------------
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 #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
aasha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r383781222
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
 ##########
 @@ -582,10 +592,20 @@ void dumpTable(String dbName, String tblName, String validTxnList, Path dbRoot,
     }
     MmContext mmCtx = MmContext.createIfNeeded(tableSpec.tableHandle);
     tuple.replicationSpec.setRepl(true);
-    new TableExport(
-        exportPaths, tableSpec, tuple.replicationSpec, hiveDb, distCpDoAsUser, conf, mmCtx).write();
-
+    Path replDataDir = new Path(dumproot, EximUtil.DATA_PATH_NAME);
+    new TableExport(exportPaths, tableSpec, tuple.replicationSpec, hiveDb, distCpDoAsUser, conf, mmCtx).write();
+    if (conf.getBoolVar(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY) ||
 
 Review comment:
   This should n't reach here

----------------------------------------------------------------
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 #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
aasha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r386130273
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AbstractEventHandler.java
 ##########
 @@ -71,4 +85,31 @@ public long fromEventId() {
   public long toEventId() {
     return event.getEventId();
   }
+
+  public void writeFileEntry(String dbName, Table table, String file, BufferedWriter fileListWriter,
 
 Review comment:
   This can be in util class like before. 

----------------------------------------------------------------
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] pkumarsinha closed pull request #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
pkumarsinha closed pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911
 
 
   

----------------------------------------------------------------
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 #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
aasha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r386129803
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
 ##########
 @@ -582,10 +586,19 @@ void dumpTable(String dbName, String tblName, String validTxnList, Path dbRoot,
     }
     MmContext mmCtx = MmContext.createIfNeeded(tableSpec.tableHandle);
     tuple.replicationSpec.setRepl(true);
-    new TableExport(
-        exportPaths, tableSpec, tuple.replicationSpec, hiveDb, distCpDoAsUser, conf, mmCtx).write();
-
+    List<ReplPathMapping> replPathMappings = new TableExport(
+            exportPaths, tableSpec, tuple.replicationSpec, hiveDb, distCpDoAsUser, conf, mmCtx).write(false);
     replLogger.tableLog(tblName, tableSpec.tableHandle.getTableType());
+    if (Utils.shouldDumpMetaDataOnly(tuple.object, conf)) {
 
 Review comment:
   Why is this check needed here? Is it not done previously

----------------------------------------------------------------
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] pkumarsinha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
pkumarsinha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r386045024
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/exec/ExportTask.java
 ##########
 @@ -53,9 +53,7 @@ public int execute() {
       work.acidPostProcess(db);
       TableExport tableExport = new TableExport(exportPaths, work.getTableSpec(),
           work.getReplicationSpec(), db, null, conf, work.getMmContext());
-      if (!tableExport.write()) {
 
 Review comment:
   Have refactored the code and same behavior is achieved in TableExport.export() now

----------------------------------------------------------------
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 #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
aasha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r388341737
 
 

 ##########
 File path: itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTablesBootstrap.java
 ##########
 @@ -264,7 +266,7 @@ public void testBootstrapAcidTablesDuringIncrementalWithConcurrentWrites() throw
     prepareIncAcidData(primaryDbName);
     // Perform concurrent writes. Bootstrap won't see the written data but the subsequent
     // incremental repl should see it. We can not inject callerVerifier since an incremental dump
-    // would not cause an ALTER DATABASE event. Instead we piggy back on
+    // would not cause an ALTER DATABASE event. Instead we piggy bEHANack on
 
 Review comment:
   typo

----------------------------------------------------------------
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] pkumarsinha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
pkumarsinha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r386110262
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
 ##########
 @@ -305,7 +309,7 @@ private Long incrementalDump(Path dumpRoot, DumpMetaData dmd, Path cmRoot, Hive
       NotificationEvent ev = evIter.next();
       lastReplId = ev.getEventId();
       Path evRoot = new Path(dumpRoot, String.valueOf(lastReplId));
-      dumpEvent(ev, evRoot, cmRoot, hiveDb);
+      dumpEvent(ev, evRoot, dumpRoot, cmRoot, hiveDb);
 
 Review comment:
   hiveDumpRoot is recieved as dumpRoot in the current method. I haven't renamed existing parameter names.

----------------------------------------------------------------
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 #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
aasha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r386130216
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AbstractEventHandler.java
 ##########
 @@ -71,4 +85,31 @@ public long fromEventId() {
   public long toEventId() {
     return event.getEventId();
   }
+
+  public void writeFileEntry(String dbName, Table table, String file, BufferedWriter fileListWriter,
+                             Context withinContext) throws IOException, LoginException {
+    HiveConf hiveConf = withinContext.hiveConf;
+    String distCpDoAsUser = hiveConf.getVar(HiveConf.ConfVars.HIVE_DISTCP_DOAS_USER);
+    if (Utils.shouldDumpMetaDataOnly(table, withinContext.hiveConf)) {
 
 Review comment:
   If its metadata only, why are we dumping data

----------------------------------------------------------------
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] pkumarsinha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
pkumarsinha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r388480128
 
 

 ##########
 File path: itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosExternalTables.java
 ##########
 @@ -904,8 +901,20 @@ public void replicationWithTableNameContainsKeywords() throws Throwable {
     return ReplicationTestUtils.externalTableBasePathWithClause(REPLICA_EXTERNAL_BASE, replica);
   }
 
-  private void assertExternalFileInfo(List<String> expected, Path externalTableInfoFile)
+  private void assertExternalFileInfo(List<String> expected, String dumplocation) throws IOException {
+    assertExternalFileInfo(expected, dumplocation, null);
+  }
+  private void assertExternalFileInfo(List<String> expected, String dumplocation, String dbName)
       throws IOException {
+    Path externalTableInfoFile = new Path(dumplocation, relativeExtInfoPath(dbName));
     ReplicationTestUtils.assertExternalFileInfo(primary, expected, externalTableInfoFile);
   }
+  private String relativeExtInfoPath(String dbName) {
+
+    if (dbName == null) {
 
 Review comment:
   No, the location of external table info file is different in bootstrap and incremental case.

----------------------------------------------------------------
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 #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
aasha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r383669184
 
 

 ##########
 File path: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
 ##########
 @@ -475,6 +475,8 @@ private static void populateLlapDaemonVarsSet(Set<String> llapDaemonVarsSetLocal
     REPL_DUMPDIR_TTL("hive.repl.dumpdir.ttl", "7d",
         new TimeValidator(TimeUnit.DAYS),
         "TTL of dump dirs before cleanup."),
+    REPL_DUMP_COPY_DATA("hive.repl.dump.copydata", true,
 
 Review comment:
   what happens when both REPL_DUMP_COPY_DATA and REPL_DUMP_METADATA_ONLY is set to true? which one takes precedence?

----------------------------------------------------------------
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] pkumarsinha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
pkumarsinha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r386348679
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AbstractEventHandler.java
 ##########
 @@ -71,4 +85,31 @@ public long fromEventId() {
   public long toEventId() {
     return event.getEventId();
   }
+
+  public void writeFileEntry(String dbName, Table table, String file, BufferedWriter fileListWriter,
 
 Review comment:
   Will move out to util.

----------------------------------------------------------------
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 #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
aasha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r388345992
 
 

 ##########
 File path: itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosExternalTables.java
 ##########
 @@ -904,8 +901,20 @@ public void replicationWithTableNameContainsKeywords() throws Throwable {
     return ReplicationTestUtils.externalTableBasePathWithClause(REPLICA_EXTERNAL_BASE, replica);
   }
 
-  private void assertExternalFileInfo(List<String> expected, Path externalTableInfoFile)
+  private void assertExternalFileInfo(List<String> expected, String dumplocation) throws IOException {
+    assertExternalFileInfo(expected, dumplocation, null);
+  }
+  private void assertExternalFileInfo(List<String> expected, String dumplocation, String dbName)
       throws IOException {
+    Path externalTableInfoFile = new Path(dumplocation, relativeExtInfoPath(dbName));
     ReplicationTestUtils.assertExternalFileInfo(primary, expected, externalTableInfoFile);
   }
+  private String relativeExtInfoPath(String dbName) {
+
+    if (dbName == null) {
 
 Review comment:
   For incremental dbname is not 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.
 
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 #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
aasha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r386130273
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AbstractEventHandler.java
 ##########
 @@ -71,4 +85,31 @@ public long fromEventId() {
   public long toEventId() {
     return event.getEventId();
   }
+
+  public void writeFileEntry(String dbName, Table table, String file, BufferedWriter fileListWriter,
 
 Review comment:
   This can be in util class like before. Don't think its appropriate in a event handler class

----------------------------------------------------------------
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 #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
aasha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r383791504
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AbstractEventHandler.java
 ##########
 @@ -71,4 +84,32 @@ public long fromEventId() {
   public long toEventId() {
     return event.getEventId();
   }
+
+  public void writeFileEntry(String dbName, String tblName, String file, BufferedWriter fileListWriter,
+                             Context withinContext) throws IOException, LoginException {
+      HiveConf hiveConf = withinContext.hiveConf;
+      boolean copyActualData = !hiveConf.getBoolVar(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY);
 
 Review comment:
   check REPL_DUMP_METADATA_EXTERNAL_TABLE

----------------------------------------------------------------
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] pkumarsinha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory

Posted by GitBox <gi...@apache.org>.
pkumarsinha commented on a change in pull request #911: HIVE-22865 Include data in replication staging directory
URL: https://github.com/apache/hive/pull/911#discussion_r388480221
 
 

 ##########
 File path: itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTablesBootstrap.java
 ##########
 @@ -264,7 +266,7 @@ public void testBootstrapAcidTablesDuringIncrementalWithConcurrentWrites() throw
     prepareIncAcidData(primaryDbName);
     // Perform concurrent writes. Bootstrap won't see the written data but the subsequent
     // incremental repl should see it. We can not inject callerVerifier since an incremental dump
-    // would not cause an ALTER DATABASE event. Instead we piggy back on
+    // would not cause an ALTER DATABASE event. Instead we piggy bEHANack on
 
 Review comment:
   Fixed

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