accumulo-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From e..@apache.org
Subject accumulo git commit: ACCUMULO-3795 fix bug: not sending the batchTimeout for regular scans
Date Fri, 26 Jun 2015 16:50:12 GMT
Repository: accumulo
Updated Branches:
  refs/heads/master 819946477 -> 51f744143


ACCUMULO-3795 fix bug: not sending the batchTimeout for regular scans


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

Branch: refs/heads/master
Commit: 51f744143bb3911ef5e6c8a4b3d91455e0003b1c
Parents: 8199464
Author: Eric C. Newton <eric.newton@gmail.com>
Authored: Fri Jun 26 12:50:03 2015 -0400
Committer: Eric C. Newton <eric.newton@gmail.com>
Committed: Fri Jun 26 12:50:03 2015 -0400

----------------------------------------------------------------------
 .../apache/accumulo/core/client/impl/ScannerIterator.java |  2 +-
 .../apache/accumulo/core/client/impl/ThriftScanner.java   |  8 +-------
 .../org/apache/accumulo/test/ScanFlushWithTimeIT.java     | 10 +++-------
 3 files changed, 5 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/51f74414/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java
b/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java
index fd91b5a..764db21 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java
@@ -119,7 +119,7 @@ public class ScannerIterator implements Iterator<Entry<Key,Value>>
{
     }
 
     scanState = new ScanState(context, tableId, authorizations, new Range(range), options.fetchedColumns,
size, options.serverSideIteratorList,
-        options.serverSideIteratorOptions, isolated, readaheadThreshold);
+        options.serverSideIteratorOptions, isolated, readaheadThreshold, options.batchTimeOut);
 
     // If we want to start readahead immediately, don't wait for hasNext to be called
     if (0l == readaheadThreshold) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/51f74414/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
index 2116cf2..5437cc4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
@@ -91,7 +91,7 @@ public class ThriftScanner {
       try {
         // not reading whole rows (or stopping on row boundries) so there is no need to enable
isolation below
         ScanState scanState = new ScanState(context, extent.getTableId(), authorizations,
range, fetchedColumns, size, serverSideIteratorList,
-            serverSideIteratorOptions, false, batchTimeOut);
+            serverSideIteratorOptions, false, Constants.SCANNER_DEFAULT_READAHEAD_THRESHOLD,
batchTimeOut);
 
         TabletType ttype = TabletType.type(extent);
         boolean waitForWrites = !serversWaitedForWrites.get(ttype).contains(server);
@@ -153,12 +153,6 @@ public class ThriftScanner {
     Map<String,Map<String,String>> serverSideIteratorOptions;
 
     public ScanState(ClientContext context, Text tableId, Authorizations authorizations,
Range range, SortedSet<Column> fetchedColumns, int size,
-        List<IterInfo> serverSideIteratorList, Map<String,Map<String,String>>
serverSideIteratorOptions, boolean isolated, long batchTimeOut) {
-      this(context, tableId, authorizations, range, fetchedColumns, size, serverSideIteratorList,
serverSideIteratorOptions, isolated,
-          Constants.SCANNER_DEFAULT_READAHEAD_THRESHOLD, batchTimeOut);
-    }
-
-    public ScanState(ClientContext context, Text tableId, Authorizations authorizations,
Range range, SortedSet<Column> fetchedColumns, int size,
         List<IterInfo> serverSideIteratorList, Map<String,Map<String,String>>
serverSideIteratorOptions, boolean isolated, long readaheadThreshold,
         long batchTimeOut) {
       this.context = context;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/51f74414/test/src/main/java/org/apache/accumulo/test/ScanFlushWithTimeIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/ScanFlushWithTimeIT.java b/test/src/main/java/org/apache/accumulo/test/ScanFlushWithTimeIT.java
index cb28856..0edd2e8 100644
--- a/test/src/main/java/org/apache/accumulo/test/ScanFlushWithTimeIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ScanFlushWithTimeIT.java
@@ -57,9 +57,7 @@ public class ScanFlushWithTimeIT extends AccumuloClusterHarness {
     c.tableOperations().attachIterator(tableName, setting);
     log.info("Splitting the table");
     SortedSet<Text> partitionKeys = new TreeSet<>();
-    for (int i = 1; i < 10; i++) {
-      partitionKeys.add(new Text("" + i));
-    }
+    partitionKeys.add(new Text("5"));
     c.tableOperations().addSplits(tableName, partitionKeys);
     log.info("waiting for zookeeper propagation");
     UtilWaitThread.sleep(5 * 1000);
@@ -76,15 +74,13 @@ public class ScanFlushWithTimeIT extends AccumuloClusterHarness {
     log.info("Scanner");
     Scanner s = c.createScanner(tableName, Authorizations.EMPTY);
     s.setBatchTimeout(500, TimeUnit.MILLISECONDS);
-    s.setBatchSize(100);
-    testScanner(s, 500);
+    testScanner(s, 1200);
 
     log.info("IsolatedScanner");
     IsolatedScanner is = new IsolatedScanner(s);
-    is.setBatchTimeout(500, TimeUnit.MILLISECONDS);
     is.setReadaheadThreshold(1);
     // buffers an entire row
-    testScanner(is, 1200);
+    testScanner(is, 2200);
 
     log.info("BatchScanner");
     BatchScanner bs = c.createBatchScanner(tableName, Authorizations.EMPTY, 5);


Mime
View raw message