You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2015/10/19 05:22:37 UTC

[1/2] phoenix git commit: PHOENIX-2313 TypeMismatchException thrown while querying a table that has an index with a Boolean

Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-0.98 192c0aca5 -> 6ba8f9b51


PHOENIX-2313 TypeMismatchException thrown while querying a table that has an index with a Boolean


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

Branch: refs/heads/4.x-HBase-0.98
Commit: 4653babe786c5c5b5899f5d8831bff3f2f5685fa
Parents: 192c0ac
Author: James Taylor <jt...@salesforce.com>
Authored: Sun Oct 18 15:38:03 2015 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Sun Oct 18 20:20:31 2015 -0700

----------------------------------------------------------------------
 .../end2end/index/BaseMutableIndexIT.java       |  11 +-
 .../apache/phoenix/schema/types/PBoolean.java   | 208 +++++++++----------
 .../phoenix/schema/types/PDataTypeTest.java     |  33 +++
 3 files changed, 144 insertions(+), 108 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/4653babe/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseMutableIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseMutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseMutableIndexIT.java
index 68998cf..ea12245 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseMutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseMutableIndexIT.java
@@ -118,12 +118,17 @@ public abstract class BaseMutableIndexIT extends BaseHBaseManagedTimeIT {
             "create " + (localIndex ? "LOCAL" : "") + " index i on t(b)");
         conn1.createStatement().execute("upsert into t values(1,true,'foo')");
         conn1.createStatement().execute("upsert into t values(2,false,'foo')");
+        conn1.createStatement().execute("upsert into t values(3)");
         conn1.commit();
         ResultSet rs = conn1.createStatement().executeQuery("select b from t");
-        rs.next();
-        assertEquals(true, rs.getBoolean(1));
-        rs.next();
+        assertTrue(rs.next());
+        assertEquals(false, rs.getBoolean(1));
+        assertTrue(rs.wasNull());
+        assertTrue(rs.next());
         assertEquals(false, rs.getBoolean(1));
