You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by je...@apache.org on 2014/02/10 08:55:37 UTC

git commit: Test Failures Fixes

Updated Branches:
  refs/heads/4.0.0 53f7d3ce8 -> 350017b17


Test Failures Fixes


Project: http://git-wip-us.apache.org/repos/asf/incubator-phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-phoenix/commit/350017b1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-phoenix/tree/350017b1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-phoenix/diff/350017b1

Branch: refs/heads/4.0.0
Commit: 350017b1764acbb3f88026e3c2f06d786b618387
Parents: 53f7d3c
Author: Jeffrey Zhong <jz...@JZhongs-MacBook-Pro.local>
Authored: Sun Feb 9 23:47:48 2014 -0800
Committer: Jeffrey Zhong <jz...@JZhongs-MacBook-Pro.local>
Committed: Sun Feb 9 23:54:01 2014 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/index/Indexer.java  | 40 ++++++-----
 .../apache/phoenix/client/ClientKeyValue.java   | 71 ++++----------------
 ...ALReplayWithIndexWritesAndCompressedWAL.java | 21 +++---
 3 files changed, 46 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/350017b1/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/Indexer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/Indexer.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/Indexer.java
index aa9df58..f958c81 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/Indexer.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/Indexer.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.hbase.index;
 
 import static org.apache.hadoop.hbase.index.util.IndexManagementUtil.rethrowIndexingException;
 
-import java.io.DataInput;
-import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -50,17 +48,6 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
-import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
-import org.apache.hadoop.hbase.regionserver.ScanType;
-import org.apache.hadoop.hbase.regionserver.Store;
-import org.apache.hadoop.hbase.regionserver.wal.HLog;
-import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
-import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
-import org.apache.hadoop.hbase.util.Pair;
-
-import com.google.common.collect.Multimap;
 import org.apache.hadoop.hbase.index.builder.IndexBuildManager;
 import org.apache.hadoop.hbase.index.builder.IndexBuilder;
 import org.apache.hadoop.hbase.index.builder.IndexBuildingFailureException;
@@ -73,8 +60,19 @@ import org.apache.hadoop.hbase.index.write.IndexWriter;
 import org.apache.hadoop.hbase.index.write.recovery.PerRegionIndexWriteCache;
 import org.apache.hadoop.hbase.index.write.recovery.StoreFailuresInCachePolicy;
 import org.apache.hadoop.hbase.index.write.recovery.TrackingParallelWriterIndexCommitter;
+import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
+import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
+import org.apache.hadoop.hbase.regionserver.ScanType;
+import org.apache.hadoop.hbase.regionserver.Store;
+import org.apache.hadoop.hbase.regionserver.wal.HLog;
+import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.util.MetaDataUtil;
 
