You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ec...@apache.org on 2012/09/01 02:40:08 UTC

svn commit: r1379683 - in /hive/trunk: common/src/java/org/apache/hadoop/hive/conf/ conf/ ql/src/java/org/apache/hadoop/hive/ql/parse/ ql/src/test/queries/clientpositive/ ql/src/test/results/clientpositive/

Author: ecapriolo
Date: Sat Sep  1 00:40:07 2012
New Revision: 1379683

URL: http://svn.apache.org/viewvc?rev=1379683&view=rev
Log:
HIVE-3068 Export table metadata as JSON on table drop (Andrew Chalfant via egc)

Added:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/MetaDataExportListener.java
    hive/trunk/ql/src/test/queries/clientpositive/metadata_export_drop.q
    hive/trunk/ql/src/test/results/clientpositive/metadata_export_drop.q.out
Modified:
    hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hive/trunk/conf/hive-default.xml.template

Modified: hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1379683&r1=1379682&r2=1379683&view=diff
==============================================================================
--- hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Sat Sep  1 00:40:07 2012
@@ -325,6 +325,11 @@ public class HiveConf extends Configurat
     METASTORE_END_FUNCTION_LISTENERS("hive.metastore.end.function.listeners", ""),
     METASTORE_PART_INHERIT_TBL_PROPS("hive.metastore.partition.inherit.table.properties",""),
 
+    // Parameters for exporting metadata on table drop (requires the use of the)
+    // org.apache.hadoop.hive.ql.parse.MetaDataExportListener preevent listener
+    METADATA_EXPORT_LOCATION("hive.metadata.export.location", ""),
+    MOVE_EXPORTED_METADATA_TO_TRASH("hive.metadata.move.exported.metadata.to.trash", true),
+
     // CLI
     CLIIGNOREERRORS("hive.cli.errors.ignore", false),
     CLIPRINTCURRENTDB("hive.cli.print.current.db", false),

Modified: hive/trunk/conf/hive-default.xml.template
URL: http://svn.apache.org/viewvc/hive/trunk/conf/hive-default.xml.template?rev=1379683&r1=1379682&r2=1379683&view=diff
==============================================================================
--- hive/trunk/conf/hive-default.xml.template (original)
+++ hive/trunk/conf/hive-default.xml.template Sat Sep  1 00:40:07 2012
@@ -256,6 +256,18 @@
 </property>
 
 <property>
