You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by me...@apache.org on 2015/06/23 21:00:05 UTC

[1/5] drill git commit: DRILL-3285: Part 5--Split old hacky next() into separate methods.

Repository: drill
Updated Branches:
  refs/heads/master 9c125b0d9 -> 711992f22


DRILL-3285: Part 5--Split old hacky next() into separate methods.

Split the original public next() method (which was hacked to handle an extra,
initial call to read the schema batch) into:
- new loadInitialSchema() (for handling the call for the schema)
- modified next() (for handling normal calls from ResultSet.next())
- new private nextRowInternally() (for common code)

Pulled invariant afterFirstBatch up out of bogus-batch loop.


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

Branch: refs/heads/master
Commit: 711992f22ae6d6dfc43bdb4c01bf8f921d175b38
Parents: 22232d4
Author: dbarclay <db...@maprtech.com>
Authored: Wed Jun 10 13:49:47 2015 -0700
Committer: Mehant Baid <me...@gmail.com>
Committed: Mon Jun 22 13:05:04 2015 -0700

----------------------------------------------------------------------
 .../org/apache/drill/jdbc/impl/DrillCursor.java | 124 +++++++++++++------
 .../drill/jdbc/impl/DrillResultSetImpl.java     |   2 +-
 2 files changed, 84 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/711992f2/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