+import com.google.common.collect.Multimap;
+
 /**
  * Do all the work of managing index updates from a single coprocessor. All Puts/Delets are passed
  * to an {@link IndexBuilder} to determine the actual updates to make.
@@ -288,6 +286,12 @@ public class Indexer extends BaseRegionObserver {
     Map<ImmutableBytesPtr, MultiMutation> mutations =
         new HashMap<ImmutableBytesPtr, MultiMutation>();
 
+    Durability defaultDurability = Durability.SYNC_WAL;
+    if(c.getEnvironment().getRegion() != null) {
+    	defaultDurability = c.getEnvironment().getRegion().getTableDesc().getDurability();
+    	defaultDurability = (defaultDurability == Durability.USE_DEFAULT) ? 
+    			Durability.SYNC_WAL : defaultDurability;
+    }
     Durability durability = Durability.SKIP_WAL;
     for (int i = 0; i < miniBatchOp.size(); i++) {
       // remove the batch keyvalue marker - its added for all puts
@@ -311,9 +315,10 @@ public class Indexer extends BaseRegionObserver {
         continue;
       }
       
-      // figure out if this is batch is durable or not
-      if (m.getDurability().ordinal() > durability.ordinal()) {
-        durability = m.getDurability();
+      Durability effectiveDurablity = (m.getDurability() == Durability.USE_DEFAULT) ? 
+    		  defaultDurability : m.getDurability();
+      if (effectiveDurablity.ordinal() > durability.ordinal()) {
+        durability = effectiveDurablity;
       }
 
       // add the mutation to the batch set
@@ -694,4 +699,5 @@ public class Indexer extends BaseRegionObserver {
     properties.put(Indexer.INDEX_BUILDER_CONF_KEY, builder.getName());
     desc.addCoprocessor(Indexer.class.getName(), null, Coprocessor.PRIORITY_USER, properties);
   }
-}
\ No newline at end of file
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/350017b1/phoenix-core/src/main/java/org/apache/phoenix/client/ClientKeyValue.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/client/ClientKeyValue.java b/phoenix-core/src/main/java/org/apache/phoenix/client/ClientKeyValue.java
index 90e8e0b..9b75e75 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/client/ClientKeyValue.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/client/ClientKeyValue.java
@@ -17,10 +17,6 @@
  */
 package org.apache.phoenix.client;
 
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.hadoop.hbase.HConstants;
@@ -53,7 +49,7 @@ public class ClientKeyValue extends KeyValue {
   private Type type;
   private long ts;
   private ImmutableBytesWritable value;
-  private byte[] bytes = null;
+  private KeyValue delegate = null;
 
   /**
    * @param row must not be <tt>null</tt>
@@ -306,39 +302,6 @@ public class ClientKeyValue extends KeyValue {
     return this.matchingFamily(family) && matchingQualifier(qualifier);
   }
 
-  public void write(DataOutput out) throws IOException {
-    // we need to simulate the keyvalue writing, but actually step through each buffer.
-    //start with keylength
-    long longkeylength = KeyValue.KEY_INFRASTRUCTURE_SIZE + row.getLength() + family.getLength()
-        + qualifier.getLength();
-    if (longkeylength > Integer.MAX_VALUE) {
-      throw new IllegalArgumentException("keylength " + longkeylength + " > " + Integer.MAX_VALUE);
-    }
-    // need to figure out the max length before we start
-    int length = this.getLength();
-    out.writeInt(length);
-
-    // write the actual data
-    int keylength = (int) longkeylength;
-    out.writeInt(keylength);
-    int vlength = value == null ? 0 : value.getLength();
-    out.writeInt(vlength);
-    out.writeShort((short) (row.getLength() & 0x0000ffff));
-    out.write(this.row.get(), this.row.getOffset(), this.row.getLength());
-    out.writeByte((byte) (family.getLength() & 0x0000ff));
-    if (family.getLength() != 0) {
-      out.write(this.family.get(), this.family.getOffset(), this.family.getLength());
-    }
-    if (qualifier != NULL) {
-      out.write(this.qualifier.get(), this.qualifier.getOffset(), this.qualifier.getLength());
-    }
-    out.writeLong(ts);
-    out.writeByte(this.type.getCode());
-    if (this.value != NULL) {
-      out.write(this.value.get(), this.value.getOffset(), this.value.getLength());
-    }
-  }
-
   @Override
   public int getLength() {
     return KEYVALUE_INFRASTRUCTURE_SIZE + KeyValue.ROW_LENGTH_SIZE + row.getLength()
@@ -450,27 +413,17 @@ public class ClientKeyValue extends KeyValue {
         + " does not support a single backing buffer.");
   }
 
+  @SuppressWarnings("deprecation")
   @Override
   public byte[] getBuffer() {
-	if(this.bytes != null) return this.bytes;
-	try {
-		ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
-		DataOutputStream out = new DataOutputStream(byteStream);
-		try {
-			write(out);
-			this.bytes = byteStream.toByteArray();
-			return this.bytes;
-		} finally {
-			if (out != null) {
-				out.close();
-				out = null;
-			}
-		}
-	} catch (IOException ioe) {
-		throw new UnsupportedOperationException(
-				ClientKeyValue.class.getSimpleName()
-						+ " can not being serialized to a single backing buffer. Due to " + ioe);
-	}
+    if (this.delegate != null) {
+      return this.delegate.getBuffer();
+    }
+    this.delegate =
+        new KeyValue(row.get(), row.getOffset(), row.getLength(), family.get(), family.getOffset(),
+            family.getLength(), qualifier.get(), qualifier.getOffset(), qualifier.getLength(),
+            this.ts, type, value.get(), value.getOffset(), value.getLength());
+    return this.delegate.getBuffer();
   }
-  
-}
\ No newline at end of file
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/350017b1/phoenix-core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayWithIndexWritesAndCompressedWAL.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayWithIndexWritesAndCompressedWAL.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayWithIndexWritesAndCompressedWAL.java
index b8c78c9..003f2b1 100644
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayWithIndexWritesAndCompressedWAL.java
+++ b/phoenix-core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayWithIndexWritesAndCompressedWAL.java
@@ -25,6 +25,12 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.index.IndexTestingUtils;
+import org.apache.hadoop.hbase.index.TableName;
+import org.apache.hadoop.hbase.index.covered.example.ColumnGroup;
+import org.apache.hadoop.hbase.index.covered.example.CoveredColumn;
+import org.apache.hadoop.hbase.index.covered.example.CoveredColumnIndexSpecifierBuilder;
+import org.apache.hadoop.hbase.index.covered.example.CoveredColumnIndexer;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionServerAccounting;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
@@ -36,13 +42,6 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.mockito.Mockito;
 
-import org.apache.hadoop.hbase.index.IndexTestingUtils;
-import org.apache.hadoop.hbase.index.TableName;
-import org.apache.hadoop.hbase.index.covered.example.ColumnGroup;
-import org.apache.hadoop.hbase.index.covered.example.CoveredColumn;
-import org.apache.hadoop.hbase.index.covered.example.CoveredColumnIndexSpecifierBuilder;
-import org.apache.hadoop.hbase.index.covered.example.CoveredColumnIndexer;
-
 /**
  * For pre-0.94.9 instances, this class tests correctly deserializing WALEdits w/o compression. Post
  * 0.94.9 we can support a custom {@link WALEditCodec}, which handles reading/writing the compressed
@@ -172,8 +171,9 @@ public class TestWALReplayWithIndexWritesAndCompressedWAL {
     Mockito.when(mockServerName.getServerName()).thenReturn(tableNameStr + ",1234");
     Mockito.when(mockRS.getServerName()).thenReturn(mockServerName);
     HRegion region = new HRegion(basedir, wal, this.fs, this.conf, hri, htd, mockRS);
-    long seqid = region.initialize();
-    
+    region.initialize();
+    region.getSequenceId().set(0);
+
     //make an attempted write to the primary that should also be indexed
     byte[] rowkey = Bytes.toBytes("indexed_row_key");
     Put p = new Put(rowkey);
@@ -274,4 +274,5 @@ public class TestWALReplayWithIndexWritesAndCompressedWAL {
 
     return count;
   }
-}
\ No newline at end of file
+}
+