+  <name>hive.metadata.export.location</name>
+  <value></value>
+  <description>When used in conjunction with the org.apache.hadoop.hive.ql.parse.MetaDataExportListener pre event listener, it is the location to which the metadata will be exported. The default is an empty string, which results in the metadata being exported to the current user's home directory on HDFS.</description>
+</property>
+
+<property>
+  <name>hive.metadata.move.exported.metadata.to.trash</name>
+  <value></value>
+  <description>When used in conjunction with the org.apache.hadoop.hive.ql.parse.MetaDataExportListener pre event listener, this setting determines if the metadata that is exported will subsequently be moved to the user's trash directory alongside the dropped table data. This ensures that the metadata will be cleaned up along with the dropped table data.</description>
+</property>
+
+<property>
   <name>hive.metastore.end.function.listeners</name>
   <value></value>
   <description>list of comma separated listeners for the end of metastore functions.</description>

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/MetaDataExportListener.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/MetaDataExportListener.java?rev=1379683&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/MetaDataExportListener.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/MetaDataExportListener.java Sat Sep  1 00:40:07 2012
@@ -0,0 +1,110 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.parse;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
+import org.apache.hadoop.hive.metastore.MetaStorePreEventListener;
+import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.events.PreDropTableEvent;
+import org.apache.hadoop.hive.metastore.events.PreEventContext;
+import org.apache.hadoop.hive.metastore.events.PreEventContext.PreEventType;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+/**
+ * This class listens for drop events and, if set, exports the table's metadata as JSON
+ * to the trash of the user performing the drop */
+public class MetaDataExportListener extends MetaStorePreEventListener {
+  public static final Log LOG = LogFactory.getLog(MetaDataExportListener.class);
+  private static final SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd-HH-mm-ss");
+
+  /** Configure the export listener */
+  public MetaDataExportListener(Configuration config) {
+    super(config);
+  }
+
+  /** Export the metadata to a given path, and then move it to the user's trash */
+  private void export_meta_data(PreDropTableEvent tableEvent) throws MetaException {
+    FileSystem fs = null;
+    Table tbl = tableEvent.getTable();
+    String name = tbl.getTableName();
+    org.apache.hadoop.hive.ql.metadata.Table mTbl =
+      new org.apache.hadoop.hive.ql.metadata.Table(tbl);
+    HMSHandler handler = tableEvent.getHandler();
+    HiveConf hiveconf = handler.getHiveConf();
+    Warehouse wh = new Warehouse(hiveconf);
+    Path tblPath = new Path(tbl.getSd().getLocation());
+    fs = wh.getFs(tblPath);
+    Date now = new Date();
+    String dateString = sdf.format(now);
+    String exportPathString = hiveconf.getVar(HiveConf.ConfVars.METADATA_EXPORT_LOCATION);
+    boolean moveMetadataToTrash =
+      hiveconf.getBoolVar(HiveConf.ConfVars.MOVE_EXPORTED_METADATA_TO_TRASH);
+    Path exportPath = null;
+    if (exportPathString != null && exportPathString.length() == 0) {
+      exportPath = fs.getHomeDirectory();
+    } else {
+      exportPath = new Path(exportPathString);
+    }
+    Path metaPath = new Path(exportPath, name + "." + dateString);
+    LOG.info("Exporting the metadata of table " +
+      tbl.toString() + " to path " + metaPath.toString());
+    try {
+      fs.mkdirs(metaPath);
+    } catch (IOException e) {
+      throw new MetaException(e.getMessage());
+    }
+    Path outFile = new Path(metaPath, name + ".metadata");
+    try {
+      SessionState.getConsole().printInfo("Beginning metadata export");
+      EximUtil.createExportDump(fs, outFile, mTbl, null);
+      if (moveMetadataToTrash == true) {
+        wh.deleteDir(metaPath, true);
+      }
+    } catch (IOException e) {
+      throw new MetaException(e.getMessage());
+    } catch (SemanticException e) {
+      throw new MetaException(e.getMessage());
+    }
+  }
+
+  /**
+   * Listen for an event; if it is a DROP_TABLE event, call export_meta_data
+   * */
+  @Override
+  public void onEvent(PreEventContext context) throws MetaException, NoSuchObjectException,
+      InvalidOperationException {
+    if (context.getEventType() == PreEventType.DROP_TABLE) {
+      export_meta_data((PreDropTableEvent) context);
+    }
+  }
+
+}

Added: hive/trunk/ql/src/test/queries/clientpositive/metadata_export_drop.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/metadata_export_drop.q?rev=1379683&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/metadata_export_drop.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/metadata_export_drop.q Sat Sep  1 00:40:07 2012
@@ -0,0 +1,9 @@
+create table tmp_meta_export_listener_drop_test (foo string);
+dfs -mkdir ../build/ql/test/data/exports/;
+set hive.metastore.pre.event.listeners=org.apache.hadoop.hive.ql.parse.MetaDataExportListener;
+set hive.metadata.export.location=../build/ql/test/data/exports/;
+set hive.move.exported.metadata.to.trash=false;
+drop table tmp_meta_export_listener_drop_test;
+dfs -rmr ../build/ql/test/data/exports/tmp_meta_export_listener_drop_test.*;
+dfs -rmr ../build/ql/test/data/exports/
+

Added: hive/trunk/ql/src/test/results/clientpositive/metadata_export_drop.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/metadata_export_drop.q.out?rev=1379683&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/metadata_export_drop.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/metadata_export_drop.q.out Sat Sep  1 00:40:07 2012
@@ -0,0 +1,14 @@
+PREHOOK: query: create table tmp_meta_export_listener_drop_test (foo string)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table tmp_meta_export_listener_drop_test (foo string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@tmp_meta_export_listener_drop_test
+PREHOOK: query: drop table tmp_meta_export_listener_drop_test
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@tmp_meta_export_listener_drop_test
+PREHOOK: Output: default@tmp_meta_export_listener_drop_test
+POSTHOOK: query: drop table tmp_meta_export_listener_drop_test
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@tmp_meta_export_listener_drop_test
+POSTHOOK: Output: default@tmp_meta_export_listener_drop_test
+#### A masked pattern was here ####