You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by th...@apache.org on 2017/08/31 00:22:24 UTC

[5/6] hive git commit: HIVE-17100 : Improve HS2 operation logs for REPL commands (Sankar Hariappan, reviewed by Anishek Agarwal, Thejas Nair)

http://git-wip-us.apache.org/repos/asf/hive/blob/70cb7f0b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/ReplState.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/ReplState.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/ReplState.java
new file mode 100644
index 0000000..a003d37
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/ReplState.java
@@ -0,0 +1,57 @@
+/**
+ * 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.repl;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.map.SerializationConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class ReplState {
+  @JsonIgnoreProperties
+  private static final Logger REPL_LOG = LoggerFactory.getLogger("ReplState");
+
+  @JsonIgnoreProperties
+  private static final ObjectMapper mapper = new ObjectMapper(); // Thread-safe.
+
+  static {
+    mapper.configure(SerializationConfig.Feature.AUTO_DETECT_GETTERS, false);
+    mapper.configure(SerializationConfig.Feature.AUTO_DETECT_IS_GETTERS, false);
+    mapper.configure(SerializationConfig.Feature.AUTO_DETECT_FIELDS, false);
+  }
+
+  public enum LogTag {
+    START,
+    TABLE_DUMP,
+    FUNCTION_DUMP,
+    EVENT_DUMP,
+    TABLE_LOAD,
+    FUNCTION_LOAD,
+    EVENT_LOAD,
+    END
+  }
+
+  public void log(LogTag tag) {
+    try {
+      REPL_LOG.info("REPL::{}: {}", tag.name(), mapper.writeValueAsString(this));
+    } catch (Exception exception) {
+      REPL_LOG.error("Could not serialize REPL log: {}", exception.getMessage());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/70cb7f0b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java
index f40c703..a48a17e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java
@@ -31,6 +31,7 @@ import com.google.common.collect.Collections2;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.util.Collection;
 import java.util.List;
 
 public class Utils {
@@ -64,14 +65,18 @@ public class Utils {
   public static Iterable<? extends String> matchesTbl(Hive db, String dbName, String tblPattern)
       throws HiveException {
     if (tblPattern == null) {
-      return Collections2.filter(db.getAllTables(dbName),
-          tableName -> {
-            assert tableName != null;
-            return !tableName.toLowerCase().startsWith(
-                SemanticAnalyzer.VALUES_TMP_TABLE_NAME_PREFIX.toLowerCase());
-          });
+      return getAllTables(db, dbName);
     } else {
       return db.getTablesByPattern(dbName, tblPattern);
     }
   }
+
+  public static Collection<String> getAllTables(Hive db, String dbName) throws HiveException {
+    return Collections2.filter(db.getAllTables(dbName),
+            tableName -> {
+              assert tableName != null;
+              return !tableName.toLowerCase().startsWith(
+                      SemanticAnalyzer.VALUES_TMP_TABLE_NAME_PREFIX.toLowerCase());
+            });
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/70cb7f0b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/BootstrapDumpLogger.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/BootstrapDumpLogger.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/BootstrapDumpLogger.java
new file mode 100644
index 0000000..1b0ec59
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/BootstrapDumpLogger.java
@@ -0,0 +1,71 @@
+/**
+ * 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.repl.dump.log;
+
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.ql.parse.repl.dump.log.state.BootstrapDumpBegin;
+import org.apache.hadoop.hive.ql.parse.repl.dump.log.state.BootstrapDumpEnd;
+import org.apache.hadoop.hive.ql.parse.repl.dump.log.state.BootstrapDumpFunction;
+import org.apache.hadoop.hive.ql.parse.repl.dump.log.state.BootstrapDumpTable;
+import org.apache.hadoop.hive.ql.parse.repl.ReplLogger;
+import org.apache.hadoop.hive.ql.parse.repl.ReplState.LogTag;
+
+public class BootstrapDumpLogger extends ReplLogger {
+  private String dbName;
+  private String dumpDir;
+  private long estimatedNumTables;
+  private long estimatedNumFunctions;
+  private long tableSeqNo;
+  private long functionSeqNo;
+
+  public BootstrapDumpLogger(String dbName, String dumpDir,
+                             int estimatedNumTables, int estimatedNumFunctions) {
+    this.dbName = dbName;
+    this.dumpDir = dumpDir;
+    this.estimatedNumTables = estimatedNumTables;
+    this.estimatedNumFunctions = estimatedNumFunctions;
+    this.tableSeqNo = 0;
+    this.functionSeqNo = 0;
+  }
+
+  @Override
+  public void startLog() {
+    (new BootstrapDumpBegin(dbName, estimatedNumTables, estimatedNumFunctions))
+            .log(LogTag.START);
+  }
+
+  @Override
+  public void tableLog(String tableName, TableType tableType) {
+    tableSeqNo++;
+    (new BootstrapDumpTable(dbName, tableName, tableType, tableSeqNo, estimatedNumTables))
+            .log(LogTag.TABLE_DUMP);
+  }
+
+  @Override
+  public void functionLog(String funcName) {
+    functionSeqNo++;
+    (new BootstrapDumpFunction(dbName, funcName, functionSeqNo, estimatedNumFunctions))
+            .log(LogTag.FUNCTION_DUMP);
+  }
+
+  @Override
+  public void endLog(String lastReplId) {
+    (new BootstrapDumpEnd(dbName, tableSeqNo, functionSeqNo, dumpDir, lastReplId))
+            .log(LogTag.END);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/70cb7f0b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/IncrementalDumpLogger.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/IncrementalDumpLogger.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/IncrementalDumpLogger.java
new file mode 100644
index 0000000..03144df
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/IncrementalDumpLogger.java
@@ -0,0 +1,55 @@
+/**
+ * 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.repl.dump.log;
+
+import org.apache.hadoop.hive.ql.parse.repl.dump.log.state.IncrementalDumpBegin;
+import org.apache.hadoop.hive.ql.parse.repl.dump.log.state.IncrementalDumpEnd;
+import org.apache.hadoop.hive.ql.parse.repl.dump.log.state.IncrementalDumpEvent;
+import org.apache.hadoop.hive.ql.parse.repl.ReplLogger;
+import org.apache.hadoop.hive.ql.parse.repl.ReplState.LogTag;
+
+public class IncrementalDumpLogger extends ReplLogger {
+  private String dbName;
+  private String dumpDir;
+  private long estimatedNumEvents;
+  private long eventSeqNo;
+
+  public IncrementalDumpLogger(String dbName, String dumpDir, long estimatedNumEvents) {
+    this.dbName = dbName;
+    this.dumpDir = dumpDir;
+    this.estimatedNumEvents = estimatedNumEvents;
+    this.eventSeqNo = 0;
+  }
+
+  @Override
+  public void startLog() {
+    (new IncrementalDumpBegin(dbName, estimatedNumEvents)).log(LogTag.START);
+  }
+
+  @Override
+  public void eventLog(String eventId, String eventType) {
+    eventSeqNo++;
+    (new IncrementalDumpEvent(dbName, eventId, eventType, eventSeqNo, estimatedNumEvents))
+            .log(LogTag.EVENT_DUMP);
+  }
+
+  @Override
+  public void endLog(String lastReplId) {
+    (new IncrementalDumpEnd(dbName, eventSeqNo, dumpDir, lastReplId)).log(LogTag.END);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/70cb7f0b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/BootstrapDumpBegin.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/BootstrapDumpBegin.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/BootstrapDumpBegin.java
new file mode 100644
index 0000000..3c4ebd3
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/BootstrapDumpBegin.java
@@ -0,0 +1,47 @@
+/**
+ * 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.repl.dump.log.state;
+
+import org.apache.hadoop.hive.ql.parse.repl.ReplState;
+import org.apache.hadoop.hive.ql.parse.repl.DumpType;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+public class BootstrapDumpBegin extends ReplState {
+  @JsonProperty
+  private String dbName;
+
+  @JsonProperty
+  private DumpType dumpType;
+
+  @JsonProperty
+  private Long estimatedNumTables;
+
+  @JsonProperty
+  private Long estimatedNumFunctions;
+
+  @JsonProperty
+  private Long dumpStartTime;
+
+  public BootstrapDumpBegin(String dbName, long estimatedNumTables, long estimatedNumFunctions) {
+    this.dbName = dbName;
+    this.dumpType = DumpType.BOOTSTRAP;
+    this.estimatedNumTables = estimatedNumTables;
+    this.estimatedNumFunctions = estimatedNumFunctions;
+    this.dumpStartTime = System.currentTimeMillis() / 1000;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/70cb7f0b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/BootstrapDumpEnd.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/BootstrapDumpEnd.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/BootstrapDumpEnd.java
new file mode 100644
index 0000000..a4dcfb5
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/BootstrapDumpEnd.java
@@ -0,0 +1,59 @@
+/**
+ * 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.repl.dump.log.state;
+
+import org.apache.hadoop.hive.ql.parse.repl.ReplState;
+import org.apache.hadoop.hive.ql.parse.repl.DumpType;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+public class BootstrapDumpEnd extends ReplState {
+  @JsonProperty
+  private String dbName;
+
+  @JsonProperty
+  private DumpType dumpType;
+
+  @JsonProperty
+  private Long actualNumTables;
+
+  @JsonProperty
+  private Long actualNumFunctions;
+
+  @JsonProperty
+  private Long dumpEndTime;
+
+  @JsonProperty
+  private String dumpDir;
+
+  @JsonProperty
+  private String lastReplId;
+
+  public BootstrapDumpEnd(String dbName,
+                          long actualNumTables,
+                          long actualNumFunctions,
+                          String dumpDir,
+                          String lastReplId) {
+    this.dbName = dbName;
+    this.dumpType = DumpType.BOOTSTRAP;
+    this.actualNumTables = actualNumTables;
+    this.actualNumFunctions = actualNumFunctions;
+    this.dumpEndTime = System.currentTimeMillis() / 1000;
+    this.dumpDir = dumpDir;
+    this.lastReplId = lastReplId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/70cb7f0b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/BootstrapDumpFunction.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/BootstrapDumpFunction.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/BootstrapDumpFunction.java
new file mode 100644
index 0000000..a414daa
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/BootstrapDumpFunction.java
@@ -0,0 +1,44 @@
+/**
+ * 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.repl.dump.log.state;
+
+import org.apache.hadoop.hive.ql.parse.repl.ReplState;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+public class BootstrapDumpFunction extends ReplState {
+  @JsonProperty
+  private String dbName;
+
+  @JsonProperty
+  private String functionName;
+
+  @JsonProperty
+  private String functionsDumpProgress;
+
+  @JsonProperty
+  private Long dumpTime;
+
+  public BootstrapDumpFunction(String dbName, String funcName,
+                               long functionSeqNo, long estimatedNumFunctions) {
+    this.dbName = dbName;
+    this.functionName = funcName;
+    this.functionsDumpProgress = new String(new StringBuilder()
+                                  .append(functionSeqNo).append("/").append(estimatedNumFunctions));
+    this.dumpTime = System.currentTimeMillis() / 1000;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/70cb7f0b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/BootstrapDumpTable.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/BootstrapDumpTable.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/BootstrapDumpTable.java
new file mode 100644
index 0000000..4bf4905
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/BootstrapDumpTable.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.parse.repl.dump.log.state;
+
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.ql.parse.repl.ReplState;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+public class BootstrapDumpTable extends ReplState {
+  @JsonProperty
+  private String dbName;
+
+  @JsonProperty
+  private String tableName;
+
+  @JsonProperty
+  private String tableType;
+
+  @JsonProperty
+  private String tablesDumpProgress;
+
+  @JsonProperty
+  private Long dumpTime;
+
+  public BootstrapDumpTable(String dbName,
+                            String tableName,
+                            TableType tableType,
+                            long tableSeqNo,
+                            long estimatedNumTables) {
+    this.dbName = dbName;
+    this.tableName = tableName;
+    this.tableType = tableType.name();
+    this.tablesDumpProgress = new String(new StringBuilder()
+                                        .append(tableSeqNo).append("/").append(estimatedNumTables));
+    this.dumpTime = System.currentTimeMillis() / 1000;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/70cb7f0b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/IncrementalDumpBegin.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/IncrementalDumpBegin.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/IncrementalDumpBegin.java
new file mode 100644
index 0000000..166d359
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/IncrementalDumpBegin.java
@@ -0,0 +1,43 @@
+/**
+ * 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.repl.dump.log.state;
+
+import org.apache.hadoop.hive.ql.parse.repl.ReplState;
+import org.apache.hadoop.hive.ql.parse.repl.DumpType;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+public class IncrementalDumpBegin extends ReplState {
+  @JsonProperty
+  private String dbName;
+
+  @JsonProperty
+  private DumpType dumpType;
+
+  @JsonProperty
+  private Long estimatedNumEvents;
+
+  @JsonProperty
+  private Long dumpStartTime;
+
+  public IncrementalDumpBegin(String dbName, long estimatedNumEvents) {
+    this.dbName = dbName;
+    this.dumpType = DumpType.INCREMENTAL;
+    this.estimatedNumEvents = estimatedNumEvents;
+    this.dumpStartTime = System.currentTimeMillis() / 1000;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/70cb7f0b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/IncrementalDumpEnd.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/IncrementalDumpEnd.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/IncrementalDumpEnd.java
new file mode 100644
index 0000000..a7cec21
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/IncrementalDumpEnd.java
@@ -0,0 +1,54 @@
+/**
+ * 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.repl.dump.log.state;
+
+import org.apache.hadoop.hive.ql.parse.repl.ReplState;
+import org.apache.hadoop.hive.ql.parse.repl.DumpType;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+public class IncrementalDumpEnd extends ReplState {
+  @JsonProperty
+  private String dbName;
+
+  @JsonProperty
+  private DumpType dumpType;
+
+  @JsonProperty
+  private Long actualNumEvents;
+
+  @JsonProperty
+  private Long dumpEndTime;
+
+  @JsonProperty
+  private String dumpDir;
+
+  @JsonProperty
+  private String lastReplId;
+
+  public IncrementalDumpEnd(String dbName,
+                            long actualNumEvents,
+                            String dumpDir,
+                            String lastReplId) {
+    this.dbName = dbName;
+    this.dumpType = DumpType.INCREMENTAL;
+    this.actualNumEvents = actualNumEvents;
+    this.dumpEndTime = System.currentTimeMillis() / 1000;
+    this.dumpDir = dumpDir;
+    this.lastReplId = lastReplId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/70cb7f0b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/IncrementalDumpEvent.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/IncrementalDumpEvent.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/IncrementalDumpEvent.java
new file mode 100644
index 0000000..ebeb468
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/log/state/IncrementalDumpEvent.java
@@ -0,0 +1,51 @@
+/**
+ * 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.repl.dump.log.state;
+
+import org.apache.hadoop.hive.ql.parse.repl.ReplState;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+public class IncrementalDumpEvent extends ReplState {
+  @JsonProperty
+  private String dbName;
+
+  @JsonProperty
+  private String eventId;
+
+  @JsonProperty
+  private String eventType;
+
+  @JsonProperty
+  private String eventsDumpProgress;
+
+  @JsonProperty
+  private Long dumpTime;
+
+  public IncrementalDumpEvent(String dbName,
+                              String eventId,
+                              String eventType,
+                              long eventSeqNo,
+                              long estimatedNumEvents) {
+    this.dbName = dbName;
+    this.eventId = eventId;
+    this.eventType = eventType;
+    this.eventsDumpProgress = new String(new StringBuilder()
+                                        .append(eventSeqNo).append("/").append(estimatedNumEvents));
+    this.dumpTime = System.currentTimeMillis() / 1000;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/70cb7f0b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/BootstrapLoadLogger.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/BootstrapLoadLogger.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/BootstrapLoadLogger.java
new file mode 100644
index 0000000..739bd12
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/BootstrapLoadLogger.java
@@ -0,0 +1,66 @@
+/**
+ * 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.repl.load.log;
+
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.ql.parse.repl.load.log.state.*;
+import org.apache.hadoop.hive.ql.parse.repl.ReplLogger;
+import org.apache.hadoop.hive.ql.parse.repl.ReplState.LogTag;
+
+public class BootstrapLoadLogger extends ReplLogger {
+  private String dbName;
+  private String dumpDir;
+  private long numTables;
+  private long numFunctions;
+  private long tableSeqNo;
+  private long functionSeqNo;
+
+  public BootstrapLoadLogger(String dbName, String dumpDir, long numTables, long numFunctions) {
+    this.dbName = dbName;
+    this.dumpDir = dumpDir;
+    this.numTables = numTables;
+    this.numFunctions = numFunctions;
+    this.tableSeqNo = 0;
+    this.functionSeqNo = 0;
+  }
+
+  @Override
+  public void startLog() {
+    (new BootstrapLoadBegin(dbName, dumpDir, numTables, numFunctions)).log(LogTag.START);
+  }
+
+  @Override
+  public void tableLog(String tableName, TableType tableType) {
+    tableSeqNo++;
+    (new BootstrapLoadTable(dbName, tableName, tableType, tableSeqNo, numTables))
+            .log(LogTag.TABLE_LOAD);
+  }
+
+  @Override
+  public void functionLog(String funcName) {
+    functionSeqNo++;
+    (new BootstrapLoadFunction(dbName, funcName, functionSeqNo, numFunctions))
+            .log(LogTag.FUNCTION_LOAD);
+  }
+
+  @Override
+  public void endLog(String lastReplId) {
+    (new BootstrapLoadEnd(dbName, numTables, numFunctions, dumpDir, lastReplId))
+            .log(LogTag.END);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/70cb7f0b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/IncrementalLoadLogger.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/IncrementalLoadLogger.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/IncrementalLoadLogger.java
new file mode 100644
index 0000000..d2114f0
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/IncrementalLoadLogger.java
@@ -0,0 +1,55 @@
+/**
+ * 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.repl.load.log;
+
+import org.apache.hadoop.hive.ql.parse.repl.load.log.state.IncrementalLoadBegin;
+import org.apache.hadoop.hive.ql.parse.repl.load.log.state.IncrementalLoadEnd;
+import org.apache.hadoop.hive.ql.parse.repl.load.log.state.IncrementalLoadEvent;
+import org.apache.hadoop.hive.ql.parse.repl.ReplLogger;
+import org.apache.hadoop.hive.ql.parse.repl.ReplState.LogTag;
+
+public class IncrementalLoadLogger extends ReplLogger {
+  private String dbName;
+  private String dumpDir;
+  private long numEvents;
+  private long eventSeqNo;
+
+  public IncrementalLoadLogger(String dbName, String dumpDir, int numEvents) {
+    this.dbName = dbName;
+    this.dumpDir = dumpDir;
+    this.numEvents = numEvents;
+    this.eventSeqNo = 0;
+  }
+
+  @Override
+  public void startLog() {
+    (new IncrementalLoadBegin(dbName, dumpDir, numEvents)).log(LogTag.START);
+  }
+
+  @Override
+  public void eventLog(String eventId, String eventType) {
+    eventSeqNo++;
+    (new IncrementalLoadEvent(dbName, eventId, eventType, eventSeqNo, numEvents))
+            .log(LogTag.EVENT_LOAD);
+  }
+
+  @Override
+  public void endLog(String lastReplId) {
+    (new IncrementalLoadEnd(dbName, numEvents, dumpDir, lastReplId)).log(LogTag.END);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/70cb7f0b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/BootstrapLoadBegin.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/BootstrapLoadBegin.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/BootstrapLoadBegin.java
new file mode 100644
index 0000000..5574454
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/BootstrapLoadBegin.java
@@ -0,0 +1,51 @@
+/**
+ * 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.repl.load.log.state;
+
+import org.apache.hadoop.hive.ql.parse.repl.ReplState;
+import org.apache.hadoop.hive.ql.parse.repl.DumpType;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+public class BootstrapLoadBegin extends ReplState {
+  @JsonProperty
+  private String dbName;
+
+  @JsonProperty
+  private String dumpDir;
+
+  @JsonProperty
+  private DumpType loadType;
+
+  @JsonProperty
+  private Long numTables;
+
+  @JsonProperty
+  private Long numFunctions;
+
+  @JsonProperty
+  private Long loadStartTime;
+
+  public BootstrapLoadBegin(String dbName, String dumpDir, long numTables, long numFunctions) {
+    this.dbName = dbName;
+    this.dumpDir = dumpDir;
+    this.loadType = DumpType.BOOTSTRAP;
+    this.numTables = numTables;
+    this.numFunctions = numFunctions;
+    this.loadStartTime = System.currentTimeMillis() / 1000;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/70cb7f0b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/BootstrapLoadEnd.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/BootstrapLoadEnd.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/BootstrapLoadEnd.java
new file mode 100644
index 0000000..3db8144
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/BootstrapLoadEnd.java
@@ -0,0 +1,59 @@
+/**
+ * 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.repl.load.log.state;
+
+import org.apache.hadoop.hive.ql.parse.repl.ReplState;
+import org.apache.hadoop.hive.ql.parse.repl.DumpType;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+public class BootstrapLoadEnd extends ReplState {
+  @JsonProperty
+  private String dbName;
+
+  @JsonProperty
+  private DumpType loadType;
+
+  @JsonProperty
+  private Long numTables;
+
+  @JsonProperty
+  private Long numFunctions;
+
+  @JsonProperty
+  private Long loadEndTime;
+
+  @JsonProperty
+  private String dumpDir;
+
+  @JsonProperty
+  private String lastReplId;
+
+  public BootstrapLoadEnd(String dbName,
+                          long numTables,
+                          long numFunctions,
+                          String dumpDir,
+                          String lastReplId) {
+    this.dbName = dbName;
+    this.loadType = DumpType.BOOTSTRAP;
+    this.numTables = numTables;
+    this.numFunctions = numFunctions;
+    this.loadEndTime = System.currentTimeMillis() / 1000;
+    this.dumpDir = dumpDir;
+    this.lastReplId = lastReplId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/70cb7f0b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/BootstrapLoadFunction.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/BootstrapLoadFunction.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/BootstrapLoadFunction.java
new file mode 100644
index 0000000..e2434ce
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/BootstrapLoadFunction.java
@@ -0,0 +1,44 @@
+/**
+ * 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.repl.load.log.state;
+
+import org.apache.hadoop.hive.ql.parse.repl.ReplState;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+public class BootstrapLoadFunction extends ReplState {
+  @JsonProperty
+  private String dbName;
+
+  @JsonProperty
+  private String functionName;
+
+  @JsonProperty
+  private String functionsLoadProgress;
+
+  @JsonProperty
+  private Long loadTime;
+
+  public BootstrapLoadFunction(String dbName, String funcName,
+                               long functionSeqNo, long numFunctions) {
+    this.dbName = dbName;
+    this.functionName = funcName;
+    this.functionsLoadProgress = new String(new StringBuilder()
+                                          .append(functionSeqNo).append("/").append(numFunctions));
+    this.loadTime = System.currentTimeMillis() / 1000;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/70cb7f0b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/BootstrapLoadTable.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/BootstrapLoadTable.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/BootstrapLoadTable.java
new file mode 100644
index 0000000..63ff467
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/BootstrapLoadTable.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.parse.repl.load.log.state;
+
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.ql.parse.repl.ReplState;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+public class BootstrapLoadTable extends ReplState {
+  @JsonProperty
+  private String dbName;
+
+  @JsonProperty
+  private String tableName;
+
+  @JsonProperty
+  private String tableType;
+
+  @JsonProperty
+  private String tablesLoadProgress;
+
+  @JsonProperty
+  private Long loadTime;
+
+  public BootstrapLoadTable(String dbName,
+                            String tableName,
+                            TableType tableType,
+                            long tableSeqNo,
+                            long numTables) {
+    this.dbName = dbName;
+    this.tableName = tableName;
+    this.tableType = tableType.name();
+    this.tablesLoadProgress = new String(new StringBuilder()
+                                          .append(tableSeqNo).append("/").append(numTables));
+    this.loadTime = System.currentTimeMillis() / 1000;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/70cb7f0b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/IncrementalLoadBegin.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/IncrementalLoadBegin.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/IncrementalLoadBegin.java
new file mode 100644
index 0000000..93f1d7b
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/IncrementalLoadBegin.java
@@ -0,0 +1,47 @@
+/**
+ * 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.repl.load.log.state;
+
+import org.apache.hadoop.hive.ql.parse.repl.ReplState;
+import org.apache.hadoop.hive.ql.parse.repl.DumpType;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+public class IncrementalLoadBegin extends ReplState {
+  @JsonProperty
+  private String dbName;
+
+  @JsonProperty
+  private String dumpDir;
+
+  @JsonProperty
+  private DumpType loadType;
+
+  @JsonProperty
+  private Long numEvents;
+
+  @JsonProperty
+  private Long loadStartTime;
+
+  public IncrementalLoadBegin(String dbName, String dumpDir, long numEvents) {
+    this.dbName = dbName;
+    this.dumpDir = dumpDir;
+    this.loadType = DumpType.INCREMENTAL;
+    this.numEvents = numEvents;
+    this.loadStartTime = System.currentTimeMillis() / 1000;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/70cb7f0b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/IncrementalLoadEnd.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/IncrementalLoadEnd.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/IncrementalLoadEnd.java
new file mode 100644
index 0000000..f633c72
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/IncrementalLoadEnd.java
@@ -0,0 +1,54 @@
+/**
+ * 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.repl.load.log.state;
+
+import org.apache.hadoop.hive.ql.parse.repl.ReplState;
+import org.apache.hadoop.hive.ql.parse.repl.DumpType;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+public class IncrementalLoadEnd extends ReplState {
+  @JsonProperty
+  private String dbName;
+
+  @JsonProperty
+  private DumpType loadType;
+
+  @JsonProperty
+  private Long numEvents;
+
+  @JsonProperty
+  private Long loadEndTime;
+
+  @JsonProperty
+  private String dumpDir;
+
+  @JsonProperty
+  private String lastReplId;
+
+  public IncrementalLoadEnd(String dbName,
+                            long numEvents,
+                            String dumpDir,
+                            String lastReplId) {
+    this.dbName = dbName;
+    this.loadType = DumpType.INCREMENTAL;
+    this.numEvents = numEvents;
+    this.loadEndTime = System.currentTimeMillis() / 1000;
+    this.dumpDir = dumpDir;
+    this.lastReplId = lastReplId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/70cb7f0b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/IncrementalLoadEvent.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/IncrementalLoadEvent.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/IncrementalLoadEvent.java
new file mode 100644
index 0000000..224aeb2
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/log/state/IncrementalLoadEvent.java
@@ -0,0 +1,51 @@
+/**
+ * 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.repl.load.log.state;
+
+import org.apache.hadoop.hive.ql.parse.repl.ReplState;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+public class IncrementalLoadEvent extends ReplState {
+  @JsonProperty
+  private String dbName;
+
+  @JsonProperty
+  private String eventId;
+
+  @JsonProperty
+  private String eventType;
+
+  @JsonProperty
+  private String eventsLoadProgress;
+
+  @JsonProperty
+  private Long loadTime;
+
+  public IncrementalLoadEvent(String dbName,
+                              String eventId,
+                              String eventType,
+                              long eventSeqNo,
+                              long numEvents) {
+    this.dbName = dbName;
+    this.eventId = eventId;
+    this.eventType = eventType;
+    this.eventsLoadProgress = new String(new StringBuilder()
+                                            .append(eventSeqNo).append("/").append(numEvents));
+    this.loadTime = System.currentTimeMillis() / 1000;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/70cb7f0b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CreateFunctionHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CreateFunctionHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CreateFunctionHandler.java
index 3f176aa..caf6f3f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CreateFunctionHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CreateFunctionHandler.java
@@ -49,12 +49,20 @@ import java.util.List;
 import static org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.toReadEntity;
 
 public class CreateFunctionHandler extends AbstractMessageHandler {
+  private String functionName;
+
+  public String getFunctionName() {
+    return functionName;
+  }
+
   @Override
   public List<Task<? extends Serializable>> handle(Context context)
       throws SemanticException {
     try {
       FunctionDescBuilder builder = new FunctionDescBuilder(context);
       CreateFunctionDesc descToLoad = builder.build();
+      this.functionName = builder.metadata.function.getFunctionName();
+
       context.log.debug("Loading function desc : {}", descToLoad.toString());
       Task<FunctionWork> createTask = TaskFactory.get(
           new FunctionWork(descToLoad), context.hiveConf

http://git-wip-us.apache.org/repos/asf/hive/blob/70cb7f0b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/MessageHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/MessageHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/MessageHandler.java
index 8daff6d..3ccd639 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/MessageHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/MessageHandler.java
@@ -45,9 +45,10 @@ public interface MessageHandler {
   UpdatedMetaDataTracker getUpdatedMetadata();
 
   class Context {
-    final String dbName, tableName, location;
-    final Task<? extends Serializable> precursor;
-    DumpMetaData dmd;
+    public String dbName;
+    public final String tableName, location;
+    public final Task<? extends Serializable> precursor;
+    public DumpMetaData dmd;
     final HiveConf hiveConf;
     final Hive db;
     final org.apache.hadoop.hive.ql.Context nestedContext;
@@ -83,7 +84,7 @@ public interface MessageHandler {
       return StringUtils.isEmpty(tableName);
     }
 
-    boolean isDbNameEmpty() {
+    public boolean isDbNameEmpty() {
       return StringUtils.isEmpty(dbName);
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/70cb7f0b/ql/src/java/org/apache/hadoop/hive/ql/plan/ImportTableDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ImportTableDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ImportTableDesc.java
index a4fb378..1770046 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ImportTableDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ImportTableDesc.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.ql.exec.Task;
@@ -34,11 +35,8 @@ import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
-import org.apache.hadoop.hive.ql.parse.EximUtil;
 import org.apache.hadoop.hive.ql.parse.ReplicationSpec;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
-import org.apache.hadoop.hive.ql.plan.CreateTableDesc;
-import org.apache.hadoop.hive.ql.plan.CreateViewDesc;
 
 /**
  * ImportTableDesc.
@@ -56,7 +54,7 @@ public class ImportTableDesc {
     this.dbName = dbName;
     this.table = table;
 
-    switch (getTableType()) {
+    switch (getDescType()) {
       case TABLE:
         this.createTblDesc = new CreateTableDesc(dbName,
                 table.getTableName(),
@@ -122,7 +120,7 @@ public class ImportTableDesc {
     }
   }
 
-  public TYPE getTableType() {
+  public TYPE getDescType() {
     if (table.isView() || table.isMaterializedView()) {
       return TYPE.VIEW;
     }
@@ -143,7 +141,7 @@ public class ImportTableDesc {
   }
 
   public void setReplicationSpec(ReplicationSpec replSpec) {
-    switch (getTableType()) {
+    switch (getDescType()) {
       case TABLE:
         createTblDesc.setReplicationSpec(replSpec);
         break;
@@ -154,20 +152,20 @@ public class ImportTableDesc {
   }
 
   public void setExternal(boolean isExternal) {
-    if (TYPE.TABLE.equals(getTableType())) {
+    if (TYPE.TABLE.equals(getDescType())) {
       createTblDesc.setExternal(isExternal);
     }
   }
 
   public boolean isExternal() {
-    if (TYPE.TABLE.equals(getTableType())) {
+    if (TYPE.TABLE.equals(getDescType())) {
       return createTblDesc.isExternal();
     }
     return false;
   }
 
   public void setLocation(String location) {
-    switch (getTableType()) {
+    switch (getDescType()) {
       case TABLE:
         createTblDesc.setLocation(location);
         break;
@@ -178,7 +176,7 @@ public class ImportTableDesc {
   }
 
   public String getLocation() {
-    switch (getTableType()) {
+    switch (getDescType()) {
       case TABLE:
         return createTblDesc.getLocation();
       case VIEW:
@@ -188,7 +186,7 @@ public class ImportTableDesc {
   }
 
   public void setTableName(String tableName) throws SemanticException {
-    switch (getTableType()) {
+    switch (getDescType()) {
       case TABLE:
         createTblDesc.setTableName(tableName);
         break;
@@ -201,7 +199,7 @@ public class ImportTableDesc {
   }
 
   public String getTableName() throws SemanticException {
-    switch (getTableType()) {
+    switch (getDescType()) {
       case TABLE:
         return createTblDesc.getTableName();
       case VIEW:
@@ -213,7 +211,7 @@ public class ImportTableDesc {
   }
 
   public List<FieldSchema> getPartCols() {
-    switch (getTableType()) {
+    switch (getDescType()) {
       case TABLE:
         return createTblDesc.getPartCols();
       case VIEW:
@@ -223,7 +221,7 @@ public class ImportTableDesc {
   }
 
   public List<FieldSchema> getCols() {
-    switch (getTableType()) {
+    switch (getDescType()) {
       case TABLE:
         return createTblDesc.getCols();
       case VIEW:
@@ -233,7 +231,7 @@ public class ImportTableDesc {
   }
 
   public Map<String, String> getTblProps() {
-    switch (getTableType()) {
+    switch (getDescType()) {
       case TABLE:
         return createTblDesc.getTblProps();
       case VIEW:
@@ -243,7 +241,7 @@ public class ImportTableDesc {
   }
 
   public String getInputFormat() {
-    switch (getTableType()) {
+    switch (getDescType()) {
       case TABLE:
         return createTblDesc.getInputFormat();
       case VIEW:
@@ -253,7 +251,7 @@ public class ImportTableDesc {
   }
 
   public String getOutputFormat() {
-    switch (getTableType()) {
+    switch (getDescType()) {
       case TABLE:
         return createTblDesc.getOutputFormat();
       case VIEW:
@@ -263,7 +261,7 @@ public class ImportTableDesc {
   }
 
   public String getSerName() {
-    switch (getTableType()) {
+    switch (getDescType()) {
       case TABLE:
         return createTblDesc.getSerName();
       case VIEW:
@@ -273,7 +271,7 @@ public class ImportTableDesc {
   }
 
   public Map<String, String> getSerdeProps() {
-    switch (getTableType()) {
+    switch (getDescType()) {
       case TABLE:
         return createTblDesc.getSerdeProps();
       case VIEW:
@@ -283,14 +281,14 @@ public class ImportTableDesc {
   }
 
   public List<String> getBucketCols() {
-    if (TYPE.TABLE.equals(getTableType())) {
+    if (TYPE.TABLE.equals(getDescType())) {
       return createTblDesc.getBucketCols();
     }
     return null;
   }
 
   public List<Order> getSortCols() {
-    if (TYPE.TABLE.equals(getTableType())) {
+    if (TYPE.TABLE.equals(getDescType())) {
       return createTblDesc.getSortCols();
     }
     return null;
@@ -300,7 +298,7 @@ public class ImportTableDesc {
    * @param replaceMode Determine if this CreateTable should behave like a replace-into alter instead
    */
   public void setReplaceMode(boolean replaceMode) {
-    switch (getTableType()) {
+    switch (getDescType()) {
       case TABLE:
         createTblDesc.setReplaceMode(replaceMode);
         break;
@@ -315,7 +313,7 @@ public class ImportTableDesc {
 
   public Task<? extends Serializable> getCreateTableTask(HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs,
       HiveConf conf) {
-    switch (getTableType()) {
+    switch (getDescType()) {
       case TABLE:
         return TaskFactory.get(new DDLWork(inputs, outputs, createTblDesc), conf);
       case VIEW:
@@ -332,4 +330,13 @@ public class ImportTableDesc {
   public boolean isMaterializedView() {
     return table.isMaterializedView();
   }
+
+  public TableType tableType() {
+    if (isView()) {
+      return TableType.VIRTUAL_VIEW;
+    } else if (isMaterializedView()) {
+      return TableType.MATERIALIZED_VIEW;
+    }
+    return TableType.MANAGED_TABLE;
+  }
 }