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 2016/05/06 00:25:03 UTC

[1/3] phoenix git commit: PHOENIX-2824 PhoenixTransactionalIndexer rollback doesn't work correctly

Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.0 01cc2a02b -> 4900b0b3c


PHOENIX-2824 PhoenixTransactionalIndexer rollback doesn't work correctly


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

Branch: refs/heads/4.x-HBase-1.0
Commit: 0302e539f60f7267f39f77f36cff152a338b7e17
Parents: ac9f928
Author: James Taylor <ja...@apache.org>
Authored: Thu May 5 15:58:02 2016 -0700
Committer: James Taylor <ja...@apache.org>
Committed: Thu May 5 17:22:31 2016 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/tx/TxCheckpointIT.java   | 22 +++++++++++++-----
 .../index/PhoenixTransactionalIndexer.java      | 24 ++++++++++++--------
 2 files changed, 30 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/0302e539/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
index c7fc053..11b587a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
@@ -37,7 +37,6 @@ import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
 import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.jdbc.PhoenixConnection;
-import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
@@ -49,10 +48,10 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
-import com.google.common.collect.Maps;
-
 import co.cask.tephra.Transaction.VisibilityLevel;
 
+import com.google.common.collect.Maps;
+
 @RunWith(Parameterized.class)
 public class TxCheckpointIT extends BaseHBaseManagedTimeIT {
 	
@@ -110,14 +109,25 @@ public class TxCheckpointIT extends BaseHBaseManagedTimeIT {
     }
     
     @Test
-    public void testRollbackOfUncommittedDelete() throws Exception {
+    public void testRollbackOfUncommittedDeleteSingleCol() throws Exception {
+        String indexDDL = "CREATE "+(localIndex? "LOCAL " : "")+"INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE(v2)";
+        testRollbackOfUncommittedDelete(indexDDL);
+    }
+
+    @Test
+    public void testRollbackOfUncommittedDeleteMultiCol() throws Exception {
+        String indexDDL = "CREATE "+(localIndex? "LOCAL " : "")+"INDEX " + indexName + " ON " + fullTableName + " (v1, v2)";
+        testRollbackOfUncommittedDelete(indexDDL);
+    }
+    
+    private void testRollbackOfUncommittedDelete(String indexDDL) throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
         try {
             Statement stmt = conn.createStatement();
             stmt.execute("CREATE TABLE " + fullTableName + "(k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)"+(!mutable? " IMMUTABLE_ROWS=true" : ""));
-            stmt.execute("CREATE "+(localIndex? "LOCAL " : "")+"INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE(v2)");
+            stmt.execute(indexDDL);
             
             stmt.executeUpdate("upsert into " + fullTableName + " values('x1', 'y1', 'a1')");
             stmt.executeUpdate("upsert into " + fullTableName + " values('x2', 'y2', 'a2')");
@@ -182,7 +192,7 @@ public class TxCheckpointIT extends BaseHBaseManagedTimeIT {
             //assert two rows in index table
             rs = stmt.executeQuery("select k, v1, v2 from " + fullTableName + " ORDER BY v1");
             assertTrue(rs.next());
-            assertEquals("x1", rs.getString(1));
+            assertEquals("x1", rs.getString(1)); // fails here
             assertEquals("y1", rs.getString(2));
             assertEquals("a1", rs.getString(3));
             assertTrue(rs.next());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0302e539/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixTransactionalIndexer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixTransactionalIndexer.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixTransactionalIndexer.java
index 134c50c..b3606f6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixTransactionalIndexer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixTransactionalIndexer.java
@@ -26,6 +26,7 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.ListIterator;
 import java.util.Map;
 import java.util.Set;
 
@@ -72,11 +73,6 @@ import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.ServerUtil;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import com.google.common.primitives.Longs;
-
 import co.cask.tephra.Transaction;
 import co.cask.tephra.Transaction;
 import co.cask.tephra.Transaction.VisibilityLevel;
@@ -85,6 +81,11 @@ import co.cask.tephra.TxConstants;
 import co.cask.tephra.TxConstants;
 import co.cask.tephra.hbase10.TransactionAwareHTable;
 
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import com.google.common.primitives.Longs;
+
 /**
  * Do all the work of managing index updates for a transactional table from a single coprocessor. Since the transaction
  * manager essentially time orders writes through conflict detection, the logic to maintain a secondary index is quite a
@@ -321,17 +322,20 @@ public class PhoenixTransactionalIndexer extends BaseRegionObserver {
                     boolean hasPuts = false;
                     LinkedList<Cell> singleTimeCells = Lists.newLinkedList();
                     long writePtr;
+                    Cell cell = cells.get(i);
                     do {
-                        Cell cell = cells.get(i);
                         hasPuts |= cell.getTypeByte() == KeyValue.Type.Put.getCode();
                         writePtr = cell.getTimestamp();
+                        ListIterator<Cell> it = singleTimeCells.listIterator();
                         do {
                             // Add at the beginning of the list to match the expected HBase
                             // newest to oldest sort order (which TxTableState relies on
-                            // with the Result.getLatestColumnValue() calls).
-                            singleTimeCells.addFirst(cell);
-                        } while (++i < nCells && cells.get(i).getTimestamp() == writePtr);
-                    } while (i < nCells && cells.get(i).getTimestamp() <= readPtr);
+                            // with the Result.getLatestColumnValue() calls). However, we
+                            // still want to add Cells in the expected order for each time
+                            // bound as otherwise we won't find it in our old state.
+                            it.add(cell);
+                        } while (++i < nCells && (cell=cells.get(i)).getTimestamp() == writePtr);
+                    } while (i < nCells && cell.getTimestamp() <= readPtr);
                     
                     // Generate point delete markers for the prior row deletion of the old index value.
                     // The write timestamp is the next timestamp, not the current timestamp,


[3/3] phoenix git commit: Merge branch '4.x-HBase-1.0' of https://git-wip-us.apache.org/repos/asf/phoenix into 4.x-HBase-1.0

Posted by ja...@apache.org.
Merge branch '4.x-HBase-1.0' of https://git-wip-us.apache.org/repos/asf/phoenix into 4.x-HBase-1.0


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

Branch: refs/heads/4.x-HBase-1.0
Commit: 4900b0b3c929396fcf082387a35e67b803ade2ff
Parents: 174fefd 01cc2a0
Author: James Taylor <ja...@apache.org>
Authored: Thu May 5 17:24:32 2016 -0700
Committer: James Taylor <ja...@apache.org>
Committed: Thu May 5 17:24:32 2016 -0700

----------------------------------------------------------------------
 .gitignore                                      |   1 +
 examples/STOCK_SYMBOL.sql                       |   1 -
 examples/WEB_STAT_QUERIES.sql                   |   1 -
 .../org/apache/phoenix/end2end/ArrayIT.java     |  55 ++++++++-
 .../end2end/BaseClientManagedTimeIT.java        |   2 +-
 .../phoenix/end2end/BaseHBaseManagedTimeIT.java |   2 +-
 .../BaseHBaseManagedTimeTableReuseIT.java       |  44 +++-----
 .../BaseOwnClusterClientManagedTimeIT.java      |   2 +-
 .../BaseOwnClusterHBaseManagedTimeIT.java       |   2 +-
 .../apache/phoenix/end2end/CreateSchemaIT.java  |   2 +-
 .../org/apache/phoenix/end2end/DateTimeIT.java  |  39 +++++++
 .../apache/phoenix/end2end/DistinctCountIT.java |  76 ++++++-------
 .../apache/phoenix/end2end/DropSchemaIT.java    |  21 +++-
 .../apache/phoenix/end2end/DynamicColumnIT.java | 112 +++++++++----------
 .../apache/phoenix/end2end/DynamicUpsertIT.java |  11 +-
 .../phoenix/end2end/InMemoryOrderByIT.java      |   6 +-
 .../org/apache/phoenix/end2end/OrderByIT.java   |  11 +-
 .../apache/phoenix/end2end/PercentileIT.java    |  76 +++----------
 .../phoenix/end2end/SpooledOrderByIT.java       |   4 +-
 .../phoenix/end2end/ToCharFunctionIT.java       |  28 ++---
 .../apache/phoenix/end2end/UpsertValuesIT.java  |   1 +
 .../phoenix/compile/StatementContext.java       |   5 +-
 .../phoenix/coprocessor/MetaDataProtocol.java   |   2 +-
 .../query/ConnectionQueryServicesImpl.java      |   6 +-
 .../java/org/apache/phoenix/util/DateUtil.java  |  16 ++-
 .../java/org/apache/phoenix/query/BaseTest.java |  84 +++++++++++++-
 .../org/apache/phoenix/util/DateUtilTest.java   |  25 +++++
 .../phoenix/end2end/QueryServerBasicsIT.java    |  41 ++++---
 28 files changed, 424 insertions(+), 252 deletions(-)
----------------------------------------------------------------------



[2/3] phoenix git commit: PHOENIX-2879 HivePhoenixStoreIT fails with timeout if JDK 1.7 used (Sergey Soldatov)

Posted by ja...@apache.org.
PHOENIX-2879 HivePhoenixStoreIT fails with timeout if JDK 1.7 used (Sergey Soldatov)


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

Branch: refs/heads/4.x-HBase-1.0
Commit: 174fefd7e4be864c07cbfd9319048f97050f07f4
Parents: 0302e53
Author: James Taylor <ja...@apache.org>
Authored: Thu May 5 16:11:43 2016 -0700
Committer: James Taylor <ja...@apache.org>
Committed: Thu May 5 17:22:34 2016 -0700

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/174fefd7/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 8f52433..8fb48f8 100644
--- a/pom.xml
+++ b/pom.xml
@@ -310,7 +310,7 @@
                  <forkCount>${numForkedIT}</forkCount>
                  <runOrder>alphabetical</runOrder>
                  <reuseForks>false</reuseForks>
-                 <argLine>-enableassertions -Xmx2000m -XX:MaxPermSize=128m -Djava.security.egd=file:/dev/./urandom "-Djava.library.path=${hadoop.library.path}${path.separator}${java.library.path}"</argLine>
+                 <argLine>-enableassertions -Xmx2000m -XX:MaxPermSize=256m -Djava.security.egd=file:/dev/./urandom "-Djava.library.path=${hadoop.library.path}${path.separator}${java.library.path}"</argLine>
                  <redirectTestOutputToFile>${test.output.tofile}</redirectTestOutputToFile>
                  <testSourceDirectory>${basedir}/src/it/java</testSourceDirectory>
                  <groups>org.apache.phoenix.end2end.NeedsOwnMiniClusterTest</groups>