You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2009/08/02 06:39:50 UTC

svn commit: r799987 - in /hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/client/tableindexed/migration: ./ IndexSpecificationArray.java MoveIndexMetaData.java

Author: stack
Date: Sun Aug  2 04:39:50 2009
New Revision: 799987

URL: http://svn.apache.org/viewvc?rev=799987&view=rev
Log:
HBASE-1726 Migration to 0.20 looses index information from HTableDescriptor

Added:
    hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/client/tableindexed/migration/
    hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/client/tableindexed/migration/IndexSpecificationArray.java
    hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/client/tableindexed/migration/MoveIndexMetaData.java

Added: hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/client/tableindexed/migration/IndexSpecificationArray.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/client/tableindexed/migration/IndexSpecificationArray.java?rev=799987&view=auto
==============================================================================
--- hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/client/tableindexed/migration/IndexSpecificationArray.java (added)
+++ hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/client/tableindexed/migration/IndexSpecificationArray.java Sun Aug  2 04:39:50 2009
@@ -0,0 +1,51 @@
+package org.apache.hadoop.hbase.client.tableindexed.migration;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.client.tableindexed.IndexSpecification;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Holds an array of index specifications. Copied from 0.20 hbase to migrate indexes.
+ */
+public class IndexSpecificationArray implements Writable {
+
+    private IndexSpecification[] indexSpecifications;
+
+    public IndexSpecificationArray() {
+    // FOr writable
+    }
+
+    public IndexSpecificationArray(final IndexSpecification[] specs) {
+        this.indexSpecifications = specs;
+    }
+
+    public void readFields(final DataInput in) throws IOException {
+        int size = in.readInt();
+        indexSpecifications = new IndexSpecification[size];
+        for (int i = 0; i < size; i++) {
+            indexSpecifications[i] = new IndexSpecification();
+            indexSpecifications[i].readFields(in);
+        }
+
+    }
+
+    public void write(final DataOutput out) throws IOException {
+        out.writeInt(indexSpecifications.length);
+        for (IndexSpecification indexSpec : indexSpecifications) {
+            indexSpec.write(out);
+        }
+    }
+
+    /**
+     * Get indexSpecifications.
+     * 
+     * @return indexSpecifications
+     */
+    public IndexSpecification[] getIndexSpecifications() {
+        return indexSpecifications;
+    }
+
+}

Added: hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/client/tableindexed/migration/MoveIndexMetaData.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/client/tableindexed/migration/MoveIndexMetaData.java?rev=799987&view=auto
==============================================================================
--- hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/client/tableindexed/migration/MoveIndexMetaData.java (added)
+++ hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/client/tableindexed/migration/MoveIndexMetaData.java Sun Aug  2 04:39:50 2009
@@ -0,0 +1,50 @@
+package org.apache.hadoop.hbase.client.tableindexed.migration;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.tableindexed.IndexSpecification;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * One time migration account for index metadata change for 0.20. (Not handled by hbase migration).
+ */
+public class MoveIndexMetaData {
+
+    private static final byte[] INDEXES_KEY = Bytes.toBytes("INDEXES");
+
+    public static void main(final String[] args) {
+        try {
+            HBaseAdmin admin = new HBaseAdmin(new HBaseConfiguration());
+            for (HTableDescriptor tableDesc : admin.listTables()) {
+                if (!tableDesc.getIndexes().isEmpty()) {
+                    admin.disableTable(tableDesc.getName());
+                    writeToTable(tableDesc.getIndexes(), tableDesc);
+                    admin.modifyTable(tableDesc.getName(), tableDesc);
+                    admin.enableTable(tableDesc.getName());
+                }
+            }
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private static void writeToTable(final Collection<IndexSpecification> indexes, final HTableDescriptor tableDesc) {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutputStream dos = new DataOutputStream(baos);
+        IndexSpecificationArray indexArray = new IndexSpecificationArray(indexes.toArray(new IndexSpecification[0]));
+
+        try {
+            indexArray.write(dos);
+            dos.flush();
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+        tableDesc.setValue(INDEXES_KEY, baos.toByteArray());
+    }
+}