tajo-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jihoon...@apache.org
Subject [16/19] git commit: TAJO-936: TestStorages::testSplitable is failed occasionally. (jinho)
Date Tue, 15 Jul 2014 07:16:55 GMT
TAJO-936: TestStorages::testSplitable is failed occasionally. (jinho)


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

Branch: refs/heads/index_support
Commit: aee7874faa45e3ba2eda3157683cf20111cacea5
Parents: 48dbfd9
Author: jinossy <jinossy@gmail.com>
Authored: Tue Jul 15 14:25:51 2014 +0900
Committer: jinossy <jinossy@gmail.com>
Committed: Tue Jul 15 14:25:51 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |  2 +
 .../org/apache/tajo/storage/rcfile/RCFile.java  |  1 +
 .../org/apache/tajo/storage/TestStorages.java   | 58 +++++++++++++++++++-
 3 files changed, 60 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/aee7874f/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 3803ded..473db19 100644
--- a/CHANGES
+++ b/CHANGES
@@ -84,6 +84,8 @@ Release 0.9.0 - unreleased
 
   BUG FIXES
 
+    TAJO-936: TestStorages::testSplitable is failed occasionally. (jinho)
+
     TAJO-673: Assign proper number of tasks when inserting into partitioned table. (jaehwa)
 
     TAJO-916: SubQuery::computeStatFromTasks occasionally fail.

http://git-wip-us.apache.org/repos/asf/tajo/blob/aee7874f/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java b/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java
index 1da41e4..4cc37b3 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java
@@ -1380,6 +1380,7 @@ public class RCFile {
 
       in.readFully(sync); // read sync bytes
       headerEnd = in.getPos();
+      lastSeenSyncPos = headerEnd; //initial sync position
       readBytes += headerEnd;
     }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/aee7874f/tajo-storage/src/test/java/org/apache/tajo/storage/TestStorages.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestStorages.java b/tajo-storage/src/test/java/org/apache/tajo/storage/TestStorages.java
index 944aa5c..1cf1ecf 100644
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/TestStorages.java
+++ b/tajo-storage/src/test/java/org/apache/tajo/storage/TestStorages.java
@@ -23,7 +23,9 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.tajo.QueryId;
 import org.apache.tajo.TajoIdProtos;
-import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.common.TajoDataTypes.Type;
@@ -175,6 +177,60 @@ public class TestStorages {
 	}
 
   @Test
+  public void testRCFileSplitable() throws IOException {
+    if (storeType == StoreType.RCFILE) {
+      Schema schema = new Schema();
+      schema.addColumn("id", Type.INT4);
+      schema.addColumn("age", Type.INT8);
+
+      TableMeta meta = CatalogUtil.newTableMeta(storeType);
+      Path tablePath = new Path(testDir, "Splitable.data");
+      Appender appender = StorageManagerFactory.getStorageManager(conf).getAppender(meta,
schema, tablePath);
+      appender.enableStats();
+      appender.init();
+      int tupleNum = 10000;
+      VTuple vTuple;
+
+      for (int i = 0; i < tupleNum; i++) {
+        vTuple = new VTuple(2);
+        vTuple.put(0, DatumFactory.createInt4(i + 1));
+        vTuple.put(1, DatumFactory.createInt8(25l));
+        appender.addTuple(vTuple);
+      }
+      appender.close();
+      TableStats stat = appender.getStats();
+      assertEquals(tupleNum, stat.getNumRows().longValue());
+
+      FileStatus status = fs.getFileStatus(tablePath);
+      long fileLen = status.getLen();
+      long randomNum = 122; // header size
+
+      FileFragment[] tablets = new FileFragment[2];
+      tablets[0] = new FileFragment("Splitable", tablePath, 0, randomNum);
+      tablets[1] = new FileFragment("Splitable", tablePath, randomNum, (fileLen - randomNum));
+
+      Scanner scanner = StorageManagerFactory.getStorageManager(conf).getScanner(meta, schema,
tablets[0], schema);
+      assertTrue(scanner.isSplittable());
+      scanner.init();
+      int tupleCnt = 0;
+      while (scanner.next() != null) {
+        tupleCnt++;
+      }
+      scanner.close();
+
+      scanner = StorageManagerFactory.getStorageManager(conf).getScanner(meta, schema, tablets[1],
schema);
+      assertTrue(scanner.isSplittable());
+      scanner.init();
+      while (scanner.next() != null) {
+        tupleCnt++;
+      }
+      scanner.close();
+
+      assertEquals(tupleNum, tupleCnt);
+    }
+  }
+
+  @Test
   public void testProjection() throws IOException {
     Schema schema = new Schema();
     schema.addColumn("id", Type.INT4);


Mime
View raw message