+        assertTrue(rs.next());
+        assertEquals(true, rs.getBoolean(1));
+        assertFalse(rs.next());
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4653babe/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBoolean.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBoolean.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBoolean.java
index 9892426..f90d70b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBoolean.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBoolean.java
@@ -27,122 +27,120 @@ import com.google.common.primitives.Booleans;
 
 public class PBoolean extends PDataType<Boolean> {
 
-  public static final PBoolean INSTANCE = new PBoolean();
+    public static final PBoolean INSTANCE = new PBoolean();
 
-  private PBoolean() {
-    super("BOOLEAN", Types.BOOLEAN, Boolean.class, null, 21);
-  }
+    private PBoolean() {
+        super("BOOLEAN", Types.BOOLEAN, Boolean.class, null, 21);
+    }
 
-  @Override
-  public byte[] toBytes(Object object) {
-    if (object == null) {
-      // TODO: review - return null?
-      throw newIllegalDataException(this + " may not be null");
+    @Override
+    public byte[] toBytes(Object object) {
+        if (object == null) {
+            // TODO: review - return null?
+            throw newIllegalDataException(this + " may not be null");
+        }
+        return ((Boolean) object).booleanValue() ? TRUE_BYTES : FALSE_BYTES;
     }
-    return ((Boolean) object).booleanValue() ? TRUE_BYTES : FALSE_BYTES;
-  }
-
-  @Override
-  public int toBytes(Object object, byte[] bytes, int offset) {
-    if (object == null) {
-      // TODO: review - return null?
-      throw newIllegalDataException(this + " may not be null");
+
+    @Override
+    public int toBytes(Object object, byte[] bytes, int offset) {
+        if (object == null) {
+            // TODO: review - return null?
+            throw newIllegalDataException(this + " may not be null");
+        }
+        bytes[offset] = ((Boolean) object).booleanValue() ? TRUE_BYTE : FALSE_BYTE;
+        return BOOLEAN_LENGTH;
     }
-    bytes[offset] = ((Boolean) object).booleanValue() ? TRUE_BYTE : FALSE_BYTE;
-    return BOOLEAN_LENGTH;
-  }
-
-  @Override
-  public byte[] toBytes(Object object, SortOrder sortOrder) {
-    if (object == null) {
-      // TODO: review - return null?
-      throw newIllegalDataException(this + " may not be null");
+
+    @Override
+    public byte[] toBytes(Object object, SortOrder sortOrder) {
+        if (object == null) {
+            // TODO: review - return null?
+            throw newIllegalDataException(this + " may not be null");
+        }
+        return ((Boolean) object).booleanValue() ^ sortOrder == SortOrder.ASC ?
+                FALSE_BYTES :
+                    TRUE_BYTES;
     }
-    return ((Boolean) object).booleanValue() ^ sortOrder == SortOrder.ASC ?
-        TRUE_BYTES :
-        FALSE_BYTES;
-  }
-
-  @Override
-  public Boolean toObject(byte[] bytes, int offset, int length, PDataType actualType,
-      SortOrder sortOrder, Integer maxLength, Integer scale) {
-    Preconditions.checkNotNull(sortOrder);
-    if (length == 0) {
-      return null;
+
+    @Override
+    public Boolean toObject(byte[] bytes, int offset, int length, PDataType actualType,
+            SortOrder sortOrder, Integer maxLength, Integer scale) {
+        Preconditions.checkNotNull(sortOrder);
+        if (length == 0) {
+            return null;
+        }
+        if (actualType == this) {
+            if (length > 1) {
+                throw newIllegalDataException("BOOLEAN may only be a single byte");
+            }
+            return ((bytes[offset] == FALSE_BYTE ^ sortOrder == SortOrder.DESC) ?
+                    Boolean.FALSE :
+                        Boolean.TRUE);
+        } else if (actualType == PDecimal.INSTANCE) {
+            // false translated to the ZERO_BYTE
+            return sortOrder == SortOrder.DESC ? SortOrder.invert(bytes[offset]) != ZERO_BYTE : bytes[offset] != ZERO_BYTE;
+        }
+        throwConstraintViolationException(actualType, this);
+        return null;
     }
-    if (actualType == this) {
-      if (length > 1) {
-        throw newIllegalDataException("BOOLEAN may only be a single byte");
-      }
-      return ((bytes[offset] == FALSE_BYTE ^ sortOrder == SortOrder.DESC) ?
-          Boolean.FALSE :
-          Boolean.TRUE);
-    } else if (actualType == PDecimal.INSTANCE) {
-      // false translated to the ZERO_BYTE
-      return ((bytes[offset] == ZERO_BYTE ^ sortOrder == SortOrder.DESC) ?
-          Boolean.FALSE :
-          Boolean.TRUE);
+
+    @Override
+    public boolean isCoercibleTo(PDataType targetType) {
+        return super.isCoercibleTo(targetType) || targetType.equals(PBinary.INSTANCE);
     }
-    throwConstraintViolationException(actualType, this);
-    return null;
-  }
-
-  @Override
-  public boolean isCoercibleTo(PDataType targetType) {
-    return super.isCoercibleTo(targetType) || targetType.equals(PBinary.INSTANCE);
-  }
-
-  @Override
-  public boolean isCastableTo(PDataType targetType) {
-    // Allow cast to BOOLEAN so it can be used in an index or group by
-    return super.isCastableTo(targetType) || targetType.equals(PDecimal.INSTANCE);
-  }
-
-  @Override
-  public boolean isFixedWidth() {
-    return true;
-  }
-
-  @Override
-  public Integer getByteSize() {
-    return BOOLEAN_LENGTH;
-  }
-
-  @Override
-  public int estimateByteSize(Object o) {
-    return BOOLEAN_LENGTH;
-  }
-
-  @Override
-  public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
-    return Booleans.compare((Boolean) lhs, (Boolean) rhs);
-  }
-
-  @Override
-  public Object toObject(String value) {
-    return Boolean.parseBoolean(value);
-  }
-
-  @Override
-  public Object toObject(Object object, PDataType actualType) {
-    if (actualType == this || object == null) {
-      return object;
+
+    @Override
+    public boolean isCastableTo(PDataType targetType) {
+        // Allow cast to BOOLEAN so it can be used in an index or group by
+        return super.isCastableTo(targetType) || targetType.equals(PDecimal.INSTANCE);
     }
-    if (actualType == PVarbinary.INSTANCE || actualType == PBinary.INSTANCE) {
-      byte[] bytes = (byte[]) object;
-      return toObject(bytes, 0, bytes.length);
+
+    @Override
+    public boolean isFixedWidth() {
+        return true;
     }
+
+    @Override
+    public Integer getByteSize() {
+        return BOOLEAN_LENGTH;
+    }
+
+    @Override
+    public int estimateByteSize(Object o) {
+        return BOOLEAN_LENGTH;
+    }
+
+    @Override
+    public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+        return Booleans.compare((Boolean) lhs, (Boolean) rhs);
+    }
+
+    @Override
+    public Object toObject(String value) {
+        return Boolean.parseBoolean(value);
+    }
+
+    @Override
+    public Object toObject(Object object, PDataType actualType) {
+        if (actualType == this || object == null) {
+            return object;
+        }
+        if (actualType == PVarbinary.INSTANCE || actualType == PBinary.INSTANCE) {
+            byte[] bytes = (byte[]) object;
+            return toObject(bytes, 0, bytes.length);
+        }
         if (actualType == PDecimal.INSTANCE) {
-            return ((BigDecimal) object).equals(BigDecimal.ONE) ? Boolean.TRUE : Boolean.FALSE;
+            return ((BigDecimal) object).equals(BigDecimal.ZERO) ? Boolean.FALSE : Boolean.TRUE;
         }
-    return throwConstraintViolationException(actualType, this);
-  }
-
-  @Override
-  public Object getSampleValue(Integer maxLength, Integer arrayLength) {
-    return RANDOM.get().nextBoolean();
-  }
-  
+        return throwConstraintViolationException(actualType, this);
+    }
+
+    @Override
+    public Object getSampleValue(Integer maxLength, Integer arrayLength) {
+        return RANDOM.get().nextBoolean();
+    }
+
     @Override
     public PhoenixArrayFactory getArrayFactory() {
         return new PhoenixArrayFactory() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4653babe/phoenix-core/src/test/java/org/apache/phoenix/schema/types/PDataTypeTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/schema/types/PDataTypeTest.java b/phoenix-core/src/test/java/org/apache/phoenix/schema/types/PDataTypeTest.java
index 5657c22..7a04aeb 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/schema/types/PDataTypeTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/schema/types/PDataTypeTest.java
@@ -1799,4 +1799,37 @@ public class PDataTypeTest {
             assertTrue(Bytes.compareTo(array, QueryConstants.DESC_SEPARATOR_BYTE_ARRAY) <= 0);
         }
     }
+    
+    @Test
+    public void testBoolean() {
+        byte[] bytes = PBoolean.INSTANCE.toBytes(Boolean.TRUE);
+        assertEquals(1, bytes[0]);
+        bytes = PBoolean.INSTANCE.toBytes(Boolean.FALSE);
+        assertEquals(0, bytes[0]);
+        
+        bytes = PBoolean.INSTANCE.toBytes(Boolean.TRUE, SortOrder.DESC);
+        assertEquals(0, bytes[0]);
+        bytes = PBoolean.INSTANCE.toBytes(Boolean.FALSE, SortOrder.DESC);
+        assertEquals(1, bytes[0]);
+        
+        Object dec = PDecimal.INSTANCE.toObject(Boolean.TRUE, PBoolean.INSTANCE);
+        bytes = PDecimal.INSTANCE.toBytes(dec);
+        Object b = PBoolean.INSTANCE.toObject(bytes, 0, bytes.length, PDecimal.INSTANCE, SortOrder.ASC);
+        assertEquals(true, b);
+        
+        dec = PDecimal.INSTANCE.toObject(Boolean.FALSE, PBoolean.INSTANCE);
+        bytes = PDecimal.INSTANCE.toBytes(dec);
+        b = PBoolean.INSTANCE.toObject(bytes, 0, bytes.length, PDecimal.INSTANCE, SortOrder.ASC);
+        assertEquals(false, b);
+        
+        dec = PDecimal.INSTANCE.toObject(Boolean.TRUE, PBoolean.INSTANCE);
+        bytes = PDecimal.INSTANCE.toBytes(dec, SortOrder.DESC);
+        b = PBoolean.INSTANCE.toObject(bytes, 0, bytes.length, PDecimal.INSTANCE, SortOrder.DESC);
+        assertEquals(true, b);
+        
+        dec = PDecimal.INSTANCE.toObject(Boolean.FALSE, PBoolean.INSTANCE);
+        bytes = PDecimal.INSTANCE.toBytes(dec, SortOrder.DESC);
+        b = PBoolean.INSTANCE.toObject(bytes, 0, bytes.length, PDecimal.INSTANCE, SortOrder.DESC);
+        assertEquals(false, b);
+    }
 }


[2/2] phoenix git commit: PHOENIX-2335 Fix flapping MR-related tests

Posted by ja...@apache.org.
PHOENIX-2335 Fix flapping MR-related tests


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

Branch: refs/heads/4.x-HBase-0.98
Commit: 6ba8f9b51a75edaec17ac513993691b612e832e9
Parents: 4653bab
Author: James Taylor <jt...@salesforce.com>
Authored: Sun Oct 18 17:05:30 2015 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Sun Oct 18 20:20:49 2015 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/ConnectionUtilIT.java | 11 +++--------
 .../apache/phoenix/mapreduce/CsvBulkLoadToolIT.java  | 15 ++++++---------
 .../org/apache/phoenix/mapreduce/IndexToolIT.java    | 13 +++++--------
 3 files changed, 14 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6ba8f9b5/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionUtilIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionUtilIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionUtilIT.java
index babad81..65d2d37 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionUtilIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionUtilIT.java
@@ -19,7 +19,7 @@
 package org.apache.phoenix.end2end;
 
 import static org.apache.phoenix.query.BaseTest.setUpConfigForMiniCluster;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
 
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.phoenix.jdbc.PhoenixDriver;
 import org.apache.phoenix.mapreduce.util.ConnectionUtil;
-import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -77,13 +76,9 @@ public class ConnectionUtilIT {
 	@AfterClass
 	public static void tearDownAfterClass() throws Exception {
 		try {
-			PhoenixDriver.INSTANCE.close();
+            DriverManager.deregisterDriver(PhoenixDriver.INSTANCE);
 		} finally {
-			try {
-				DriverManager.deregisterDriver(PhoenixDriver.INSTANCE);
-			} finally {
-				hbaseTestUtil.shutdownMiniCluster();
-			}
+		    hbaseTestUtil.shutdownMiniCluster();
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6ba8f9b5/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/CsvBulkLoadToolIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/CsvBulkLoadToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/CsvBulkLoadToolIT.java
index 276bc47..e2ebad4 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/CsvBulkLoadToolIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/CsvBulkLoadToolIT.java
@@ -62,6 +62,7 @@ public class CsvBulkLoadToolIT {
         hbaseTestUtil.startMiniMapReduceCluster();
 
         Class.forName(PhoenixDriver.class.getName());
+        DriverManager.registerDriver(PhoenixDriver.INSTANCE);
         zkQuorum = "localhost:" + hbaseTestUtil.getZkCluster().getClientPort();
         conn = DriverManager.getConnection(PhoenixRuntime.JDBC_PROTOCOL
                 + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkQuorum);
@@ -70,19 +71,15 @@ public class CsvBulkLoadToolIT {
     @AfterClass
     public static void tearDownAfterClass() throws Exception {
         try {
-            conn.close();
+            if (conn != null) conn.close();
         } finally {
             try {
-                PhoenixDriver.INSTANCE.close();
+                DriverManager.deregisterDriver(PhoenixDriver.INSTANCE);
             } finally {
                 try {
-                    DriverManager.deregisterDriver(PhoenixDriver.INSTANCE);
-                } finally {                    
-                    try {
-                        hbaseTestUtil.shutdownMiniMapReduceCluster();
-                    } finally {
-                        hbaseTestUtil.shutdownMiniCluster();
-                    }
+                    hbaseTestUtil.shutdownMiniMapReduceCluster();
+                } finally {
+                    hbaseTestUtil.shutdownMiniCluster();
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6ba8f9b5/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/IndexToolIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/IndexToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/IndexToolIT.java
index bc85c6a..30cae36 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/IndexToolIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/IndexToolIT.java
@@ -67,6 +67,7 @@ public class IndexToolIT {
         hbaseTestUtil.startMiniCluster();
         hbaseTestUtil.startMiniMapReduceCluster();
         Class.forName(PhoenixDriver.class.getName());
+        DriverManager.registerDriver(PhoenixDriver.INSTANCE);
         zkQuorum = "localhost:" + hbaseTestUtil.getZkCluster().getClientPort();
     }
     
@@ -321,16 +322,12 @@ public class IndexToolIT {
     @AfterClass
     public static void tearDownAfterClass() throws Exception {
         try {
-            PhoenixDriver.INSTANCE.close();
+            DriverManager.deregisterDriver(PhoenixDriver.INSTANCE);
         } finally {
             try {
-                DriverManager.deregisterDriver(PhoenixDriver.INSTANCE);
-            } finally {                    
-                try {
-                    hbaseTestUtil.shutdownMiniMapReduceCluster();
-                } finally {
-                    hbaseTestUtil.shutdownMiniCluster();
-                }
+                hbaseTestUtil.shutdownMiniMapReduceCluster();
+            } finally {
+                hbaseTestUtil.shutdownMiniCluster();
             }
         }
     }