index cdf030b..5ae7509 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
@@ -56,8 +56,7 @@ class DrillCursor implements Cursor {
   /** ... corresponds to current schema. */
   private DrillColumnMetaDataList columnMetaDataList;
 
-  /** Whether we're past the special first call to this.next() from
-   * DrillResultSetImpl.execute(). */
+  /** Whether loadInitialSchema() has been called. */
   private boolean initialSchemaLoaded = false;
 
   /** Whether after first batch.  (Re skipping spurious empty batches.) */
@@ -65,7 +64,8 @@ class DrillCursor implements Cursor {
 
   /**
    * Whether the next call to this.next() should just return {@code true} rather
-   * than trying to actually advance to the next record.
+   * than calling nextRowInternally() to try to advance to the next
+   * record.
    * <p>
    *   Currently, can be true only for first call to next().
    * </p>
@@ -120,33 +120,17 @@ class DrillCursor implements Cursor {
   }
 
   /**
-   * Advances this cursor to the next row, if any, or to after the sequence of
-   * rows if no next row.  However, the first call does not advance to the first
-   * row, only reading schema information.
+   * ...
    * <p>
-   *   Is to be called (once) from {@link DrillResultSetImpl#execute()}, and
-   *   then from {@link AvaticaResultSet#next()}.
+   *   Is to be called (once) from {@link #loadInitialSchema} for
+   *   {@link DrillResultSetImpl#execute()}, and then (repeatedly) from
+   *   {@link #next()} for {@link AvaticaResultSet#next()}.
    * </p>
    *
    * @return  whether cursor is positioned at a row (false when after end of
    *   results)
    */
-  @Override
-  public boolean next() throws SQLException {
-    if (!initialSchemaLoaded) {
-      initialSchemaLoaded = true;
-      returnTrueForNextCallToNext = true;
-    } else if (returnTrueForNextCallToNext && !afterLastRow) {
-      // We have a deferred "not after end" to report--reset and report that.
-      returnTrueForNextCallToNext = false;
-      return true;
-    }
-
-    if (afterLastRow) {
-      // We're already after end of rows/records--just report that after end.
-      return false;
-    }
-
+  private boolean nextRowInternally() throws SQLException {
     if (currentRecordNumber + 1 < currentBatchHolder.getRecordCount()) {
       // Have next row in current batch--just advance index and report "at a row."
       currentRecordNumber++;
@@ -161,28 +145,29 @@ class DrillCursor implements Cursor {
         // (Apparently:)  Skip any spurious empty batches (batches that have
         // zero rows and/or null data, other than the first batch (which carries
         // the (initial) schema but no rows)).
-        while ( qrb != null
-                && ( qrb.getHeader().getRowCount() == 0
-                     || qrb.getData() == null )
-                && afterFirstBatch ) {
-          // Empty message--dispose of and try to get another.
-          logger.warn( "Spurious batch read: {}", qrb );
+        if ( afterFirstBatch ) {
+          while ( qrb != null
+                  && ( qrb.getHeader().getRowCount() == 0
+                      || qrb.getData() == null ) ) {
+            // Empty message--dispose of and try to get another.
+            logger.warn( "Spurious batch read: {}", qrb );
 
-          qrb.release();
+            qrb.release();
 
-          qrb = resultsListener.getNext();
+            qrb = resultsListener.getNext();
 
-          // NOTE:  It is unclear why this check does not check getRowCount()
-          // as the loop condition above does.
-          if ( qrb != null && qrb.getData() == null ) {
-            // Got another batch with null data--dispose of and report "no more
-            // rows".
+            // NOTE:  It is unclear why this check does not check getRowCount()
+            // as the loop condition above does.
+            if ( qrb != null && qrb.getData() == null ) {
+              // Got another batch with null data--dispose of and report "no more
+              // rows".
 
-            qrb.release();
+              qrb.release();
 
-            // NOTE:  It is unclear why this returns false but doesn't set
-            // afterLastRow (as we do when we normally return false).
-            return false;
+              // NOTE:  It is unclear why this returns false but doesn't set
+              // afterLastRow (as we do when we normally return false).
+              return false;
+            }
           }
         }
 
@@ -246,6 +231,63 @@ class DrillCursor implements Cursor {
     }
   }
 
+  /**
+   * Advances to first batch to load schema data into result set metadata.
+   * <p>
+   *   To be called once from {@link DrillResultSetImpl#execute()} before
+   *   {@link next()} is called from {@link AvaticaResultSet#next()}.
+   * <p>
+   */
+  void loadInitialSchema() throws SQLException {
+    if ( initialSchemaLoaded ) {
+      throw new IllegalStateException(
+          "loadInitialSchema() called a second time" );
+    }
+    assert ! afterLastRow : "afterLastRow already true in loadInitialSchema()";
+    assert ! afterFirstBatch : "afterLastRow already true in loadInitialSchema()";
+    assert -1 == currentRecordNumber
+        : "currentRecordNumber not -1 (is " + currentRecordNumber
+          + ") in loadInitialSchema()";
+    assert 0 == currentBatchHolder.getRecordCount()
+        : "currentBatchHolder.getRecordCount() not 0 (is "
+          + currentBatchHolder.getRecordCount() + " in loadInitialSchema()";
+
+    returnTrueForNextCallToNext = true;
+
+    nextRowInternally();
+
+    initialSchemaLoaded = true;
+  }
+
+  /**
+   * Advances this cursor to the next row, if any, or to after the sequence of
+   * rows if no next row.
+   *
+   * @return  whether cursor is positioned at a row (false when after end of
+   *   results)
+   */
+  @Override
+  public boolean next() throws SQLException {
+    if ( ! initialSchemaLoaded ) {
+      throw new IllegalStateException(
+          "next() called but loadInitialSchema() was not called" );
+    }
+    assert afterFirstBatch : "afterFirstBatch still false in next()";
+
+    if ( afterLastRow ) {
+      // We're already after end of rows/records--just report that after end.
+      return false;
+    }
+    else if ( returnTrueForNextCallToNext ) {
+      // We have a deferred "not after end" to report--reset and report that.
+      returnTrueForNextCallToNext = false;
+      return true;
+    }
+    else {
+      return nextRowInternally();
+    }
+  }
+
   @Override
   public void close() {
     // currentBatchHolder is owned by resultSet and cleaned up by

http://git-wip-us.apache.org/repos/asf/drill/blob/711992f2/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
index c682d97..31593bf 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
@@ -184,7 +184,7 @@ class DrillResultSetImpl extends AvaticaResultSet implements DrillResultSet {
 
     // Read first (schema-only) batch to initialize result-set metadata from
     // (initial) schema before Statement.execute...(...) returns result set:
-    cursor.next();
+    cursor.loadInitialSchema();
 
     return this;
   }


[2/5] drill git commit: DRILL-3285: Part 3--Invert beforeFirstBatch -> ! afterFirstBatch.

Posted by me...@apache.org.
DRILL-3285: Part 3--Invert beforeFirstBatch -> ! afterFirstBatch.


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

Branch: refs/heads/master
Commit: e050f1e17f32d60960b01757455face52050bed3
Parents: d040429
Author: dbarclay <db...@maprtech.com>
Authored: Fri Jun 12 18:42:14 2015 -0700
Committer: Mehant Baid <me...@gmail.com>
Committed: Mon Jun 22 13:05:04 2015 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/drill/jdbc/impl/DrillCursor.java    | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/e050f1e1/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
index a54422e..81b2de5 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
@@ -70,8 +70,8 @@ class DrillCursor implements Cursor {
    */
   private boolean returnTrueForNextCallToNext = false;
 
-  /** Whether on first batch.  (Re skipping spurious empty batches.) */
-  private boolean beforeFirstBatch = true;
+  /** Whether after first batch.  (Re skipping spurious empty batches.) */
+  private boolean afterFirstBatch = false;
 
   /** ... corresponds to current schema. */
   private DrillColumnMetaDataList columnMetaDataList;
@@ -156,7 +156,7 @@ class DrillCursor implements Cursor {
         while ( qrb != null
                 && ( qrb.getHeader().getRowCount() == 0
                      || qrb.getData() == null )
-                && ! beforeFirstBatch ) {
+                && afterFirstBatch ) {
           // Empty message--dispose of and try to get another.
           logger.warn( "Spurious batch read: {}", qrb );
 
@@ -178,7 +178,7 @@ class DrillCursor implements Cursor {
           }
         }
 
-        beforeFirstBatch = false;
+        afterFirstBatch = true;
 
         if (qrb == null) {
           // End of batches--clean up, set state to done, report after last row.


[4/5] drill git commit: DRILL-3285: Part 4--Reorder fields, updateColumns.

Posted by me...@apache.org.
DRILL-3285: Part 4--Reorder fields, updateColumns.


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

Branch: refs/heads/master
Commit: 22232d4d31e66e31cdb3a017cffab444223dd47e
Parents: e050f1e
Author: dbarclay <db...@maprtech.com>
Authored: Fri Jun 12 18:26:28 2015 -0700
Committer: Mehant Baid <me...@gmail.com>
Committed: Mon Jun 22 13:05:04 2015 -0700

----------------------------------------------------------------------
 .../org/apache/drill/jdbc/impl/DrillCursor.java | 40 ++++++++++----------
 1 file changed, 20 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/22232d4d/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
index 81b2de5..cdf030b 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
@@ -48,12 +48,20 @@ class DrillCursor implements Cursor {
 
   private final DrillResultSetImpl.ResultsListener resultsListener;
 
+  private final DrillAccessorList accessors = new DrillAccessorList();
+
+  /** Schema of current batch (null before first load). */
+  private BatchSchema schema;
+
+  /** ... corresponds to current schema. */
+  private DrillColumnMetaDataList columnMetaDataList;
+
   /** Whether we're past the special first call to this.next() from
    * DrillResultSetImpl.execute(). */
   private boolean initialSchemaLoaded = false;
 
-  /** Whether cursor is after the end of the sequence of records/rows. */
-  private boolean afterLastRow = false;
+  /** Whether after first batch.  (Re skipping spurious empty batches.) */
+  private boolean afterFirstBatch = false;
 
   /**
    * Whether the next call to this.next() should just return {@code true} rather
@@ -70,21 +78,13 @@ class DrillCursor implements Cursor {
    */
   private boolean returnTrueForNextCallToNext = false;
 
-  /** Whether after first batch.  (Re skipping spurious empty batches.) */
-  private boolean afterFirstBatch = false;
-
-  /** ... corresponds to current schema. */
-  private DrillColumnMetaDataList columnMetaDataList;
-
-  /** Schema of current batch (null before first load). */
-  private BatchSchema schema;
+  /** Whether cursor is after the end of the sequence of records/rows. */
+  private boolean afterLastRow = false;
 
   /** Zero-based offset of current record in record batch.
    * (Not <i>row</i> number.) */
   private int currentRecordNumber = -1;
 
-  private final DrillAccessorList accessors = new DrillAccessorList();
-
 
   /**
    *
@@ -111,6 +111,14 @@ class DrillCursor implements Cursor {
     return accessors;
   }
 
+  private void updateColumns() {
+    accessors.generateAccessors(this, currentBatchHolder);
+    columnMetaDataList.updateColumnMetaData(UNKNOWN, UNKNOWN, UNKNOWN, schema);
+    if (getResultSet().changeListener != null) {
+      getResultSet().changeListener.schemaChanged(schema);
+    }
+  }
+
   /**
    * Advances this cursor to the next row, if any, or to after the sequence of
    * rows if no next row.  However, the first call does not advance to the first
@@ -238,14 +246,6 @@ class DrillCursor implements Cursor {
     }
   }
 
-  private void updateColumns() {
-    accessors.generateAccessors(this, currentBatchHolder);
-    columnMetaDataList.updateColumnMetaData(UNKNOWN, UNKNOWN, UNKNOWN, schema);
-    if (getResultSet().changeListener != null) {
-      getResultSet().changeListener.schemaChanged(schema);
-    }
-  }
-
   @Override
   public void close() {
     // currentBatchHolder is owned by resultSet and cleaned up by


[3/5] drill git commit: DRILL-3285: Part 1--Prep., Hygiene: Mainly, adding comments.

Posted by me...@apache.org.
DRILL-3285: Part 1--Prep., Hygiene:  Mainly, adding comments.

Added/edited comments:
- field doc. comments
- method doc. comments
- branch/block comments

Removed unused recordBatchCount and getRecordBatchCount().

Added logger call for spurious batch.

Various cleanup:
- Cleaned up logger.
- Added "final" on updateColumns().
- Wrapped some lines
- Misc. comment whitespace.


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

Branch: refs/heads/master
Commit: 228be48f47ecc14b415e80b7351cc3f78957d57d
Parents: 9c125b0
Author: dbarclay <db...@maprtech.com>
Authored: Wed Jun 10 14:03:21 2015 -0700
Committer: Mehant Baid <me...@gmail.com>
Committed: Mon Jun 22 13:05:04 2015 -0700

----------------------------------------------------------------------
 .../org/apache/drill/jdbc/impl/DrillCursor.java | 112 ++++++++++++++-----
 .../drill/jdbc/impl/DrillResultSetImpl.java     |   5 +-
 2 files changed, 87 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/228be48f/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
index 13a1d95..9b54cf3 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
@@ -31,33 +31,58 @@ import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
+import org.slf4j.Logger;
+import static org.slf4j.LoggerFactory.getLogger;
 
 
 class DrillCursor implements Cursor {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillCursor.class);
+  private static final Logger logger = getLogger( DrillCursor.class );
 
   private static final String UNKNOWN = "--UNKNOWN--";
 
-  /** The associated java.sql.ResultSet implementation. */
+  /** The associated {@link java.sql.ResultSet} implementation. */
   private final DrillResultSetImpl resultSet;
 
+  /** Holds current batch of records (none before first load). */
   private final RecordBatchLoader currentBatch;
+
   private final DrillResultSetImpl.ResultsListener resultsListener;
 
-  // TODO:  Doc.:  Say what's started (set of rows?  just current result batch?)
+  /** Whether we're past the special first call to this.next() from
+   * DrillResultSetImpl.execute(). */
   private boolean started = false;
+
+  /** Whether cursor is after the end of the sequence of records/rows. */
   private boolean finished = false;
-  // TODO:  Doc.: Say what "readFirstNext" means.
+
+  /**
+   * Whether the next call to this.next() should just return {@code true} rather
+   * than trying to actually advance to the next record.
+   * <p>
+   *   Currently, can be true only for first call to next().
+   * </p>
+   * <p>
+   *   (Relates to loadInitialSchema()'s calling nextRowInternally()
+   *   one "extra" time
+   *   (extra relative to number of ResultSet.next() calls) at the beginning to
+   *   get first batch and schema before Statement.execute...(...) even returns.
+   * </p>
+   */
   private boolean redoFirstNext = false;
-  // TODO:  Doc.: First what? (First batch? record? "next" call/operation?)
+
+  /** Whether on first batch.  (Re skipping spurious empty batches.) */
   private boolean first = true;
 
+  /** ... corresponds to current schema. */
   private DrillColumnMetaDataList columnMetaDataList;
+
+  /** Schema of current batch (null before first load). */
   private BatchSchema schema;
 
-  /** Zero-based index of current record in record batch. */
+  /** Zero-based offset of current record in record batch.
+   * (Not <i>row</i> number.) */
   private int currentRecordNumber = -1;
-  private long recordBatchCount;
+
   private final DrillAccessorList accessors = new DrillAccessorList();
 
 
@@ -80,49 +105,75 @@ class DrillCursor implements Cursor {
   }
 
   @Override
-  public List<Accessor> createAccessors(List<ColumnMetaData> types, Calendar localCalendar, Factory factory) {
+  public List<Accessor> createAccessors(List<ColumnMetaData> types,
+                                        Calendar localCalendar, Factory factory) {
     columnMetaDataList = (DrillColumnMetaDataList) types;
     return accessors;
   }
 
-  // TODO:  Doc.:  Specify what the return value actually means.  (The wording
-  // "Moves to the next row" and "Whether moved" from the documentation of the
-  // implemented interface (net.hydromatic.avatica.Cursor) doesn't address
-  // moving past last row or how to evaluate "whether moved" on the first call.
-  // In particular, document what the return value indicates about whether we're
-  // currently at a valid row (or whether next() can be called again, or
-  // whatever it does indicate), especially the first time this next() called
-  // for a new result.
+  /**
+   * Advances this cursor to the next row, if any, or to after the sequence of
+   * rows if no next row.  However, the first call does not advance to the first
+   * row, only reading schema information.
+   * <p>
+   *   Is to be called (once) from {@link DrillResultSetImpl#execute()}, and
+   *   then from {@link AvaticaResultSet#next()}.
+   * </p>
+   *
+   * @return  whether cursor is positioned at a row (false when after end of
+   *   results)
+   */
   @Override
   public boolean next() throws SQLException {
     if (!started) {
       started = true;
       redoFirstNext = true;
     } else if (redoFirstNext && !finished) {
+      // We have a deferred "not after end" to report--reset and report that.
       redoFirstNext = false;
       return true;
     }
 
     if (finished) {
+      // We're already after end of rows/records--just report that after end.
       return false;
     }
 
     if (currentRecordNumber + 1 < currentBatch.getRecordCount()) {
-      // Next index is in within current batch--just increment to that record.
+      // Have next row in current batch--just advance index and report "at a row."
       currentRecordNumber++;
       return true;
     } else {
-      // Next index is not in current batch (including initial empty batch--
-      // (try to) get next batch.
+      // No (more) records in any current batch--try to get first or next batch.
+      // (First call always takes this branch.)
+
       try {
         QueryDataBatch qrb = resultsListener.getNext();
-        recordBatchCount++;
-        while (qrb != null && (qrb.getHeader().getRowCount() == 0 || qrb.getData() == null ) && !first) {
+
+        // (Apparently:)  Skip any spurious empty batches (batches that have
+        // zero rows and/or null data, other than the first batch (which carries
+        // the (initial) schema but no rows)).
+        while ( qrb != null
+                && ( qrb.getHeader().getRowCount() == 0
+                     || qrb.getData() == null )
+                && ! first ) {
+          // Empty message--dispose of and try to get another.
+          logger.warn( "Spurious batch read: {}", qrb );
+
           qrb.release();
+
           qrb = resultsListener.getNext();
-          recordBatchCount++;
-          if(qrb != null && qrb.getData()==null){
+
+          // NOTE:  It is unclear why this check does not check getRowCount()
+          // as the loop condition above does.
+          if ( qrb != null && qrb.getData() == null ) {
+            // Got another batch with null data--dispose of and report "no more
+            // rows".
+
             qrb.release();
+
+            // NOTE:  It is unclear why this returns false but doesn't set
+            // afterLastRow (as we do when we normally return false).
             return false;
           }
         }
@@ -130,11 +181,17 @@ class DrillCursor implements Cursor {
         first = false;
 
         if (qrb == null) {
-          currentBatch.clear();
+          // End of batches--clean up, set state to done, report after last row.
+
+          currentBatch.clear();  // (We load it so we clear it.)
           finished = true;
           return false;
         } else {
+          // Got next (or first) batch--reset record offset to beginning,
+          // assimilate schema if changed, ... ???
+
           currentRecordNumber = 0;
+
           final boolean changed;
           try {
             changed = currentBatch.load(qrb.getHeader().getDef(), qrb.getData());
@@ -146,6 +203,7 @@ class DrillCursor implements Cursor {
           if (changed) {
             updateColumns();
           }
+
           if (redoFirstNext && currentBatch.getRecordCount() == 0) {
             redoFirstNext = false;
           }
@@ -178,7 +236,7 @@ class DrillCursor implements Cursor {
     }
   }
 
-  void updateColumns() {
+  private void updateColumns() {
     accessors.generateAccessors(this, currentBatch);
     columnMetaDataList.updateColumnMetaData(UNKNOWN, UNKNOWN, UNKNOWN, schema);
     if (getResultSet().changeListener != null) {
@@ -186,10 +244,6 @@ class DrillCursor implements Cursor {
     }
   }
 
-  public long getRecordBatchCount() {
-    return recordBatchCount;
-  }
-
   @Override
   public void close() {
     // currentBatch is owned by resultSet and cleaned up by

http://git-wip-us.apache.org/repos/asf/drill/blob/228be48f/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
index d7fafe9..ab1eb92 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
@@ -99,7 +99,7 @@ class DrillResultSetImpl extends AvaticaResultSet implements DrillResultSet {
    *          cancelation and no QueryCanceledSqlException had been thrown yet
    *          for this ResultSet
    * @throws  AlreadyClosedSqlException  if ResultSet is closed
-   * @throws SQLException if error in calling {@link #isClosed()}
+   * @throws  SQLException  if error in calling {@link #isClosed()}
    */
   private void checkNotClosed() throws SQLException {
     if ( isClosed() ) {
@@ -181,6 +181,9 @@ class DrillResultSetImpl extends AvaticaResultSet implements DrillResultSet {
       // but JDBC client certainly could.
       throw new SQLException( "Interrupted", e );
     }
+
+    // Read first (schema-only) batch to initialize result-set metadata from
+    // (initial) schema before Statement.execute...(...) returns result set:
     cursor.next();
 
     return this;


[5/5] drill git commit: DRILL-3285: Part 2--Renaming.

Posted by me...@apache.org.
DRILL-3285: Part 2--Renaming.

Renamed state/control-flow members:
- started -> initialSchemaLoaded
- first -> beforeFirstBatch
- redoFirstNext -> returnTrueForNextCallToNext
- finished -> afterLastRow

Renamed other items:
- changed -> schemaChanged
- currentBatch -> currentBatchHolder
- DrillResultSet's currentBatch -> batchLoader


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

Branch: refs/heads/master
Commit: d040429cdafee0bfd25d1115a36922a10e2b5d96
Parents: 228be48
Author: dbarclay <db...@maprtech.com>
Authored: Fri Jun 12 18:37:58 2015 -0700
Committer: Mehant Baid <me...@gmail.com>
Committed: Mon Jun 22 13:05:04 2015 -0700

----------------------------------------------------------------------
 .../org/apache/drill/jdbc/impl/DrillCursor.java | 52 ++++++++++----------
 .../drill/jdbc/impl/DrillResultSetImpl.java     |  6 +--
 2 files changed, 30 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/d040429c/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
index 9b54cf3..a54422e 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
@@ -44,16 +44,16 @@ class DrillCursor implements Cursor {
   private final DrillResultSetImpl resultSet;
 
   /** Holds current batch of records (none before first load). */
-  private final RecordBatchLoader currentBatch;
+  private final RecordBatchLoader currentBatchHolder;
 
   private final DrillResultSetImpl.ResultsListener resultsListener;
 
   /** Whether we're past the special first call to this.next() from
    * DrillResultSetImpl.execute(). */
-  private boolean started = false;
+  private boolean initialSchemaLoaded = false;
 
   /** Whether cursor is after the end of the sequence of records/rows. */
-  private boolean finished = false;
+  private boolean afterLastRow = false;
 
   /**
    * Whether the next call to this.next() should just return {@code true} rather
@@ -68,10 +68,10 @@ class DrillCursor implements Cursor {
    *   get first batch and schema before Statement.execute...(...) even returns.
    * </p>
    */
-  private boolean redoFirstNext = false;
+  private boolean returnTrueForNextCallToNext = false;
 
   /** Whether on first batch.  (Re skipping spurious empty batches.) */
-  private boolean first = true;
+  private boolean beforeFirstBatch = true;
 
   /** ... corresponds to current schema. */
   private DrillColumnMetaDataList columnMetaDataList;
@@ -92,7 +92,7 @@ class DrillCursor implements Cursor {
    */
   DrillCursor(final DrillResultSetImpl resultSet) {
     this.resultSet = resultSet;
-    currentBatch = resultSet.currentBatch;
+    currentBatchHolder = resultSet.batchLoader;
     resultsListener = resultSet.resultsListener;
   }
 
@@ -125,21 +125,21 @@ class DrillCursor implements Cursor {
    */
   @Override
   public boolean next() throws SQLException {
-    if (!started) {
-      started = true;
-      redoFirstNext = true;
-    } else if (redoFirstNext && !finished) {
+    if (!initialSchemaLoaded) {
+      initialSchemaLoaded = true;
+      returnTrueForNextCallToNext = true;
+    } else if (returnTrueForNextCallToNext && !afterLastRow) {
       // We have a deferred "not after end" to report--reset and report that.
-      redoFirstNext = false;
+      returnTrueForNextCallToNext = false;
       return true;
     }
 
-    if (finished) {
+    if (afterLastRow) {
       // We're already after end of rows/records--just report that after end.
       return false;
     }
 
-    if (currentRecordNumber + 1 < currentBatch.getRecordCount()) {
+    if (currentRecordNumber + 1 < currentBatchHolder.getRecordCount()) {
       // Have next row in current batch--just advance index and report "at a row."
       currentRecordNumber++;
       return true;
@@ -156,7 +156,7 @@ class DrillCursor implements Cursor {
         while ( qrb != null
                 && ( qrb.getHeader().getRowCount() == 0
                      || qrb.getData() == null )
-                && ! first ) {
+                && ! beforeFirstBatch ) {
           // Empty message--dispose of and try to get another.
           logger.warn( "Spurious batch read: {}", qrb );
 
@@ -178,13 +178,13 @@ class DrillCursor implements Cursor {
           }
         }
 
-        first = false;
+        beforeFirstBatch = false;
 
         if (qrb == null) {
           // End of batches--clean up, set state to done, report after last row.
 
-          currentBatch.clear();  // (We load it so we clear it.)
-          finished = true;
+          currentBatchHolder.clear();  // (We load it so we clear it.)
+          afterLastRow = true;
           return false;
         } else {
           // Got next (or first) batch--reset record offset to beginning,
@@ -192,20 +192,22 @@ class DrillCursor implements Cursor {
 
           currentRecordNumber = 0;
 
-          final boolean changed;
+          final boolean schemaChanged;
           try {
-            changed = currentBatch.load(qrb.getHeader().getDef(), qrb.getData());
+            schemaChanged = currentBatchHolder.load(qrb.getHeader().getDef(),
+                                                    qrb.getData());
           }
           finally {
             qrb.release();
           }
-          schema = currentBatch.getSchema();
-          if (changed) {
+          schema = currentBatchHolder.getSchema();
+          if (schemaChanged) {
             updateColumns();
           }
 
-          if (redoFirstNext && currentBatch.getRecordCount() == 0) {
-            redoFirstNext = false;
+          if (returnTrueForNextCallToNext
+              && currentBatchHolder.getRecordCount() == 0) {
+            returnTrueForNextCallToNext = false;
           }
           return true;
         }
@@ -237,7 +239,7 @@ class DrillCursor implements Cursor {
   }
 
   private void updateColumns() {
-    accessors.generateAccessors(this, currentBatch);
+    accessors.generateAccessors(this, currentBatchHolder);
     columnMetaDataList.updateColumnMetaData(UNKNOWN, UNKNOWN, UNKNOWN, schema);
     if (getResultSet().changeListener != null) {
       getResultSet().changeListener.schemaChanged(schema);
@@ -246,7 +248,7 @@ class DrillCursor implements Cursor {
 
   @Override
   public void close() {
-    // currentBatch is owned by resultSet and cleaned up by
+    // currentBatchHolder is owned by resultSet and cleaned up by
     // DrillResultSet.cleanup()
 
     // listener is owned by resultSet and cleaned up by

http://git-wip-us.apache.org/repos/asf/drill/blob/d040429c/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
index ab1eb92..c682d97 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
@@ -68,7 +68,7 @@ class DrillResultSetImpl extends AvaticaResultSet implements DrillResultSet {
   private final DrillClient client;
   // TODO:  Resolve:  Since is barely manipulated here in DrillResultSetImpl,
   //  move down into DrillCursor and have this.clean() have cursor clean it.
-  final RecordBatchLoader currentBatch;
+  final RecordBatchLoader batchLoader;
   final DrillCursor cursor;
   boolean hasPendingCancelationNotification;
 
@@ -82,7 +82,7 @@ class DrillResultSetImpl extends AvaticaResultSet implements DrillResultSet {
     resultsListener = new ResultsListener( batchQueueThrottlingThreshold );
     DrillConnection c = (DrillConnection) statement.getConnection();
     DrillClient client = c.getClient();
-    currentBatch = new RecordBatchLoader(client.getAllocator());
+    batchLoader = new RecordBatchLoader(client.getAllocator());
     this.client = client;
     cursor = new DrillCursor(this);
   }
@@ -133,7 +133,7 @@ class DrillResultSetImpl extends AvaticaResultSet implements DrillResultSet {
       client.cancelQuery(resultsListener.getQueryId());
     }
     resultsListener.close();
-    currentBatch.clear();
+    batchLoader.clear();
   }
 
   @Override