You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2018/04/28 17:47:41 UTC

[1/3] hive git commit: HIVE-19288 : Implement protobuf logging hive hook. (Harish JP via Ashutosh Chauhan)

Repository: hive
Updated Branches:
  refs/heads/master 3a0e4dd40 -> 8349dbde5


http://git-wip-us.apache.org/repos/asf/hive/blob/8349dbde/ql/src/java/org/apache/hadoop/hive/ql/hooks/DatePartitionedLogger.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/DatePartitionedLogger.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/DatePartitionedLogger.java
new file mode 100644
index 0000000..c9d1b93
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/DatePartitionedLogger.java
@@ -0,0 +1,167 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.hooks;
+
+import java.io.IOException;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.yarn.util.Clock;
+
+import com.google.protobuf.MessageLite;
+import com.google.protobuf.Parser;
+
+/**
+ * Class to create proto reader and writer for a date partitioned directory structure.
+ *
+ * @param <T> The proto message type.
+ */
+public class DatePartitionedLogger<T extends MessageLite> {
+  // Everyone has permission to write, but with sticky set so that delete is restricted.
+  // This is required, since the path is same for all users and everyone writes into it.
+  private static final FsPermission DIR_PERMISSION = FsPermission.createImmutable((short)01777);
+
+  private final Parser<T> parser;
+  private final Path basePath;
+  private final Configuration conf;
+  private final Clock clock;
+  private final FileSystem fileSystem;
+
+  public DatePartitionedLogger(Parser<T> parser, Path baseDir, Configuration conf, Clock clock)
+      throws IOException {
+    this.conf = conf;
+    this.clock = clock;
+    this.parser = parser;
+    this.fileSystem = baseDir.getFileSystem(conf);
+    if (!fileSystem.exists(baseDir)) {
+      fileSystem.mkdirs(baseDir);
+      fileSystem.setPermission(baseDir, DIR_PERMISSION);
+    }
+    this.basePath = fileSystem.resolvePath(baseDir);
+  }
+
+  /**
+   * Creates a writer for the given fileName, with date as today.
+   */
+  public ProtoMessageWriter<T> getWriter(String fileName) throws IOException {
+    Path filePath = getPathForDate(getNow().toLocalDate(), fileName);
+    return new ProtoMessageWriter<>(conf, filePath, parser);
+  }
+
+  /**
+   * Creates a reader for the given filePath, no validation is done.
+   */
+  public ProtoMessageReader<T> getReader(Path filePath) throws IOException {
+    return new ProtoMessageReader<>(conf, filePath, parser);
+  }
+
+  /**
+   * Create a path for the given date and fileName. This can be used to create a reader.
+   */
+  public Path getPathForDate(LocalDate date, String fileName) throws IOException {
+    Path path = new Path(basePath, getDirForDate(date));
+    if (!fileSystem.exists(path)) {
+      fileSystem.mkdirs(path);
+      fileSystem.setPermission(path, DIR_PERMISSION);
+    }
+    return new Path(path, fileName);
+  }
+
+  /**
+   * Extract the date from the directory name, this should be a directory created by this class.
+   */
+  public LocalDate getDateFromDir(String dirName) {
+    if (!dirName.startsWith("date=")) {
+      throw new IllegalArgumentException("Invalid directory: "+ dirName);
+    }
+    return LocalDate.parse(dirName.substring(5), DateTimeFormatter.ISO_LOCAL_DATE);
+  }
+
+  /**
+   * Returns the directory name for a given date.
+   */
+  public String getDirForDate(LocalDate date) {
+    return "date=" + DateTimeFormatter.ISO_LOCAL_DATE.format(date);
+  }
+
+  /**
+   * Find next available directory, after the given directory.
+   */
+  public String getNextDirectory(String currentDir) throws IOException {
+    // Fast check, if the next day directory exists return it.
+    String nextDate = getDirForDate(getDateFromDir(currentDir).plusDays(1));
+    if (fileSystem.exists(new Path(basePath, nextDate))) {
+      return nextDate;
+    }
+    // Have to scan the directory to find min date greater than currentDir.
+    String dirName = null;
+    for (FileStatus status : fileSystem.listStatus(basePath)) {
+      String name = status.getPath().getName();
+      // String comparison is good enough, since its of form date=yyyy-MM-dd
+      if (name.compareTo(currentDir) > 0 && (dirName == null || name.compareTo(dirName) < 0)) {
+        dirName = name;
+      }
+    }
+    return dirName;
+  }
+
+  /**
+   * Returns new or changed files in the given directory. The offsets are used to find
+   * changed files.
+   */
+  public List<Path> scanForChangedFiles(String subDir, Map<String, Long> currentOffsets)
+      throws IOException {
+    Path dirPath = new Path(basePath, subDir);
+    List<Path> newFiles = new ArrayList<>();
+    if (!fileSystem.exists(dirPath)) {
+      return newFiles;
+    }
+    for (FileStatus status : fileSystem.listStatus(dirPath)) {
+      String fileName = status.getPath().getName();
+      Long offset = currentOffsets.get(fileName);
+      // If the offset was never added or offset < fileSize.
+      if (offset == null || offset < status.getLen()) {
+        newFiles.add(new Path(dirPath, fileName));
+      }
+    }
+    return newFiles;
+  }
+
+  /**
+   * Returns the current time, using the underlying clock in UTC time.
+   */
+  public LocalDateTime getNow() {
+    // Use UTC date to ensure reader date is same on all timezones.
+    return LocalDateTime.ofEpochSecond(clock.getTime() / 1000, 0, ZoneOffset.UTC);
+  }
+
+  public Configuration getConfig() {
+    return conf;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/8349dbde/ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java
new file mode 100644
index 0000000..1ae8194
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java
@@ -0,0 +1,493 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.hooks;
+
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERDATABASE;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERDATABASE_OWNER;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERPARTITION_BUCKETNUM;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERPARTITION_FILEFORMAT;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERPARTITION_LOCATION;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERPARTITION_MERGEFILES;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERPARTITION_SERDEPROPERTIES;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERPARTITION_SERIALIZER;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_ADDCOLS;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_ADDCONSTRAINT;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_ADDPARTS;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_ARCHIVE;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_BUCKETNUM;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_CLUSTER_SORT;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_COMPACT;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_DROPCONSTRAINT;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_DROPPARTS;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_EXCHANGEPARTITION;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_FILEFORMAT;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_LOCATION;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_MERGEFILES;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_PARTCOLTYPE;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_PROPERTIES;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_RENAME;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_RENAMECOL;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_RENAMEPART;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_REPLACECOLS;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_SERDEPROPERTIES;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_SERIALIZER;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_SKEWED;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_TOUCH;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_UNARCHIVE;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_UPDATEPARTSTATS;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTABLE_UPDATETABLESTATS;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERTBLPART_SKEWED_LOCATION;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERVIEW_AS;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERVIEW_PROPERTIES;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ALTERVIEW_RENAME;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.ANALYZE_TABLE;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.CACHE_METADATA;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.CREATEDATABASE;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.CREATEFUNCTION;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.CREATEMACRO;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.CREATEROLE;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.CREATETABLE;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.CREATETABLE_AS_SELECT;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.CREATEVIEW;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.DROPDATABASE;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.DROPFUNCTION;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.DROPMACRO;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.DROPROLE;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.DROPTABLE;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.DROPVIEW;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.DROPVIEW_PROPERTIES;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.EXPORT;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.IMPORT;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.KILL_QUERY;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.LOAD;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.LOCKTABLE;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.MSCK;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.QUERY;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.RELOADFUNCTION;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.TRUNCATETABLE;
+import static org.apache.hadoop.hive.ql.plan.HiveOperation.UNLOCKTABLE;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.llap.registry.impl.LlapRegistryService;
+import org.apache.hadoop.hive.ql.QueryPlan;
+import org.apache.hadoop.hive.ql.exec.ExplainTask;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.exec.mr.ExecDriver;
+import org.apache.hadoop.hive.ql.exec.tez.TezTask;
+import org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto;
+import org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry;
+import org.apache.hadoop.hive.ql.parse.ExplainConfiguration;
+import org.apache.hadoop.hive.ql.plan.ExplainWork;
+import org.apache.hadoop.hive.ql.plan.HiveOperation;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.hive.common.util.ShutdownHookManager;
+import org.json.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * Log events from hive hook using protobuf serialized format, partitioned by date.
+ */
+public class HiveProtoLoggingHook implements ExecuteWithHookContext {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveProtoLoggingHook.class.getName());
+  private static final Set<String> includedOperationSet;
+  private static final int VERSION = 1;
+
+  static {
+    // List of operation for which we log.
+    includedOperationSet = Arrays.stream(new HiveOperation[] { LOAD, EXPORT, IMPORT,
+        CREATEDATABASE, DROPDATABASE, DROPTABLE, MSCK, ALTERTABLE_ADDCOLS, ALTERTABLE_REPLACECOLS,
+        ALTERTABLE_RENAMECOL, ALTERTABLE_RENAMEPART, ALTERTABLE_UPDATEPARTSTATS,
+        ALTERTABLE_UPDATETABLESTATS, ALTERTABLE_RENAME, ALTERTABLE_DROPPARTS, ALTERTABLE_ADDPARTS,
+        ALTERTABLE_TOUCH, ALTERTABLE_ARCHIVE, ALTERTABLE_UNARCHIVE, ALTERTABLE_PROPERTIES,
+        ALTERTABLE_SERIALIZER, ALTERPARTITION_SERIALIZER, ALTERTABLE_SERDEPROPERTIES,
+        ALTERPARTITION_SERDEPROPERTIES, ALTERTABLE_CLUSTER_SORT, ANALYZE_TABLE, CACHE_METADATA,
+        ALTERTABLE_BUCKETNUM, ALTERPARTITION_BUCKETNUM, CREATEFUNCTION, DROPFUNCTION,
+        RELOADFUNCTION, CREATEMACRO, DROPMACRO, CREATEVIEW, DROPVIEW, ALTERVIEW_PROPERTIES,
+        DROPVIEW_PROPERTIES, LOCKTABLE, UNLOCKTABLE, CREATEROLE, DROPROLE, ALTERTABLE_FILEFORMAT,
+        ALTERPARTITION_FILEFORMAT, ALTERTABLE_LOCATION, ALTERPARTITION_LOCATION, CREATETABLE,
+        TRUNCATETABLE, CREATETABLE_AS_SELECT, QUERY, ALTERDATABASE, ALTERDATABASE_OWNER,
+        ALTERTABLE_MERGEFILES, ALTERPARTITION_MERGEFILES, ALTERTABLE_SKEWED,
+        ALTERTBLPART_SKEWED_LOCATION, ALTERTABLE_PARTCOLTYPE, ALTERTABLE_EXCHANGEPARTITION,
+        ALTERTABLE_DROPCONSTRAINT, ALTERTABLE_ADDCONSTRAINT, ALTERVIEW_RENAME, ALTERVIEW_AS,
+        ALTERTABLE_COMPACT, KILL_QUERY })
+            .map(HiveOperation::getOperationName)
+            .collect(Collectors.toSet());
+  }
+
+  public static final String HIVE_EVENTS_BASE_PATH = "hive.hook.proto.base-directory";
+  public static final String HIVE_HOOK_PROTO_QUEUE_CAPACITY = "hive.hook.proto.queue.capacity";
+  public static final int HIVE_HOOK_PROTO_QUEUE_CAPACITY_DEFAULT = 64;
+  private static final int WAIT_TIME = 5;
+
+  public enum EventType {
+    QUERY_SUBMITTED, QUERY_COMPLETED
+  }
+
+  public enum OtherInfoType {
+    QUERY, STATUS, TEZ, MAPRED, INVOKER_INFO, SESSION_ID, THREAD_NAME, VERSION, CLIENT_IP_ADDRESS,
+    HIVE_ADDRESS, HIVE_INSTANCE_TYPE, CONF, PERF, LLAP_APP_ID
+  }
+
+  public enum ExecutionMode {
+    MR, TEZ, LLAP, SPARK, NONE
+  }
+
+  static class EventLogger {
+    private final Clock clock;
+    private final String logFileName;
+    private final DatePartitionedLogger<HiveHookEventProto> logger;
+    private final ExecutorService eventHandler;
+    private final ExecutorService logWriter;
+
+    EventLogger(HiveConf conf, Clock clock) {
+      this.clock = clock;
+      // randomUUID is slow, since its cryptographically secure, only first query will take time.
+      this.logFileName = "hive_" + UUID.randomUUID().toString();
+      String baseDir = conf.get(HIVE_EVENTS_BASE_PATH);
+      if (baseDir == null) {
+        LOG.error(HIVE_EVENTS_BASE_PATH + " is not set, logging disabled.");
+      }
+
+      DatePartitionedLogger<HiveHookEventProto> tmpLogger = null;
+      try {
+        if (baseDir != null) {
+          tmpLogger = new DatePartitionedLogger<>(HiveHookEventProto.PARSER, new Path(baseDir),
+              conf, clock);
+        }
+      } catch (IOException e) {
+        LOG.error("Unable to intialize logger, logging disabled.", e);
+      }
+      this.logger = tmpLogger;
+      if (logger == null) {
+        eventHandler = null;
+        logWriter = null;
+        return;
+      }
+
+      int queueCapacity = conf.getInt(HIVE_HOOK_PROTO_QUEUE_CAPACITY,
+          HIVE_HOOK_PROTO_QUEUE_CAPACITY_DEFAULT);
+
+      ThreadFactory threadFactory = new ThreadFactoryBuilder().setDaemon(true)
+          .setNameFormat("Hive Hook Proto Event Handler %d").build();
+      eventHandler = new ThreadPoolExecutor(1, 1, 0, TimeUnit.MILLISECONDS,
+          new LinkedBlockingQueue<Runnable>(queueCapacity), threadFactory);
+
+      threadFactory = new ThreadFactoryBuilder().setDaemon(true)
+          .setNameFormat("Hive Hook Proto Log Writer %d").build();
+      logWriter = new ThreadPoolExecutor(1, 1, 0, TimeUnit.MILLISECONDS,
+          new LinkedBlockingQueue<Runnable>(queueCapacity), threadFactory);
+    }
+
+    void shutdown() {
+      // Wait for all the events to be written off, the order of service is important
+      for (ExecutorService service : new ExecutorService[] {eventHandler, logWriter}) {
+        if (service == null) {
+          continue;
+        }
+        service.shutdown();
+        try {
+          service.awaitTermination(WAIT_TIME, TimeUnit.SECONDS);
+        } catch (InterruptedException e) {
+          LOG.warn("Got interrupted exception while waiting for events to be flushed", e);
+        }
+      }
+    }
+
+    void handle(HookContext hookContext) {
+      if (logger == null) {
+        return;
+      }
+      try {
+        eventHandler.execute(() -> generateEvent(hookContext));
+      } catch (RejectedExecutionException e) {
+        LOG.warn("Handler queue full ignoring event: " + hookContext.getHookType());
+      }
+    }
+
+    private void generateEvent(HookContext hookContext) {
+      QueryPlan plan = hookContext.getQueryPlan();
+      if (plan == null) {
+        LOG.debug("Received null query plan.");
+        return;
+      }
+      if (!includedOperationSet.contains(plan.getOperationName())) {
+        LOG.debug("Not logging events of operation type : {}", plan.getOperationName());
+        return;
+      }
+      HiveHookEventProto event;
+      switch (hookContext.getHookType()) {
+      case PRE_EXEC_HOOK:
+        event = getPreHookEvent(hookContext);
+        break;
+      case POST_EXEC_HOOK:
+        event = getPostHookEvent(hookContext, true);
+        break;
+      case ON_FAILURE_HOOK:
+        event = getPostHookEvent(hookContext, false);
+        break;
+      default:
+        LOG.warn("Ignoring event of type: {}", hookContext.getHookType());
+        event = null;
+      }
+      if (event != null) {
+        try {
+          logWriter.execute(() -> writeEvent(event));
+        } catch (RejectedExecutionException e) {
+          LOG.warn("Writer queue full ignoring event {} for query {}",
+              hookContext.getHookType(), plan.getQueryId());
+        }
+      }
+    }
+
+    private void writeEvent(HiveHookEventProto event) {
+      try (ProtoMessageWriter<HiveHookEventProto> writer = logger.getWriter(logFileName)) {
+        writer.writeProto(event);
+        // This does not work hence, opening and closing file for every event.
+        // writer.hflush();
+      } catch (IOException e) {
+        LOG.error("Error writing proto message for query {}, eventType: {}: ",
+            event.getHiveQueryId(), event.getEventType(), e);
+      }
+    }
+
+    private HiveHookEventProto getPreHookEvent(HookContext hookContext) {
+      QueryPlan plan = hookContext.getQueryPlan();
+      LOG.info("Received pre-hook notification for: " + plan.getQueryId());
+
+      // Make a copy so that we do not modify hookContext conf.
+      HiveConf conf = new HiveConf(hookContext.getConf());
+      List<ExecDriver> mrTasks = Utilities.getMRTasks(plan.getRootTasks());
+      List<TezTask> tezTasks = Utilities.getTezTasks(plan.getRootTasks());
+      ExecutionMode executionMode = getExecutionMode(plan, mrTasks, tezTasks);
+
+      HiveHookEventProto.Builder builder = HiveHookEventProto.newBuilder();
+      builder.setEventType(EventType.QUERY_SUBMITTED.name());
+      builder.setTimestamp(plan.getQueryStartTime());
+      builder.setHiveQueryId(plan.getQueryId());
+      builder.setUser(getUser(hookContext));
+      builder.setRequestUser(getRequestUser(hookContext));
+      builder.setQueue(conf.get("mapreduce.job.queuename"));
+      builder.setExecutionMode(executionMode.name());
+      builder.addAllTablesRead(getTablesFromEntitySet(hookContext.getInputs()));
+      builder.addAllTablesWritten(getTablesFromEntitySet(hookContext.getOutputs()));
+      if (hookContext.getOperationId() != null) {
+        builder.setOperationId(hookContext.getOperationId());
+      }
+
+      try {
+        JSONObject queryObj = new JSONObject();
+        queryObj.put("queryText", plan.getQueryStr());
+        queryObj.put("queryPlan", getExplainPlan(plan, conf, hookContext));
+        addMapEntry(builder, OtherInfoType.QUERY, queryObj.toString());
+      } catch (Exception e) {
+        LOG.error("Unexpected exception while serializing json.", e);
+      }
+
+      addMapEntry(builder, OtherInfoType.TEZ, Boolean.toString(tezTasks.size() > 0));
+      addMapEntry(builder, OtherInfoType.MAPRED, Boolean.toString(mrTasks.size() > 0));
+      addMapEntry(builder, OtherInfoType.SESSION_ID, hookContext.getSessionId());
+      String logID = conf.getLogIdVar(hookContext.getSessionId());
+      addMapEntry(builder, OtherInfoType.INVOKER_INFO, logID);
+      addMapEntry(builder, OtherInfoType.THREAD_NAME, hookContext.getThreadId());
+      addMapEntry(builder, OtherInfoType.VERSION, Integer.toString(VERSION));
+      addMapEntry(builder, OtherInfoType.CLIENT_IP_ADDRESS, hookContext.getIpAddress());
+
+      String hiveInstanceAddress = hookContext.getHiveInstanceAddress();
+      if (hiveInstanceAddress == null) {
+        try {
+          hiveInstanceAddress = InetAddress.getLocalHost().getHostAddress();
+        } catch (UnknownHostException e) {
+          LOG.error("Error tyring to get localhost address: ", e);
+        }
+      }
+      addMapEntry(builder, OtherInfoType.HIVE_ADDRESS, hiveInstanceAddress);
+
+      String hiveInstanceType = hookContext.isHiveServerQuery() ? "HS2" : "CLI";
+      addMapEntry(builder, OtherInfoType.HIVE_INSTANCE_TYPE, hiveInstanceType);
+
+      ApplicationId llapId = determineLlapId(conf, executionMode);
+      if (llapId != null) {
+        addMapEntry(builder, OtherInfoType.LLAP_APP_ID, llapId.toString());
+      }
+
+      conf.stripHiddenConfigurations(conf);
+      JSONObject confObj = new JSONObject();
+      for (Map.Entry<String, String> setting : conf) {
+        confObj.put(setting.getKey(), setting.getValue());
+      }
+      addMapEntry(builder, OtherInfoType.CONF, confObj.toString());
+      return builder.build();
+    }
+
+    private HiveHookEventProto getPostHookEvent(HookContext hookContext, boolean success) {
+      QueryPlan plan = hookContext.getQueryPlan();
+      LOG.info("Received post-hook notification for: " + plan.getQueryId());
+
+      HiveHookEventProto.Builder builder = HiveHookEventProto.newBuilder();
+      builder.setEventType(EventType.QUERY_COMPLETED.name());
+      builder.setTimestamp(clock.getTime());
+      builder.setHiveQueryId(plan.getQueryId());
+      builder.setUser(getUser(hookContext));
+      builder.setRequestUser(getRequestUser(hookContext));
+      if (hookContext.getOperationId() != null) {
+        builder.setOperationId(hookContext.getOperationId());
+      }
+      addMapEntry(builder, OtherInfoType.STATUS, Boolean.toString(success));
+      JSONObject perfObj = new JSONObject(hookContext.getPerfLogger().getEndTimes());
+      addMapEntry(builder, OtherInfoType.PERF, perfObj.toString());
+
+      return builder.build();
+    }
+
+    private void addMapEntry(HiveHookEventProto.Builder builder, OtherInfoType key, String value) {
+      if (value != null) {
+        builder.addOtherInfo(
+            MapFieldEntry.newBuilder().setKey(key.name()).setValue(value).build());
+      }
+    }
+
+    private String getUser(HookContext hookContext) {
+      return hookContext.getUgi().getShortUserName();
+    }
+
+    private String getRequestUser(HookContext hookContext) {
+      String requestuser = hookContext.getUserName();
+      if (requestuser == null) {
+        requestuser = hookContext.getUgi().getUserName();
+      }
+      return requestuser;
+    }
+
+    private List<String> getTablesFromEntitySet(Set<? extends Entity> entities) {
+      List<String> tableNames = new ArrayList<>();
+      for (Entity entity : entities) {
+        if (entity.getType() == Entity.Type.TABLE) {
+          tableNames.add(entity.getTable().getDbName() + "." + entity.getTable().getTableName());
+        }
+      }
+      return tableNames;
+    }
+
+    private ExecutionMode getExecutionMode(QueryPlan plan, List<ExecDriver> mrTasks,
+        List<TezTask> tezTasks) {
+      if (tezTasks.size() > 0) {
+        // Need to go in and check if any of the tasks is running in LLAP mode.
+        for (TezTask tezTask : tezTasks) {
+          if (tezTask.getWork().getLlapMode()) {
+            return ExecutionMode.LLAP;
+          }
+        }
+        return ExecutionMode.TEZ;
+      } else if (mrTasks.size() > 0) {
+        return ExecutionMode.MR;
+      } else if (Utilities.getSparkTasks(plan.getRootTasks()).size() > 0) {
+        return ExecutionMode.SPARK;
+      } else {
+        return ExecutionMode.NONE;
+      }
+    }
+
+    private JSONObject getExplainPlan(QueryPlan plan, HiveConf conf, HookContext hookContext)
+        throws Exception {
+      // Get explain plan for the query.
+      ExplainConfiguration config = new ExplainConfiguration();
+      config.setFormatted(true);
+      ExplainWork work = new ExplainWork(null, // resFile
+          null, // pCtx
+          plan.getRootTasks(), // RootTasks
+          plan.getFetchTask(), // FetchTask
+          null, // analyzer
+          config, // explainConfig
+          null // cboInfo
+      );
+      ExplainTask explain = (ExplainTask) TaskFactory.get(work, conf);
+      explain.initialize(hookContext.getQueryState(), plan, null, null);
+      return explain.getJSONPlan(null, work);
+    }
+
+    private ApplicationId determineLlapId(HiveConf conf, ExecutionMode mode) {
+      // Note: for now, LLAP is only supported in Tez tasks. Will never come to MR; others may
+      // be added here, although this is only necessary to have extra debug information.
+      if (mode == ExecutionMode.LLAP) {
+        // In HS2, the client should have been cached already for the common case.
+        // Otherwise, this may actually introduce delay to compilation for the first query.
+        String hosts = HiveConf.getVar(conf, HiveConf.ConfVars.LLAP_DAEMON_SERVICE_HOSTS);
+        if (hosts != null && !hosts.isEmpty()) {
+          try {
+            return LlapRegistryService.getClient(conf).getApplicationId();
+          } catch (IOException e) {
+            LOG.error("Error trying to get llap instance", e);
+          }
+        } else {
+          LOG.info("Cannot determine LLAP instance on client - service hosts are not set");
+          return null;
+        }
+      }
+      return null;
+    }
+
+    // Singleton using DCL.
+    private static volatile EventLogger instance;
+    static EventLogger getInstance(HiveConf conf) {
+      if (instance == null) {
+        synchronized (EventLogger.class) {
+          if (instance == null) {
+            instance = new EventLogger(conf, SystemClock.getInstance());
+            ShutdownHookManager.addShutdownHook(instance::shutdown);
+          }
+        }
+      }
+      return instance;
+    }
+  }
+
+  @Override
+  public void run(HookContext hookContext) throws Exception {
+    try {
+      EventLogger logger = EventLogger.getInstance(hookContext.getConf());
+      logger.handle(hookContext);
+    } catch (Exception e) {
+      LOG.error("Got exceptoin while processing event: ", e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/8349dbde/ql/src/java/org/apache/hadoop/hive/ql/hooks/ProtoMessageReader.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/ProtoMessageReader.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/ProtoMessageReader.java
new file mode 100644
index 0000000..1c4296c
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/ProtoMessageReader.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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.hooks;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.SequenceFile;
+
+import com.google.protobuf.MessageLite;
+import com.google.protobuf.Parser;
+
+public class ProtoMessageReader<T extends MessageLite> implements Closeable {
+  private final Path filePath;
+  private final SequenceFile.Reader reader;
+  private final ProtoMessageWritable<T> writable;
+
+  ProtoMessageReader(Configuration conf, Path filePath, Parser<T> parser) throws IOException {
+    this.filePath = filePath;
+    this.reader = new SequenceFile.Reader(conf, SequenceFile.Reader.file(filePath));
+    this.writable = new ProtoMessageWritable<>(parser);
+  }
+
+  public Path getFilePath() {
+    return filePath;
+  }
+
+  public void setOffset(long offset) throws IOException {
+    reader.seek(offset);
+  }
+
+  public long getOffset() throws IOException {
+    return reader.getPosition();
+  }
+
+  public T readEvent() throws IOException {
+    if (!reader.next(NullWritable.get(), writable)) {
+      return null;
+    }
+    return writable.getMessage();
+  }
+
+  @Override
+  public void close() throws IOException {
+    reader.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/8349dbde/ql/src/java/org/apache/hadoop/hive/ql/hooks/ProtoMessageWritable.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/ProtoMessageWritable.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/ProtoMessageWritable.java
new file mode 100644
index 0000000..61d8449
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/ProtoMessageWritable.java
@@ -0,0 +1,101 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.hooks;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.hadoop.io.Writable;
+
+import com.google.protobuf.CodedInputStream;
+import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.MessageLite;
+import com.google.protobuf.Parser;
+
+public class ProtoMessageWritable<T extends MessageLite> implements Writable {
+  private T message;
+  private final Parser<T> parser;
+  private DataOutputStream dos;
+  private CodedOutputStream cos;
+  private DataInputStream din;
+  private CodedInputStream cin;
+
+  ProtoMessageWritable(Parser<T> parser) {
+    this.parser = parser;
+  }
+
+  public T getMessage() {
+    return message;
+  }
+
+  public void setMessage(T message) {
+    this.message = message;
+  }
+
+  private static class DataOutputStream extends OutputStream {
+    DataOutput out;
+    @Override
+    public void write(int b) throws IOException {
+      out.write(b);
+    }
+
+    @Override
+    public void write(byte b[], int off, int len) throws IOException {
+      out.write(b, off, len);
+    }
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    if (dos == null) {
+      dos = new DataOutputStream();
+      cos = CodedOutputStream.newInstance(dos);
+    }
+    dos.out = out;
+    cos.writeMessageNoTag(message);
+    cos.flush();
+  }
+
+  private static class DataInputStream extends InputStream {
+    DataInput in;
+    @Override
+    public int read() throws IOException {
+      try {
+        return in.readUnsignedByte();
+      } catch (EOFException e) {
+        return -1;
+      }
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    if (din == null) {
+      din = new DataInputStream();
+      cin = CodedInputStream.newInstance(din);
+      cin.setSizeLimit(Integer.MAX_VALUE);
+    }
+    din.in = in;
+    message = cin.readMessage(parser, null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/8349dbde/ql/src/java/org/apache/hadoop/hive/ql/hooks/ProtoMessageWriter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/ProtoMessageWriter.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/ProtoMessageWriter.java
new file mode 100644
index 0000000..ed8de93
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/ProtoMessageWriter.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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.hooks;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
+
+import com.google.protobuf.MessageLite;
+import com.google.protobuf.Parser;
+
+public class ProtoMessageWriter<T extends MessageLite> implements Closeable {
+  private final Path filePath;
+  private final SequenceFile.Writer writer;
+  private final ProtoMessageWritable<T> writable;
+
+  ProtoMessageWriter(Configuration conf, Path filePath, Parser<T> parser) throws IOException {
+    this.filePath = filePath;
+    this.writer = SequenceFile.createWriter(
+        conf,
+        SequenceFile.Writer.file(filePath),
+        SequenceFile.Writer.keyClass(NullWritable.class),
+        SequenceFile.Writer.valueClass(ProtoMessageWritable.class),
+        SequenceFile.Writer.appendIfExists(true),
+        SequenceFile.Writer.compression(CompressionType.RECORD));
+    this.writable = new ProtoMessageWritable<>(parser);
+  }
+
+  public Path getPath() {
+    return filePath;
+  }
+
+  public long getOffset() throws IOException {
+    return writer.getLength();
+  }
+
+  public void writeProto(T message) throws IOException {
+    writable.setMessage(message);
+    writer.append(NullWritable.get(), writable);
+  }
+
+  public void hflush() throws IOException {
+    writer.hflush();
+  }
+
+  @Override
+  public void close() throws IOException {
+    writer.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/8349dbde/ql/src/protobuf/HiveEvents.proto
----------------------------------------------------------------------
diff --git a/ql/src/protobuf/HiveEvents.proto b/ql/src/protobuf/HiveEvents.proto
new file mode 100644
index 0000000..eab0cc9
--- /dev/null
+++ b/ql/src/protobuf/HiveEvents.proto
@@ -0,0 +1,38 @@
+/**
+ * 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.
+ */
+option java_package = "org.apache.hadoop.hive.ql.hooks.proto";
+option java_outer_classname = "HiveHookEvents";
+
+message MapFieldEntry {
+    optional string key = 1;
+    optional string value = 2;
+}
+
+message HiveHookEventProto {
+    optional string eventType = 1;
+    optional string hiveQueryId = 2;
+    optional int64 timestamp = 3;
+    optional string executionMode = 4;
+    optional string requestUser = 5;
+    optional string queue = 6;
+    optional string user = 7;
+    optional string operationId = 8;
+    repeated string tablesWritten = 9;
+    repeated string tablesRead = 10;
+    repeated MapFieldEntry otherInfo = 50;
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/8349dbde/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestHiveProtoLoggingHook.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestHiveProtoLoggingHook.java b/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestHiveProtoLoggingHook.java
new file mode 100644
index 0000000..5e117fe
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestHiveProtoLoggingHook.java
@@ -0,0 +1,170 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.hooks;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.QueryPlan;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.hooks.HiveProtoLoggingHook.EventLogger;
+import org.apache.hadoop.hive.ql.hooks.HiveProtoLoggingHook.EventType;
+import org.apache.hadoop.hive.ql.hooks.HiveProtoLoggingHook.OtherInfoType;
+import org.apache.hadoop.hive.ql.hooks.HookContext.HookType;
+import org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto;
+import org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry;
+import org.apache.hadoop.hive.ql.log.PerfLogger;
+import org.apache.hadoop.hive.ql.plan.HiveOperation;
+import org.apache.hadoop.yarn.util.SystemClock;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+
+public class TestHiveProtoLoggingHook {
+
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+  private HiveConf conf;
+  private HookContext context;
+  private String tmpFolder;
+
+  @Before
+  public void setup() throws Exception {
+    conf = new HiveConf();
+    tmpFolder = folder.newFolder().getAbsolutePath();
+    conf.set(HiveProtoLoggingHook.HIVE_EVENTS_BASE_PATH, tmpFolder);
+    QueryState state = new QueryState.Builder().withHiveConf(conf).build();
+    @SuppressWarnings("serial")
+    QueryPlan queryPlan = new QueryPlan(HiveOperation.QUERY) {};
+    queryPlan.setQueryId("test_queryId");
+    queryPlan.setQueryStartTime(1234L);
+    queryPlan.setRootTasks(new ArrayList<>());
+    queryPlan.setInputs(new HashSet<>());
+    queryPlan.setOutputs(new HashSet<>());
+
+    PerfLogger perf = PerfLogger.getPerfLogger(conf, true);
+    context = new HookContext(queryPlan, state, null, "test_user", "192.168.10.10",
+        "hive_addr", "test_op_id", "test_session_id", "test_thread_id", true, perf, null);
+  }
+
+  @Test
+  public void testPreEventLog() throws Exception {
+    context.setHookType(HookType.PRE_EXEC_HOOK);
+    EventLogger evtLogger = new EventLogger(conf, SystemClock.getInstance());
+    evtLogger.handle(context);
+    evtLogger.shutdown();
+
+    HiveHookEventProto event = loadEvent(conf, tmpFolder);
+
+    Assert.assertEquals(EventType.QUERY_SUBMITTED.name(), event.getEventType());
+    Assert.assertEquals(1234L, event.getTimestamp());
+    Assert.assertEquals(System.getProperty("user.name"), event.getUser());
+    Assert.assertEquals("test_user", event.getRequestUser());
+    Assert.assertEquals("test_queryId", event.getHiveQueryId());
+    Assert.assertEquals("test_op_id", event.getOperationId());
+    Assert.assertEquals("NONE", event.getExecutionMode());
+
+    assertOtherInfo(event, OtherInfoType.TEZ, Boolean.FALSE.toString());
+    assertOtherInfo(event, OtherInfoType.MAPRED, Boolean.FALSE.toString());
+    assertOtherInfo(event, OtherInfoType.CLIENT_IP_ADDRESS, "192.168.10.10");
+    assertOtherInfo(event, OtherInfoType.SESSION_ID, "test_session_id");
+    assertOtherInfo(event, OtherInfoType.THREAD_NAME, "test_thread_id");
+    assertOtherInfo(event, OtherInfoType.HIVE_INSTANCE_TYPE, "HS2");
+    assertOtherInfo(event, OtherInfoType.HIVE_ADDRESS, "hive_addr");
+    assertOtherInfo(event, OtherInfoType.CONF, null);
+    assertOtherInfo(event, OtherInfoType.QUERY, null);
+  }
+
+  @Test
+  public void testPostEventLog() throws Exception {
+    context.setHookType(HookType.POST_EXEC_HOOK);
+
+    EventLogger evtLogger = new EventLogger(conf, SystemClock.getInstance());
+    evtLogger.handle(context);
+    evtLogger.shutdown();
+
+    HiveHookEventProto event = loadEvent(conf, tmpFolder);
+    Assert.assertEquals(EventType.QUERY_COMPLETED.name(), event.getEventType());
+    Assert.assertEquals(System.getProperty("user.name"), event.getUser());
+    Assert.assertEquals("test_user", event.getRequestUser());
+    Assert.assertEquals("test_queryId", event.getHiveQueryId());
+    Assert.assertEquals("test_op_id", event.getOperationId());
+
+    assertOtherInfo(event, OtherInfoType.STATUS, Boolean.TRUE.toString());
+    assertOtherInfo(event, OtherInfoType.PERF, null);
+  }
+
+  @Test
+  public void testFailureEventLog() throws Exception {
+    context.setHookType(HookType.ON_FAILURE_HOOK);
+
+    EventLogger evtLogger = new EventLogger(conf, SystemClock.getInstance());
+    evtLogger.handle(context);
+    evtLogger.shutdown();
+
+    HiveHookEventProto event = loadEvent(conf, tmpFolder);
+    Assert.assertEquals(EventType.QUERY_COMPLETED.name(), event.getEventType());
+    Assert.assertEquals(System.getProperty("user.name"), event.getUser());
+    Assert.assertEquals("test_user", event.getRequestUser());
+    Assert.assertEquals("test_queryId", event.getHiveQueryId());
+    Assert.assertEquals("test_op_id", event.getOperationId());
+
+    assertOtherInfo(event, OtherInfoType.STATUS, Boolean.FALSE.toString());
+    assertOtherInfo(event, OtherInfoType.PERF, null);
+  }
+
+  private HiveHookEventProto loadEvent(HiveConf conf, String tmpFolder)
+      throws IOException, FileNotFoundException {
+    Path path = new Path(tmpFolder);
+    FileSystem fs = path.getFileSystem(conf);
+    FileStatus[] status = fs.listStatus(path);
+    Assert.assertEquals(1, status.length);
+    status = fs.listStatus(status[0].getPath());
+    Assert.assertEquals(1, status.length);
+
+    DatePartitionedLogger<HiveHookEventProto> logger = new DatePartitionedLogger<>(
+        HiveHookEventProto.PARSER, path, conf, SystemClock.getInstance());
+    ProtoMessageReader<HiveHookEventProto> reader = logger.getReader(status[0].getPath());
+    HiveHookEventProto event = reader.readEvent();
+    Assert.assertNotNull(event);
+    return event;
+  }
+
+  private void assertOtherInfo(HiveHookEventProto event, OtherInfoType key, String value) {
+    for (MapFieldEntry otherInfo : event.getOtherInfoList()) {
+      if (otherInfo.getKey().equals(key.name())) {
+        if (value != null) {
+          Assert.assertEquals(value, otherInfo.getValue());
+        }
+        return;
+      }
+    }
+    Assert.fail("Cannot find key: " + key);
+  }
+}


[3/3] hive git commit: HIVE-19288 : Implement protobuf logging hive hook. (Harish JP via Ashutosh Chauhan)

Posted by ha...@apache.org.
HIVE-19288 : Implement protobuf logging hive hook. (Harish JP via Ashutosh Chauhan)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/8349dbde
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/8349dbde
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/8349dbde

Branch: refs/heads/master
Commit: 8349dbde55f479167e43cfd1f089e131d4271e5b
Parents: 3a0e4dd
Author: Harish Jaiprakash <hj...@hortonworks.com>
Authored: Sat Apr 28 10:47:09 2018 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Sat Apr 28 10:47:09 2018 -0700

----------------------------------------------------------------------
 .../hadoop/hive/llap/TestAsyncPbRpcProxy.java   |    4 +-
 ql/pom.xml                                      |   34 +
 .../hive/ql/hooks/proto/HiveHookEvents.java     | 2917 ++++++++++++++++++
 .../hive/ql/hooks/DatePartitionedLogger.java    |  167 +
 .../hive/ql/hooks/HiveProtoLoggingHook.java     |  493 +++
 .../hive/ql/hooks/ProtoMessageReader.java       |   66 +
 .../hive/ql/hooks/ProtoMessageWritable.java     |  101 +
 .../hive/ql/hooks/ProtoMessageWriter.java       |   71 +
 ql/src/protobuf/HiveEvents.proto                |   38 +
 .../hive/ql/hooks/TestHiveProtoLoggingHook.java |  170 +
 10 files changed, 4060 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/8349dbde/llap-client/src/test/org/apache/hadoop/hive/llap/TestAsyncPbRpcProxy.java
----------------------------------------------------------------------
diff --git a/llap-client/src/test/org/apache/hadoop/hive/llap/TestAsyncPbRpcProxy.java b/llap-client/src/test/org/apache/hadoop/hive/llap/TestAsyncPbRpcProxy.java
index b152f1c..9abbbbe 100644
--- a/llap-client/src/test/org/apache/hadoop/hive/llap/TestAsyncPbRpcProxy.java
+++ b/llap-client/src/test/org/apache/hadoop/hive/llap/TestAsyncPbRpcProxy.java
@@ -112,7 +112,9 @@ public class TestAsyncPbRpcProxy {
       super(numThreads, 1);
     }
 
-    protected void submitToExecutor(LlapProtocolClientProxy.CallableRequest request, LlapNodeId nodeId) {
+    @Override
+    protected <T extends Message, U extends Message> void submitToExecutor(
+        LlapProtocolClientProxy.CallableRequest<T, U> request, LlapNodeId nodeId) {
       numSubmissionsCounters++;
       MutableInt nodeCount = numInvocationsPerNode.get(nodeId);
       if (nodeCount == null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/8349dbde/ql/pom.xml
----------------------------------------------------------------------
diff --git a/ql/pom.xml b/ql/pom.xml
index 165610f..867a38a 100644
--- a/ql/pom.xml
+++ b/ql/pom.xml
@@ -795,6 +795,39 @@
           </plugins>
        </build>
     </profile>
+    <profile>
+      <id>protobuf</id>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>generate-protobuf-sources</id>
+                <phase>generate-sources</phase>
+                <configuration>
+                  <target>
+                    <property name="protobuf.src.dir"  location="${basedir}/src/protobuf"/>
+                    <property name="protobuf.build.dir"  location="${basedir}/src/gen/protobuf/gen-java"/>
+                    <echo>Building ql Protobuf</echo>
+                    <mkdir dir="${protobuf.build.dir}"/>
+                    <exec executable="protoc" failonerror="true">
+                      <arg value="--java_out=${protobuf.build.dir}"/>
+                      <arg value="-I=${protobuf.src.dir}"/>
+                      <arg value="${protobuf.src.dir}/HiveEvents.proto"/>
+                    </exec>
+                  </target>
+                </configuration>
+                <goals>
+                  <goal>run</goal>
+                </goals>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
   </profiles>
 
   <build>
@@ -957,6 +990,7 @@
               <sources>
                 <source>src/gen/thrift/gen-javabean</source>
                 <source>${project.build.directory}/generated-sources/java</source>
+                <source>src/gen/protobuf/gen-java</source>
               </sources>
             </configuration>
           </execution>


[2/3] hive git commit: HIVE-19288 : Implement protobuf logging hive hook. (Harish JP via Ashutosh Chauhan)

Posted by ha...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/8349dbde/ql/src/gen/protobuf/gen-java/org/apache/hadoop/hive/ql/hooks/proto/HiveHookEvents.java
----------------------------------------------------------------------
diff --git a/ql/src/gen/protobuf/gen-java/org/apache/hadoop/hive/ql/hooks/proto/HiveHookEvents.java b/ql/src/gen/protobuf/gen-java/org/apache/hadoop/hive/ql/hooks/proto/HiveHookEvents.java
new file mode 100644
index 0000000..6adda72
--- /dev/null
+++ b/ql/src/gen/protobuf/gen-java/org/apache/hadoop/hive/ql/hooks/proto/HiveHookEvents.java
@@ -0,0 +1,2917 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: HiveEvents.proto
+
+package org.apache.hadoop.hive.ql.hooks.proto;
+
+public final class HiveHookEvents {
+  private HiveHookEvents() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public interface MapFieldEntryOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // optional string key = 1;
+    /**
+     * <code>optional string key = 1;</code>
+     */
+    boolean hasKey();
+    /**
+     * <code>optional string key = 1;</code>
+     */
+    java.lang.String getKey();
+    /**
+     * <code>optional string key = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getKeyBytes();
+
+    // optional string value = 2;
+    /**
+     * <code>optional string value = 2;</code>
+     */
+    boolean hasValue();
+    /**
+     * <code>optional string value = 2;</code>
+     */
+    java.lang.String getValue();
+    /**
+     * <code>optional string value = 2;</code>
+     */
+    com.google.protobuf.ByteString
+        getValueBytes();
+  }
+  /**
+   * Protobuf type {@code org.apache.hadoop.hive.ql.hooks.proto.MapFieldEntry}
+   */
+  public static final class MapFieldEntry extends
+      com.google.protobuf.GeneratedMessage
+      implements MapFieldEntryOrBuilder {
+    // Use MapFieldEntry.newBuilder() to construct.
+    private MapFieldEntry(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private MapFieldEntry(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final MapFieldEntry defaultInstance;
+    public static MapFieldEntry getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public MapFieldEntry getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private MapFieldEntry(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              key_ = input.readBytes();
+              break;
+            }
+            case 18: {
+              bitField0_ |= 0x00000002;
+              value_ = input.readBytes();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.internal_static_org_apache_hadoop_hive_ql_hooks_proto_MapFieldEntry_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.internal_static_org_apache_hadoop_hive_ql_hooks_proto_MapFieldEntry_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry.class, org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<MapFieldEntry> PARSER =
+        new com.google.protobuf.AbstractParser<MapFieldEntry>() {
+      public MapFieldEntry parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new MapFieldEntry(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<MapFieldEntry> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // optional string key = 1;
+    public static final int KEY_FIELD_NUMBER = 1;
+    private java.lang.Object key_;
+    /**
+     * <code>optional string key = 1;</code>
+     */
+    public boolean hasKey() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional string key = 1;</code>
+     */
+    public java.lang.String getKey() {
+      java.lang.Object ref = key_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          key_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string key = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getKeyBytes() {
+      java.lang.Object ref = key_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        key_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional string value = 2;
+    public static final int VALUE_FIELD_NUMBER = 2;
+    private java.lang.Object value_;
+    /**
+     * <code>optional string value = 2;</code>
+     */
+    public boolean hasValue() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional string value = 2;</code>
+     */
+    public java.lang.String getValue() {
+      java.lang.Object ref = value_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          value_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string value = 2;</code>
+     */
+    public com.google.protobuf.ByteString
+        getValueBytes() {
+      java.lang.Object ref = value_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        value_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    private void initFields() {
+      key_ = "";
+      value_ = "";
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getKeyBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, getValueBytes());
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getKeyBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, getValueBytes());
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    public static org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code org.apache.hadoop.hive.ql.hooks.proto.MapFieldEntry}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntryOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.internal_static_org_apache_hadoop_hive_ql_hooks_proto_MapFieldEntry_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.internal_static_org_apache_hadoop_hive_ql_hooks_proto_MapFieldEntry_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry.class, org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        key_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        value_ = "";
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.internal_static_org_apache_hadoop_hive_ql_hooks_proto_MapFieldEntry_descriptor;
+      }
+
+      public org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry getDefaultInstanceForType() {
+        return org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry build() {
+        org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry buildPartial() {
+        org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry result = new org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.key_ = key_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.value_ = value_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry) {
+          return mergeFrom((org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry other) {
+        if (other == org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry.getDefaultInstance()) return this;
+        if (other.hasKey()) {
+          bitField0_ |= 0x00000001;
+          key_ = other.key_;
+          onChanged();
+        }
+        if (other.hasValue()) {
+          bitField0_ |= 0x00000002;
+          value_ = other.value_;
+          onChanged();
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // optional string key = 1;
+      private java.lang.Object key_ = "";
+      /**
+       * <code>optional string key = 1;</code>
+       */
+      public boolean hasKey() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional string key = 1;</code>
+       */
+      public java.lang.String getKey() {
+        java.lang.Object ref = key_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          key_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string key = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getKeyBytes() {
+        java.lang.Object ref = key_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          key_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string key = 1;</code>
+       */
+      public Builder setKey(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        key_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string key = 1;</code>
+       */
+      public Builder clearKey() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        key_ = getDefaultInstance().getKey();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string key = 1;</code>
+       */
+      public Builder setKeyBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        key_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional string value = 2;
+      private java.lang.Object value_ = "";
+      /**
+       * <code>optional string value = 2;</code>
+       */
+      public boolean hasValue() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional string value = 2;</code>
+       */
+      public java.lang.String getValue() {
+        java.lang.Object ref = value_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          value_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string value = 2;</code>
+       */
+      public com.google.protobuf.ByteString
+          getValueBytes() {
+        java.lang.Object ref = value_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          value_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string value = 2;</code>
+       */
+      public Builder setValue(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        value_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string value = 2;</code>
+       */
+      public Builder clearValue() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        value_ = getDefaultInstance().getValue();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string value = 2;</code>
+       */
+      public Builder setValueBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        value_ = value;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hive.ql.hooks.proto.MapFieldEntry)
+    }
+
+    static {
+      defaultInstance = new MapFieldEntry(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:org.apache.hadoop.hive.ql.hooks.proto.MapFieldEntry)
+  }
+
+  public interface HiveHookEventProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // optional string eventType = 1;
+    /**
+     * <code>optional string eventType = 1;</code>
+     */
+    boolean hasEventType();
+    /**
+     * <code>optional string eventType = 1;</code>
+     */
+    java.lang.String getEventType();
+    /**
+     * <code>optional string eventType = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getEventTypeBytes();
+
+    // optional string hiveQueryId = 2;
+    /**
+     * <code>optional string hiveQueryId = 2;</code>
+     */
+    boolean hasHiveQueryId();
+    /**
+     * <code>optional string hiveQueryId = 2;</code>
+     */
+    java.lang.String getHiveQueryId();
+    /**
+     * <code>optional string hiveQueryId = 2;</code>
+     */
+    com.google.protobuf.ByteString
+        getHiveQueryIdBytes();
+
+    // optional int64 timestamp = 3;
+    /**
+     * <code>optional int64 timestamp = 3;</code>
+     */
+    boolean hasTimestamp();
+    /**
+     * <code>optional int64 timestamp = 3;</code>
+     */
+    long getTimestamp();
+
+    // optional string executionMode = 4;
+    /**
+     * <code>optional string executionMode = 4;</code>
+     */
+    boolean hasExecutionMode();
+    /**
+     * <code>optional string executionMode = 4;</code>
+     */
+    java.lang.String getExecutionMode();
+    /**
+     * <code>optional string executionMode = 4;</code>
+     */
+    com.google.protobuf.ByteString
+        getExecutionModeBytes();
+
+    // optional string requestUser = 5;
+    /**
+     * <code>optional string requestUser = 5;</code>
+     */
+    boolean hasRequestUser();
+    /**
+     * <code>optional string requestUser = 5;</code>
+     */
+    java.lang.String getRequestUser();
+    /**
+     * <code>optional string requestUser = 5;</code>
+     */
+    com.google.protobuf.ByteString
+        getRequestUserBytes();
+
+    // optional string queue = 6;
+    /**
+     * <code>optional string queue = 6;</code>
+     */
+    boolean hasQueue();
+    /**
+     * <code>optional string queue = 6;</code>
+     */
+    java.lang.String getQueue();
+    /**
+     * <code>optional string queue = 6;</code>
+     */
+    com.google.protobuf.ByteString
+        getQueueBytes();
+
+    // optional string user = 7;
+    /**
+     * <code>optional string user = 7;</code>
+     */
+    boolean hasUser();
+    /**
+     * <code>optional string user = 7;</code>
+     */
+    java.lang.String getUser();
+    /**
+     * <code>optional string user = 7;</code>
+     */
+    com.google.protobuf.ByteString
+        getUserBytes();
+
+    // optional string operationId = 8;
+    /**
+     * <code>optional string operationId = 8;</code>
+     */
+    boolean hasOperationId();
+    /**
+     * <code>optional string operationId = 8;</code>
+     */
+    java.lang.String getOperationId();
+    /**
+     * <code>optional string operationId = 8;</code>
+     */
+    com.google.protobuf.ByteString
+        getOperationIdBytes();
+
+    // repeated string tablesWritten = 9;
+    /**
+     * <code>repeated string tablesWritten = 9;</code>
+     */
+    java.util.List<java.lang.String>
+    getTablesWrittenList();
+    /**
+     * <code>repeated string tablesWritten = 9;</code>
+     */
+    int getTablesWrittenCount();
+    /**
+     * <code>repeated string tablesWritten = 9;</code>
+     */
+    java.lang.String getTablesWritten(int index);
+    /**
+     * <code>repeated string tablesWritten = 9;</code>
+     */
+    com.google.protobuf.ByteString
+        getTablesWrittenBytes(int index);
+
+    // repeated string tablesRead = 10;
+    /**
+     * <code>repeated string tablesRead = 10;</code>
+     */
+    java.util.List<java.lang.String>
+    getTablesReadList();
+    /**
+     * <code>repeated string tablesRead = 10;</code>
+     */
+    int getTablesReadCount();
+    /**
+     * <code>repeated string tablesRead = 10;</code>
+     */
+    java.lang.String getTablesRead(int index);
+    /**
+     * <code>repeated string tablesRead = 10;</code>
+     */
+    com.google.protobuf.ByteString
+        getTablesReadBytes(int index);
+
+    // repeated .org.apache.hadoop.hive.ql.hooks.proto.MapFieldEntry otherInfo = 50;
+    /**
+     * <code>repeated .org.apache.hadoop.hive.ql.hooks.proto.MapFieldEntry otherInfo = 50;</code>
+     */
+    java.util.List<org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry> 
+        getOtherInfoList();
+    /**
+     * <code>repeated .org.apache.hadoop.hive.ql.hooks.proto.MapFieldEntry otherInfo = 50;</code>
+     */
+    org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry getOtherInfo(int index);
+    /**
+     * <code>repeated .org.apache.hadoop.hive.ql.hooks.proto.MapFieldEntry otherInfo = 50;</code>
+     */
+    int getOtherInfoCount();
+    /**
+     * <code>repeated .org.apache.hadoop.hive.ql.hooks.proto.MapFieldEntry otherInfo = 50;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntryOrBuilder> 
+        getOtherInfoOrBuilderList();
+    /**
+     * <code>repeated .org.apache.hadoop.hive.ql.hooks.proto.MapFieldEntry otherInfo = 50;</code>
+     */
+    org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntryOrBuilder getOtherInfoOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code org.apache.hadoop.hive.ql.hooks.proto.HiveHookEventProto}
+   */
+  public static final class HiveHookEventProto extends
+      com.google.protobuf.GeneratedMessage
+      implements HiveHookEventProtoOrBuilder {
+    // Use HiveHookEventProto.newBuilder() to construct.
+    private HiveHookEventProto(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private HiveHookEventProto(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final HiveHookEventProto defaultInstance;
+    public static HiveHookEventProto getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public HiveHookEventProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private HiveHookEventProto(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              eventType_ = input.readBytes();
+              break;
+            }
+            case 18: {
+              bitField0_ |= 0x00000002;
+              hiveQueryId_ = input.readBytes();
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              timestamp_ = input.readInt64();
+              break;
+            }
+            case 34: {
+              bitField0_ |= 0x00000008;
+              executionMode_ = input.readBytes();
+              break;
+            }
+            case 42: {
+              bitField0_ |= 0x00000010;
+              requestUser_ = input.readBytes();
+              break;
+            }
+            case 50: {
+              bitField0_ |= 0x00000020;
+              queue_ = input.readBytes();
+              break;
+            }
+            case 58: {
+              bitField0_ |= 0x00000040;
+              user_ = input.readBytes();
+              break;
+            }
+            case 66: {
+              bitField0_ |= 0x00000080;
+              operationId_ = input.readBytes();
+              break;
+            }
+            case 74: {
+              if (!((mutable_bitField0_ & 0x00000100) == 0x00000100)) {
+                tablesWritten_ = new com.google.protobuf.LazyStringArrayList();
+                mutable_bitField0_ |= 0x00000100;
+              }
+              tablesWritten_.add(input.readBytes());
+              break;
+            }
+            case 82: {
+              if (!((mutable_bitField0_ & 0x00000200) == 0x00000200)) {
+                tablesRead_ = new com.google.protobuf.LazyStringArrayList();
+                mutable_bitField0_ |= 0x00000200;
+              }
+              tablesRead_.add(input.readBytes());
+              break;
+            }
+            case 402: {
+              if (!((mutable_bitField0_ & 0x00000400) == 0x00000400)) {
+                otherInfo_ = new java.util.ArrayList<org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry>();
+                mutable_bitField0_ |= 0x00000400;
+              }
+              otherInfo_.add(input.readMessage(org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry.PARSER, extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000100) == 0x00000100)) {
+          tablesWritten_ = new com.google.protobuf.UnmodifiableLazyStringList(tablesWritten_);
+        }
+        if (((mutable_bitField0_ & 0x00000200) == 0x00000200)) {
+          tablesRead_ = new com.google.protobuf.UnmodifiableLazyStringList(tablesRead_);
+        }
+        if (((mutable_bitField0_ & 0x00000400) == 0x00000400)) {
+          otherInfo_ = java.util.Collections.unmodifiableList(otherInfo_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.internal_static_org_apache_hadoop_hive_ql_hooks_proto_HiveHookEventProto_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.internal_static_org_apache_hadoop_hive_ql_hooks_proto_HiveHookEventProto_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto.class, org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<HiveHookEventProto> PARSER =
+        new com.google.protobuf.AbstractParser<HiveHookEventProto>() {
+      public HiveHookEventProto parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new HiveHookEventProto(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<HiveHookEventProto> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // optional string eventType = 1;
+    public static final int EVENTTYPE_FIELD_NUMBER = 1;
+    private java.lang.Object eventType_;
+    /**
+     * <code>optional string eventType = 1;</code>
+     */
+    public boolean hasEventType() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional string eventType = 1;</code>
+     */
+    public java.lang.String getEventType() {
+      java.lang.Object ref = eventType_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          eventType_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string eventType = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getEventTypeBytes() {
+      java.lang.Object ref = eventType_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        eventType_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional string hiveQueryId = 2;
+    public static final int HIVEQUERYID_FIELD_NUMBER = 2;
+    private java.lang.Object hiveQueryId_;
+    /**
+     * <code>optional string hiveQueryId = 2;</code>
+     */
+    public boolean hasHiveQueryId() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional string hiveQueryId = 2;</code>
+     */
+    public java.lang.String getHiveQueryId() {
+      java.lang.Object ref = hiveQueryId_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          hiveQueryId_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string hiveQueryId = 2;</code>
+     */
+    public com.google.protobuf.ByteString
+        getHiveQueryIdBytes() {
+      java.lang.Object ref = hiveQueryId_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        hiveQueryId_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional int64 timestamp = 3;
+    public static final int TIMESTAMP_FIELD_NUMBER = 3;
+    private long timestamp_;
+    /**
+     * <code>optional int64 timestamp = 3;</code>
+     */
+    public boolean hasTimestamp() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional int64 timestamp = 3;</code>
+     */
+    public long getTimestamp() {
+      return timestamp_;
+    }
+
+    // optional string executionMode = 4;
+    public static final int EXECUTIONMODE_FIELD_NUMBER = 4;
+    private java.lang.Object executionMode_;
+    /**
+     * <code>optional string executionMode = 4;</code>
+     */
+    public boolean hasExecutionMode() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>optional string executionMode = 4;</code>
+     */
+    public java.lang.String getExecutionMode() {
+      java.lang.Object ref = executionMode_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          executionMode_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string executionMode = 4;</code>
+     */
+    public com.google.protobuf.ByteString
+        getExecutionModeBytes() {
+      java.lang.Object ref = executionMode_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        executionMode_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional string requestUser = 5;
+    public static final int REQUESTUSER_FIELD_NUMBER = 5;
+    private java.lang.Object requestUser_;
+    /**
+     * <code>optional string requestUser = 5;</code>
+     */
+    public boolean hasRequestUser() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    /**
+     * <code>optional string requestUser = 5;</code>
+     */
+    public java.lang.String getRequestUser() {
+      java.lang.Object ref = requestUser_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          requestUser_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string requestUser = 5;</code>
+     */
+    public com.google.protobuf.ByteString
+        getRequestUserBytes() {
+      java.lang.Object ref = requestUser_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        requestUser_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional string queue = 6;
+    public static final int QUEUE_FIELD_NUMBER = 6;
+    private java.lang.Object queue_;
+    /**
+     * <code>optional string queue = 6;</code>
+     */
+    public boolean hasQueue() {
+      return ((bitField0_ & 0x00000020) == 0x00000020);
+    }
+    /**
+     * <code>optional string queue = 6;</code>
+     */
+    public java.lang.String getQueue() {
+      java.lang.Object ref = queue_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          queue_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string queue = 6;</code>
+     */
+    public com.google.protobuf.ByteString
+        getQueueBytes() {
+      java.lang.Object ref = queue_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        queue_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional string user = 7;
+    public static final int USER_FIELD_NUMBER = 7;
+    private java.lang.Object user_;
+    /**
+     * <code>optional string user = 7;</code>
+     */
+    public boolean hasUser() {
+      return ((bitField0_ & 0x00000040) == 0x00000040);
+    }
+    /**
+     * <code>optional string user = 7;</code>
+     */
+    public java.lang.String getUser() {
+      java.lang.Object ref = user_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          user_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string user = 7;</code>
+     */
+    public com.google.protobuf.ByteString
+        getUserBytes() {
+      java.lang.Object ref = user_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        user_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional string operationId = 8;
+    public static final int OPERATIONID_FIELD_NUMBER = 8;
+    private java.lang.Object operationId_;
+    /**
+     * <code>optional string operationId = 8;</code>
+     */
+    public boolean hasOperationId() {
+      return ((bitField0_ & 0x00000080) == 0x00000080);
+    }
+    /**
+     * <code>optional string operationId = 8;</code>
+     */
+    public java.lang.String getOperationId() {
+      java.lang.Object ref = operationId_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          operationId_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string operationId = 8;</code>
+     */
+    public com.google.protobuf.ByteString
+        getOperationIdBytes() {
+      java.lang.Object ref = operationId_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        operationId_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // repeated string tablesWritten = 9;
+    public static final int TABLESWRITTEN_FIELD_NUMBER = 9;
+    private com.google.protobuf.LazyStringList tablesWritten_;
+    /**
+     * <code>repeated string tablesWritten = 9;</code>
+     */
+    public java.util.List<java.lang.String>
+        getTablesWrittenList() {
+      return tablesWritten_;
+    }
+    /**
+     * <code>repeated string tablesWritten = 9;</code>
+     */
+    public int getTablesWrittenCount() {
+      return tablesWritten_.size();
+    }
+    /**
+     * <code>repeated string tablesWritten = 9;</code>
+     */
+    public java.lang.String getTablesWritten(int index) {
+      return tablesWritten_.get(index);
+    }
+    /**
+     * <code>repeated string tablesWritten = 9;</code>
+     */
+    public com.google.protobuf.ByteString
+        getTablesWrittenBytes(int index) {
+      return tablesWritten_.getByteString(index);
+    }
+
+    // repeated string tablesRead = 10;
+    public static final int TABLESREAD_FIELD_NUMBER = 10;
+    private com.google.protobuf.LazyStringList tablesRead_;
+    /**
+     * <code>repeated string tablesRead = 10;</code>
+     */
+    public java.util.List<java.lang.String>
+        getTablesReadList() {
+      return tablesRead_;
+    }
+    /**
+     * <code>repeated string tablesRead = 10;</code>
+     */
+    public int getTablesReadCount() {
+      return tablesRead_.size();
+    }
+    /**
+     * <code>repeated string tablesRead = 10;</code>
+     */
+    public java.lang.String getTablesRead(int index) {
+      return tablesRead_.get(index);
+    }
+    /**
+     * <code>repeated string tablesRead = 10;</code>
+     */
+    public com.google.protobuf.ByteString
+        getTablesReadBytes(int index) {
+      return tablesRead_.getByteString(index);
+    }
+
+    // repeated .org.apache.hadoop.hive.ql.hooks.proto.MapFieldEntry otherInfo = 50;
+    public static final int OTHERINFO_FIELD_NUMBER = 50;
+    private java.util.List<org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry> otherInfo_;
+    /**
+     * <code>repeated .org.apache.hadoop.hive.ql.hooks.proto.MapFieldEntry otherInfo = 50;</code>
+     */
+    public java.util.List<org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry> getOtherInfoList() {
+      return otherInfo_;
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hive.ql.hooks.proto.MapFieldEntry otherInfo = 50;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntryOrBuilder> 
+        getOtherInfoOrBuilderList() {
+      return otherInfo_;
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hive.ql.hooks.proto.MapFieldEntry otherInfo = 50;</code>
+     */
+    public int getOtherInfoCount() {
+      return otherInfo_.size();
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hive.ql.hooks.proto.MapFieldEntry otherInfo = 50;</code>
+     */
+    public org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry getOtherInfo(int index) {
+      return otherInfo_.get(index);
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hive.ql.hooks.proto.MapFieldEntry otherInfo = 50;</code>
+     */
+    public org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntryOrBuilder getOtherInfoOrBuilder(
+        int index) {
+      return otherInfo_.get(index);
+    }
+
+    private void initFields() {
+      eventType_ = "";
+      hiveQueryId_ = "";
+      timestamp_ = 0L;
+      executionMode_ = "";
+      requestUser_ = "";
+      queue_ = "";
+      user_ = "";
+      operationId_ = "";
+      tablesWritten_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      tablesRead_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      otherInfo_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getEventTypeBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, getHiveQueryIdBytes());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeInt64(3, timestamp_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeBytes(4, getExecutionModeBytes());
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeBytes(5, getRequestUserBytes());
+      }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        output.writeBytes(6, getQueueBytes());
+      }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        output.writeBytes(7, getUserBytes());
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        output.writeBytes(8, getOperationIdBytes());
+      }
+      for (int i = 0; i < tablesWritten_.size(); i++) {
+        output.writeBytes(9, tablesWritten_.getByteString(i));
+      }
+      for (int i = 0; i < tablesRead_.size(); i++) {
+        output.writeBytes(10, tablesRead_.getByteString(i));
+      }
+      for (int i = 0; i < otherInfo_.size(); i++) {
+        output.writeMessage(50, otherInfo_.get(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getEventTypeBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, getHiveQueryIdBytes());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt64Size(3, timestamp_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(4, getExecutionModeBytes());
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(5, getRequestUserBytes());
+      }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(6, getQueueBytes());
+      }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(7, getUserBytes());
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(8, getOperationIdBytes());
+      }
+      {
+        int dataSize = 0;
+        for (int i = 0; i < tablesWritten_.size(); i++) {
+          dataSize += com.google.protobuf.CodedOutputStream
+            .computeBytesSizeNoTag(tablesWritten_.getByteString(i));
+        }
+        size += dataSize;
+        size += 1 * getTablesWrittenList().size();
+      }
+      {
+        int dataSize = 0;
+        for (int i = 0; i < tablesRead_.size(); i++) {
+          dataSize += com.google.protobuf.CodedOutputStream
+            .computeBytesSizeNoTag(tablesRead_.getByteString(i));
+        }
+        size += dataSize;
+        size += 1 * getTablesReadList().size();
+      }
+      for (int i = 0; i < otherInfo_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(50, otherInfo_.get(i));
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    public static org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code org.apache.hadoop.hive.ql.hooks.proto.HiveHookEventProto}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.internal_static_org_apache_hadoop_hive_ql_hooks_proto_HiveHookEventProto_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.internal_static_org_apache_hadoop_hive_ql_hooks_proto_HiveHookEventProto_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto.class, org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getOtherInfoFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        eventType_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        hiveQueryId_ = "";
+        bitField0_ = (bitField0_ & ~0x00000002);
+        timestamp_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        executionMode_ = "";
+        bitField0_ = (bitField0_ & ~0x00000008);
+        requestUser_ = "";
+        bitField0_ = (bitField0_ & ~0x00000010);
+        queue_ = "";
+        bitField0_ = (bitField0_ & ~0x00000020);
+        user_ = "";
+        bitField0_ = (bitField0_ & ~0x00000040);
+        operationId_ = "";
+        bitField0_ = (bitField0_ & ~0x00000080);
+        tablesWritten_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000100);
+        tablesRead_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000200);
+        if (otherInfoBuilder_ == null) {
+          otherInfo_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000400);
+        } else {
+          otherInfoBuilder_.clear();
+        }
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.internal_static_org_apache_hadoop_hive_ql_hooks_proto_HiveHookEventProto_descriptor;
+      }
+
+      public org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto build() {
+        org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto buildPartial() {
+        org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto result = new org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.eventType_ = eventType_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.hiveQueryId_ = hiveQueryId_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.timestamp_ = timestamp_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.executionMode_ = executionMode_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        result.requestUser_ = requestUser_;
+        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
+          to_bitField0_ |= 0x00000020;
+        }
+        result.queue_ = queue_;
+        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
+          to_bitField0_ |= 0x00000040;
+        }
+        result.user_ = user_;
+        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
+          to_bitField0_ |= 0x00000080;
+        }
+        result.operationId_ = operationId_;
+        if (((bitField0_ & 0x00000100) == 0x00000100)) {
+          tablesWritten_ = new com.google.protobuf.UnmodifiableLazyStringList(
+              tablesWritten_);
+          bitField0_ = (bitField0_ & ~0x00000100);
+        }
+        result.tablesWritten_ = tablesWritten_;
+        if (((bitField0_ & 0x00000200) == 0x00000200)) {
+          tablesRead_ = new com.google.protobuf.UnmodifiableLazyStringList(
+              tablesRead_);
+          bitField0_ = (bitField0_ & ~0x00000200);
+        }
+        result.tablesRead_ = tablesRead_;
+        if (otherInfoBuilder_ == null) {
+          if (((bitField0_ & 0x00000400) == 0x00000400)) {
+            otherInfo_ = java.util.Collections.unmodifiableList(otherInfo_);
+            bitField0_ = (bitField0_ & ~0x00000400);
+          }
+          result.otherInfo_ = otherInfo_;
+        } else {
+          result.otherInfo_ = otherInfoBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto) {
+          return mergeFrom((org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto other) {
+        if (other == org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto.getDefaultInstance()) return this;
+        if (other.hasEventType()) {
+          bitField0_ |= 0x00000001;
+          eventType_ = other.eventType_;
+          onChanged();
+        }
+        if (other.hasHiveQueryId()) {
+          bitField0_ |= 0x00000002;
+          hiveQueryId_ = other.hiveQueryId_;
+          onChanged();
+        }
+        if (other.hasTimestamp()) {
+          setTimestamp(other.getTimestamp());
+        }
+        if (other.hasExecutionMode()) {
+          bitField0_ |= 0x00000008;
+          executionMode_ = other.executionMode_;
+          onChanged();
+        }
+        if (other.hasRequestUser()) {
+          bitField0_ |= 0x00000010;
+          requestUser_ = other.requestUser_;
+          onChanged();
+        }
+        if (other.hasQueue()) {
+          bitField0_ |= 0x00000020;
+          queue_ = other.queue_;
+          onChanged();
+        }
+        if (other.hasUser()) {
+          bitField0_ |= 0x00000040;
+          user_ = other.user_;
+          onChanged();
+        }
+        if (other.hasOperationId()) {
+          bitField0_ |= 0x00000080;
+          operationId_ = other.operationId_;
+          onChanged();
+        }
+        if (!other.tablesWritten_.isEmpty()) {
+          if (tablesWritten_.isEmpty()) {
+            tablesWritten_ = other.tablesWritten_;
+            bitField0_ = (bitField0_ & ~0x00000100);
+          } else {
+            ensureTablesWrittenIsMutable();
+            tablesWritten_.addAll(other.tablesWritten_);
+          }
+          onChanged();
+        }
+        if (!other.tablesRead_.isEmpty()) {
+          if (tablesRead_.isEmpty()) {
+            tablesRead_ = other.tablesRead_;
+            bitField0_ = (bitField0_ & ~0x00000200);
+          } else {
+            ensureTablesReadIsMutable();
+            tablesRead_.addAll(other.tablesRead_);
+          }
+          onChanged();
+        }
+        if (otherInfoBuilder_ == null) {
+          if (!other.otherInfo_.isEmpty()) {
+            if (otherInfo_.isEmpty()) {
+              otherInfo_ = other.otherInfo_;
+              bitField0_ = (bitField0_ & ~0x00000400);
+            } else {
+              ensureOtherInfoIsMutable();
+              otherInfo_.addAll(other.otherInfo_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.otherInfo_.isEmpty()) {
+            if (otherInfoBuilder_.isEmpty()) {
+              otherInfoBuilder_.dispose();
+              otherInfoBuilder_ = null;
+              otherInfo_ = other.otherInfo_;
+              bitField0_ = (bitField0_ & ~0x00000400);
+              otherInfoBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getOtherInfoFieldBuilder() : null;
+            } else {
+              otherInfoBuilder_.addAllMessages(other.otherInfo_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // optional string eventType = 1;
+      private java.lang.Object eventType_ = "";
+      /**
+       * <code>optional string eventType = 1;</code>
+       */
+      public boolean hasEventType() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional string eventType = 1;</code>
+       */
+      public java.lang.String getEventType() {
+        java.lang.Object ref = eventType_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          eventType_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string eventType = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getEventTypeBytes() {
+        java.lang.Object ref = eventType_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          eventType_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string eventType = 1;</code>
+       */
+      public Builder setEventType(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        eventType_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string eventType = 1;</code>
+       */
+      public Builder clearEventType() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        eventType_ = getDefaultInstance().getEventType();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string eventType = 1;</code>
+       */
+      public Builder setEventTypeBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        eventType_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional string hiveQueryId = 2;
+      private java.lang.Object hiveQueryId_ = "";
+      /**
+       * <code>optional string hiveQueryId = 2;</code>
+       */
+      public boolean hasHiveQueryId() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional string hiveQueryId = 2;</code>
+       */
+      public java.lang.String getHiveQueryId() {
+        java.lang.Object ref = hiveQueryId_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          hiveQueryId_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string hiveQueryId = 2;</code>
+       */
+      public com.google.protobuf.ByteString
+          getHiveQueryIdBytes() {
+        java.lang.Object ref = hiveQueryId_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          hiveQueryId_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string hiveQueryId = 2;</code>
+       */
+      public Builder setHiveQueryId(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        hiveQueryId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string hiveQueryId = 2;</code>
+       */
+      public Builder clearHiveQueryId() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        hiveQueryId_ = getDefaultInstance().getHiveQueryId();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string hiveQueryId = 2;</code>
+       */
+      public Builder setHiveQueryIdBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        hiveQueryId_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional int64 timestamp = 3;
+      private long timestamp_ ;
+      /**
+       * <code>optional int64 timestamp = 3;</code>
+       */
+      public boolean hasTimestamp() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>optional int64 timestamp = 3;</code>
+       */
+      public long getTimestamp() {
+        return timestamp_;
+      }
+      /**
+       * <code>optional int64 timestamp = 3;</code>
+       */
+      public Builder setTimestamp(long value) {
+        bitField0_ |= 0x00000004;
+        timestamp_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int64 timestamp = 3;</code>
+       */
+      public Builder clearTimestamp() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        timestamp_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // optional string executionMode = 4;
+      private java.lang.Object executionMode_ = "";
+      /**
+       * <code>optional string executionMode = 4;</code>
+       */
+      public boolean hasExecutionMode() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>optional string executionMode = 4;</code>
+       */
+      public java.lang.String getExecutionMode() {
+        java.lang.Object ref = executionMode_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          executionMode_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string executionMode = 4;</code>
+       */
+      public com.google.protobuf.ByteString
+          getExecutionModeBytes() {
+        java.lang.Object ref = executionMode_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          executionMode_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string executionMode = 4;</code>
+       */
+      public Builder setExecutionMode(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000008;
+        executionMode_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string executionMode = 4;</code>
+       */
+      public Builder clearExecutionMode() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        executionMode_ = getDefaultInstance().getExecutionMode();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string executionMode = 4;</code>
+       */
+      public Builder setExecutionModeBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000008;
+        executionMode_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional string requestUser = 5;
+      private java.lang.Object requestUser_ = "";
+      /**
+       * <code>optional string requestUser = 5;</code>
+       */
+      public boolean hasRequestUser() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      /**
+       * <code>optional string requestUser = 5;</code>
+       */
+      public java.lang.String getRequestUser() {
+        java.lang.Object ref = requestUser_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          requestUser_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string requestUser = 5;</code>
+       */
+      public com.google.protobuf.ByteString
+          getRequestUserBytes() {
+        java.lang.Object ref = requestUser_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          requestUser_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string requestUser = 5;</code>
+       */
+      public Builder setRequestUser(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000010;
+        requestUser_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string requestUser = 5;</code>
+       */
+      public Builder clearRequestUser() {
+        bitField0_ = (bitField0_ & ~0x00000010);
+        requestUser_ = getDefaultInstance().getRequestUser();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string requestUser = 5;</code>
+       */
+      public Builder setRequestUserBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000010;
+        requestUser_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional string queue = 6;
+      private java.lang.Object queue_ = "";
+      /**
+       * <code>optional string queue = 6;</code>
+       */
+      public boolean hasQueue() {
+        return ((bitField0_ & 0x00000020) == 0x00000020);
+      }
+      /**
+       * <code>optional string queue = 6;</code>
+       */
+      public java.lang.String getQueue() {
+        java.lang.Object ref = queue_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          queue_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string queue = 6;</code>
+       */
+      public com.google.protobuf.ByteString
+          getQueueBytes() {
+        java.lang.Object ref = queue_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          queue_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string queue = 6;</code>
+       */
+      public Builder setQueue(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000020;
+        queue_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string queue = 6;</code>
+       */
+      public Builder clearQueue() {
+        bitField0_ = (bitField0_ & ~0x00000020);
+        queue_ = getDefaultInstance().getQueue();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string queue = 6;</code>
+       */
+      public Builder setQueueBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000020;
+        queue_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional string user = 7;
+      private java.lang.Object user_ = "";
+      /**
+       * <code>optional string user = 7;</code>
+       */
+      public boolean hasUser() {
+        return ((bitField0_ & 0x00000040) == 0x00000040);
+      }
+      /**
+       * <code>optional string user = 7;</code>
+       */
+      public java.lang.String getUser() {
+        java.lang.Object ref = user_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          user_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string user = 7;</code>
+       */
+      public com.google.protobuf.ByteString
+          getUserBytes() {
+        java.lang.Object ref = user_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          user_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string user = 7;</code>
+       */
+      public Builder setUser(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000040;
+        user_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string user = 7;</code>
+       */
+      public Builder clearUser() {
+        bitField0_ = (bitField0_ & ~0x00000040);
+        user_ = getDefaultInstance().getUser();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string user = 7;</code>
+       */
+      public Builder setUserBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000040;
+        user_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional string operationId = 8;
+      private java.lang.Object operationId_ = "";
+      /**
+       * <code>optional string operationId = 8;</code>
+       */
+      public boolean hasOperationId() {
+        return ((bitField0_ & 0x00000080) == 0x00000080);
+      }
+      /**
+       * <code>optional string operationId = 8;</code>
+       */
+      public java.lang.String getOperationId() {
+        java.lang.Object ref = operationId_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          operationId_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string operationId = 8;</code>
+       */
+      public com.google.protobuf.ByteString
+          getOperationIdBytes() {
+        java.lang.Object ref = operationId_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          operationId_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string operationId = 8;</code>
+       */
+      public Builder setOperationId(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000080;
+        operationId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string operationId = 8;</code>
+       */
+      public Builder clearOperationId() {
+        bitField0_ = (bitField0_ & ~0x00000080);
+        operationId_ = getDefaultInstance().getOperationId();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string operationId = 8;</code>
+       */
+      public Builder setOperationIdBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000080;
+        operationId_ = value;
+        onChanged();
+        return this;
+      }
+
+      // repeated string tablesWritten = 9;
+      private com.google.protobuf.LazyStringList tablesWritten_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      private void ensureTablesWrittenIsMutable() {
+        if (!((bitField0_ & 0x00000100) == 0x00000100)) {
+          tablesWritten_ = new com.google.protobuf.LazyStringArrayList(tablesWritten_);
+          bitField0_ |= 0x00000100;
+         }
+      }
+      /**
+       * <code>repeated string tablesWritten = 9;</code>
+       */
+      public java.util.List<java.lang.String>
+          getTablesWrittenList() {
+        return java.util.Collections.unmodifiableList(tablesWritten_);
+      }
+      /**
+       * <code>repeated string tablesWritten = 9;</code>
+       */
+      public int getTablesWrittenCount() {
+        return tablesWritten_.size();
+      }
+      /**
+       * <code>repeated string tablesWritten = 9;</code>
+       */
+      public java.lang.String getTablesWritten(int index) {
+        return tablesWritten_.get(index);
+      }
+      /**
+       * <code>repeated string tablesWritten = 9;</code>
+       */
+      public com.google.protobuf.ByteString
+          getTablesWrittenBytes(int index) {
+        return tablesWritten_.getByteString(index);
+      }
+      /**
+       * <code>repeated string tablesWritten = 9;</code>
+       */
+      public Builder setTablesWritten(
+          int index, java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureTablesWrittenIsMutable();
+        tablesWritten_.set(index, value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string tablesWritten = 9;</code>
+       */
+      public Builder addTablesWritten(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureTablesWrittenIsMutable();
+        tablesWritten_.add(value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string tablesWritten = 9;</code>
+       */
+      public Builder addAllTablesWritten(
+          java.lang.Iterable<java.lang.String> values) {
+        ensureTablesWrittenIsMutable();
+        super.addAll(values, tablesWritten_);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string tablesWritten = 9;</code>
+       */
+      public Builder clearTablesWritten() {
+        tablesWritten_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000100);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string tablesWritten = 9;</code>
+       */
+      public Builder addTablesWrittenBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureTablesWrittenIsMutable();
+        tablesWritten_.add(value);
+        onChanged();
+        return this;
+      }
+
+      // repeated string tablesRead = 10;
+      private com.google.protobuf.LazyStringList tablesRead_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      private void ensureTablesReadIsMutable() {
+        if (!((bitField0_ & 0x00000200) == 0x00000200)) {
+          tablesRead_ = new com.google.protobuf.LazyStringArrayList(tablesRead_);
+          bitField0_ |= 0x00000200;
+         }
+      }
+      /**
+       * <code>repeated string tablesRead = 10;</code>
+       */
+      public java.util.List<java.lang.String>
+          getTablesReadList() {
+        return java.util.Collections.unmodifiableList(tablesRead_);
+      }
+      /**
+       * <code>repeated string tablesRead = 10;</code>
+       */
+      public int getTablesReadCount() {
+        return tablesRead_.size();
+      }
+      /**
+       * <code>repeated string tablesRead = 10;</code>
+       */
+      public java.lang.String getTablesRead(int index) {
+        return tablesRead_.get(index);
+      }
+      /**
+       * <code>repeated string tablesRead = 10;</code>
+       */
+      public com.google.protobuf.ByteString
+          getTablesReadBytes(int index) {
+        return tablesRead_.getByteString(index);
+      }
+      /**
+       * <code>repeated string tablesRead = 10;</code>
+       */
+      public Builder setTablesRead(
+          int index, java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureTablesReadIsMutable();
+        tablesRead_.set(index, value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string tablesRead = 10;</code>
+       */
+      public Builder addTablesRead(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureTablesReadIsMutable();
+        tablesRead_.add(value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string tablesRead = 10;</code>
+       */
+      public Builder addAllTablesRead(
+          java.lang.Iterable<java.lang.String> values) {
+        ensureTablesReadIsMutable();
+        super.addAll(values, tablesRead_);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string tablesRead = 10;</code>
+       */
+      public Builder clearTablesRead() {
+        tablesRead_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000200);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string tablesRead = 10

<TRUNCATED>